aboutsummaryrefslogtreecommitdiffstats
path: root/kernel/rcu/tree.c
diff options
context:
space:
mode:
Diffstat (limited to 'kernel/rcu/tree.c')
-rw-r--r--kernel/rcu/tree.c409
1 files changed, 282 insertions, 127 deletions
diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
index 6c6569e0586c..8ce77d9ac716 100644
--- a/kernel/rcu/tree.c
+++ b/kernel/rcu/tree.c
@@ -57,6 +57,9 @@
#include <linux/slab.h>
#include <linux/sched/isolation.h>
#include <linux/sched/clock.h>
+#include <linux/vmalloc.h>
+#include <linux/mm.h>
+#include <linux/kasan.h>
#include "../time/tick-internal.h"
#include "tree.h"
@@ -175,6 +178,15 @@ module_param(gp_init_delay, int, 0444);
static int gp_cleanup_delay;
module_param(gp_cleanup_delay, int, 0444);
+/*
+ * This rcu parameter is runtime-read-only. It reflects
+ * a minimum allowed number of objects which can be cached
+ * per-CPU. Object size is equal to one page. This value
+ * can be changed at boot time.
+ */
+static int rcu_min_cached_objs = 2;
+module_param(rcu_min_cached_objs, int, 0444);
+
/* Retrieve RCU kthreads priority for rcutorture */
int rcu_get_gp_kthreads_prio(void)
{
@@ -954,7 +966,6 @@ void __rcu_irq_enter_check_tick(void)
/**
* rcu_nmi_enter - inform RCU of entry to NMI context
- * @irq: Is this call from rcu_irq_enter?
*
* If the CPU was idle from RCU's viewpoint, update rdp->dynticks and
* rdp->dynticks_nmi_nesting to let the RCU grace-period handling know
@@ -990,8 +1001,11 @@ noinstr void rcu_nmi_enter(void)
rcu_dynticks_eqs_exit();
// ... but is watching here.
- if (!in_nmi())
+ if (!in_nmi()) {
+ instrumentation_begin();
rcu_cleanup_after_idle();
+ instrumentation_end();
+ }
instrumentation_begin();
// instrumentation for the noinstr rcu_dynticks_curr_cpu_in_eqs()
@@ -1638,7 +1652,7 @@ static void rcu_gp_slow(int delay)
if (delay > 0 &&
!(rcu_seq_ctr(rcu_state.gp_seq) %
(rcu_num_nodes * PER_RCU_NODE_PERIOD * delay)))
- schedule_timeout_uninterruptible(delay);
+ schedule_timeout_idle(delay);
}
static unsigned long sleep_duration;
@@ -1661,7 +1675,7 @@ static void rcu_gp_torture_wait(void)
duration = xchg(&sleep_duration, 0UL);
if (duration > 0) {
pr_alert("%s: Waiting %lu jiffies\n", __func__, duration);
- schedule_timeout_uninterruptible(duration);
+ schedule_timeout_idle(duration);
pr_alert("%s: Wait complete\n", __func__);
}
}
@@ -2443,6 +2457,7 @@ static void rcu_do_batch(struct rcu_data *rdp)
local_irq_save(flags);
rcu_nocb_lock(rdp);
count = -rcl.len;
+ rdp->n_cbs_invoked += count;
trace_rcu_batch_end(rcu_state.name, count, !!rcl.head, need_resched(),
is_idle_task(current), rcu_is_callbacks_kthread());
@@ -2726,7 +2741,7 @@ static void rcu_cpu_kthread(unsigned int cpu)
}
*statusp = RCU_KTHREAD_YIELDING;
trace_rcu_utilization(TPS("Start CPU kthread@rcu_yield"));
- schedule_timeout_interruptible(2);
+ schedule_timeout_idle(2);
trace_rcu_utilization(TPS("End CPU kthread@rcu_yield"));
*statusp = RCU_KTHREAD_WAITING;
}
@@ -2876,6 +2891,7 @@ __call_rcu(struct rcu_head *head, rcu_callback_t func)
head->func = func;
head->next = NULL;
local_irq_save(flags);
+ kasan_record_aux_stack(head);
rdp = this_cpu_ptr(&rcu_data);
/* Add the callback to our list. */
@@ -2894,8 +2910,8 @@ __call_rcu(struct rcu_head *head, rcu_callback_t func)
return; // Enqueued onto ->nocb_bypass, so just leave.
// If no-CBs CPU gets here, rcu_nocb_try_bypass() acquired ->nocb_lock.
rcu_segcblist_enqueue(&rdp->cblist, head);
- if (__is_kfree_rcu_offset((unsigned long)func))
- trace_rcu_kfree_callback(rcu_state.name, head,
+ if (__is_kvfree_rcu_offset((unsigned long)func))
+ trace_rcu_kvfree_callback(rcu_state.name, head,
(unsigned long)func,
rcu_segcblist_n_cbs(&rdp->cblist));
else
@@ -2957,53 +2973,53 @@ EXPORT_SYMBOL_GPL(call_rcu);
/* Maximum number of jiffies to wait before draining a batch. */
#define KFREE_DRAIN_JIFFIES (HZ / 50)
#define KFREE_N_BATCHES 2
-
-/*
- * This macro defines how many entries the "records" array
- * will contain. It is based on the fact that the size of
- * kfree_rcu_bulk_data structure becomes exactly one page.
- */
-#define KFREE_BULK_MAX_ENTR ((PAGE_SIZE / sizeof(void *)) - 3)
+#define FREE_N_CHANNELS 2
/**
- * struct kfree_rcu_bulk_data - single block to store kfree_rcu() pointers
+ * struct kvfree_rcu_bulk_data - single block to store kvfree_rcu() pointers
* @nr_records: Number of active pointers in the array
- * @records: Array of the kfree_rcu() pointers
* @next: Next bulk object in the block chain
- * @head_free_debug: For debug, when CONFIG_DEBUG_OBJECTS_RCU_HEAD is set
+ * @records: Array of the kvfree_rcu() pointers
*/
-struct kfree_rcu_bulk_data {
+struct kvfree_rcu_bulk_data {
unsigned long nr_records;
- void *records[KFREE_BULK_MAX_ENTR];
- struct kfree_rcu_bulk_data *next;
- struct rcu_head *head_free_debug;
+ struct kvfree_rcu_bulk_data *next;
+ void *records[];
};
+/*
+ * This macro defines how many entries the "records" array
+ * will contain. It is based on the fact that the size of
+ * kvfree_rcu_bulk_data structure becomes exactly one page.
+ */
+#define KVFREE_BULK_MAX_ENTR \
+ ((PAGE_SIZE - sizeof(struct kvfree_rcu_bulk_data)) / sizeof(void *))
+
/**
* struct kfree_rcu_cpu_work - single batch of kfree_rcu() requests
* @rcu_work: Let queue_rcu_work() invoke workqueue handler after grace period
* @head_free: List of kfree_rcu() objects waiting for a grace period
- * @bhead_free: Bulk-List of kfree_rcu() objects waiting for a grace period
+ * @bkvhead_free: Bulk-List of kvfree_rcu() objects waiting for a grace period
* @krcp: Pointer to @kfree_rcu_cpu structure
*/
struct kfree_rcu_cpu_work {
struct rcu_work rcu_work;
struct rcu_head *head_free;
- struct kfree_rcu_bulk_data *bhead_free;
+ struct kvfree_rcu_bulk_data *bkvhead_free[FREE_N_CHANNELS];
struct kfree_rcu_cpu *krcp;
};
/**
* struct kfree_rcu_cpu - batch up kfree_rcu() requests for RCU grace period
* @head: List of kfree_rcu() objects not yet waiting for a grace period
- * @bhead: Bulk-List of kfree_rcu() objects not yet waiting for a grace period
- * @bcached: Keeps at most one object for later reuse when build chain blocks
+ * @bkvhead: Bulk-List of kvfree_rcu() objects not yet waiting for a grace period
* @krw_arr: Array of batches of kfree_rcu() objects waiting for a grace period
* @lock: Synchronize access to this structure
* @monitor_work: Promote @head to @head_free after KFREE_DRAIN_JIFFIES
* @monitor_todo: Tracks whether a @monitor_work delayed work is pending
- * @initialized: The @lock and @rcu_work fields have been initialized
+ * @initialized: The @rcu_work fields have been initialized
+ * @count: Number of objects for which GP not started
*
* This is a per-CPU structure. The reason that it is not included in
* the rcu_data structure is to permit this code to be extracted from
@@ -3012,28 +3028,84 @@ struct kfree_rcu_cpu_work {
*/
struct kfree_rcu_cpu {
struct rcu_head *head;
- struct kfree_rcu_bulk_data *bhead;
- struct kfree_rcu_bulk_data *bcached;
+ struct kvfree_rcu_bulk_data *bkvhead[FREE_N_CHANNELS];
struct kfree_rcu_cpu_work krw_arr[KFREE_N_BATCHES];
- spinlock_t lock;
+ raw_spinlock_t lock;
struct delayed_work monitor_work;
bool monitor_todo;
bool initialized;
- // Number of objects for which GP not started
int count;
+
+ /*
+ * A simple cache list that contains objects for
+ * reuse purpose. In order to save some per-cpu
+ * space the list is singular. Even though it is
+ * lockless an access has to be protected by the
+ * per-cpu lock.
+ */
+ struct llist_head bkvcache;
+ int nr_bkv_objs;
};
-static DEFINE_PER_CPU(struct kfree_rcu_cpu, krc);
+static DEFINE_PER_CPU(struct kfree_rcu_cpu, krc) = {
+ .lock = __RAW_SPIN_LOCK_UNLOCKED(krc.lock),
+};
static __always_inline void
-debug_rcu_head_unqueue_bulk(struct rcu_head *head)
+debug_rcu_bhead_unqueue(struct kvfree_rcu_bulk_data *bhead)
{
#ifdef CONFIG_DEBUG_OBJECTS_RCU_HEAD
- for (; head; head = head->next)
- debug_rcu_head_unqueue(head);
+ int i;
+
+ for (i = 0; i < bhead->nr_records; i++)
+ debug_rcu_head_unqueue((struct rcu_head *)(bhead->records[i]));
#endif
}
+static inline struct kfree_rcu_cpu *
+krc_this_cpu_lock(unsigned long *flags)
+{
+ struct kfree_rcu_cpu *krcp;
+
+ local_irq_save(*flags); // For safely calling this_cpu_ptr().
+ krcp = this_cpu_ptr(&krc);
+ raw_spin_lock(&krcp->lock);
+
+ return krcp;
+}
+
+static inline void
+krc_this_cpu_unlock(struct kfree_rcu_cpu *krcp, unsigned long flags)
+{
+ raw_spin_unlock(&krcp->lock);
+ local_irq_restore(flags);
+}
+
+static inline struct kvfree_rcu_bulk_data *
+get_cached_bnode(struct kfree_rcu_cpu *krcp)
+{
+ if (!krcp->nr_bkv_objs)
+ return NULL;
+
+ krcp->nr_bkv_objs--;
+ return (struct kvfree_rcu_bulk_data *)
+ llist_del_first(&krcp->bkvcache);
+}
+
+static inline bool
+put_cached_bnode(struct kfree_rcu_cpu *krcp,
+ struct kvfree_rcu_bulk_data *bnode)
+{
+ // Check the limit.
+ if (krcp->nr_bkv_objs >= rcu_min_cached_objs)
+ return false;
+
+ llist_add((struct llist_node *) bnode, &krcp->bkvcache);
+ krcp->nr_bkv_objs++;
+ return true;
+
+}
+
/*
* This function is invoked in workqueue context after a grace period.
* It frees all the objects queued on ->bhead_free or ->head_free.
@@ -3041,38 +3113,63 @@ debug_rcu_head_unqueue_bulk(struct rcu_head *head)
static void kfree_rcu_work(struct work_struct *work)
{
unsigned long flags;
+ struct kvfree_rcu_bulk_data *bkvhead[FREE_N_CHANNELS], *bnext;
struct rcu_head *head, *next;
- struct kfree_rcu_bulk_data *bhead, *bnext;
struct kfree_rcu_cpu *krcp;
struct kfree_rcu_cpu_work *krwp;
+ int i, j;
krwp = container_of(to_rcu_work(work),
struct kfree_rcu_cpu_work, rcu_work);
krcp = krwp->krcp;
- spin_lock_irqsave(&krcp->lock, flags);
- head = krwp->head_free;
- krwp->head_free = NULL;
- bhead = krwp->bhead_free;
- krwp->bhead_free = NULL;
- spin_unlock_irqrestore(&krcp->lock, flags);
-
- /* "bhead" is now private, so traverse locklessly. */
- for (; bhead; bhead = bnext) {
- bnext = bhead->next;
- debug_rcu_head_unqueue_bulk(bhead->head_free_debug);
+ raw_spin_lock_irqsave(&krcp->lock, flags);
+ // Channels 1 and 2.
+ for (i = 0; i < FREE_N_CHANNELS; i++) {
+ bkvhead[i] = krwp->bkvhead_free[i];
+ krwp->bkvhead_free[i] = NULL;
+ }
- rcu_lock_acquire(&rcu_callback_map);
- trace_rcu_invoke_kfree_bulk_callback(rcu_state.name,
- bhead->nr_records, bhead->records);
+ // Channel 3.
+ head = krwp->head_free;
+ krwp->head_free = NULL;
+ raw_spin_unlock_irqrestore(&krcp->lock, flags);
+
+ // Handle two first channels.
+ for (i = 0; i < FREE_N_CHANNELS; i++) {
+ for (; bkvhead[i]; bkvhead[i] = bnext) {
+ bnext = bkvhead[i]->next;
+ debug_rcu_bhead_unqueue(bkvhead[i]);
+
+ rcu_lock_acquire(&rcu_callback_map);
+ if (i == 0) { // kmalloc() / kfree().
+ trace_rcu_invoke_kfree_bulk_callback(
+ rcu_state.name, bkvhead[i]->nr_records,
+ bkvhead[i]->records);
+
+ kfree_bulk(bkvhead[i]->nr_records,
+ bkvhead[i]->records);
+ } else { // vmalloc() / vfree().
+ for (j = 0; j < bkvhead[i]->nr_records; j++) {
+ trace_rcu_invoke_kvfree_callback(
+ rcu_state.name,
+ bkvhead[i]->records[j], 0);
+
+ vfree(bkvhead[i]->records[j]);
+ }
+ }
+ rcu_lock_release(&rcu_callback_map);
- kfree_bulk(bhead->nr_records, bhead->records);
- rcu_lock_release(&rcu_callback_map);
+ krcp = krc_this_cpu_lock(&flags);
+ if (put_cached_bnode(krcp, bkvhead[i]))
+ bkvhead[i] = NULL;
+ krc_this_cpu_unlock(krcp, flags);
- if (cmpxchg(&krcp->bcached, NULL, bhead))
- free_page((unsigned long) bhead);
+ if (bkvhead[i])
+ free_page((unsigned long) bkvhead[i]);
- cond_resched_tasks_rcu_qs();
+ cond_resched_tasks_rcu_qs();
+ }
}
/*
@@ -3082,14 +3179,15 @@ static void kfree_rcu_work(struct work_struct *work)
*/
for (; head; head = next) {
unsigned long offset = (unsigned long)head->func;
+ void *ptr = (void *)head - offset;
next = head->next;
- debug_rcu_head_unqueue(head);
+ debug_rcu_head_unqueue((struct rcu_head *)ptr);
rcu_lock_acquire(&rcu_callback_map);
- trace_rcu_invoke_kfree_callback(rcu_state.name, head, offset);
+ trace_rcu_invoke_kvfree_callback(rcu_state.name, head, offset);
- if (!WARN_ON_ONCE(!__is_kfree_rcu_offset(offset)))
- kfree((void *)head - offset);
+ if (!WARN_ON_ONCE(!__is_kvfree_rcu_offset(offset)))
+ kvfree(ptr);
rcu_lock_release(&rcu_callback_map);
cond_resched_tasks_rcu_qs();
@@ -3105,8 +3203,8 @@ static void kfree_rcu_work(struct work_struct *work)
static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
{
struct kfree_rcu_cpu_work *krwp;
- bool queued = false;
- int i;
+ bool repeat = false;
+ int i, j;
lockdep_assert_held(&krcp->lock);
@@ -3114,21 +3212,25 @@ static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
krwp = &(krcp->krw_arr[i]);
/*
- * Try to detach bhead or head and attach it over any
+ * Try to detach bkvhead or head and attach it over any
* available corresponding free channel. It can be that
* a previous RCU batch is in progress, it means that
* immediately to queue another one is not possible so
* return false to tell caller to retry.
*/
- if ((krcp->bhead && !krwp->bhead_free) ||
+ if ((krcp->bkvhead[0] && !krwp->bkvhead_free[0]) ||
+ (krcp->bkvhead[1] && !krwp->bkvhead_free[1]) ||
(krcp->head && !krwp->head_free)) {
- /* Channel 1. */
- if (!krwp->bhead_free) {
- krwp->bhead_free = krcp->bhead;
- krcp->bhead = NULL;
+ // Channel 1 corresponds to SLAB ptrs.
+ // Channel 2 corresponds to vmalloc ptrs.
+ for (j = 0; j < FREE_N_CHANNELS; j++) {
+ if (!krwp->bkvhead_free[j]) {
+ krwp->bkvhead_free[j] = krcp->bkvhead[j];
+ krcp->bkvhead[j] = NULL;
+ }
}
- /* Channel 2. */
+ // Channel 3 corresponds to emergency path.
if (!krwp->head_free) {
krwp->head_free = krcp->head;
krcp->head = NULL;
@@ -3137,17 +3239,21 @@ static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
WRITE_ONCE(krcp->count, 0);
/*
- * One work is per one batch, so there are two "free channels",
- * "bhead_free" and "head_free" the batch can handle. It can be
- * that the work is in the pending state when two channels have
- * been detached following each other, one by one.
+ * One work is per one batch, so there are three
+ * "free channels", the batch can handle. It can
+ * be that the work is in the pending state when
+ * channels have been detached following by each
+ * other.
*/
queue_rcu_work(system_wq, &krwp->rcu_work);
- queued = true;
}
+
+ // Repeat if any "free" corresponding channel is still busy.
+ if (krcp->bkvhead[0] || krcp->bkvhead[1] || krcp->head)
+ repeat = true;
}
- return queued;
+ return !repeat;
}
static inline void kfree_rcu_drain_unlock(struct kfree_rcu_cpu *krcp,
@@ -3157,14 +3263,14 @@ static inline void kfree_rcu_drain_unlock(struct kfree_rcu_cpu *krcp,
krcp->monitor_todo = false;
if (queue_kfree_rcu_work(krcp)) {
// Success! Our job is done here.
- spin_unlock_irqrestore(&krcp->lock, flags);
+ raw_spin_unlock_irqrestore(&krcp->lock, flags);
return;
}
// Previous RCU batch still in progress, try again later.
krcp->monitor_todo = true;
schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
- spin_unlock_irqrestore(&krcp->lock, flags);
+ raw_spin_unlock_irqrestore(&krcp->lock, flags);
}
/*
@@ -3177,32 +3283,50 @@ static void kfree_rcu_monitor(struct work_struct *work)
struct kfree_rcu_cpu *krcp = container_of(work, struct kfree_rcu_cpu,
monitor_work.work);
- spin_lock_irqsave(&krcp->lock, flags);
+ raw_spin_lock_irqsave(&krcp->lock, flags);
if (krcp->monitor_todo)
kfree_rcu_drain_unlock(krcp, flags);
else
- spin_unlock_irqrestore(&krcp->lock, flags);
+ raw_spin_unlock_irqrestore(&krcp->lock, flags);
}
static inline bool
-kfree_call_rcu_add_ptr_to_bulk(struct kfree_rcu_cpu *krcp,
- struct rcu_head *head, rcu_callback_t func)
+kvfree_call_rcu_add_ptr_to_bulk(struct kfree_rcu_cpu *krcp, void *ptr)
{
- struct kfree_rcu_bulk_data *bnode;
+ struct kvfree_rcu_bulk_data *bnode;
+ int idx;
if (unlikely(!krcp->initialized))
return false;
lockdep_assert_held(&krcp->lock);
+ idx = !!is_vmalloc_addr(ptr);
/* Check if a new block is required. */
- if (!krcp->bhead ||
- krcp->bhead->nr_records == KFREE_BULK_MAX_ENTR) {
- bnode = xchg(&krcp->bcached, NULL);
+ if (!krcp->bkvhead[idx] ||
+ krcp->bkvhead[idx]->nr_records == KVFREE_BULK_MAX_ENTR) {
+ bnode = get_cached_bnode(krcp);
if (!bnode) {
- WARN_ON_ONCE(sizeof(struct kfree_rcu_bulk_data) > PAGE_SIZE);
+ /*
+ * To keep this path working on raw non-preemptible
+ * sections, prevent the optional entry into the
+ * allocator as it uses sleeping locks. In fact, even
+ * if the caller of kfree_rcu() is preemptible, this
+ * path still is not, as krcp->lock is a raw spinlock.
+ * With additional page pre-allocation in the works,
+ * hitting this return is going to be much less likely.
+ */
+ if (IS_ENABLED(CONFIG_PREEMPT_RT))
+ return false;
- bnode = (struct kfree_rcu_bulk_data *)
+ /*
+ * NOTE: For one argument of kvfree_rcu() we can
+ * drop the lock and get the page in sleepable
+ * context. That would allow to maintain an array
+ * for the CONFIG_PREEMPT_RT as well if no cached
+ * pages are available.
+ */
+ bnode = (struct kvfree_rcu_bulk_data *)
__get_free_page(GFP_NOWAIT | __GFP_NOWARN);
}
@@ -3212,53 +3336,62 @@ kfree_call_rcu_add_ptr_to_bulk(struct kfree_rcu_cpu *krcp,
/* Initialize the new block. */
bnode->nr_records = 0;
- bnode->next = krcp->bhead;
- bnode->head_free_debug = NULL;
+ bnode->next = krcp->bkvhead[idx];
/* Attach it to the head. */
- krcp->bhead = bnode;
+ krcp->bkvhead[idx] = bnode;
}
-#ifdef CONFIG_DEBUG_OBJECTS_RCU_HEAD
- head->func = func;
- head->next = krcp->bhead->head_free_debug;
- krcp->bhead->head_free_debug = head;
-#endif
-
/* Finally insert. */
- krcp->bhead->records[krcp->bhead->nr_records++] =
- (void *) head - (unsigned long) func;
+ krcp->bkvhead[idx]->records
+ [krcp->bkvhead[idx]->nr_records++] = ptr;
return true;
}
/*
- * Queue a request for lazy invocation of kfree_bulk()/kfree() after a grace
- * period. Please note there are two paths are maintained, one is the main one
- * that uses kfree_bulk() interface and second one is emergency one, that is
- * used only when the main path can not be maintained temporary, due to memory
- * pressure.
+ * Queue a request for lazy invocation of appropriate free routine after a
+ * grace period. Please note there are three paths are maintained, two are the
+ * main ones that use array of pointers interface and third one is emergency
+ * one, that is used only when the main path can not be maintained temporary,
+ * due to memory pressure.
*
- * Each kfree_call_rcu() request is added to a batch. The batch will be drained
+ * Each kvfree_call_rcu() request is added to a batch. The batch will be drained
* every KFREE_DRAIN_JIFFIES number of jiffies. All the objects in the batch will
* be free'd in workqueue context. This allows us to: batch requests together to
- * reduce the number of grace periods during heavy kfree_rcu() load.
+ * reduce the number of grace periods during heavy kfree_rcu()/kvfree_rcu() load.
*/
-void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
+void kvfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
{
unsigned long flags;
struct kfree_rcu_cpu *krcp;
+ bool success;
+ void *ptr;
- local_irq_save(flags); // For safely calling this_cpu_ptr().
- krcp = this_cpu_ptr(&krc);
- if (krcp->initialized)
- spin_lock(&krcp->lock);
+ if (head) {
+ ptr = (void *) head - (unsigned long) func;
+ } else {
+ /*
+ * Please note there is a limitation for the head-less
+ * variant, that is why there is a clear rule for such
+ * objects: it can be used from might_sleep() context
+ * only. For other places please embed an rcu_head to
+ * your data.
+ */
+ might_sleep();
+ ptr = (unsigned long *) func;
+ }
+
+ krcp = krc_this_cpu_lock(&flags);
// Queue the object but don't yet schedule the batch.
- if (debug_rcu_head_queue(head)) {
+ if (debug_rcu_head_queue(ptr)) {
// Probable double kfree_rcu(), just leak.
WARN_ONCE(1, "%s(): Double-freed call. rcu_head %p\n",
__func__, head);
+
+ // Mark as success and leave.
+ success = true;
goto unlock_return;
}
@@ -3266,10 +3399,16 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
* Under high memory pressure GFP_NOWAIT can fail,
* in that case the emergency path is maintained.
*/
- if (unlikely(!kfree_call_rcu_add_ptr_to_bulk(krcp, head, func))) {
+ success = kvfree_call_rcu_add_ptr_to_bulk(krcp, ptr);
+ if (!success) {
+ if (head == NULL)
+ // Inline if kvfree_rcu(one_arg) call.
+ goto unlock_return;
+
head->func = func;
head->next = krcp->head;
krcp->head = head;
+ success = true;
}
WRITE_ONCE(krcp->count, krcp->count + 1);
@@ -3282,11 +3421,20 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
}
unlock_return:
- if (krcp->initialized)
- spin_unlock(&krcp->lock);
- local_irq_restore(flags);
+ krc_this_cpu_unlock(krcp, flags);
+
+ /*
+ * Inline kvfree() after synchronize_rcu(). We can do
+ * it from might_sleep() context only, so the current
+ * CPU can pass the QS state.
+ */
+ if (!success) {
+ debug_rcu_head_unqueue((struct rcu_head *) ptr);
+ synchronize_rcu();
+ kvfree(ptr);
+ }
}
-EXPORT_SYMBOL_GPL(kfree_call_rcu);
+EXPORT_SYMBOL_GPL(kvfree_call_rcu);
static unsigned long
kfree_rcu_shrink_count(struct shrinker *shrink, struct shrink_control *sc)
@@ -3315,11 +3463,11 @@ kfree_rcu_shrink_scan(struct shrinker *shrink, struct shrink_control *sc)
struct kfree_rcu_cpu *krcp = per_cpu_ptr(&krc, cpu);
count = krcp->count;
- spin_lock_irqsave(&krcp->lock, flags);
+ raw_spin_lock_irqsave(&krcp->lock, flags);
if (krcp->monitor_todo)
kfree_rcu_drain_unlock(krcp, flags);
else
- spin_unlock_irqrestore(&krcp->lock, flags);
+ raw_spin_unlock_irqrestore(&krcp->lock, flags);
sc->nr_to_scan -= count;
freed += count;
@@ -3328,7 +3476,7 @@ kfree_rcu_shrink_scan(struct shrinker *shrink, struct shrink_control *sc)
break;
}
- return freed;
+ return freed == 0 ? SHRINK_STOP : freed;
}
static struct shrinker kfree_rcu_shrinker = {
@@ -3346,15 +3494,15 @@ void __init kfree_rcu_scheduler_running(void)
for_each_online_cpu(cpu) {
struct kfree_rcu_cpu *krcp = per_cpu_ptr(&krc, cpu);
- spin_lock_irqsave(&krcp->lock, flags);
+ raw_spin_lock_irqsave(&krcp->lock, flags);
if (!krcp->head || krcp->monitor_todo) {
- spin_unlock_irqrestore(&krcp->lock, flags);
+ raw_spin_unlock_irqrestore(&krcp->lock, flags);
continue;
}
krcp->monitor_todo = true;
schedule_delayed_work_on(cpu, &krcp->monitor_work,
KFREE_DRAIN_JIFFIES);
- spin_unlock_irqrestore(&krcp->lock, flags);
+ raw_spin_unlock_irqrestore(&krcp->lock, flags);
}
}
@@ -3842,10 +3990,9 @@ void rcu_cpu_starting(unsigned int cpu)
{
unsigned long flags;
unsigned long mask;
- int nbits;
- unsigned long oldmask;
struct rcu_data *rdp;
struct rcu_node *rnp;
+ bool newcpu;
if (per_cpu(rcu_cpu_started, cpu))
return;
@@ -3857,12 +4004,10 @@ void rcu_cpu_starting(unsigned int cpu)
mask = rdp->grpmask;
raw_spin_lock_irqsave_rcu_node(rnp, flags);
WRITE_ONCE(rnp->qsmaskinitnext, rnp->qsmaskinitnext | mask);
- oldmask = rnp->expmaskinitnext;
+ newcpu = !(rnp->expmaskinitnext & mask);
rnp->expmaskinitnext |= mask;
- oldmask ^= rnp->expmaskinitnext;
- nbits = bitmap_weight(&oldmask, BITS_PER_LONG);
/* Allow lockless access for expedited grace periods. */
- smp_store_release(&rcu_state.ncpus, rcu_state.ncpus + nbits); /* ^^^ */
+ smp_store_release(&rcu_state.ncpus, rcu_state.ncpus + newcpu); /* ^^^ */
ASSERT_EXCLUSIVE_WRITER(rcu_state.ncpus);
rcu_gpnum_ovf(rnp, rdp); /* Offline-induced counter wrap? */
rdp->rcu_onl_gp_seq = READ_ONCE(rcu_state.gp_seq);
@@ -4249,13 +4394,23 @@ static void __init kfree_rcu_batch_init(void)
for_each_possible_cpu(cpu) {
struct kfree_rcu_cpu *krcp = per_cpu_ptr(&krc, cpu);
+ struct kvfree_rcu_bulk_data *bnode;
- spin_lock_init(&krcp->lock);
for (i = 0; i < KFREE_N_BATCHES; i++) {
INIT_RCU_WORK(&krcp->krw_arr[i].rcu_work, kfree_rcu_work);
krcp->krw_arr[i].krcp = krcp;
}
+ for (i = 0; i < rcu_min_cached_objs; i++) {
+ bnode = (struct kvfree_rcu_bulk_data *)
+ __get_free_page(GFP_NOWAIT | __GFP_NOWARN);
+
+ if (bnode)
+ put_cached_bnode(krcp, bnode);
+ else
+ pr_err("Failed to preallocate for %d CPU!\n", cpu);
+ }
+
INIT_DELAYED_WORK(&krcp->monitor_work, kfree_rcu_monitor);
krcp->initialized = true;
}