mirror of
https://git.kernel.org/pub/scm/linux/kernel/git/stable/linux.git
synced 2025-01-01 10:45:49 +00:00
d3a9f82ec5
Provide nbcon_atomic_flush_pending() to perform flushing of all registered nbcon consoles using their write_atomic() callback. Unlike console_flush_all(), nbcon_atomic_flush_pending() will only flush up through the newest record at the time of the call. This prevents a CPU from printing unbounded when other CPUs are adding records. If new records are added while flushing, it is expected that the dedicated printer threads will print those records. If the printer thread is not available (which is always the case at this point in the rework), nbcon_atomic_flush_pending() _will_ flush all records in the ringbuffer. Unlike console_flush_all(), nbcon_atomic_flush_pending() will fully flush one console before flushing the next. This helps to guarantee that a block of pending records (such as a stack trace in an emergency situation) can be printed atomically at once before releasing console ownership. nbcon_atomic_flush_pending() is safe in any context because it uses write_atomic() and acquires with unsafe_takeover disabled. Co-developed-by: John Ogness <john.ogness@linutronix.de> Signed-off-by: John Ogness <john.ogness@linutronix.de> Signed-off-by: Thomas Gleixner (Intel) <tglx@linutronix.de> Reviewed-by: Petr Mladek <pmladek@suse.com> Link: https://lore.kernel.org/r/20240820063001.36405-21-john.ogness@linutronix.de Signed-off-by: Petr Mladek <pmladek@suse.com>
1249 lines
38 KiB
C
1249 lines
38 KiB
C
// SPDX-License-Identifier: GPL-2.0-only
|
|
// Copyright (C) 2022 Linutronix GmbH, John Ogness
|
|
// Copyright (C) 2022 Intel, Thomas Gleixner
|
|
|
|
#include <linux/atomic.h>
|
|
#include <linux/bug.h>
|
|
#include <linux/console.h>
|
|
#include <linux/delay.h>
|
|
#include <linux/errno.h>
|
|
#include <linux/export.h>
|
|
#include <linux/init.h>
|
|
#include <linux/irqflags.h>
|
|
#include <linux/minmax.h>
|
|
#include <linux/percpu.h>
|
|
#include <linux/preempt.h>
|
|
#include <linux/slab.h>
|
|
#include <linux/smp.h>
|
|
#include <linux/stddef.h>
|
|
#include <linux/string.h>
|
|
#include <linux/types.h>
|
|
#include "internal.h"
|
|
#include "printk_ringbuffer.h"
|
|
/*
|
|
* Printk console printing implementation for consoles which does not depend
|
|
* on the legacy style console_lock mechanism.
|
|
*
|
|
* The state of the console is maintained in the "nbcon_state" atomic
|
|
* variable.
|
|
*
|
|
* The console is locked when:
|
|
*
|
|
* - The 'prio' field contains the priority of the context that owns the
|
|
* console. Only higher priority contexts are allowed to take over the
|
|
* lock. A value of 0 (NBCON_PRIO_NONE) means the console is not locked.
|
|
*
|
|
* - The 'cpu' field denotes on which CPU the console is locked. It is used
|
|
* to prevent busy waiting on the same CPU. Also it informs the lock owner
|
|
* that it has lost the lock in a more complex scenario when the lock was
|
|
* taken over by a higher priority context, released, and taken on another
|
|
* CPU with the same priority as the interrupted owner.
|
|
*
|
|
* The acquire mechanism uses a few more fields:
|
|
*
|
|
* - The 'req_prio' field is used by the handover approach to make the
|
|
* current owner aware that there is a context with a higher priority
|
|
* waiting for the friendly handover.
|
|
*
|
|
* - The 'unsafe' field allows to take over the console in a safe way in the
|
|
* middle of emitting a message. The field is set only when accessing some
|
|
* shared resources or when the console device is manipulated. It can be
|
|
* cleared, for example, after emitting one character when the console
|
|
* device is in a consistent state.
|
|
*
|
|
* - The 'unsafe_takeover' field is set when a hostile takeover took the
|
|
* console in an unsafe state. The console will stay in the unsafe state
|
|
* until re-initialized.
|
|
*
|
|
* The acquire mechanism uses three approaches:
|
|
*
|
|
* 1) Direct acquire when the console is not owned or is owned by a lower
|
|
* priority context and is in a safe state.
|
|
*
|
|
* 2) Friendly handover mechanism uses a request/grant handshake. It is used
|
|
* when the current owner has lower priority and the console is in an
|
|
* unsafe state.
|
|
*
|
|
* The requesting context:
|
|
*
|
|
* a) Sets its priority into the 'req_prio' field.
|
|
*
|
|
* b) Waits (with a timeout) for the owning context to unlock the
|
|
* console.
|
|
*
|
|
* c) Takes the lock and clears the 'req_prio' field.
|
|
*
|
|
* The owning context:
|
|
*
|
|
* a) Observes the 'req_prio' field set on exit from the unsafe
|
|
* console state.
|
|
*
|
|
* b) Gives up console ownership by clearing the 'prio' field.
|
|
*
|
|
* 3) Unsafe hostile takeover allows to take over the lock even when the
|
|
* console is an unsafe state. It is used only in panic() by the final
|
|
* attempt to flush consoles in a try and hope mode.
|
|
*
|
|
* Note that separate record buffers are used in panic(). As a result,
|
|
* the messages can be read and formatted without any risk even after
|
|
* using the hostile takeover in unsafe state.
|
|
*
|
|
* The release function simply clears the 'prio' field.
|
|
*
|
|
* All operations on @console::nbcon_state are atomic cmpxchg based to
|
|
* handle concurrency.
|
|
*
|
|
* The acquire/release functions implement only minimal policies:
|
|
*
|
|
* - Preference for higher priority contexts.
|
|
* - Protection of the panic CPU.
|
|
*
|
|
* All other policy decisions must be made at the call sites:
|
|
*
|
|
* - What is marked as an unsafe section.
|
|
* - Whether to spin-wait if there is already an owner and the console is
|
|
* in an unsafe state.
|
|
* - Whether to attempt an unsafe hostile takeover.
|
|
*
|
|
* The design allows to implement the well known:
|
|
*
|
|
* acquire()
|
|
* output_one_printk_record()
|
|
* release()
|
|
*
|
|
* The output of one printk record might be interrupted with a higher priority
|
|
* context. The new owner is supposed to reprint the entire interrupted record
|
|
* from scratch.
|
|
*/
|
|
|
|
/**
|
|
* nbcon_state_set - Helper function to set the console state
|
|
* @con: Console to update
|
|
* @new: The new state to write
|
|
*
|
|
* Only to be used when the console is not yet or no longer visible in the
|
|
* system. Otherwise use nbcon_state_try_cmpxchg().
|
|
*/
|
|
static inline void nbcon_state_set(struct console *con, struct nbcon_state *new)
|
|
{
|
|
atomic_set(&ACCESS_PRIVATE(con, nbcon_state), new->atom);
|
|
}
|
|
|
|
/**
|
|
* nbcon_state_read - Helper function to read the console state
|
|
* @con: Console to read
|
|
* @state: The state to store the result
|
|
*/
|
|
static inline void nbcon_state_read(struct console *con, struct nbcon_state *state)
|
|
{
|
|
state->atom = atomic_read(&ACCESS_PRIVATE(con, nbcon_state));
|
|
}
|
|
|
|
/**
|
|
* nbcon_state_try_cmpxchg() - Helper function for atomic_try_cmpxchg() on console state
|
|
* @con: Console to update
|
|
* @cur: Old/expected state
|
|
* @new: New state
|
|
*
|
|
* Return: True on success. False on fail and @cur is updated.
|
|
*/
|
|
static inline bool nbcon_state_try_cmpxchg(struct console *con, struct nbcon_state *cur,
|
|
struct nbcon_state *new)
|
|
{
|
|
return atomic_try_cmpxchg(&ACCESS_PRIVATE(con, nbcon_state), &cur->atom, new->atom);
|
|
}
|
|
|
|
/**
|
|
* nbcon_seq_read - Read the current console sequence
|
|
* @con: Console to read the sequence of
|
|
*
|
|
* Return: Sequence number of the next record to print on @con.
|
|
*/
|
|
u64 nbcon_seq_read(struct console *con)
|
|
{
|
|
unsigned long nbcon_seq = atomic_long_read(&ACCESS_PRIVATE(con, nbcon_seq));
|
|
|
|
return __ulseq_to_u64seq(prb, nbcon_seq);
|
|
}
|
|
|
|
/**
|
|
* nbcon_seq_force - Force console sequence to a specific value
|
|
* @con: Console to work on
|
|
* @seq: Sequence number value to set
|
|
*
|
|
* Only to be used during init (before registration) or in extreme situations
|
|
* (such as panic with CONSOLE_REPLAY_ALL).
|
|
*/
|
|
void nbcon_seq_force(struct console *con, u64 seq)
|
|
{
|
|
/*
|
|
* If the specified record no longer exists, the oldest available record
|
|
* is chosen. This is especially important on 32bit systems because only
|
|
* the lower 32 bits of the sequence number are stored. The upper 32 bits
|
|
* are derived from the sequence numbers available in the ringbuffer.
|
|
*/
|
|
u64 valid_seq = max_t(u64, seq, prb_first_valid_seq(prb));
|
|
|
|
atomic_long_set(&ACCESS_PRIVATE(con, nbcon_seq), __u64seq_to_ulseq(valid_seq));
|
|
}
|
|
|
|
/**
|
|
* nbcon_seq_try_update - Try to update the console sequence number
|
|
* @ctxt: Pointer to an acquire context that contains
|
|
* all information about the acquire mode
|
|
* @new_seq: The new sequence number to set
|
|
*
|
|
* @ctxt->seq is updated to the new value of @con::nbcon_seq (expanded to
|
|
* the 64bit value). This could be a different value than @new_seq if
|
|
* nbcon_seq_force() was used or the current context no longer owns the
|
|
* console. In the later case, it will stop printing anyway.
|
|
*/
|
|
static void nbcon_seq_try_update(struct nbcon_context *ctxt, u64 new_seq)
|
|
{
|
|
unsigned long nbcon_seq = __u64seq_to_ulseq(ctxt->seq);
|
|
struct console *con = ctxt->console;
|
|
|
|
if (atomic_long_try_cmpxchg(&ACCESS_PRIVATE(con, nbcon_seq), &nbcon_seq,
|
|
__u64seq_to_ulseq(new_seq))) {
|
|
ctxt->seq = new_seq;
|
|
} else {
|
|
ctxt->seq = nbcon_seq_read(con);
|
|
}
|
|
}
|
|
|
|
/**
|
|
* nbcon_context_try_acquire_direct - Try to acquire directly
|
|
* @ctxt: The context of the caller
|
|
* @cur: The current console state
|
|
*
|
|
* Acquire the console when it is released. Also acquire the console when
|
|
* the current owner has a lower priority and the console is in a safe state.
|
|
*
|
|
* Return: 0 on success. Otherwise, an error code on failure. Also @cur
|
|
* is updated to the latest state when failed to modify it.
|
|
*
|
|
* Errors:
|
|
*
|
|
* -EPERM: A panic is in progress and this is not the panic CPU.
|
|
* Or the current owner or waiter has the same or higher
|
|
* priority. No acquire method can be successful in
|
|
* this case.
|
|
*
|
|
* -EBUSY: The current owner has a lower priority but the console
|
|
* in an unsafe state. The caller should try using
|
|
* the handover acquire method.
|
|
*/
|
|
static int nbcon_context_try_acquire_direct(struct nbcon_context *ctxt,
|
|
struct nbcon_state *cur)
|
|
{
|
|
unsigned int cpu = smp_processor_id();
|
|
struct console *con = ctxt->console;
|
|
struct nbcon_state new;
|
|
|
|
do {
|
|
/*
|
|
* Panic does not imply that the console is owned. However, it
|
|
* is critical that non-panic CPUs during panic are unable to
|
|
* acquire ownership in order to satisfy the assumptions of
|
|
* nbcon_waiter_matches(). In particular, the assumption that
|
|
* lower priorities are ignored during panic.
|
|
*/
|
|
if (other_cpu_in_panic())
|
|
return -EPERM;
|
|
|
|
if (ctxt->prio <= cur->prio || ctxt->prio <= cur->req_prio)
|
|
return -EPERM;
|
|
|
|
if (cur->unsafe)
|
|
return -EBUSY;
|
|
|
|
/*
|
|
* The console should never be safe for a direct acquire
|
|
* if an unsafe hostile takeover has ever happened.
|
|
*/
|
|
WARN_ON_ONCE(cur->unsafe_takeover);
|
|
|
|
new.atom = cur->atom;
|
|
new.prio = ctxt->prio;
|
|
new.req_prio = NBCON_PRIO_NONE;
|
|
new.unsafe = cur->unsafe_takeover;
|
|
new.cpu = cpu;
|
|
|
|
} while (!nbcon_state_try_cmpxchg(con, cur, &new));
|
|
|
|
return 0;
|
|
}
|
|
|
|
static bool nbcon_waiter_matches(struct nbcon_state *cur, int expected_prio)
|
|
{
|
|
/*
|
|
* The request context is well defined by the @req_prio because:
|
|
*
|
|
* - Only a context with a priority higher than the owner can become
|
|
* a waiter.
|
|
* - Only a context with a priority higher than the waiter can
|
|
* directly take over the request.
|
|
* - There are only three priorities.
|
|
* - Only one CPU is allowed to request PANIC priority.
|
|
* - Lower priorities are ignored during panic() until reboot.
|
|
*
|
|
* As a result, the following scenario is *not* possible:
|
|
*
|
|
* 1. This context is currently a waiter.
|
|
* 2. Another context with a higher priority than this context
|
|
* directly takes ownership.
|
|
* 3. The higher priority context releases the ownership.
|
|
* 4. Another lower priority context takes the ownership.
|
|
* 5. Another context with the same priority as this context
|
|
* creates a request and starts waiting.
|
|
*
|
|
* Event #1 implies this context is EMERGENCY.
|
|
* Event #2 implies the new context is PANIC.
|
|
* Event #3 occurs when panic() has flushed the console.
|
|
* Events #4 and #5 are not possible due to the other_cpu_in_panic()
|
|
* check in nbcon_context_try_acquire_direct().
|
|
*/
|
|
|
|
return (cur->req_prio == expected_prio);
|
|
}
|
|
|
|
/**
|
|
* nbcon_context_try_acquire_requested - Try to acquire after having
|
|
* requested a handover
|
|
* @ctxt: The context of the caller
|
|
* @cur: The current console state
|
|
*
|
|
* This is a helper function for nbcon_context_try_acquire_handover().
|
|
* It is called when the console is in an unsafe state. The current
|
|
* owner will release the console on exit from the unsafe region.
|
|
*
|
|
* Return: 0 on success and @cur is updated to the new console state.
|
|
* Otherwise an error code on failure.
|
|
*
|
|
* Errors:
|
|
*
|
|
* -EPERM: A panic is in progress and this is not the panic CPU
|
|
* or this context is no longer the waiter.
|
|
*
|
|
* -EBUSY: The console is still locked. The caller should
|
|
* continue waiting.
|
|
*
|
|
* Note: The caller must still remove the request when an error has occurred
|
|
* except when this context is no longer the waiter.
|
|
*/
|
|
static int nbcon_context_try_acquire_requested(struct nbcon_context *ctxt,
|
|
struct nbcon_state *cur)
|
|
{
|
|
unsigned int cpu = smp_processor_id();
|
|
struct console *con = ctxt->console;
|
|
struct nbcon_state new;
|
|
|
|
/* Note that the caller must still remove the request! */
|
|
if (other_cpu_in_panic())
|
|
return -EPERM;
|
|
|
|
/*
|
|
* Note that the waiter will also change if there was an unsafe
|
|
* hostile takeover.
|
|
*/
|
|
if (!nbcon_waiter_matches(cur, ctxt->prio))
|
|
return -EPERM;
|
|
|
|
/* If still locked, caller should continue waiting. */
|
|
if (cur->prio != NBCON_PRIO_NONE)
|
|
return -EBUSY;
|
|
|
|
/*
|
|
* The previous owner should have never released ownership
|
|
* in an unsafe region.
|
|
*/
|
|
WARN_ON_ONCE(cur->unsafe);
|
|
|
|
new.atom = cur->atom;
|
|
new.prio = ctxt->prio;
|
|
new.req_prio = NBCON_PRIO_NONE;
|
|
new.unsafe = cur->unsafe_takeover;
|
|
new.cpu = cpu;
|
|
|
|
if (!nbcon_state_try_cmpxchg(con, cur, &new)) {
|
|
/*
|
|
* The acquire could fail only when it has been taken
|
|
* over by a higher priority context.
|
|
*/
|
|
WARN_ON_ONCE(nbcon_waiter_matches(cur, ctxt->prio));
|
|
return -EPERM;
|
|
}
|
|
|
|
/* Handover success. This context now owns the console. */
|
|
return 0;
|
|
}
|
|
|
|
/**
|
|
* nbcon_context_try_acquire_handover - Try to acquire via handover
|
|
* @ctxt: The context of the caller
|
|
* @cur: The current console state
|
|
*
|
|
* The function must be called only when the context has higher priority
|
|
* than the current owner and the console is in an unsafe state.
|
|
* It is the case when nbcon_context_try_acquire_direct() returns -EBUSY.
|
|
*
|
|
* The function sets "req_prio" field to make the current owner aware of
|
|
* the request. Then it waits until the current owner releases the console,
|
|
* or an even higher context takes over the request, or timeout expires.
|
|
*
|
|
* The current owner checks the "req_prio" field on exit from the unsafe
|
|
* region and releases the console. It does not touch the "req_prio" field
|
|
* so that the console stays reserved for the waiter.
|
|
*
|
|
* Return: 0 on success. Otherwise, an error code on failure. Also @cur
|
|
* is updated to the latest state when failed to modify it.
|
|
*
|
|
* Errors:
|
|
*
|
|
* -EPERM: A panic is in progress and this is not the panic CPU.
|
|
* Or a higher priority context has taken over the
|
|
* console or the handover request.
|
|
*
|
|
* -EBUSY: The current owner is on the same CPU so that the hand
|
|
* shake could not work. Or the current owner is not
|
|
* willing to wait (zero timeout). Or the console does
|
|
* not enter the safe state before timeout passed. The
|
|
* caller might still use the unsafe hostile takeover
|
|
* when allowed.
|
|
*
|
|
* -EAGAIN: @cur has changed when creating the handover request.
|
|
* The caller should retry with direct acquire.
|
|
*/
|
|
static int nbcon_context_try_acquire_handover(struct nbcon_context *ctxt,
|
|
struct nbcon_state *cur)
|
|
{
|
|
unsigned int cpu = smp_processor_id();
|
|
struct console *con = ctxt->console;
|
|
struct nbcon_state new;
|
|
int timeout;
|
|
int request_err = -EBUSY;
|
|
|
|
/*
|
|
* Check that the handover is called when the direct acquire failed
|
|
* with -EBUSY.
|
|
*/
|
|
WARN_ON_ONCE(ctxt->prio <= cur->prio || ctxt->prio <= cur->req_prio);
|
|
WARN_ON_ONCE(!cur->unsafe);
|
|
|
|
/* Handover is not possible on the same CPU. */
|
|
if (cur->cpu == cpu)
|
|
return -EBUSY;
|
|
|
|
/*
|
|
* Console stays unsafe after an unsafe takeover until re-initialized.
|
|
* Waiting is not going to help in this case.
|
|
*/
|
|
if (cur->unsafe_takeover)
|
|
return -EBUSY;
|
|
|
|
/* Is the caller willing to wait? */
|
|
if (ctxt->spinwait_max_us == 0)
|
|
return -EBUSY;
|
|
|
|
/*
|
|
* Setup a request for the handover. The caller should try to acquire
|
|
* the console directly when the current state has been modified.
|
|
*/
|
|
new.atom = cur->atom;
|
|
new.req_prio = ctxt->prio;
|
|
if (!nbcon_state_try_cmpxchg(con, cur, &new))
|
|
return -EAGAIN;
|
|
|
|
cur->atom = new.atom;
|
|
|
|
/* Wait until there is no owner and then acquire the console. */
|
|
for (timeout = ctxt->spinwait_max_us; timeout >= 0; timeout--) {
|
|
/* On successful acquire, this request is cleared. */
|
|
request_err = nbcon_context_try_acquire_requested(ctxt, cur);
|
|
if (!request_err)
|
|
return 0;
|
|
|
|
/*
|
|
* If the acquire should be aborted, it must be ensured
|
|
* that the request is removed before returning to caller.
|
|
*/
|
|
if (request_err == -EPERM)
|
|
break;
|
|
|
|
udelay(1);
|
|
|
|
/* Re-read the state because some time has passed. */
|
|
nbcon_state_read(con, cur);
|
|
}
|
|
|
|
/* Timed out or aborted. Carefully remove handover request. */
|
|
do {
|
|
/*
|
|
* No need to remove request if there is a new waiter. This
|
|
* can only happen if a higher priority context has taken over
|
|
* the console or the handover request.
|
|
*/
|
|
if (!nbcon_waiter_matches(cur, ctxt->prio))
|
|
return -EPERM;
|
|
|
|
/* Unset request for handover. */
|
|
new.atom = cur->atom;
|
|
new.req_prio = NBCON_PRIO_NONE;
|
|
if (nbcon_state_try_cmpxchg(con, cur, &new)) {
|
|
/*
|
|
* Request successfully unset. Report failure of
|
|
* acquiring via handover.
|
|
*/
|
|
cur->atom = new.atom;
|
|
return request_err;
|
|
}
|
|
|
|
/*
|
|
* Unable to remove request. Try to acquire in case
|
|
* the owner has released the lock.
|
|
*/
|
|
} while (nbcon_context_try_acquire_requested(ctxt, cur));
|
|
|
|
/* Lucky timing. The acquire succeeded while removing the request. */
|
|
return 0;
|
|
}
|
|
|
|
/**
|
|
* nbcon_context_try_acquire_hostile - Acquire via unsafe hostile takeover
|
|
* @ctxt: The context of the caller
|
|
* @cur: The current console state
|
|
*
|
|
* Acquire the console even in the unsafe state.
|
|
*
|
|
* It can be permitted by setting the 'allow_unsafe_takeover' field only
|
|
* by the final attempt to flush messages in panic().
|
|
*
|
|
* Return: 0 on success. -EPERM when not allowed by the context.
|
|
*/
|
|
static int nbcon_context_try_acquire_hostile(struct nbcon_context *ctxt,
|
|
struct nbcon_state *cur)
|
|
{
|
|
unsigned int cpu = smp_processor_id();
|
|
struct console *con = ctxt->console;
|
|
struct nbcon_state new;
|
|
|
|
if (!ctxt->allow_unsafe_takeover)
|
|
return -EPERM;
|
|
|
|
/* Ensure caller is allowed to perform unsafe hostile takeovers. */
|
|
if (WARN_ON_ONCE(ctxt->prio != NBCON_PRIO_PANIC))
|
|
return -EPERM;
|
|
|
|
/*
|
|
* Check that try_acquire_direct() and try_acquire_handover() returned
|
|
* -EBUSY in the right situation.
|
|
*/
|
|
WARN_ON_ONCE(ctxt->prio <= cur->prio || ctxt->prio <= cur->req_prio);
|
|
WARN_ON_ONCE(cur->unsafe != true);
|
|
|
|
do {
|
|
new.atom = cur->atom;
|
|
new.cpu = cpu;
|
|
new.prio = ctxt->prio;
|
|
new.unsafe |= cur->unsafe_takeover;
|
|
new.unsafe_takeover |= cur->unsafe;
|
|
|
|
} while (!nbcon_state_try_cmpxchg(con, cur, &new));
|
|
|
|
return 0;
|
|
}
|
|
|
|
static struct printk_buffers panic_nbcon_pbufs;
|
|
|
|
/**
|
|
* nbcon_context_try_acquire - Try to acquire nbcon console
|
|
* @ctxt: The context of the caller
|
|
*
|
|
* Context: Under @ctxt->con->device_lock() or local_irq_save().
|
|
* Return: True if the console was acquired. False otherwise.
|
|
*
|
|
* If the caller allowed an unsafe hostile takeover, on success the
|
|
* caller should check the current console state to see if it is
|
|
* in an unsafe state. Otherwise, on success the caller may assume
|
|
* the console is not in an unsafe state.
|
|
*/
|
|
static bool nbcon_context_try_acquire(struct nbcon_context *ctxt)
|
|
{
|
|
unsigned int cpu = smp_processor_id();
|
|
struct console *con = ctxt->console;
|
|
struct nbcon_state cur;
|
|
int err;
|
|
|
|
nbcon_state_read(con, &cur);
|
|
try_again:
|
|
err = nbcon_context_try_acquire_direct(ctxt, &cur);
|
|
if (err != -EBUSY)
|
|
goto out;
|
|
|
|
err = nbcon_context_try_acquire_handover(ctxt, &cur);
|
|
if (err == -EAGAIN)
|
|
goto try_again;
|
|
if (err != -EBUSY)
|
|
goto out;
|
|
|
|
err = nbcon_context_try_acquire_hostile(ctxt, &cur);
|
|
out:
|
|
if (err)
|
|
return false;
|
|
|
|
/* Acquire succeeded. */
|
|
|
|
/* Assign the appropriate buffer for this context. */
|
|
if (atomic_read(&panic_cpu) == cpu)
|
|
ctxt->pbufs = &panic_nbcon_pbufs;
|
|
else
|
|
ctxt->pbufs = con->pbufs;
|
|
|
|
/* Set the record sequence for this context to print. */
|
|
ctxt->seq = nbcon_seq_read(ctxt->console);
|
|
|
|
return true;
|
|
}
|
|
|
|
static bool nbcon_owner_matches(struct nbcon_state *cur, int expected_cpu,
|
|
int expected_prio)
|
|
{
|
|
/*
|
|
* A similar function, nbcon_waiter_matches(), only deals with
|
|
* EMERGENCY and PANIC priorities. However, this function must also
|
|
* deal with the NORMAL priority, which requires additional checks
|
|
* and constraints.
|
|
*
|
|
* For the case where preemption and interrupts are disabled, it is
|
|
* enough to also verify that the owning CPU has not changed.
|
|
*
|
|
* For the case where preemption or interrupts are enabled, an
|
|
* external synchronization method *must* be used. In particular,
|
|
* the driver-specific locking mechanism used in device_lock()
|
|
* (including disabling migration) should be used. It prevents
|
|
* scenarios such as:
|
|
*
|
|
* 1. [Task A] owns a context with NBCON_PRIO_NORMAL on [CPU X] and
|
|
* is scheduled out.
|
|
* 2. Another context takes over the lock with NBCON_PRIO_EMERGENCY
|
|
* and releases it.
|
|
* 3. [Task B] acquires a context with NBCON_PRIO_NORMAL on [CPU X]
|
|
* and is scheduled out.
|
|
* 4. [Task A] gets running on [CPU X] and sees that the console is
|
|
* still owned by a task on [CPU X] with NBON_PRIO_NORMAL. Thus
|
|
* [Task A] thinks it is the owner when it is not.
|
|
*/
|
|
|
|
if (cur->prio != expected_prio)
|
|
return false;
|
|
|
|
if (cur->cpu != expected_cpu)
|
|
return false;
|
|
|
|
return true;
|
|
}
|
|
|
|
/**
|
|
* nbcon_context_release - Release the console
|
|
* @ctxt: The nbcon context from nbcon_context_try_acquire()
|
|
*/
|
|
static void nbcon_context_release(struct nbcon_context *ctxt)
|
|
{
|
|
unsigned int cpu = smp_processor_id();
|
|
struct console *con = ctxt->console;
|
|
struct nbcon_state cur;
|
|
struct nbcon_state new;
|
|
|
|
nbcon_state_read(con, &cur);
|
|
|
|
do {
|
|
if (!nbcon_owner_matches(&cur, cpu, ctxt->prio))
|
|
break;
|
|
|
|
new.atom = cur.atom;
|
|
new.prio = NBCON_PRIO_NONE;
|
|
|
|
/*
|
|
* If @unsafe_takeover is set, it is kept set so that
|
|
* the state remains permanently unsafe.
|
|
*/
|
|
new.unsafe |= cur.unsafe_takeover;
|
|
|
|
} while (!nbcon_state_try_cmpxchg(con, &cur, &new));
|
|
|
|
ctxt->pbufs = NULL;
|
|
}
|
|
|
|
/**
|
|
* nbcon_context_can_proceed - Check whether ownership can proceed
|
|
* @ctxt: The nbcon context from nbcon_context_try_acquire()
|
|
* @cur: The current console state
|
|
*
|
|
* Return: True if this context still owns the console. False if
|
|
* ownership was handed over or taken.
|
|
*
|
|
* Must be invoked when entering the unsafe state to make sure that it still
|
|
* owns the lock. Also must be invoked when exiting the unsafe context
|
|
* to eventually free the lock for a higher priority context which asked
|
|
* for the friendly handover.
|
|
*
|
|
* It can be called inside an unsafe section when the console is just
|
|
* temporary in safe state instead of exiting and entering the unsafe
|
|
* state.
|
|
*
|
|
* Also it can be called in the safe context before doing an expensive
|
|
* safe operation. It does not make sense to do the operation when
|
|
* a higher priority context took the lock.
|
|
*
|
|
* When this function returns false then the calling context no longer owns
|
|
* the console and is no longer allowed to go forward. In this case it must
|
|
* back out immediately and carefully. The buffer content is also no longer
|
|
* trusted since it no longer belongs to the calling context.
|
|
*/
|
|
static bool nbcon_context_can_proceed(struct nbcon_context *ctxt, struct nbcon_state *cur)
|
|
{
|
|
unsigned int cpu = smp_processor_id();
|
|
|
|
/* Make sure this context still owns the console. */
|
|
if (!nbcon_owner_matches(cur, cpu, ctxt->prio))
|
|
return false;
|
|
|
|
/* The console owner can proceed if there is no waiter. */
|
|
if (cur->req_prio == NBCON_PRIO_NONE)
|
|
return true;
|
|
|
|
/*
|
|
* A console owner within an unsafe region is always allowed to
|
|
* proceed, even if there are waiters. It can perform a handover
|
|
* when exiting the unsafe region. Otherwise the waiter will
|
|
* need to perform an unsafe hostile takeover.
|
|
*/
|
|
if (cur->unsafe)
|
|
return true;
|
|
|
|
/* Waiters always have higher priorities than owners. */
|
|
WARN_ON_ONCE(cur->req_prio <= cur->prio);
|
|
|
|
/*
|
|
* Having a safe point for take over and eventually a few
|
|
* duplicated characters or a full line is way better than a
|
|
* hostile takeover. Post processing can take care of the garbage.
|
|
* Release and hand over.
|
|
*/
|
|
nbcon_context_release(ctxt);
|
|
|
|
/*
|
|
* It is not clear whether the waiter really took over ownership. The
|
|
* outermost callsite must make the final decision whether console
|
|
* ownership is needed for it to proceed. If yes, it must reacquire
|
|
* ownership (possibly hostile) before carefully proceeding.
|
|
*
|
|
* The calling context no longer owns the console so go back all the
|
|
* way instead of trying to implement reacquire heuristics in tons of
|
|
* places.
|
|
*/
|
|
return false;
|
|
}
|
|
|
|
/**
|
|
* nbcon_can_proceed - Check whether ownership can proceed
|
|
* @wctxt: The write context that was handed to the write function
|
|
*
|
|
* Return: True if this context still owns the console. False if
|
|
* ownership was handed over or taken.
|
|
*
|
|
* It is used in nbcon_enter_unsafe() to make sure that it still owns the
|
|
* lock. Also it is used in nbcon_exit_unsafe() to eventually free the lock
|
|
* for a higher priority context which asked for the friendly handover.
|
|
*
|
|
* It can be called inside an unsafe section when the console is just
|
|
* temporary in safe state instead of exiting and entering the unsafe state.
|
|
*
|
|
* Also it can be called in the safe context before doing an expensive safe
|
|
* operation. It does not make sense to do the operation when a higher
|
|
* priority context took the lock.
|
|
*
|
|
* When this function returns false then the calling context no longer owns
|
|
* the console and is no longer allowed to go forward. In this case it must
|
|
* back out immediately and carefully. The buffer content is also no longer
|
|
* trusted since it no longer belongs to the calling context.
|
|
*/
|
|
bool nbcon_can_proceed(struct nbcon_write_context *wctxt)
|
|
{
|
|
struct nbcon_context *ctxt = &ACCESS_PRIVATE(wctxt, ctxt);
|
|
struct console *con = ctxt->console;
|
|
struct nbcon_state cur;
|
|
|
|
nbcon_state_read(con, &cur);
|
|
|
|
return nbcon_context_can_proceed(ctxt, &cur);
|
|
}
|
|
EXPORT_SYMBOL_GPL(nbcon_can_proceed);
|
|
|
|
#define nbcon_context_enter_unsafe(c) __nbcon_context_update_unsafe(c, true)
|
|
#define nbcon_context_exit_unsafe(c) __nbcon_context_update_unsafe(c, false)
|
|
|
|
/**
|
|
* __nbcon_context_update_unsafe - Update the unsafe bit in @con->nbcon_state
|
|
* @ctxt: The nbcon context from nbcon_context_try_acquire()
|
|
* @unsafe: The new value for the unsafe bit
|
|
*
|
|
* Return: True if the unsafe state was updated and this context still
|
|
* owns the console. Otherwise false if ownership was handed
|
|
* over or taken.
|
|
*
|
|
* This function allows console owners to modify the unsafe status of the
|
|
* console.
|
|
*
|
|
* When this function returns false then the calling context no longer owns
|
|
* the console and is no longer allowed to go forward. In this case it must
|
|
* back out immediately and carefully. The buffer content is also no longer
|
|
* trusted since it no longer belongs to the calling context.
|
|
*
|
|
* Internal helper to avoid duplicated code.
|
|
*/
|
|
static bool __nbcon_context_update_unsafe(struct nbcon_context *ctxt, bool unsafe)
|
|
{
|
|
struct console *con = ctxt->console;
|
|
struct nbcon_state cur;
|
|
struct nbcon_state new;
|
|
|
|
nbcon_state_read(con, &cur);
|
|
|
|
do {
|
|
/*
|
|
* The unsafe bit must not be cleared if an
|
|
* unsafe hostile takeover has occurred.
|
|
*/
|
|
if (!unsafe && cur.unsafe_takeover)
|
|
goto out;
|
|
|
|
if (!nbcon_context_can_proceed(ctxt, &cur))
|
|
return false;
|
|
|
|
new.atom = cur.atom;
|
|
new.unsafe = unsafe;
|
|
} while (!nbcon_state_try_cmpxchg(con, &cur, &new));
|
|
|
|
cur.atom = new.atom;
|
|
out:
|
|
return nbcon_context_can_proceed(ctxt, &cur);
|
|
}
|
|
|
|
/**
|
|
* nbcon_enter_unsafe - Enter an unsafe region in the driver
|
|
* @wctxt: The write context that was handed to the write function
|
|
*
|
|
* Return: True if this context still owns the console. False if
|
|
* ownership was handed over or taken.
|
|
*
|
|
* When this function returns false then the calling context no longer owns
|
|
* the console and is no longer allowed to go forward. In this case it must
|
|
* back out immediately and carefully. The buffer content is also no longer
|
|
* trusted since it no longer belongs to the calling context.
|
|
*/
|
|
bool nbcon_enter_unsafe(struct nbcon_write_context *wctxt)
|
|
{
|
|
struct nbcon_context *ctxt = &ACCESS_PRIVATE(wctxt, ctxt);
|
|
|
|
return nbcon_context_enter_unsafe(ctxt);
|
|
}
|
|
EXPORT_SYMBOL_GPL(nbcon_enter_unsafe);
|
|
|
|
/**
|
|
* nbcon_exit_unsafe - Exit an unsafe region in the driver
|
|
* @wctxt: The write context that was handed to the write function
|
|
*
|
|
* Return: True if this context still owns the console. False if
|
|
* ownership was handed over or taken.
|
|
*
|
|
* When this function returns false then the calling context no longer owns
|
|
* the console and is no longer allowed to go forward. In this case it must
|
|
* back out immediately and carefully. The buffer content is also no longer
|
|
* trusted since it no longer belongs to the calling context.
|
|
*/
|
|
bool nbcon_exit_unsafe(struct nbcon_write_context *wctxt)
|
|
{
|
|
struct nbcon_context *ctxt = &ACCESS_PRIVATE(wctxt, ctxt);
|
|
|
|
return nbcon_context_exit_unsafe(ctxt);
|
|
}
|
|
EXPORT_SYMBOL_GPL(nbcon_exit_unsafe);
|
|
|
|
/**
|
|
* nbcon_emit_next_record - Emit a record in the acquired context
|
|
* @wctxt: The write context that will be handed to the write function
|
|
*
|
|
* Return: True if this context still owns the console. False if
|
|
* ownership was handed over or taken.
|
|
*
|
|
* When this function returns false then the calling context no longer owns
|
|
* the console and is no longer allowed to go forward. In this case it must
|
|
* back out immediately and carefully. The buffer content is also no longer
|
|
* trusted since it no longer belongs to the calling context. If the caller
|
|
* wants to do more it must reacquire the console first.
|
|
*
|
|
* When true is returned, @wctxt->ctxt.backlog indicates whether there are
|
|
* still records pending in the ringbuffer,
|
|
*/
|
|
static bool nbcon_emit_next_record(struct nbcon_write_context *wctxt)
|
|
{
|
|
struct nbcon_context *ctxt = &ACCESS_PRIVATE(wctxt, ctxt);
|
|
struct console *con = ctxt->console;
|
|
bool is_extended = console_srcu_read_flags(con) & CON_EXTENDED;
|
|
struct printk_message pmsg = {
|
|
.pbufs = ctxt->pbufs,
|
|
};
|
|
unsigned long con_dropped;
|
|
struct nbcon_state cur;
|
|
unsigned long dropped;
|
|
|
|
/*
|
|
* The printk buffers are filled within an unsafe section. This
|
|
* prevents NBCON_PRIO_NORMAL and NBCON_PRIO_EMERGENCY from
|
|
* clobbering each other.
|
|
*/
|
|
|
|
if (!nbcon_context_enter_unsafe(ctxt))
|
|
return false;
|
|
|
|
ctxt->backlog = printk_get_next_message(&pmsg, ctxt->seq, is_extended, true);
|
|
if (!ctxt->backlog)
|
|
return nbcon_context_exit_unsafe(ctxt);
|
|
|
|
/*
|
|
* @con->dropped is not protected in case of an unsafe hostile
|
|
* takeover. In that situation the update can be racy so
|
|
* annotate it accordingly.
|
|
*/
|
|
con_dropped = data_race(READ_ONCE(con->dropped));
|
|
|
|
dropped = con_dropped + pmsg.dropped;
|
|
if (dropped && !is_extended)
|
|
console_prepend_dropped(&pmsg, dropped);
|
|
|
|
if (!nbcon_context_exit_unsafe(ctxt))
|
|
return false;
|
|
|
|
/* For skipped records just update seq/dropped in @con. */
|
|
if (pmsg.outbuf_len == 0)
|
|
goto update_con;
|
|
|
|
/* Initialize the write context for driver callbacks. */
|
|
wctxt->outbuf = &pmsg.pbufs->outbuf[0];
|
|
wctxt->len = pmsg.outbuf_len;
|
|
nbcon_state_read(con, &cur);
|
|
wctxt->unsafe_takeover = cur.unsafe_takeover;
|
|
|
|
if (con->write_atomic) {
|
|
con->write_atomic(con, wctxt);
|
|
} else {
|
|
/*
|
|
* This function should never be called for legacy consoles.
|
|
* Handle it as if ownership was lost and try to continue.
|
|
*/
|
|
WARN_ON_ONCE(1);
|
|
nbcon_context_release(ctxt);
|
|
return false;
|
|
}
|
|
|
|
/*
|
|
* Since any dropped message was successfully output, reset the
|
|
* dropped count for the console.
|
|
*/
|
|
dropped = 0;
|
|
update_con:
|
|
/*
|
|
* The dropped count and the sequence number are updated within an
|
|
* unsafe section. This limits update races to the panic context and
|
|
* allows the panic context to win.
|
|
*/
|
|
|
|
if (!nbcon_context_enter_unsafe(ctxt))
|
|
return false;
|
|
|
|
if (dropped != con_dropped) {
|
|
/* Counterpart to the READ_ONCE() above. */
|
|
WRITE_ONCE(con->dropped, dropped);
|
|
}
|
|
|
|
nbcon_seq_try_update(ctxt, pmsg.seq + 1);
|
|
|
|
return nbcon_context_exit_unsafe(ctxt);
|
|
}
|
|
|
|
/**
|
|
* nbcon_get_default_prio - The appropriate nbcon priority to use for nbcon
|
|
* printing on the current CPU
|
|
*
|
|
* Context: Any context.
|
|
* Return: The nbcon_prio to use for acquiring an nbcon console in this
|
|
* context for printing.
|
|
*
|
|
* The function is safe for reading per-CPU data in any context because
|
|
* preemption is disabled if the current CPU is in the panic state.
|
|
*/
|
|
enum nbcon_prio nbcon_get_default_prio(void)
|
|
{
|
|
if (this_cpu_in_panic())
|
|
return NBCON_PRIO_PANIC;
|
|
|
|
return NBCON_PRIO_NORMAL;
|
|
}
|
|
|
|
/*
|
|
* __nbcon_atomic_flush_pending_con - Flush specified nbcon console using its
|
|
* write_atomic() callback
|
|
* @con: The nbcon console to flush
|
|
* @stop_seq: Flush up until this record
|
|
*
|
|
* Return: 0 if @con was flushed up to @stop_seq Otherwise, error code on
|
|
* failure.
|
|
*
|
|
* Errors:
|
|
*
|
|
* -EPERM: Unable to acquire console ownership.
|
|
*
|
|
* -EAGAIN: Another context took over ownership while printing.
|
|
*
|
|
* -ENOENT: A record before @stop_seq is not available.
|
|
*
|
|
* If flushing up to @stop_seq was not successful, it only makes sense for the
|
|
* caller to try again when -EAGAIN was returned. When -EPERM is returned,
|
|
* this context is not allowed to acquire the console. When -ENOENT is
|
|
* returned, it cannot be expected that the unfinalized record will become
|
|
* available.
|
|
*/
|
|
static int __nbcon_atomic_flush_pending_con(struct console *con, u64 stop_seq)
|
|
{
|
|
struct nbcon_write_context wctxt = { };
|
|
struct nbcon_context *ctxt = &ACCESS_PRIVATE(&wctxt, ctxt);
|
|
int err = 0;
|
|
|
|
ctxt->console = con;
|
|
ctxt->spinwait_max_us = 2000;
|
|
ctxt->prio = nbcon_get_default_prio();
|
|
|
|
if (!nbcon_context_try_acquire(ctxt))
|
|
return -EPERM;
|
|
|
|
while (nbcon_seq_read(con) < stop_seq) {
|
|
/*
|
|
* nbcon_emit_next_record() returns false when the console was
|
|
* handed over or taken over. In both cases the context is no
|
|
* longer valid.
|
|
*/
|
|
if (!nbcon_emit_next_record(&wctxt))
|
|
return -EAGAIN;
|
|
|
|
if (!ctxt->backlog) {
|
|
/* Are there reserved but not yet finalized records? */
|
|
if (nbcon_seq_read(con) < stop_seq)
|
|
err = -ENOENT;
|
|
break;
|
|
}
|
|
}
|
|
|
|
nbcon_context_release(ctxt);
|
|
return err;
|
|
}
|
|
|
|
/**
|
|
* nbcon_atomic_flush_pending_con - Flush specified nbcon console using its
|
|
* write_atomic() callback
|
|
* @con: The nbcon console to flush
|
|
* @stop_seq: Flush up until this record
|
|
*
|
|
* This will stop flushing before @stop_seq if another context has ownership.
|
|
* That context is then responsible for the flushing. Likewise, if new records
|
|
* are added while this context was flushing and there is no other context
|
|
* to handle the printing, this context must also flush those records.
|
|
*/
|
|
static void nbcon_atomic_flush_pending_con(struct console *con, u64 stop_seq)
|
|
{
|
|
unsigned long flags;
|
|
int err;
|
|
|
|
again:
|
|
/*
|
|
* Atomic flushing does not use console driver synchronization (i.e.
|
|
* it does not hold the port lock for uart consoles). Therefore IRQs
|
|
* must be disabled to avoid being interrupted and then calling into
|
|
* a driver that will deadlock trying to acquire console ownership.
|
|
*/
|
|
local_irq_save(flags);
|
|
|
|
err = __nbcon_atomic_flush_pending_con(con, stop_seq);
|
|
|
|
local_irq_restore(flags);
|
|
|
|
/*
|
|
* If there was a new owner (-EPERM, -EAGAIN), that context is
|
|
* responsible for completing.
|
|
*
|
|
* Do not wait for records not yet finalized (-ENOENT) to avoid a
|
|
* possible deadlock. They will either get flushed by the writer or
|
|
* eventually skipped on panic CPU.
|
|
*/
|
|
if (err)
|
|
return;
|
|
|
|
/*
|
|
* If flushing was successful but more records are available, this
|
|
* context must flush those remaining records because there is no
|
|
* other context that will do it.
|
|
*/
|
|
if (prb_read_valid(prb, nbcon_seq_read(con), NULL)) {
|
|
stop_seq = prb_next_reserve_seq(prb);
|
|
goto again;
|
|
}
|
|
}
|
|
|
|
/**
|
|
* __nbcon_atomic_flush_pending - Flush all nbcon consoles using their
|
|
* write_atomic() callback
|
|
* @stop_seq: Flush up until this record
|
|
*/
|
|
static void __nbcon_atomic_flush_pending(u64 stop_seq)
|
|
{
|
|
struct console *con;
|
|
int cookie;
|
|
|
|
cookie = console_srcu_read_lock();
|
|
for_each_console_srcu(con) {
|
|
short flags = console_srcu_read_flags(con);
|
|
|
|
if (!(flags & CON_NBCON))
|
|
continue;
|
|
|
|
if (!console_is_usable(con, flags))
|
|
continue;
|
|
|
|
if (nbcon_seq_read(con) >= stop_seq)
|
|
continue;
|
|
|
|
nbcon_atomic_flush_pending_con(con, stop_seq);
|
|
}
|
|
console_srcu_read_unlock(cookie);
|
|
}
|
|
|
|
/**
|
|
* nbcon_atomic_flush_pending - Flush all nbcon consoles using their
|
|
* write_atomic() callback
|
|
*
|
|
* Flush the backlog up through the currently newest record. Any new
|
|
* records added while flushing will not be flushed if there is another
|
|
* context available to handle the flushing. This is to avoid one CPU
|
|
* printing unbounded because other CPUs continue to add records.
|
|
*/
|
|
void nbcon_atomic_flush_pending(void)
|
|
{
|
|
__nbcon_atomic_flush_pending(prb_next_reserve_seq(prb));
|
|
}
|
|
|
|
/**
|
|
* nbcon_alloc - Allocate and init the nbcon console specific data
|
|
* @con: Console to initialize
|
|
*
|
|
* Return: True if the console was fully allocated and initialized.
|
|
* Otherwise @con must not be registered.
|
|
*
|
|
* When allocation and init was successful, the console must be properly
|
|
* freed using nbcon_free() once it is no longer needed.
|
|
*/
|
|
bool nbcon_alloc(struct console *con)
|
|
{
|
|
struct nbcon_state state = { };
|
|
|
|
nbcon_state_set(con, &state);
|
|
atomic_long_set(&ACCESS_PRIVATE(con, nbcon_seq), 0);
|
|
|
|
if (con->flags & CON_BOOT) {
|
|
/*
|
|
* Boot console printing is synchronized with legacy console
|
|
* printing, so boot consoles can share the same global printk
|
|
* buffers.
|
|
*/
|
|
con->pbufs = &printk_shared_pbufs;
|
|
} else {
|
|
con->pbufs = kmalloc(sizeof(*con->pbufs), GFP_KERNEL);
|
|
if (!con->pbufs) {
|
|
con_printk(KERN_ERR, con, "failed to allocate printing buffer\n");
|
|
return false;
|
|
}
|
|
}
|
|
|
|
return true;
|
|
}
|
|
|
|
/**
|
|
* nbcon_free - Free and cleanup the nbcon console specific data
|
|
* @con: Console to free/cleanup nbcon data
|
|
*/
|
|
void nbcon_free(struct console *con)
|
|
{
|
|
struct nbcon_state state = { };
|
|
|
|
nbcon_state_set(con, &state);
|
|
|
|
/* Boot consoles share global printk buffers. */
|
|
if (!(con->flags & CON_BOOT))
|
|
kfree(con->pbufs);
|
|
|
|
con->pbufs = NULL;
|
|
}
|
|
|
|
/**
|
|
* nbcon_device_try_acquire - Try to acquire nbcon console and enter unsafe
|
|
* section
|
|
* @con: The nbcon console to acquire
|
|
*
|
|
* Context: Under the locking mechanism implemented in
|
|
* @con->device_lock() including disabling migration.
|
|
* Return: True if the console was acquired. False otherwise.
|
|
*
|
|
* Console drivers will usually use their own internal synchronization
|
|
* mechasism to synchronize between console printing and non-printing
|
|
* activities (such as setting baud rates). However, nbcon console drivers
|
|
* supporting atomic consoles may also want to mark unsafe sections when
|
|
* performing non-printing activities in order to synchronize against their
|
|
* atomic_write() callback.
|
|
*
|
|
* This function acquires the nbcon console using priority NBCON_PRIO_NORMAL
|
|
* and marks it unsafe for handover/takeover.
|
|
*/
|
|
bool nbcon_device_try_acquire(struct console *con)
|
|
{
|
|
struct nbcon_context *ctxt = &ACCESS_PRIVATE(con, nbcon_device_ctxt);
|
|
|
|
cant_migrate();
|
|
|
|
memset(ctxt, 0, sizeof(*ctxt));
|
|
ctxt->console = con;
|
|
ctxt->prio = NBCON_PRIO_NORMAL;
|
|
|
|
if (!nbcon_context_try_acquire(ctxt))
|
|
return false;
|
|
|
|
if (!nbcon_context_enter_unsafe(ctxt))
|
|
return false;
|
|
|
|
return true;
|
|
}
|
|
EXPORT_SYMBOL_GPL(nbcon_device_try_acquire);
|
|
|
|
/**
|
|
* nbcon_device_release - Exit unsafe section and release the nbcon console
|
|
* @con: The nbcon console acquired in nbcon_device_try_acquire()
|
|
*/
|
|
void nbcon_device_release(struct console *con)
|
|
{
|
|
struct nbcon_context *ctxt = &ACCESS_PRIVATE(con, nbcon_device_ctxt);
|
|
|
|
if (!nbcon_context_exit_unsafe(ctxt))
|
|
return;
|
|
|
|
nbcon_context_release(ctxt);
|
|
}
|
|
EXPORT_SYMBOL_GPL(nbcon_device_release);
|