[PATCH] cobalt/posix/process: pipeline: abstract kernel event handlers
Philippe Gerum
rpm at xenomai.org
Thu Dec 17 19:02:46 CET 2020
From: Philippe Gerum <rpm at xenomai.org>
Although there are significant commonalities between the I-pipe and
Dovetail when it comes to dealing with synchronous kernel events,
there is no strict 1:1 mapping between the two kernel interfaces.
As an initial step, move all the code handling the kernel events to
the I-pipe section. We may exploit commonalities between the I-pipe
and Dovetail in this area as we gradually merge support for the
latter.
No functional change is introduced.
Signed-off-by: Philippe Gerum <rpm at xenomai.org>
---
.../cobalt/kernel/ipipe/pipeline/kevents.h | 31 +
kernel/cobalt/ipipe/Makefile | 4 +-
kernel/cobalt/ipipe/kevents.c | 860 ++++++++++++++++++
kernel/cobalt/posix/process.c | 846 +----------------
kernel/cobalt/posix/process.h | 4 +
kernel/cobalt/posix/signal.c | 5 +
kernel/cobalt/posix/signal.h | 2 +
kernel/cobalt/thread.c | 4 +-
kernel/cobalt/trace/cobalt-core.h | 12 +-
9 files changed, 930 insertions(+), 838 deletions(-)
create mode 100644 include/cobalt/kernel/ipipe/pipeline/kevents.h
create mode 100644 kernel/cobalt/ipipe/kevents.c
diff --git a/include/cobalt/kernel/ipipe/pipeline/kevents.h b/include/cobalt/kernel/ipipe/pipeline/kevents.h
new file mode 100644
index 000000000..30425a96b
--- /dev/null
+++ b/include/cobalt/kernel/ipipe/pipeline/kevents.h
@@ -0,0 +1,31 @@
+/*
+ * SPDX-License-Identifier: GPL-2.0
+ *
+ * Copyright (C) 2019 Philippe Gerum <rpm at xenomai.org>
+ */
+
+#ifndef _COBALT_KERNEL_IPIPE_KEVENTS_H
+#define _COBALT_KERNEL_IPIPE_KEVENTS_H
+
+struct cobalt_process;
+struct cobalt_thread;
+
+static inline
+int pipeline_attach_process(struct cobalt_process *process)
+{
+ return 0;
+}
+
+static inline
+void pipeline_detach_process(struct cobalt_process *process)
+{ }
+
+int pipeline_prepare_current(void);
+
+void pipeline_attach_current(struct xnthread *thread);
+
+int pipeline_trap_kevents(void);
+
+void pipeline_enable_kevents(void);
+
+#endif /* !_COBALT_KERNEL_IPIPE_KEVENTS_H */
diff --git a/kernel/cobalt/ipipe/Makefile b/kernel/cobalt/ipipe/Makefile
index 6021008fb..5170bb32b 100644
--- a/kernel/cobalt/ipipe/Makefile
+++ b/kernel/cobalt/ipipe/Makefile
@@ -1,3 +1,5 @@
+ccflags-y += -Ikernel
+
obj-y += pipeline.o
-pipeline-y := init.o intr.o
+pipeline-y := init.o intr.o kevents.o
diff --git a/kernel/cobalt/ipipe/kevents.c b/kernel/cobalt/ipipe/kevents.c
new file mode 100644
index 000000000..ba584677c
--- /dev/null
+++ b/kernel/cobalt/ipipe/kevents.c
@@ -0,0 +1,860 @@
+/*
+ * SPDX-License-Identifier: GPL-2.0
+ *
+ * Copyright (C) 2001-2014 Philippe Gerum <rpm at xenomai.org>.
+ * Copyright (C) 2001-2014 The Xenomai project <http://www.xenomai.org>
+ * Copyright (C) 2006 Gilles Chanteperdrix <gilles.chanteperdrix at xenomai.org>
+ *
+ * SMP support Copyright (C) 2004 The HYADES project <http://www.hyades-itea.org>
+ * RTAI/fusion Copyright (C) 2004 The RTAI project <http://www.rtai.org>
+ */
+
+#include <linux/ipipe.h>
+#include <linux/ipipe_tickdev.h>
+#include <linux/ptrace.h>
+#include <pipeline/kevents.h>
+#include <cobalt/kernel/sched.h>
+#include <cobalt/kernel/thread.h>
+#include <cobalt/kernel/vdso.h>
+#include <rtdm/driver.h>
+#include <trace/events/cobalt-core.h>
+#include "../posix/process.h"
+#include "../posix/thread.h"
+#include "../posix/memory.h"
+
+static void detach_current(void);
+
+static inline struct cobalt_process *
+process_from_thread(struct xnthread *thread)
+{
+ return container_of(thread, struct cobalt_thread, threadbase)->process;
+}
+
+#ifdef IPIPE_KEVT_PTRESUME
+
+static void stop_debugged_process(struct xnthread *thread)
+{
+ struct cobalt_process *process = process_from_thread(thread);
+ struct cobalt_thread *cth;
+
+ if (process->debugged_threads > 0)
+ return;
+
+ list_for_each_entry(cth, &process->thread_list, next) {
+ if (&cth->threadbase == thread)
+ continue;
+
+ xnthread_suspend(&cth->threadbase, XNDBGSTOP, XN_INFINITE,
+ XN_RELATIVE, NULL);
+ }
+}
+
+static void resume_debugged_process(struct cobalt_process *process)
+{
+ struct cobalt_thread *cth;
+
+ xnsched_lock();
+
+ list_for_each_entry(cth, &process->thread_list, next)
+ if (xnthread_test_state(&cth->threadbase, XNDBGSTOP))
+ xnthread_resume(&cth->threadbase, XNDBGSTOP);
+
+ xnsched_unlock();
+}
+
+#else /* !IPIPE_KEVT_PTRESUME */
+
+static inline void stop_debugged_process(struct xnthread *thread)
+{
+}
+
+static inline void resume_debugged_process(struct cobalt_process *process)
+{
+}
+
+#endif /* !IPIPE_KEVT_PTRESUME */
+
+/* called with nklock held */
+static void register_debugged_thread(struct xnthread *thread)
+{
+ struct cobalt_process *process = process_from_thread(thread);
+
+ xnthread_set_state(thread, XNSSTEP);
+
+ stop_debugged_process(thread);
+ process->debugged_threads++;
+
+ if (xnthread_test_state(thread, XNRELAX))
+ xnthread_suspend(thread, XNDBGSTOP, XN_INFINITE, XN_RELATIVE,
+ NULL);
+}
+
+/* called with nklock held */
+static void unregister_debugged_thread(struct xnthread *thread)
+{
+ struct cobalt_process *process = process_from_thread(thread);
+
+ process->debugged_threads--;
+ xnthread_clear_state(thread, XNSSTEP);
+
+ if (process->debugged_threads == 0)
+ resume_debugged_process(process);
+}
+
+static inline int handle_exception(struct ipipe_trap_data *d)
+{
+ struct xnthread *thread;
+ struct xnsched *sched;
+
+ sched = xnsched_current();
+ thread = sched->curr;
+
+ trace_cobalt_thread_fault(xnarch_fault_pc(d),
+ xnarch_fault_trap(d));
+
+ if (xnthread_test_state(thread, XNROOT))
+ return 0;
+
+#ifdef IPIPE_KEVT_USERINTRET
+ if (xnarch_fault_bp_p(d) && user_mode(d->regs)) {
+ spl_t s;
+
+ XENO_WARN_ON(CORE, xnthread_test_state(thread, XNRELAX));
+ xnlock_get_irqsave(&nklock, s);
+ xnthread_set_info(thread, XNCONTHI);
+ ipipe_enable_user_intret_notifier();
+ stop_debugged_process(thread);
+ xnlock_put_irqrestore(&nklock, s);
+ xnsched_run();
+ }
+#endif
+
+ if (xnarch_fault_fpu_p(d)) {
+#ifdef CONFIG_XENO_ARCH_FPU
+ spl_t s;
+
+ /* FPU exception received in primary mode. */
+ splhigh(s);
+ if (xnarch_handle_fpu_fault(sched->fpuholder, thread, d)) {
+ sched->fpuholder = thread;
+ splexit(s);
+ return 1;
+ }
+ splexit(s);
+#endif /* CONFIG_XENO_ARCH_FPU */
+#if LINUX_VERSION_CODE >= KERNEL_VERSION(4, 16, 0)
+ printk("invalid use of FPU in Xenomai context at %pS\n",
+ (void *)xnarch_fault_pc(d));
+#else
+ print_symbol("invalid use of FPU in Xenomai context at %s\n",
+ xnarch_fault_pc(d));
+#endif
+ }
+
+ /*
+ * If we experienced a trap on behalf of a shadow thread
+ * running in primary mode, move it to the Linux domain,
+ * leaving the kernel process the exception.
+ */
+#if defined(CONFIG_XENO_OPT_DEBUG_COBALT) || defined(CONFIG_XENO_OPT_DEBUG_USER)
+ if (!user_mode(d->regs)) {
+ xntrace_panic_freeze();
+ printk(XENO_WARNING
+ "switching %s to secondary mode after exception #%u in "
+ "kernel-space at 0x%lx (pid %d)\n", thread->name,
+ xnarch_fault_trap(d),
+ xnarch_fault_pc(d),
+ xnthread_host_pid(thread));
+ xntrace_panic_dump();
+ } else if (xnarch_fault_notify(d)) /* Don't report debug traps */
+ printk(XENO_WARNING
+ "switching %s to secondary mode after exception #%u from "
+ "user-space at 0x%lx (pid %d)\n", thread->name,
+ xnarch_fault_trap(d),
+ xnarch_fault_pc(d),
+ xnthread_host_pid(thread));
+#endif
+
+ if (xnarch_fault_pf_p(d))
+ /*
+ * The page fault counter is not SMP-safe, but it's a
+ * simple indicator that something went wrong wrt
+ * memory locking anyway.
+ */
+ xnstat_counter_inc(&thread->stat.pf);
+
+ xnthread_relax(xnarch_fault_notify(d), SIGDEBUG_MIGRATE_FAULT);
+
+ return 0;
+}
+
+static int handle_mayday_event(struct pt_regs *regs)
+{
+ XENO_BUG_ON(COBALT, !xnthread_test_state(xnthread_current(), XNUSER));
+
+ xnthread_relax(0, 0);
+
+ return KEVENT_PROPAGATE;
+}
+
+int ipipe_trap_hook(struct ipipe_trap_data *data)
+{
+ if (data->exception == IPIPE_TRAP_MAYDAY)
+ return handle_mayday_event(data->regs);
+
+ /*
+ * No migration is possible on behalf of the head domain, so
+ * the following access is safe.
+ */
+ raw_cpu_ptr(&cobalt_machine_cpudata)->faults[data->exception]++;
+
+ if (handle_exception(data))
+ return KEVENT_STOP;
+
+ /*
+ * CAUTION: access faults must be propagated downstream
+ * whichever domain caused them, so that we don't spuriously
+ * raise a fatal error when some Linux fixup code is available
+ * to recover from the fault.
+ */
+ return KEVENT_PROPAGATE;
+}
+
+/*
+ * Legacy idle hook, unconditionally allow entering the idle state.
+ */
+bool ipipe_enter_idle_hook(void)
+{
+ return true;
+}
+
+#ifdef CONFIG_SMP
+
+static int handle_setaffinity_event(struct ipipe_cpu_migration_data *d)
+{
+ struct task_struct *p = d->task;
+ struct xnthread *thread;
+ spl_t s;
+
+ thread = xnthread_from_task(p);
+ if (thread == NULL)
+ return KEVENT_PROPAGATE;
+
+ /*
+ * Detect a Cobalt thread sleeping in primary mode which is
+ * required to migrate to another CPU by the host kernel.
+ *
+ * We may NOT fix up thread->sched immediately using the
+ * passive migration call, because that latter always has to
+ * take place on behalf of the target thread itself while
+ * running in secondary mode. Therefore, that thread needs to
+ * go through secondary mode first, then move back to primary
+ * mode, so that affinity_ok() does the fixup work.
+ *
+ * We force this by sending a SIGSHADOW signal to the migrated
+ * thread, asking it to switch back to primary mode from the
+ * handler, at which point the interrupted syscall may be
+ * restarted.
+ */
+ xnlock_get_irqsave(&nklock, s);
+
+ if (xnthread_test_state(thread, XNTHREAD_BLOCK_BITS & ~XNRELAX))
+ xnthread_signal(thread, SIGSHADOW, SIGSHADOW_ACTION_HARDEN);
+
+ xnlock_put_irqrestore(&nklock, s);
+
+ return KEVENT_PROPAGATE;
+}
+
+static inline bool affinity_ok(struct task_struct *p) /* nklocked, IRQs off */
+{
+ struct xnthread *thread = xnthread_from_task(p);
+ struct xnsched *sched;
+ int cpu = task_cpu(p);
+
+ /*
+ * To maintain consistency between both Cobalt and host
+ * schedulers, reflecting a thread migration to another CPU
+ * into the Cobalt scheduler state must happen from secondary
+ * mode only, on behalf of the migrated thread itself once it
+ * runs on the target CPU.
+ *
+ * This means that the Cobalt scheduler state regarding the
+ * CPU information lags behind the host scheduler state until
+ * the migrated thread switches back to primary mode
+ * (i.e. task_cpu(p) != xnsched_cpu(xnthread_from_task(p)->sched)).
+ * This is ok since Cobalt does not schedule such thread until then.
+ *
+ * check_affinity() detects when a Cobalt thread switching
+ * back to primary mode did move to another CPU earlier while
+ * in secondary mode. If so, do the fixups to reflect the
+ * change.
+ */
+ if (!xnsched_threading_cpu(cpu)) {
+ /*
+ * The thread is about to switch to primary mode on a
+ * non-rt CPU, which is damn wrong and hopeless.
+ * Whine and cancel that thread.
+ */
+ printk(XENO_WARNING "thread %s[%d] switched to non-rt CPU%d, aborted.\n",
+ thread->name, xnthread_host_pid(thread), cpu);
+ /*
+ * Can't call xnthread_cancel() from a migration
+ * point, that would break. Since we are on the wakeup
+ * path to hardening, just raise XNCANCELD to catch it
+ * in xnthread_harden().
+ */
+ xnthread_set_info(thread, XNCANCELD);
+ return false;
+ }
+
+ sched = xnsched_struct(cpu);
+ if (sched == thread->sched)
+ return true;
+
+ /*
+ * The current thread moved to a supported real-time CPU,
+ * which is not part of its original affinity mask
+ * though. Assume user wants to extend this mask.
+ */
+ if (!cpumask_test_cpu(cpu, &thread->affinity))
+ cpumask_set_cpu(cpu, &thread->affinity);
+
+ xnthread_run_handler_stack(thread, move_thread, cpu);
+ xnthread_migrate_passive(thread, sched);
+
+ return true;
+}
+
+#else /* !CONFIG_SMP */
+
+struct ipipe_cpu_migration_data;
+
+static int handle_setaffinity_event(struct ipipe_cpu_migration_data *d)
+{
+ return KEVENT_PROPAGATE;
+}
+
+static inline bool affinity_ok(struct task_struct *p)
+{
+ return true;
+}
+
+#endif /* CONFIG_SMP */
+
+void ipipe_migration_hook(struct task_struct *p) /* hw IRQs off */
+{
+ struct xnthread *thread = xnthread_from_task(p);
+
+ xnlock_get(&nklock);
+
+ /*
+ * We fire the handler before the thread is migrated, so that
+ * thread->sched does not change between paired invocations of
+ * relax_thread/harden_thread handlers.
+ */
+ xnthread_run_handler_stack(thread, harden_thread);
+ if (affinity_ok(p))
+ xnthread_resume(thread, XNRELAX);
+
+#ifdef IPIPE_KEVT_USERINTRET
+ /*
+ * In case we migrated independently of the user return notifier, clear
+ * XNCONTHI here and also disable the notifier - we are already done.
+ */
+ if (unlikely(xnthread_test_info(thread, XNCONTHI))) {
+ xnthread_clear_info(thread, XNCONTHI);
+ ipipe_disable_user_intret_notifier();
+ }
+#endif
+
+ /* Unregister as debugged thread in case we postponed this. */
+ if (unlikely(xnthread_test_state(thread, XNSSTEP)))
+ unregister_debugged_thread(thread);
+
+ xnlock_put(&nklock);
+
+ xnsched_run();
+}
+
+#ifdef CONFIG_XENO_OPT_HOSTRT
+
+static IPIPE_DEFINE_SPINLOCK(__hostrtlock);
+
+static int handle_hostrt_event(struct ipipe_hostrt_data *hostrt)
+{
+ unsigned long flags;
+ urwstate_t tmp;
+
+ /*
+ * The locking strategy is twofold:
+ * - The spinlock protects against concurrent updates from within the
+ * Linux kernel and against preemption by Xenomai
+ * - The unsynced R/W block is for lockless read-only access.
+ */
+ raw_spin_lock_irqsave(&__hostrtlock, flags);
+
+ unsynced_write_block(&tmp, &nkvdso->hostrt_data.lock) {
+ nkvdso->hostrt_data.live = 1;
+ nkvdso->hostrt_data.cycle_last = hostrt->cycle_last;
+ nkvdso->hostrt_data.mask = hostrt->mask;
+ nkvdso->hostrt_data.mult = hostrt->mult;
+ nkvdso->hostrt_data.shift = hostrt->shift;
+ nkvdso->hostrt_data.wall_sec = hostrt->wall_time_sec;
+ nkvdso->hostrt_data.wall_nsec = hostrt->wall_time_nsec;
+ nkvdso->hostrt_data.wtom_sec = hostrt->wall_to_monotonic.tv_sec;
+ nkvdso->hostrt_data.wtom_nsec = hostrt->wall_to_monotonic.tv_nsec;
+ }
+
+ raw_spin_unlock_irqrestore(&__hostrtlock, flags);
+
+ return KEVENT_PROPAGATE;
+}
+
+static inline void init_hostrt(void)
+{
+ unsynced_rw_init(&nkvdso->hostrt_data.lock);
+ nkvdso->hostrt_data.live = 0;
+}
+
+#else /* !CONFIG_XENO_OPT_HOSTRT */
+
+struct ipipe_hostrt_data;
+
+static inline int handle_hostrt_event(struct ipipe_hostrt_data *hostrt)
+{
+ return KEVENT_PROPAGATE;
+}
+
+static inline void init_hostrt(void) { }
+
+#endif /* !CONFIG_XENO_OPT_HOSTRT */
+
+static void __handle_taskexit_event(struct task_struct *p)
+{
+ struct cobalt_ppd *sys_ppd;
+ struct xnthread *thread;
+ spl_t s;
+
+ /*
+ * We are called for both kernel and user shadows over the
+ * root thread.
+ */
+ secondary_mode_only();
+
+ thread = xnthread_current();
+ XENO_BUG_ON(COBALT, thread == NULL);
+ trace_cobalt_shadow_unmap(thread);
+
+ xnlock_get_irqsave(&nklock, s);
+
+ if (xnthread_test_state(thread, XNSSTEP))
+ unregister_debugged_thread(thread);
+
+ xnsched_run();
+
+ xnlock_put_irqrestore(&nklock, s);
+
+ xnthread_run_handler_stack(thread, exit_thread);
+
+ if (xnthread_test_state(thread, XNUSER)) {
+ cobalt_umm_free(&cobalt_kernel_ppd.umm, thread->u_window);
+ thread->u_window = NULL;
+ sys_ppd = cobalt_ppd_get(0);
+ if (atomic_dec_and_test(&sys_ppd->refcnt))
+ cobalt_remove_process(cobalt_current_process());
+ }
+}
+
+static int handle_taskexit_event(struct task_struct *p) /* p == current */
+{
+ __handle_taskexit_event(p);
+
+ /*
+ * __xnthread_cleanup() -> ... -> finalize_thread
+ * handler. From that point, the TCB is dropped. Be careful of
+ * not treading on stale memory within @thread.
+ */
+ __xnthread_cleanup(xnthread_current());
+
+ detach_current();
+
+ return KEVENT_PROPAGATE;
+}
+
+static int handle_schedule_event(struct task_struct *next_task)
+{
+ struct task_struct *prev_task;
+ struct xnthread *next;
+ sigset_t pending;
+ spl_t s;
+
+ cobalt_signal_yield();
+
+ prev_task = current;
+ next = xnthread_from_task(next_task);
+ if (next == NULL)
+ goto out;
+
+ xnlock_get_irqsave(&nklock, s);
+
+ /*
+ * Track tasks leaving the ptraced state. Check both SIGSTOP
+ * (NPTL) and SIGINT (LinuxThreads) to detect ptrace
+ * continuation.
+ */
+ if (xnthread_test_state(next, XNSSTEP)) {
+ if (signal_pending(next_task)) {
+ /*
+ * Do not grab the sighand lock here: it's
+ * useless, and we already own the runqueue
+ * lock, so this would expose us to deadlock
+ * situations on SMP.
+ */
+ sigorsets(&pending,
+ &next_task->pending.signal,
+ &next_task->signal->shared_pending.signal);
+ if (sigismember(&pending, SIGSTOP) ||
+ sigismember(&pending, SIGINT))
+ goto no_ptrace;
+ }
+
+ /*
+ * Do not unregister before the thread migrated.
+ * unregister_debugged_thread will then be called by our
+ * ipipe_migration_hook.
+ */
+ if (!xnthread_test_info(next, XNCONTHI))
+ unregister_debugged_thread(next);
+
+ xnthread_set_localinfo(next, XNHICCUP);
+ }
+
+no_ptrace:
+ xnlock_put_irqrestore(&nklock, s);
+
+ /*
+ * Do basic sanity checks on the incoming thread state.
+ * NOTE: we allow ptraced threads to run shortly in order to
+ * properly recover from a stopped state.
+ */
+ if (!XENO_WARN(COBALT, !xnthread_test_state(next, XNRELAX),
+ "hardened thread %s[%d] running in Linux domain?! "
+ "(status=0x%x, sig=%d, prev=%s[%d])",
+ next->name, task_pid_nr(next_task),
+ xnthread_get_state(next),
+ signal_pending(next_task),
+ prev_task->comm, task_pid_nr(prev_task)))
+ XENO_WARN(COBALT,
+ !(next_task->ptrace & PT_PTRACED) &&
+ !xnthread_test_state(next, XNDORMANT)
+ && xnthread_test_state(next, XNPEND),
+ "blocked thread %s[%d] rescheduled?! "
+ "(status=0x%x, sig=%d, prev=%s[%d])",
+ next->name, task_pid_nr(next_task),
+ xnthread_get_state(next),
+ signal_pending(next_task), prev_task->comm,
+ task_pid_nr(prev_task));
+out:
+ return KEVENT_PROPAGATE;
+}
+
+static int handle_sigwake_event(struct task_struct *p)
+{
+ struct xnthread *thread;
+ sigset_t pending;
+ spl_t s;
+
+ thread = xnthread_from_task(p);
+ if (thread == NULL)
+ return KEVENT_PROPAGATE;
+
+ xnlock_get_irqsave(&nklock, s);
+
+ /*
+ * CAUTION: __TASK_TRACED is not set in p->state yet. This
+ * state bit will be set right after we return, when the task
+ * is woken up.
+ */
+ if ((p->ptrace & PT_PTRACED) && !xnthread_test_state(thread, XNSSTEP)) {
+ /* We already own the siglock. */
+ sigorsets(&pending,
+ &p->pending.signal,
+ &p->signal->shared_pending.signal);
+
+ if (sigismember(&pending, SIGTRAP) ||
+ sigismember(&pending, SIGSTOP)
+ || sigismember(&pending, SIGINT))
+ register_debugged_thread(thread);
+ }
+
+ if (xnthread_test_state(thread, XNRELAX))
+ goto out;
+
+ /*
+ * If kicking a shadow thread in primary mode, make sure Linux
+ * won't schedule in its mate under our feet as a result of
+ * running signal_wake_up(). The Xenomai scheduler must remain
+ * in control for now, until we explicitly relax the shadow
+ * thread to allow for processing the pending signals. Make
+ * sure we keep the additional state flags unmodified so that
+ * we don't break any undergoing ptrace.
+ */
+ if (p->state & (TASK_INTERRUPTIBLE|TASK_UNINTERRUPTIBLE))
+ cobalt_set_task_state(p, p->state | TASK_NOWAKEUP);
+
+ /*
+ * Allow a thread stopped for debugging to resume briefly in order to
+ * migrate to secondary mode. xnthread_relax will reapply XNDBGSTOP.
+ */
+ if (xnthread_test_state(thread, XNDBGSTOP))
+ xnthread_resume(thread, XNDBGSTOP);
+
+ __xnthread_kick(thread);
+out:
+ xnsched_run();
+
+ xnlock_put_irqrestore(&nklock, s);
+
+ return KEVENT_PROPAGATE;
+}
+
+static int handle_cleanup_event(struct mm_struct *mm)
+{
+ struct cobalt_process *old, *process;
+ struct cobalt_ppd *sys_ppd;
+ struct xnthread *curr;
+
+ /*
+ * We are NOT called for exiting kernel shadows.
+ * cobalt_current_process() is cleared if we get there after
+ * handle_task_exit(), so we need to restore this context
+ * pointer temporarily.
+ */
+ process = cobalt_search_process(mm);
+ old = cobalt_set_process(process);
+ sys_ppd = cobalt_ppd_get(0);
+ if (sys_ppd != &cobalt_kernel_ppd) {
+ bool running_exec;
+
+ /*
+ * Detect a userland shadow running exec(), i.e. still
+ * attached to the current linux task (no prior
+ * detach_current). In this case, we emulate a task
+ * exit, since the Xenomai binding shall not survive
+ * the exec() syscall. Since the process will keep on
+ * running though, we have to disable the event
+ * notifier manually for it.
+ */
+ curr = xnthread_current();
+ running_exec = curr && (current->flags & PF_EXITING) == 0;
+ if (running_exec) {
+ __handle_taskexit_event(current);
+ ipipe_disable_notifier(current);
+ }
+ if (atomic_dec_and_test(&sys_ppd->refcnt))
+ cobalt_remove_process(process);
+ if (running_exec) {
+ __xnthread_cleanup(curr);
+ detach_current();
+ }
+ }
+
+ /*
+ * CAUTION: Do not override a state change caused by
+ * cobalt_remove_process().
+ */
+ if (cobalt_current_process() == process)
+ cobalt_set_process(old);
+
+ return KEVENT_PROPAGATE;
+}
+
+static inline int handle_clockfreq_event(unsigned int *p)
+{
+ unsigned int newfreq = *p;
+
+ xnclock_update_freq(newfreq);
+
+ return KEVENT_PROPAGATE;
+}
+
+#ifdef IPIPE_KEVT_USERINTRET
+static int handle_user_return(struct task_struct *task)
+{
+ struct xnthread *thread;
+ spl_t s;
+ int err;
+
+ ipipe_disable_user_intret_notifier();
+
+ thread = xnthread_from_task(task);
+ if (thread == NULL)
+ return KEVENT_PROPAGATE;
+
+ if (xnthread_test_info(thread, XNCONTHI)) {
+ xnlock_get_irqsave(&nklock, s);
+ xnthread_clear_info(thread, XNCONTHI);
+ xnlock_put_irqrestore(&nklock, s);
+
+ err = xnthread_harden();
+
+ /*
+ * XNCONTHI may or may not have been re-applied if
+ * harden bailed out due to pending signals. Make sure
+ * it is set in that case.
+ */
+ if (err == -ERESTARTSYS) {
+ xnlock_get_irqsave(&nklock, s);
+ xnthread_set_info(thread, XNCONTHI);
+ xnlock_put_irqrestore(&nklock, s);
+ }
+ }
+
+ return KEVENT_PROPAGATE;
+}
+#endif /* IPIPE_KEVT_USERINTRET */
+
+#ifdef IPIPE_KEVT_PTRESUME
+int handle_ptrace_resume(struct ipipe_ptrace_resume_data *resume)
+{
+ struct xnthread *thread;
+ spl_t s;
+
+ thread = xnthread_from_task(resume->task);
+ if (thread == NULL)
+ return KEVENT_PROPAGATE;
+
+ if (resume->request == PTRACE_SINGLESTEP &&
+ xnthread_test_state(thread, XNSSTEP)) {
+ xnlock_get_irqsave(&nklock, s);
+
+ xnthread_resume(thread, XNDBGSTOP);
+ unregister_debugged_thread(thread);
+
+ xnlock_put_irqrestore(&nklock, s);
+ }
+
+ return KEVENT_PROPAGATE;
+}
+#endif /* IPIPE_KEVT_PTRESUME */
+
+int ipipe_kevent_hook(int kevent, void *data)
+{
+ int ret;
+
+ switch (kevent) {
+ case IPIPE_KEVT_SCHEDULE:
+ ret = handle_schedule_event(data);
+ break;
+ case IPIPE_KEVT_SIGWAKE:
+ ret = handle_sigwake_event(data);
+ break;
+ case IPIPE_KEVT_EXIT:
+ ret = handle_taskexit_event(data);
+ break;
+ case IPIPE_KEVT_CLEANUP:
+ ret = handle_cleanup_event(data);
+ break;
+ case IPIPE_KEVT_HOSTRT:
+ ret = handle_hostrt_event(data);
+ break;
+ case IPIPE_KEVT_SETAFFINITY:
+ ret = handle_setaffinity_event(data);
+ break;
+#ifdef IPIPE_KEVT_CLOCKFREQ
+ case IPIPE_KEVT_CLOCKFREQ:
+ ret = handle_clockfreq_event(data);
+ break;
+#endif
+#ifdef IPIPE_KEVT_USERINTRET
+ case IPIPE_KEVT_USERINTRET:
+ ret = handle_user_return(data);
+ break;
+#endif
+#ifdef IPIPE_KEVT_PTRESUME
+ case IPIPE_KEVT_PTRESUME:
+ ret = handle_ptrace_resume(data);
+ break;
+#endif
+ default:
+ ret = KEVENT_PROPAGATE;
+ }
+
+ return ret;
+}
+
+#ifdef CONFIG_MMU
+
+int pipeline_prepare_current(void)
+{
+ struct task_struct *p = current;
+ kernel_siginfo_t si;
+
+ if ((p->mm->def_flags & VM_LOCKED) == 0) {
+ memset(&si, 0, sizeof(si));
+ si.si_signo = SIGDEBUG;
+ si.si_code = SI_QUEUE;
+ si.si_int = SIGDEBUG_NOMLOCK | sigdebug_marker;
+ send_sig_info(SIGDEBUG, &si, p);
+ return 0;
+ }
+
+ return __ipipe_disable_ondemand_mappings(p);
+}
+
+static inline int get_mayday_prot(void)
+{
+ return PROT_READ|PROT_EXEC;
+}
+
+#else /* !CONFIG_MMU */
+
+int pipeline_prepare_current(void)
+{
+ return 0;
+}
+
+static inline int get_mayday_prot(void)
+{
+ /*
+ * Until we stop backing /dev/mem with the mayday page, we
+ * can't ask for PROT_EXEC since the former does not define
+ * mmap capabilities, and default ones won't allow an
+ * executable mapping with MAP_SHARED. In the NOMMU case, this
+ * is (currently) not an issue.
+ */
+ return PROT_READ;
+}
+
+#endif /* !CONFIG_MMU */
+
+void pipeline_attach_current(struct xnthread *thread)
+{
+ struct ipipe_threadinfo *p;
+
+ p = ipipe_current_threadinfo();
+ p->thread = thread;
+ p->process = cobalt_search_process(current->mm);
+}
+
+static void detach_current(void)
+{
+ struct ipipe_threadinfo *p = ipipe_current_threadinfo();
+ p->thread = NULL;
+ p->process = NULL;
+}
+
+int pipeline_trap_kevents(void)
+{
+ init_hostrt();
+ ipipe_set_hooks(ipipe_root_domain, IPIPE_SYSCALL|IPIPE_KEVENT);
+ ipipe_set_hooks(&xnsched_realtime_domain, IPIPE_SYSCALL|IPIPE_TRAP);
+
+ return 0;
+}
+
+void pipeline_enable_kevents(void)
+{
+ ipipe_enable_notifier(current);
+}
diff --git a/kernel/cobalt/posix/process.c b/kernel/cobalt/posix/process.c
index 8351d28fb..9bc6082d0 100644
--- a/kernel/cobalt/posix/process.c
+++ b/kernel/cobalt/posix/process.c
@@ -32,12 +32,10 @@
#include <linux/slab.h>
#include <linux/cred.h>
#include <linux/file.h>
-#include <linux/ptrace.h>
#include <linux/sched.h>
#include <linux/signal.h>
#include <linux/kallsyms.h>
-#include <linux/ipipe.h>
-#include <linux/ipipe_tickdev.h>
+#include <pipeline/kevents.h>
#include <cobalt/kernel/sched.h>
#include <cobalt/kernel/heap.h>
#include <cobalt/kernel/synch.h>
@@ -94,12 +92,6 @@ struct cobalt_resources cobalt_global_resources = {
.schedq = LIST_HEAD_INIT(cobalt_global_resources.schedq),
};
-static inline struct cobalt_process *
-process_from_thread(struct xnthread *thread)
-{
- return container_of(thread, struct cobalt_thread, threadbase)->process;
-}
-
static unsigned __attribute__((pure)) process_hash_crunch(struct mm_struct *mm)
{
unsigned long hash = ((unsigned long)mm - PAGE_OFFSET) / sizeof(*mm);
@@ -185,7 +177,7 @@ static void *lookup_context(int xid)
return priv;
}
-static void remove_process(struct cobalt_process *process)
+void cobalt_remove_process(struct cobalt_process *process)
{
struct xnthread_personality *personality;
void *priv;
@@ -567,67 +559,6 @@ int cobalt_yield(xnticks_t min, xnticks_t max)
}
EXPORT_SYMBOL_GPL(cobalt_yield);
-static inline void init_uthread_info(struct xnthread *thread)
-{
- struct ipipe_threadinfo *p;
-
- p = ipipe_current_threadinfo();
- p->thread = thread;
- p->process = cobalt_search_process(current->mm);
-}
-
-static inline void clear_threadinfo(void)
-{
- struct ipipe_threadinfo *p = ipipe_current_threadinfo();
- p->thread = NULL;
- p->process = NULL;
-}
-
-#ifdef CONFIG_MMU
-
-static inline int disable_ondemand_memory(void)
-{
- struct task_struct *p = current;
- kernel_siginfo_t si;
-
- if ((p->mm->def_flags & VM_LOCKED) == 0) {
- memset(&si, 0, sizeof(si));
- si.si_signo = SIGDEBUG;
- si.si_code = SI_QUEUE;
- si.si_int = SIGDEBUG_NOMLOCK | sigdebug_marker;
- send_sig_info(SIGDEBUG, &si, p);
- return 0;
- }
-
- return __ipipe_disable_ondemand_mappings(p);
-}
-
-static inline int get_mayday_prot(void)
-{
- return PROT_READ|PROT_EXEC;
-}
-
-#else /* !CONFIG_MMU */
-
-static inline int disable_ondemand_memory(void)
-{
- return 0;
-}
-
-static inline int get_mayday_prot(void)
-{
- /*
- * Until we stop backing /dev/mem with the mayday page, we
- * can't ask for PROT_EXEC since the former does not define
- * mmap capabilities, and default ones won't allow an
- * executable mapping with MAP_SHARED. In the NOMMU case, this
- * is (currently) not an issue.
- */
- return PROT_READ;
-}
-
-#endif /* !CONFIG_MMU */
-
/**
* @fn int cobalt_map_user(struct xnthread *thread, __u32 __user *u_winoff)
* @internal
@@ -675,7 +606,7 @@ int cobalt_map_user(struct xnthread *thread, __u32 __user *u_winoff)
if (!access_wok(u_winoff, sizeof(*u_winoff)))
return -EFAULT;
- ret = disable_ondemand_memory();
+ ret = pipeline_prepare_current();
if (ret)
return ret;
@@ -696,7 +627,7 @@ int cobalt_map_user(struct xnthread *thread, __u32 __user *u_winoff)
*/
xnthread_init_shadow_tcb(thread);
xnthread_suspend(thread, XNRELAX, XN_INFINITE, XN_RELATIVE, NULL);
- init_uthread_info(thread);
+ pipeline_attach_current(thread);
xnthread_set_state(thread, XNMAPPED);
xndebug_shadow_init(thread);
sys_ppd = cobalt_ppd_get(0);
@@ -709,7 +640,7 @@ int cobalt_map_user(struct xnthread *thread, __u32 __user *u_winoff)
* it.
*/
xnthread_run_handler(thread, map_thread);
- ipipe_enable_notifier(current);
+ pipeline_enable_kevents(current);
attr.mode = 0;
attr.entry = NULL;
@@ -726,456 +657,7 @@ int cobalt_map_user(struct xnthread *thread, __u32 __user *u_winoff)
return 0;
}
-#ifdef IPIPE_KEVT_PTRESUME
-static void stop_debugged_process(struct xnthread *thread)
-{
- struct cobalt_process *process = process_from_thread(thread);
- struct cobalt_thread *cth;
-
- if (process->debugged_threads > 0)
- return;
-
- list_for_each_entry(cth, &process->thread_list, next) {
- if (&cth->threadbase == thread)
- continue;
-
- xnthread_suspend(&cth->threadbase, XNDBGSTOP, XN_INFINITE,
- XN_RELATIVE, NULL);
- }
-}
-
-static void resume_debugged_process(struct cobalt_process *process)
-{
- struct cobalt_thread *cth;
-
- xnsched_lock();
-
- list_for_each_entry(cth, &process->thread_list, next)
- if (xnthread_test_state(&cth->threadbase, XNDBGSTOP))
- xnthread_resume(&cth->threadbase, XNDBGSTOP);
-
- xnsched_unlock();
-}
-
-#else /* IPIPE_KEVT_PTRESUME unavailable */
-
-static inline void stop_debugged_process(struct xnthread *thread)
-{
-}
-
-static inline void resume_debugged_process(struct cobalt_process *process)
-{
-}
-#endif /* IPIPE_KEVT_PTRESUME unavailable */
-
-/* called with nklock held */
-static void cobalt_register_debugged_thread(struct xnthread *thread)
-{
- struct cobalt_process *process = process_from_thread(thread);
-
- xnthread_set_state(thread, XNSSTEP);
-
- stop_debugged_process(thread);
- process->debugged_threads++;
-
- if (xnthread_test_state(thread, XNRELAX))
- xnthread_suspend(thread, XNDBGSTOP, XN_INFINITE, XN_RELATIVE,
- NULL);
-}
-
-/* called with nklock held */
-static void cobalt_unregister_debugged_thread(struct xnthread *thread)
-{
- struct cobalt_process *process = process_from_thread(thread);
-
- process->debugged_threads--;
- xnthread_clear_state(thread, XNSSTEP);
-
- if (process->debugged_threads == 0)
- resume_debugged_process(process);
-}
-
-static inline int handle_exception(struct ipipe_trap_data *d)
-{
- struct xnthread *thread;
- struct xnsched *sched;
-
- sched = xnsched_current();
- thread = sched->curr;
-
- trace_cobalt_thread_fault(d);
-
- if (xnthread_test_state(thread, XNROOT))
- return 0;
-
-#ifdef IPIPE_KEVT_USERINTRET
- if (xnarch_fault_bp_p(d) && user_mode(d->regs)) {
- spl_t s;
-
- XENO_WARN_ON(CORE, xnthread_test_state(thread, XNRELAX));
- xnlock_get_irqsave(&nklock, s);
- xnthread_set_info(thread, XNCONTHI);
- ipipe_enable_user_intret_notifier();
- stop_debugged_process(thread);
- xnlock_put_irqrestore(&nklock, s);
- xnsched_run();
- }
-#endif
-
- if (xnarch_fault_fpu_p(d)) {
-#ifdef CONFIG_XENO_ARCH_FPU
- spl_t s;
-
- /* FPU exception received in primary mode. */
- splhigh(s);
- if (xnarch_handle_fpu_fault(sched->fpuholder, thread, d)) {
- sched->fpuholder = thread;
- splexit(s);
- return 1;
- }
- splexit(s);
-#endif /* CONFIG_XENO_ARCH_FPU */
-#if LINUX_VERSION_CODE >= KERNEL_VERSION(4, 16, 0)
- printk("invalid use of FPU in Xenomai context at %pS\n",
- (void *)xnarch_fault_pc(d));
-#else
- print_symbol("invalid use of FPU in Xenomai context at %s\n",
- xnarch_fault_pc(d));
-#endif
- }
-
- /*
- * If we experienced a trap on behalf of a shadow thread
- * running in primary mode, move it to the Linux domain,
- * leaving the kernel process the exception.
- */
-#if defined(CONFIG_XENO_OPT_DEBUG_COBALT) || defined(CONFIG_XENO_OPT_DEBUG_USER)
- if (!user_mode(d->regs)) {
- xntrace_panic_freeze();
- printk(XENO_WARNING
- "switching %s to secondary mode after exception #%u in "
- "kernel-space at 0x%lx (pid %d)\n", thread->name,
- xnarch_fault_trap(d),
- xnarch_fault_pc(d),
- xnthread_host_pid(thread));
- xntrace_panic_dump();
- } else if (xnarch_fault_notify(d)) /* Don't report debug traps */
- printk(XENO_WARNING
- "switching %s to secondary mode after exception #%u from "
- "user-space at 0x%lx (pid %d)\n", thread->name,
- xnarch_fault_trap(d),
- xnarch_fault_pc(d),
- xnthread_host_pid(thread));
-#endif
-
- if (xnarch_fault_pf_p(d))
- /*
- * The page fault counter is not SMP-safe, but it's a
- * simple indicator that something went wrong wrt
- * memory locking anyway.
- */
- xnstat_counter_inc(&thread->stat.pf);
-
- xnthread_relax(xnarch_fault_notify(d), SIGDEBUG_MIGRATE_FAULT);
-
- return 0;
-}
-
-static int handle_mayday_event(struct pt_regs *regs)
-{
- XENO_BUG_ON(COBALT, !xnthread_test_state(xnthread_current(), XNUSER));
-
- xnthread_relax(0, 0);
-
- return KEVENT_PROPAGATE;
-}
-
-int ipipe_trap_hook(struct ipipe_trap_data *data)
-{
- if (data->exception == IPIPE_TRAP_MAYDAY)
- return handle_mayday_event(data->regs);
-
- /*
- * No migration is possible on behalf of the head domain, so
- * the following access is safe.
- */
- raw_cpu_ptr(&cobalt_machine_cpudata)->faults[data->exception]++;
-
- if (handle_exception(data))
- return KEVENT_STOP;
-
- /*
- * CAUTION: access faults must be propagated downstream
- * whichever domain caused them, so that we don't spuriously
- * raise a fatal error when some Linux fixup code is available
- * to recover from the fault.
- */
- return KEVENT_PROPAGATE;
-}
-
-/*
- * Legacy idle hook, unconditionally allow entering the idle state.
- */
-bool ipipe_enter_idle_hook(void)
-{
- return true;
-}
-
-#ifdef CONFIG_SMP
-
-static int handle_setaffinity_event(struct ipipe_cpu_migration_data *d)
-{
- struct task_struct *p = d->task;
- struct xnthread *thread;
- spl_t s;
-
- thread = xnthread_from_task(p);
- if (thread == NULL)
- return KEVENT_PROPAGATE;
-
- /*
- * Detect a Cobalt thread sleeping in primary mode which is
- * required to migrate to another CPU by the host kernel.
- *
- * We may NOT fix up thread->sched immediately using the
- * passive migration call, because that latter always has to
- * take place on behalf of the target thread itself while
- * running in secondary mode. Therefore, that thread needs to
- * go through secondary mode first, then move back to primary
- * mode, so that affinity_ok() does the fixup work.
- *
- * We force this by sending a SIGSHADOW signal to the migrated
- * thread, asking it to switch back to primary mode from the
- * handler, at which point the interrupted syscall may be
- * restarted.
- */
- xnlock_get_irqsave(&nklock, s);
-
- if (xnthread_test_state(thread, XNTHREAD_BLOCK_BITS & ~XNRELAX))
- xnthread_signal(thread, SIGSHADOW, SIGSHADOW_ACTION_HARDEN);
-
- xnlock_put_irqrestore(&nklock, s);
-
- return KEVENT_PROPAGATE;
-}
-
-static inline bool affinity_ok(struct task_struct *p) /* nklocked, IRQs off */
-{
- struct xnthread *thread = xnthread_from_task(p);
- struct xnsched *sched;
- int cpu = task_cpu(p);
-
- /*
- * To maintain consistency between both Cobalt and host
- * schedulers, reflecting a thread migration to another CPU
- * into the Cobalt scheduler state must happen from secondary
- * mode only, on behalf of the migrated thread itself once it
- * runs on the target CPU.
- *
- * This means that the Cobalt scheduler state regarding the
- * CPU information lags behind the host scheduler state until
- * the migrated thread switches back to primary mode
- * (i.e. task_cpu(p) != xnsched_cpu(xnthread_from_task(p)->sched)).
- * This is ok since Cobalt does not schedule such thread until then.
- *
- * check_affinity() detects when a Cobalt thread switching
- * back to primary mode did move to another CPU earlier while
- * in secondary mode. If so, do the fixups to reflect the
- * change.
- */
- if (!xnsched_threading_cpu(cpu)) {
- /*
- * The thread is about to switch to primary mode on a
- * non-rt CPU, which is damn wrong and hopeless.
- * Whine and cancel that thread.
- */
- printk(XENO_WARNING "thread %s[%d] switched to non-rt CPU%d, aborted.\n",
- thread->name, xnthread_host_pid(thread), cpu);
- /*
- * Can't call xnthread_cancel() from a migration
- * point, that would break. Since we are on the wakeup
- * path to hardening, just raise XNCANCELD to catch it
- * in xnthread_harden().
- */
- xnthread_set_info(thread, XNCANCELD);
- return false;
- }
-
- sched = xnsched_struct(cpu);
- if (sched == thread->sched)
- return true;
-
- /*
- * The current thread moved to a supported real-time CPU,
- * which is not part of its original affinity mask
- * though. Assume user wants to extend this mask.
- */
- if (!cpumask_test_cpu(cpu, &thread->affinity))
- cpumask_set_cpu(cpu, &thread->affinity);
-
- xnthread_run_handler_stack(thread, move_thread, cpu);
- xnthread_migrate_passive(thread, sched);
-
- return true;
-}
-
-#else /* !CONFIG_SMP */
-
-struct ipipe_cpu_migration_data;
-
-static int handle_setaffinity_event(struct ipipe_cpu_migration_data *d)
-{
- return KEVENT_PROPAGATE;
-}
-
-static inline bool affinity_ok(struct task_struct *p)
-{
- return true;
-}
-
-#endif /* CONFIG_SMP */
-
-void ipipe_migration_hook(struct task_struct *p) /* hw IRQs off */
-{
- struct xnthread *thread = xnthread_from_task(p);
-
- xnlock_get(&nklock);
-
- /*
- * We fire the handler before the thread is migrated, so that
- * thread->sched does not change between paired invocations of
- * relax_thread/harden_thread handlers.
- */
- xnthread_run_handler_stack(thread, harden_thread);
- if (affinity_ok(p))
- xnthread_resume(thread, XNRELAX);
-
-#ifdef IPIPE_KEVT_USERINTRET
- /*
- * In case we migrated independently of the user return notifier, clear
- * XNCONTHI here and also disable the notifier - we are already done.
- */
- if (unlikely(xnthread_test_info(thread, XNCONTHI))) {
- xnthread_clear_info(thread, XNCONTHI);
- ipipe_disable_user_intret_notifier();
- }
-#endif
-
- /* Unregister as debugged thread in case we postponed this. */
- if (unlikely(xnthread_test_state(thread, XNSSTEP)))
- cobalt_unregister_debugged_thread(thread);
-
- xnlock_put(&nklock);
-
- xnsched_run();
-}
-
-#ifdef CONFIG_XENO_OPT_HOSTRT
-
-static IPIPE_DEFINE_SPINLOCK(__hostrtlock);
-
-static int handle_hostrt_event(struct ipipe_hostrt_data *hostrt)
-{
- unsigned long flags;
- urwstate_t tmp;
-
- /*
- * The locking strategy is twofold:
- * - The spinlock protects against concurrent updates from within the
- * Linux kernel and against preemption by Xenomai
- * - The unsynced R/W block is for lockless read-only access.
- */
- raw_spin_lock_irqsave(&__hostrtlock, flags);
-
- unsynced_write_block(&tmp, &nkvdso->hostrt_data.lock) {
- nkvdso->hostrt_data.live = 1;
- nkvdso->hostrt_data.cycle_last = hostrt->cycle_last;
- nkvdso->hostrt_data.mask = hostrt->mask;
- nkvdso->hostrt_data.mult = hostrt->mult;
- nkvdso->hostrt_data.shift = hostrt->shift;
- nkvdso->hostrt_data.wall_sec = hostrt->wall_time_sec;
- nkvdso->hostrt_data.wall_nsec = hostrt->wall_time_nsec;
- nkvdso->hostrt_data.wtom_sec = hostrt->wall_to_monotonic.tv_sec;
- nkvdso->hostrt_data.wtom_nsec = hostrt->wall_to_monotonic.tv_nsec;
- }
-
- raw_spin_unlock_irqrestore(&__hostrtlock, flags);
-
- return KEVENT_PROPAGATE;
-}
-
-static inline void init_hostrt(void)
-{
- unsynced_rw_init(&nkvdso->hostrt_data.lock);
- nkvdso->hostrt_data.live = 0;
-}
-
-#else /* !CONFIG_XENO_OPT_HOSTRT */
-
-struct ipipe_hostrt_data;
-
-static inline int handle_hostrt_event(struct ipipe_hostrt_data *hostrt)
-{
- return KEVENT_PROPAGATE;
-}
-
-static inline void init_hostrt(void) { }
-
-#endif /* !CONFIG_XENO_OPT_HOSTRT */
-
-static void __handle_taskexit_event(struct task_struct *p)
-{
- struct cobalt_ppd *sys_ppd;
- struct xnthread *thread;
- spl_t s;
-
- /*
- * We are called for both kernel and user shadows over the
- * root thread.
- */
- secondary_mode_only();
-
- thread = xnthread_current();
- XENO_BUG_ON(COBALT, thread == NULL);
- trace_cobalt_shadow_unmap(thread);
-
- xnlock_get_irqsave(&nklock, s);
-
- if (xnthread_test_state(thread, XNSSTEP))
- cobalt_unregister_debugged_thread(thread);
-
- xnsched_run();
-
- xnlock_put_irqrestore(&nklock, s);
-
- xnthread_run_handler_stack(thread, exit_thread);
-
- if (xnthread_test_state(thread, XNUSER)) {
- cobalt_umm_free(&cobalt_kernel_ppd.umm, thread->u_window);
- thread->u_window = NULL;
- sys_ppd = cobalt_ppd_get(0);
- if (atomic_dec_and_test(&sys_ppd->refcnt))
- remove_process(cobalt_current_process());
- }
-}
-
-static int handle_taskexit_event(struct task_struct *p) /* p == current */
-{
- __handle_taskexit_event(p);
-
- /*
- * __xnthread_cleanup() -> ... -> finalize_thread
- * handler. From that point, the TCB is dropped. Be careful of
- * not treading on stale memory within @thread.
- */
- __xnthread_cleanup(xnthread_current());
-
- clear_threadinfo();
-
- return KEVENT_PROPAGATE;
-}
-
-static inline void signal_yield(void)
+void cobalt_signal_yield(void)
{
spl_t s;
@@ -1190,308 +672,6 @@ static inline void signal_yield(void)
xnlock_put_irqrestore(&nklock, s);
}
-static int handle_schedule_event(struct task_struct *next_task)
-{
- struct task_struct *prev_task;
- struct xnthread *next;
- sigset_t pending;
- spl_t s;
-
- signal_yield();
-
- prev_task = current;
- next = xnthread_from_task(next_task);
- if (next == NULL)
- goto out;
-
- xnlock_get_irqsave(&nklock, s);
-
- /*
- * Track tasks leaving the ptraced state. Check both SIGSTOP
- * (NPTL) and SIGINT (LinuxThreads) to detect ptrace
- * continuation.
- */
- if (xnthread_test_state(next, XNSSTEP)) {
- if (signal_pending(next_task)) {
- /*
- * Do not grab the sighand lock here: it's
- * useless, and we already own the runqueue
- * lock, so this would expose us to deadlock
- * situations on SMP.
- */
- sigorsets(&pending,
- &next_task->pending.signal,
- &next_task->signal->shared_pending.signal);
- if (sigismember(&pending, SIGSTOP) ||
- sigismember(&pending, SIGINT))
- goto no_ptrace;
- }
-
- /*
- * Do not unregister before the thread migrated.
- * cobalt_unregister_debugged_thread will then be called by our
- * ipipe_migration_hook.
- */
- if (!xnthread_test_info(next, XNCONTHI))
- cobalt_unregister_debugged_thread(next);
-
- xnthread_set_localinfo(next, XNHICCUP);
- }
-
-no_ptrace:
- xnlock_put_irqrestore(&nklock, s);
-
- /*
- * Do basic sanity checks on the incoming thread state.
- * NOTE: we allow ptraced threads to run shortly in order to
- * properly recover from a stopped state.
- */
- if (!XENO_WARN(COBALT, !xnthread_test_state(next, XNRELAX),
- "hardened thread %s[%d] running in Linux domain?! "
- "(status=0x%x, sig=%d, prev=%s[%d])",
- next->name, task_pid_nr(next_task),
- xnthread_get_state(next),
- signal_pending(next_task),
- prev_task->comm, task_pid_nr(prev_task)))
- XENO_WARN(COBALT,
- !(next_task->ptrace & PT_PTRACED) &&
- !xnthread_test_state(next, XNDORMANT)
- && xnthread_test_state(next, XNPEND),
- "blocked thread %s[%d] rescheduled?! "
- "(status=0x%x, sig=%d, prev=%s[%d])",
- next->name, task_pid_nr(next_task),
- xnthread_get_state(next),
- signal_pending(next_task), prev_task->comm,
- task_pid_nr(prev_task));
-out:
- return KEVENT_PROPAGATE;
-}
-
-static int handle_sigwake_event(struct task_struct *p)
-{
- struct xnthread *thread;
- sigset_t pending;
- spl_t s;
-
- thread = xnthread_from_task(p);
- if (thread == NULL)
- return KEVENT_PROPAGATE;
-
- xnlock_get_irqsave(&nklock, s);
-
- /*
- * CAUTION: __TASK_TRACED is not set in p->state yet. This
- * state bit will be set right after we return, when the task
- * is woken up.
- */
- if ((p->ptrace & PT_PTRACED) && !xnthread_test_state(thread, XNSSTEP)) {
- /* We already own the siglock. */
- sigorsets(&pending,
- &p->pending.signal,
- &p->signal->shared_pending.signal);
-
- if (sigismember(&pending, SIGTRAP) ||
- sigismember(&pending, SIGSTOP)
- || sigismember(&pending, SIGINT))
- cobalt_register_debugged_thread(thread);
- }
-
- if (xnthread_test_state(thread, XNRELAX))
- goto out;
-
- /*
- * If kicking a shadow thread in primary mode, make sure Linux
- * won't schedule in its mate under our feet as a result of
- * running signal_wake_up(). The Xenomai scheduler must remain
- * in control for now, until we explicitly relax the shadow
- * thread to allow for processing the pending signals. Make
- * sure we keep the additional state flags unmodified so that
- * we don't break any undergoing ptrace.
- */
- if (p->state & (TASK_INTERRUPTIBLE|TASK_UNINTERRUPTIBLE))
- cobalt_set_task_state(p, p->state | TASK_NOWAKEUP);
-
- /*
- * Allow a thread stopped for debugging to resume briefly in order to
- * migrate to secondary mode. xnthread_relax will reapply XNDBGSTOP.
- */
- if (xnthread_test_state(thread, XNDBGSTOP))
- xnthread_resume(thread, XNDBGSTOP);
-
- __xnthread_kick(thread);
-out:
- xnsched_run();
-
- xnlock_put_irqrestore(&nklock, s);
-
- return KEVENT_PROPAGATE;
-}
-
-static int handle_cleanup_event(struct mm_struct *mm)
-{
- struct cobalt_process *old, *process;
- struct cobalt_ppd *sys_ppd;
- struct xnthread *curr;
-
- /*
- * We are NOT called for exiting kernel shadows.
- * cobalt_current_process() is cleared if we get there after
- * handle_task_exit(), so we need to restore this context
- * pointer temporarily.
- */
- process = cobalt_search_process(mm);
- old = cobalt_set_process(process);
- sys_ppd = cobalt_ppd_get(0);
- if (sys_ppd != &cobalt_kernel_ppd) {
- bool running_exec;
-
- /*
- * Detect a userland shadow running exec(), i.e. still
- * attached to the current linux task (no prior
- * clear_threadinfo). In this case, we emulate a task
- * exit, since the Xenomai binding shall not survive
- * the exec() syscall. Since the process will keep on
- * running though, we have to disable the event
- * notifier manually for it.
- */
- curr = xnthread_current();
- running_exec = curr && (current->flags & PF_EXITING) == 0;
- if (running_exec) {
- __handle_taskexit_event(current);
- ipipe_disable_notifier(current);
- }
- if (atomic_dec_and_test(&sys_ppd->refcnt))
- remove_process(process);
- if (running_exec) {
- __xnthread_cleanup(curr);
- clear_threadinfo();
- }
- }
-
- /*
- * CAUTION: Do not override a state change caused by
- * remove_process().
- */
- if (cobalt_current_process() == process)
- cobalt_set_process(old);
-
- return KEVENT_PROPAGATE;
-}
-
-static inline int handle_clockfreq_event(unsigned int *p)
-{
- unsigned int newfreq = *p;
-
- xnclock_update_freq(newfreq);
-
- return KEVENT_PROPAGATE;
-}
-
-#ifdef IPIPE_KEVT_USERINTRET
-static int handle_user_return(struct task_struct *task)
-{
- struct xnthread *thread;
- spl_t s;
- int err;
-
- ipipe_disable_user_intret_notifier();
-
- thread = xnthread_from_task(task);
- if (thread == NULL)
- return KEVENT_PROPAGATE;
-
- if (xnthread_test_info(thread, XNCONTHI)) {
- xnlock_get_irqsave(&nklock, s);
- xnthread_clear_info(thread, XNCONTHI);
- xnlock_put_irqrestore(&nklock, s);
-
- err = xnthread_harden();
-
- /*
- * XNCONTHI may or may not have been re-applied if
- * harden bailed out due to pending signals. Make sure
- * it is set in that case.
- */
- if (err == -ERESTARTSYS) {
- xnlock_get_irqsave(&nklock, s);
- xnthread_set_info(thread, XNCONTHI);
- xnlock_put_irqrestore(&nklock, s);
- }
- }
-
- return KEVENT_PROPAGATE;
-}
-#endif /* IPIPE_KEVT_USERINTRET */
-
-#ifdef IPIPE_KEVT_PTRESUME
-int handle_ptrace_resume(struct ipipe_ptrace_resume_data *resume)
-{
- struct xnthread *thread;
- spl_t s;
-
- thread = xnthread_from_task(resume->task);
- if (thread == NULL)
- return KEVENT_PROPAGATE;
-
- if (resume->request == PTRACE_SINGLESTEP &&
- xnthread_test_state(thread, XNSSTEP)) {
- xnlock_get_irqsave(&nklock, s);
-
- xnthread_resume(thread, XNDBGSTOP);
- cobalt_unregister_debugged_thread(thread);
-
- xnlock_put_irqrestore(&nklock, s);
- }
-
- return KEVENT_PROPAGATE;
-}
-#endif /* IPIPE_KEVT_PTRESUME */
-
-int ipipe_kevent_hook(int kevent, void *data)
-{
- int ret;
-
- switch (kevent) {
- case IPIPE_KEVT_SCHEDULE:
- ret = handle_schedule_event(data);
- break;
- case IPIPE_KEVT_SIGWAKE:
- ret = handle_sigwake_event(data);
- break;
- case IPIPE_KEVT_EXIT:
- ret = handle_taskexit_event(data);
- break;
- case IPIPE_KEVT_CLEANUP:
- ret = handle_cleanup_event(data);
- break;
- case IPIPE_KEVT_HOSTRT:
- ret = handle_hostrt_event(data);
- break;
- case IPIPE_KEVT_SETAFFINITY:
- ret = handle_setaffinity_event(data);
- break;
-#ifdef IPIPE_KEVT_CLOCKFREQ
- case IPIPE_KEVT_CLOCKFREQ:
- ret = handle_clockfreq_event(data);
- break;
-#endif
-#ifdef IPIPE_KEVT_USERINTRET
- case IPIPE_KEVT_USERINTRET:
- ret = handle_user_return(data);
- break;
-#endif
-#ifdef IPIPE_KEVT_PTRESUME
- case IPIPE_KEVT_PTRESUME:
- ret = handle_ptrace_resume(data);
- break;
-#endif
- default:
- ret = KEVENT_PROPAGATE;
- }
-
- return ret;
-}
-
static int attach_process(struct cobalt_process *process)
{
struct cobalt_ppd *p = &process->sys_ppd;
@@ -1505,6 +685,10 @@ static int attach_process(struct cobalt_process *process)
cobalt_umm_set_name(&p->umm, "private heap[%d]", task_pid_nr(current));
+ ret = pipeline_attach_process(process);
+ if (ret)
+ goto fail_pipeline;
+
exe_path = get_exe_path(current);
if (IS_ERR(exe_path)) {
printk(XENO_WARNING
@@ -1522,8 +706,10 @@ static int attach_process(struct cobalt_process *process)
return 0;
fail_hash:
+ pipeline_detach_process(process);
if (p->exe_path)
kfree(p->exe_path);
+fail_pipeline:
cobalt_umm_destroy(&p->umm);
return ret;
@@ -1688,14 +874,16 @@ __init int cobalt_init(void)
if (ret)
goto fail_siginit;
- init_hostrt();
- ipipe_set_hooks(ipipe_root_domain, IPIPE_SYSCALL|IPIPE_KEVENT);
- ipipe_set_hooks(&xnsched_realtime_domain, IPIPE_SYSCALL|IPIPE_TRAP);
+ ret = pipeline_trap_kevents();
+ if (ret)
+ goto fail_kevents;
if (gid_arg != -1)
printk(XENO_INFO "allowing access to group %d\n", gid_arg);
return 0;
+fail_kevents:
+ cobalt_signal_cleanup();
fail_siginit:
cobalt_unregister_personality(0);
fail_register:
diff --git a/kernel/cobalt/posix/process.h b/kernel/cobalt/posix/process.h
index 375b4cbbb..3a38ae639 100644
--- a/kernel/cobalt/posix/process.h
+++ b/kernel/cobalt/posix/process.h
@@ -149,6 +149,10 @@ void cobalt_del_resource(struct cobalt_resnode *node)
list_del(&node->next);
}
+void cobalt_remove_process(struct cobalt_process *process);
+
+void cobalt_signal_yield(void);
+
extern struct xnthread_personality *cobalt_personalities[];
extern struct xnthread_personality cobalt_personality;
diff --git a/kernel/cobalt/posix/signal.c b/kernel/cobalt/posix/signal.c
index 75514d84f..862a644c0 100644
--- a/kernel/cobalt/posix/signal.c
+++ b/kernel/cobalt/posix/signal.c
@@ -614,3 +614,8 @@ __init int cobalt_signal_init(void)
return 0;
}
+
+__init void cobalt_signal_cleanup(void)
+{
+ xnheap_vfree(sigpending_mem);
+}
diff --git a/kernel/cobalt/posix/signal.h b/kernel/cobalt/posix/signal.h
index e506842e7..7a0b4b22b 100644
--- a/kernel/cobalt/posix/signal.h
+++ b/kernel/cobalt/posix/signal.h
@@ -110,4 +110,6 @@ COBALT_SYSCALL_DECL(sigqueue,
int cobalt_signal_init(void);
+void cobalt_signal_cleanup(void);
+
#endif /* !_COBALT_POSIX_SIGNAL_H */
diff --git a/kernel/cobalt/thread.c b/kernel/cobalt/thread.c
index 49f749f1c..a882bcc45 100644
--- a/kernel/cobalt/thread.c
+++ b/kernel/cobalt/thread.c
@@ -38,6 +38,7 @@
#include <cobalt/kernel/select.h>
#include <cobalt/kernel/lock.h>
#include <cobalt/kernel/thread.h>
+#include <pipeline/kevents.h>
#include <trace/events/cobalt-core.h>
#include "debug.h"
@@ -2502,7 +2503,6 @@ static inline void init_kthread_info(struct xnthread *thread)
*/
int xnthread_map(struct xnthread *thread, struct completion *done)
{
- struct task_struct *p = current;
int ret;
spl_t s;
@@ -2521,7 +2521,7 @@ int xnthread_map(struct xnthread *thread, struct completion *done)
xnthread_set_state(thread, XNMAPPED);
xndebug_shadow_init(thread);
xnthread_run_handler(thread, map_thread);
- ipipe_enable_notifier(p);
+ pipeline_enable_kevents();
/*
* CAUTION: Soon after xnthread_init() has returned,
diff --git a/kernel/cobalt/trace/cobalt-core.h b/kernel/cobalt/trace/cobalt-core.h
index a9e14815b..96eec3664 100644
--- a/kernel/cobalt/trace/cobalt-core.h
+++ b/kernel/cobalt/trace/cobalt-core.h
@@ -377,20 +377,20 @@ TRACE_EVENT(cobalt_thread_resume,
);
TRACE_EVENT(cobalt_thread_fault,
- TP_PROTO(struct ipipe_trap_data *td),
- TP_ARGS(td),
+ TP_PROTO(unsigned long ip, unsigned int type),
+ TP_ARGS(ip, type),
TP_STRUCT__entry(
- __field(void *, ip)
+ __field(unsigned long, ip)
__field(unsigned int, type)
),
TP_fast_assign(
- __entry->ip = (void *)xnarch_fault_pc(td);
- __entry->type = xnarch_fault_trap(td);
+ __entry->ip = ip;
+ __entry->type = type;
),
- TP_printk("ip=%p type=%x",
+ TP_printk("ip=%#lx type=%#x",
__entry->ip, __entry->type)
);
--
2.26.2
More information about the Xenomai
mailing list