Fix: do not generate packet at destroy after stop
[lttng-modules.git] / lib / ringbuffer / ring_buffer_frontend.c
index db33d04cba8dc7ba8f23d4b90ee0658cbe1888c4..42cf317a01d56482ae07401b8d1d8123b332a1d5 100644 (file)
@@ -1,7 +1,22 @@
 /*
  * ring_buffer_frontend.c
  *
- * (C) Copyright 2005-2010 - Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
+ * Copyright (C) 2005-2012 Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
+ *
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; only
+ * version 2.1 of the License.
+ *
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
+ *
  *
  * Ring buffer wait-free buffer synchronization. Producer-consumer and flight
  * recorder (overwrite) modes. See thesis:
@@ -34,8 +49,6 @@
  *   - splice one subbuffer worth of data to a pipe
  *   - splice the data from pipe to disk/network
  *   - put_subbuf
- *
- * Dual LGPL v2.1/GPL v2 license.
  */
 
 #include <linux/delay.h>
@@ -47,6 +60,8 @@
 #include "../../wrapper/ringbuffer/frontend.h"
 #include "../../wrapper/ringbuffer/iterator.h"
 #include "../../wrapper/ringbuffer/nohz.h"
+#include "../../wrapper/atomic.h"
+#include "../../wrapper/percpu-defs.h"
 
 /*
  * Internal structure representing offsets to use at a sub-buffer switch.
@@ -76,6 +91,9 @@ EXPORT_PER_CPU_SYMBOL(lib_ring_buffer_nesting);
 static
 void lib_ring_buffer_print_errors(struct channel *chan,
                                  struct lib_ring_buffer *buf, int cpu);
+static
+void _lib_ring_buffer_switch_remote(struct lib_ring_buffer *buf,
+               enum switch_mode mode);
 
 /*
  * Must be called under cpu hotplug protection.
@@ -103,7 +121,7 @@ void lib_ring_buffer_free(struct lib_ring_buffer *buf)
 void lib_ring_buffer_reset(struct lib_ring_buffer *buf)
 {
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned int i;
 
        /*
@@ -161,7 +179,7 @@ EXPORT_SYMBOL_GPL(channel_reset);
 int lib_ring_buffer_create(struct lib_ring_buffer *buf,
                           struct channel_backend *chanb, int cpu)
 {
-       const struct lib_ring_buffer_config *config = chanb->config;
+       const struct lib_ring_buffer_config *config = &chanb->config;
        struct channel *chan = container_of(chanb, struct channel, backend);
        void *priv = chanb->priv;
        size_t subbuf_header_size;
@@ -203,6 +221,7 @@ int lib_ring_buffer_create(struct lib_ring_buffer *buf,
        }
 
        init_waitqueue_head(&buf->read_wait);
+       init_waitqueue_head(&buf->write_wait);
        raw_spin_lock_init(&buf->raw_tick_nohz_spinlock);
 
        /*
@@ -252,7 +271,7 @@ static void switch_buffer_timer(unsigned long data)
 {
        struct lib_ring_buffer *buf = (struct lib_ring_buffer *)data;
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
 
        /*
         * Only flush buffers periodically if readers are active.
@@ -274,7 +293,7 @@ static void switch_buffer_timer(unsigned long data)
 static void lib_ring_buffer_start_switch_timer(struct lib_ring_buffer *buf)
 {
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
 
        if (!chan->switch_timer_interval || buf->switch_timer_enabled)
                return;
@@ -310,7 +329,7 @@ static void read_buffer_timer(unsigned long data)
 {
        struct lib_ring_buffer *buf = (struct lib_ring_buffer *)data;
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
 
        CHAN_WARN_ON(chan, !buf->backend.allocated);
 
@@ -334,7 +353,7 @@ static void read_buffer_timer(unsigned long data)
 static void lib_ring_buffer_start_read_timer(struct lib_ring_buffer *buf)
 {
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
 
        if (config->wakeup != RING_BUFFER_WAKEUP_BY_TIMER
            || !chan->read_timer_interval
@@ -359,7 +378,7 @@ static void lib_ring_buffer_start_read_timer(struct lib_ring_buffer *buf)
 static void lib_ring_buffer_stop_read_timer(struct lib_ring_buffer *buf)
 {
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
 
        if (config->wakeup != RING_BUFFER_WAKEUP_BY_TIMER
            || !chan->read_timer_interval
@@ -388,7 +407,7 @@ static void lib_ring_buffer_stop_read_timer(struct lib_ring_buffer *buf)
  *     Returns the success/failure of the operation. (%NOTIFY_OK, %NOTIFY_BAD)
  */
 static
-int __cpuinit lib_ring_buffer_cpu_hp_callback(struct notifier_block *nb,
+int lib_ring_buffer_cpu_hp_callback(struct notifier_block *nb,
                                              unsigned long action,
                                              void *hcpu)
 {
@@ -396,7 +415,7 @@ int __cpuinit lib_ring_buffer_cpu_hp_callback(struct notifier_block *nb,
        struct channel *chan = container_of(nb, struct channel,
                                            cpu_hp_notifier);
        struct lib_ring_buffer *buf = per_cpu_ptr(chan->backend.buf, cpu);
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
 
        if (!chan->cpu_hp_enable)
                return NOTIFY_DONE;
@@ -451,7 +470,7 @@ static int notrace ring_buffer_tick_nohz_callback(struct notifier_block *nb,
 {
        struct channel *chan = container_of(nb, struct channel,
                                            tick_nohz_notifier);
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        struct lib_ring_buffer *buf;
        int cpu = smp_processor_id();
 
@@ -483,16 +502,16 @@ static int notrace ring_buffer_tick_nohz_callback(struct notifier_block *nb,
                raw_spin_unlock(&buf->raw_tick_nohz_spinlock);
                break;
        case TICK_NOHZ_STOP:
-               spin_lock(&__get_cpu_var(ring_buffer_nohz_lock));
+               spin_lock(lttng_this_cpu_ptr(&ring_buffer_nohz_lock));
                lib_ring_buffer_stop_switch_timer(buf);
                lib_ring_buffer_stop_read_timer(buf);
-               spin_unlock(&__get_cpu_var(ring_buffer_nohz_lock));
+               spin_unlock(lttng_this_cpu_ptr(&ring_buffer_nohz_lock));
                break;
        case TICK_NOHZ_RESTART:
-               spin_lock(&__get_cpu_var(ring_buffer_nohz_lock));
+               spin_lock(lttng_this_cpu_ptr(&ring_buffer_nohz_lock));
                lib_ring_buffer_start_read_timer(buf);
                lib_ring_buffer_start_switch_timer(buf);
-               spin_unlock(&__get_cpu_var(ring_buffer_nohz_lock));
+               spin_unlock(lttng_this_cpu_ptr(&ring_buffer_nohz_lock));
                break;
        }
 
@@ -523,7 +542,7 @@ void notrace lib_ring_buffer_tick_nohz_restart(void)
  */
 static void channel_unregister_notifiers(struct channel *chan)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        int cpu;
 
        channel_iterator_unregister_notifiers(chan);
@@ -569,8 +588,68 @@ static void channel_unregister_notifiers(struct channel *chan)
        channel_backend_unregister_notifiers(&chan->backend);
 }
 
+static void lib_ring_buffer_set_quiescent(struct lib_ring_buffer *buf)
+{
+       if (!buf->quiescent) {
+               buf->quiescent = true;
+               _lib_ring_buffer_switch_remote(buf, SWITCH_FLUSH);
+       }
+}
+
+static void lib_ring_buffer_clear_quiescent(struct lib_ring_buffer *buf)
+{
+       buf->quiescent = false;
+}
+
+void lib_ring_buffer_set_quiescent_channel(struct channel *chan)
+{
+       int cpu;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
+
+       if (config->alloc == RING_BUFFER_ALLOC_PER_CPU) {
+               get_online_cpus();
+               for_each_channel_cpu(cpu, chan) {
+                       struct lib_ring_buffer *buf = per_cpu_ptr(chan->backend.buf,
+                                                             cpu);
+
+                       lib_ring_buffer_set_quiescent(buf);
+               }
+               put_online_cpus();
+       } else {
+               struct lib_ring_buffer *buf = chan->backend.buf;
+
+               lib_ring_buffer_set_quiescent(buf);
+       }
+}
+EXPORT_SYMBOL_GPL(lib_ring_buffer_set_quiescent_channel);
+
+void lib_ring_buffer_clear_quiescent_channel(struct channel *chan)
+{
+       int cpu;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
+
+       if (config->alloc == RING_BUFFER_ALLOC_PER_CPU) {
+               get_online_cpus();
+               for_each_channel_cpu(cpu, chan) {
+                       struct lib_ring_buffer *buf = per_cpu_ptr(chan->backend.buf,
+                                                             cpu);
+
+                       lib_ring_buffer_clear_quiescent(buf);
+               }
+               put_online_cpus();
+       } else {
+               struct lib_ring_buffer *buf = chan->backend.buf;
+
+               lib_ring_buffer_clear_quiescent(buf);
+       }
+}
+EXPORT_SYMBOL_GPL(lib_ring_buffer_clear_quiescent_channel);
+
 static void channel_free(struct channel *chan)
 {
+       if (chan->backend.release_priv_ops) {
+               chan->backend.release_priv_ops(chan->backend.priv_ops);
+       }
        channel_iterator_free(chan);
        channel_backend_free(&chan->backend);
        kfree(chan);
@@ -707,7 +786,7 @@ void channel_release(struct kref *kref)
 void *channel_destroy(struct channel *chan)
 {
        int cpu;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        void *priv;
 
        channel_unregister_notifiers(chan);
@@ -726,7 +805,7 @@ void *channel_destroy(struct channel *chan)
                                                           chan->backend.priv,
                                                           cpu);
                        if (buf->backend.allocated)
-                               lib_ring_buffer_switch_slow(buf, SWITCH_FLUSH);
+                               lib_ring_buffer_set_quiescent(buf);
                        /*
                         * Perform flush before writing to finalized.
                         */
@@ -740,7 +819,7 @@ void *channel_destroy(struct channel *chan)
                if (config->cb.buffer_finalize)
                        config->cb.buffer_finalize(buf, chan->backend.priv, -1);
                if (buf->backend.allocated)
-                       lib_ring_buffer_switch_slow(buf, SWITCH_FLUSH);
+                       lib_ring_buffer_set_quiescent(buf);
                /*
                 * Perform flush before writing to finalized.
                 */
@@ -751,8 +830,8 @@ void *channel_destroy(struct channel *chan)
        ACCESS_ONCE(chan->finalized) = 1;
        wake_up_interruptible(&chan->hp_wait);
        wake_up_interruptible(&chan->read_wait);
-       kref_put(&chan->ref, channel_release);
        priv = chan->backend.priv;
+       kref_put(&chan->ref, channel_release);
        return priv;
 }
 EXPORT_SYMBOL_GPL(channel_destroy);
@@ -775,7 +854,7 @@ int lib_ring_buffer_open_read(struct lib_ring_buffer *buf)
        if (!atomic_long_add_unless(&buf->active_readers, 1, 1))
                return -EBUSY;
        kref_get(&chan->ref);
-       smp_mb__after_atomic_inc();
+       lttng_smp_mb__after_atomic();
        return 0;
 }
 EXPORT_SYMBOL_GPL(lib_ring_buffer_open_read);
@@ -785,7 +864,7 @@ void lib_ring_buffer_release_read(struct lib_ring_buffer *buf)
        struct channel *chan = buf->backend.chan;
 
        CHAN_WARN_ON(chan, atomic_long_read(&buf->active_readers) != 1);
-       smp_mb__before_atomic_dec();
+       lttng_smp_mb__before_atomic();
        atomic_long_dec(&buf->active_readers);
        kref_put(&chan->ref, channel_release);
 }
@@ -817,7 +896,7 @@ int lib_ring_buffer_snapshot(struct lib_ring_buffer *buf,
                             unsigned long *consumed, unsigned long *produced)
 {
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned long consumed_cur, write_offset;
        int finalized;
 
@@ -867,6 +946,8 @@ EXPORT_SYMBOL_GPL(lib_ring_buffer_snapshot);
 
 /**
  * lib_ring_buffer_put_snapshot - move consumed counter forward
+ *
+ * Should only be called from consumer context.
  * @buf: ring buffer
  * @consumed_new: new consumed count value
  */
@@ -888,6 +969,8 @@ void lib_ring_buffer_move_consumer(struct lib_ring_buffer *buf,
        while ((long) consumed - (long) consumed_new < 0)
                consumed = atomic_long_cmpxchg(&buf->consumed, consumed,
                                               consumed_new);
+       /* Wake-up the metadata producer */
+       wake_up_interruptible(&buf->write_wait);
 }
 EXPORT_SYMBOL_GPL(lib_ring_buffer_move_consumer);
 
@@ -904,11 +987,18 @@ int lib_ring_buffer_get_subbuf(struct lib_ring_buffer *buf,
                               unsigned long consumed)
 {
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned long consumed_cur, consumed_idx, commit_count, write_offset;
        int ret;
        int finalized;
 
+       if (buf->get_subbuf) {
+               /*
+                * Reader is trying to get a subbuffer twice.
+                */
+               CHAN_WARN_ON(chan, 1);
+               return -EBUSY;
+       }
 retry:
        finalized = ACCESS_ONCE(buf->finalized);
        /*
@@ -997,7 +1087,7 @@ retry:
         */
        if (((commit_count - chan->backend.subbuf_size)
             & chan->commit_count_mask)
-           - (buf_trunc(consumed_cur, chan)
+           - (buf_trunc(consumed, chan)
               >> chan->backend.num_subbuf_order)
            != 0)
                goto nodata;
@@ -1006,7 +1096,7 @@ retry:
         * Check that we are not about to read the same subbuffer in
         * which the writer head is.
         */
-       if (subbuf_trunc(write_offset, chan) - subbuf_trunc(consumed_cur, chan)
+       if (subbuf_trunc(write_offset, chan) - subbuf_trunc(consumed, chan)
            == 0)
                goto nodata;
 
@@ -1050,7 +1140,7 @@ void lib_ring_buffer_put_subbuf(struct lib_ring_buffer *buf)
 {
        struct lib_ring_buffer_backend *bufb = &buf->backend;
        struct channel *chan = bufb->chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned long read_sb_bindex, consumed_idx, consumed;
 
        CHAN_WARN_ON(chan, atomic_long_read(&buf->active_readers) != 1);
@@ -1109,7 +1199,7 @@ void lib_ring_buffer_print_subbuffer_errors(struct lib_ring_buffer *buf,
                                            unsigned long cons_offset,
                                            int cpu)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned long cons_idx, commit_count, commit_count_sb;
 
        cons_idx = subbuf_index(cons_offset, chan);
@@ -1135,15 +1225,9 @@ void lib_ring_buffer_print_buffer_errors(struct lib_ring_buffer *buf,
                                         struct channel *chan,
                                         void *priv, int cpu)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned long write_offset, cons_offset;
 
-       /*
-        * Can be called in the error path of allocation when
-        * trans_channel_data is not yet set.
-        */
-       if (!chan)
-               return;
        /*
         * No need to order commit_count, write_offset and cons_offset reads
         * because we execute at teardown when no more writer nor reader
@@ -1152,7 +1236,7 @@ void lib_ring_buffer_print_buffer_errors(struct lib_ring_buffer *buf,
        write_offset = v_read(config, &buf->offset);
        cons_offset = atomic_long_read(&buf->consumed);
        if (write_offset != cons_offset)
-               printk(KERN_WARNING
+               printk(KERN_DEBUG
                       "ring buffer %s, cpu %d: "
                       "non-consumed data\n"
                       "  [ %lu bytes written, %lu bytes read ]\n",
@@ -1171,27 +1255,34 @@ static
 void lib_ring_buffer_print_errors(struct channel *chan,
                                  struct lib_ring_buffer *buf, int cpu)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        void *priv = chan->backend.priv;
 
-       printk(KERN_DEBUG "ring buffer %s, cpu %d: %lu records written, "
-                         "%lu records overrun\n",
-                         chan->backend.name, cpu,
-                         v_read(config, &buf->records_count),
-                         v_read(config, &buf->records_overrun));
-
-       if (v_read(config, &buf->records_lost_full)
-           || v_read(config, &buf->records_lost_wrap)
-           || v_read(config, &buf->records_lost_big))
-               printk(KERN_WARNING
-                      "ring buffer %s, cpu %d: records were lost. Caused by:\n"
-                      "  [ %lu buffer full, %lu nest buffer wrap-around, "
-                      "%lu event too big ]\n",
-                      chan->backend.name, cpu,
-                      v_read(config, &buf->records_lost_full),
-                      v_read(config, &buf->records_lost_wrap),
-                      v_read(config, &buf->records_lost_big));
-
+       if (!strcmp(chan->backend.name, "relay-metadata")) {
+               printk(KERN_DEBUG "ring buffer %s: %lu records written, "
+                       "%lu records overrun\n",
+                       chan->backend.name,
+                       v_read(config, &buf->records_count),
+                       v_read(config, &buf->records_overrun));
+       } else {
+               printk(KERN_DEBUG "ring buffer %s, cpu %d: %lu records written, "
+                       "%lu records overrun\n",
+                       chan->backend.name, cpu,
+                       v_read(config, &buf->records_count),
+                       v_read(config, &buf->records_overrun));
+
+               if (v_read(config, &buf->records_lost_full)
+                   || v_read(config, &buf->records_lost_wrap)
+                   || v_read(config, &buf->records_lost_big))
+                       printk(KERN_WARNING
+                               "ring buffer %s, cpu %d: records were lost. Caused by:\n"
+                               "  [ %lu buffer full, %lu nest buffer wrap-around, "
+                               "%lu event too big ]\n",
+                               chan->backend.name, cpu,
+                               v_read(config, &buf->records_lost_full),
+                               v_read(config, &buf->records_lost_wrap),
+                               v_read(config, &buf->records_lost_big));
+       }
        lib_ring_buffer_print_buffer_errors(buf, chan, priv, cpu);
 }
 
@@ -1206,7 +1297,7 @@ void lib_ring_buffer_switch_old_start(struct lib_ring_buffer *buf,
                                      struct switch_offsets *offsets,
                                      u64 tsc)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned long oldidx = subbuf_index(offsets->old, chan);
        unsigned long commit_count;
 
@@ -1230,10 +1321,10 @@ void lib_ring_buffer_switch_old_start(struct lib_ring_buffer *buf,
        commit_count = v_read(config, &buf->commit_hot[oldidx].cc);
        /* Check if the written buffer has to be delivered */
        lib_ring_buffer_check_deliver(config, buf, chan, offsets->old,
-                                     commit_count, oldidx);
+                                     commit_count, oldidx, tsc);
        lib_ring_buffer_write_commit_counter(config, buf, chan, oldidx,
-                                            offsets->old, commit_count,
-                                            config->cb.subbuffer_header_size());
+                       offsets->old + config->cb.subbuffer_header_size(),
+                       commit_count);
 }
 
 /*
@@ -1250,7 +1341,7 @@ void lib_ring_buffer_switch_old_end(struct lib_ring_buffer *buf,
                                    struct switch_offsets *offsets,
                                    u64 tsc)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned long oldidx = subbuf_index(offsets->old - 1, chan);
        unsigned long commit_count, padding_size, data_size;
 
@@ -1274,10 +1365,9 @@ void lib_ring_buffer_switch_old_end(struct lib_ring_buffer *buf,
        v_add(config, padding_size, &buf->commit_hot[oldidx].cc);
        commit_count = v_read(config, &buf->commit_hot[oldidx].cc);
        lib_ring_buffer_check_deliver(config, buf, chan, offsets->old - 1,
-                                     commit_count, oldidx);
+                                     commit_count, oldidx, tsc);
        lib_ring_buffer_write_commit_counter(config, buf, chan, oldidx,
-                                            offsets->old, commit_count,
-                                            padding_size);
+                       offsets->old + padding_size, commit_count);
 }
 
 /*
@@ -1293,7 +1383,7 @@ void lib_ring_buffer_switch_new_start(struct lib_ring_buffer *buf,
                                      struct switch_offsets *offsets,
                                      u64 tsc)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        unsigned long beginidx = subbuf_index(offsets->begin, chan);
        unsigned long commit_count;
 
@@ -1317,17 +1407,19 @@ void lib_ring_buffer_switch_new_start(struct lib_ring_buffer *buf,
        commit_count = v_read(config, &buf->commit_hot[beginidx].cc);
        /* Check if the written buffer has to be delivered */
        lib_ring_buffer_check_deliver(config, buf, chan, offsets->begin,
-                                     commit_count, beginidx);
+                                     commit_count, beginidx, tsc);
        lib_ring_buffer_write_commit_counter(config, buf, chan, beginidx,
-                                            offsets->begin, commit_count,
-                                            config->cb.subbuffer_header_size());
+                       offsets->begin + config->cb.subbuffer_header_size(),
+                       commit_count);
 }
 
 /*
  * lib_ring_buffer_switch_new_end: finish switching current subbuffer
  *
- * The only remaining threads could be the ones with pending commits. They will
- * have to do the deliver themselves.
+ * Calls subbuffer_set_data_size() to set the data size of the current
+ * sub-buffer. We do not need to perform check_deliver nor commit here,
+ * since this task will be done by the "commit" of the event for which
+ * we are currently doing the space reservation.
  */
 static
 void lib_ring_buffer_switch_new_end(struct lib_ring_buffer *buf,
@@ -1335,34 +1427,12 @@ void lib_ring_buffer_switch_new_end(struct lib_ring_buffer *buf,
                                            struct switch_offsets *offsets,
                                            u64 tsc)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
-       unsigned long endidx = subbuf_index(offsets->end - 1, chan);
-       unsigned long commit_count, padding_size, data_size;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
+       unsigned long endidx, data_size;
 
+       endidx = subbuf_index(offsets->end - 1, chan);
        data_size = subbuf_offset(offsets->end - 1, chan) + 1;
-       padding_size = chan->backend.subbuf_size - data_size;
        subbuffer_set_data_size(config, &buf->backend, endidx, data_size);
-
-       /*
-        * Order all writes to buffer before the commit count update that will
-        * determine that the subbuffer is full.
-        */
-       if (config->ipi == RING_BUFFER_IPI_BARRIER) {
-               /*
-                * Must write slot data before incrementing commit count.  This
-                * compiler barrier is upgraded into a smp_mb() by the IPI sent
-                * by get_subbuf().
-                */
-               barrier();
-       } else
-               smp_wmb();
-       v_add(config, padding_size, &buf->commit_hot[endidx].cc);
-       commit_count = v_read(config, &buf->commit_hot[endidx].cc);
-       lib_ring_buffer_check_deliver(config, buf, chan, offsets->end - 1,
-                                 commit_count, endidx);
-       lib_ring_buffer_write_commit_counter(config, buf, chan, endidx,
-                                            offsets->end, commit_count,
-                                            padding_size);
 }
 
 /*
@@ -1377,8 +1447,8 @@ int lib_ring_buffer_try_switch_slow(enum switch_mode mode,
                                    struct switch_offsets *offsets,
                                    u64 *tsc)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
-       unsigned long off;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
+       unsigned long off, reserve_commit_diff;
 
        offsets->begin = v_read(config, &buf->offset);
        offsets->old = offsets->begin;
@@ -1403,23 +1473,68 @@ int lib_ring_buffer_try_switch_slow(enum switch_mode mode,
         * (records and header timestamps) are visible to the reader. This is
         * required for quiescence guarantees for the fusion merge.
         */
-       if (mode == SWITCH_FLUSH || off > 0) {
-               if (unlikely(off == 0)) {
-                       /*
-                        * The client does not save any header information.
-                        * Don't switch empty subbuffer on finalize, because it
-                        * is invalid to deliver a completely empty subbuffer.
-                        */
-                       if (!config->cb.subbuffer_header_size())
+       if (mode != SWITCH_FLUSH && !off)
+               return -1;      /* we do not have to switch : buffer is empty */
+
+       if (unlikely(off == 0)) {
+               unsigned long sb_index, commit_count;
+
+               /*
+                * We are performing a SWITCH_FLUSH. At this stage, there are no
+                * concurrent writes into the buffer.
+                *
+                * The client does not save any header information.  Don't
+                * switch empty subbuffer on finalize, because it is invalid to
+                * deliver a completely empty subbuffer.
+                */
+               if (!config->cb.subbuffer_header_size())
+                       return -1;
+
+               /* Test new buffer integrity */
+               sb_index = subbuf_index(offsets->begin, chan);
+               commit_count = v_read(config,
+                               &buf->commit_cold[sb_index].cc_sb);
+               reserve_commit_diff =
+                 (buf_trunc(offsets->begin, chan)
+                  >> chan->backend.num_subbuf_order)
+                 - (commit_count & chan->commit_count_mask);
+               if (likely(reserve_commit_diff == 0)) {
+                       /* Next subbuffer not being written to. */
+                       if (unlikely(config->mode != RING_BUFFER_OVERWRITE &&
+                               subbuf_trunc(offsets->begin, chan)
+                                - subbuf_trunc((unsigned long)
+                                    atomic_long_read(&buf->consumed), chan)
+                               >= chan->backend.buf_size)) {
+                               /*
+                                * We do not overwrite non consumed buffers
+                                * and we are full : don't switch.
+                                */
                                return -1;
+                       } else {
+                               /*
+                                * Next subbuffer not being written to, and we
+                                * are either in overwrite mode or the buffer is
+                                * not full. It's safe to write in this new
+                                * subbuffer.
+                                */
+                       }
+               } else {
                        /*
-                        * Need to write the subbuffer start header on finalize.
+                        * Next subbuffer reserve offset does not match the
+                        * commit offset. Don't perform switch in
+                        * producer-consumer and overwrite mode.  Caused by
+                        * either a writer OOPS or too many nested writes over a
+                        * reserve/commit pair.
                         */
-                       offsets->switch_old_start = 1;
+                       return -1;
                }
-               offsets->begin = subbuf_align(offsets->begin, chan);
-       } else
-               return -1;      /* we do not have to switch : buffer is empty */
+
+               /*
+                * Need to write the subbuffer start header on finalize.
+                */
+               offsets->switch_old_start = 1;
+       }
+       offsets->begin = subbuf_align(offsets->begin, chan);
        /* Note: old points to the next subbuf at offset 0 */
        offsets->end = offsets->begin;
        return 0;
@@ -1436,7 +1551,7 @@ int lib_ring_buffer_try_switch_slow(enum switch_mode mode,
 void lib_ring_buffer_switch_slow(struct lib_ring_buffer *buf, enum switch_mode mode)
 {
        struct channel *chan = buf->backend.chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        struct switch_offsets offsets;
        unsigned long oldidx;
        u64 tsc;
@@ -1484,6 +1599,54 @@ void lib_ring_buffer_switch_slow(struct lib_ring_buffer *buf, enum switch_mode m
 }
 EXPORT_SYMBOL_GPL(lib_ring_buffer_switch_slow);
 
+static void remote_switch(void *info)
+{
+       struct lib_ring_buffer *buf = info;
+
+       lib_ring_buffer_switch_slow(buf, SWITCH_ACTIVE);
+}
+
+static void _lib_ring_buffer_switch_remote(struct lib_ring_buffer *buf,
+               enum switch_mode mode)
+{
+       struct channel *chan = buf->backend.chan;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
+       int ret;
+
+       /*
+        * With global synchronization we don't need to use the IPI scheme.
+        */
+       if (config->sync == RING_BUFFER_SYNC_GLOBAL) {
+               lib_ring_buffer_switch_slow(buf, mode);
+               return;
+       }
+
+       /*
+        * Taking lock on CPU hotplug to ensure two things: first, that the
+        * target cpu is not taken concurrently offline while we are within
+        * smp_call_function_single() (I don't trust that get_cpu() on the
+        * _local_ CPU actually inhibit CPU hotplug for the _remote_ CPU (to be
+        * confirmed)). Secondly, if it happens that the CPU is not online, our
+        * own call to lib_ring_buffer_switch_slow() needs to be protected from
+        * CPU hotplug handlers, which can also perform a remote subbuffer
+        * switch.
+        */
+       get_online_cpus();
+       ret = smp_call_function_single(buf->backend.cpu,
+                                remote_switch, buf, 1);
+       if (ret) {
+               /* Remote CPU is offline, do it ourself. */
+               lib_ring_buffer_switch_slow(buf, mode);
+       }
+       put_online_cpus();
+}
+
+void lib_ring_buffer_switch_remote(struct lib_ring_buffer *buf)
+{
+       _lib_ring_buffer_switch_remote(buf, SWITCH_ACTIVE);
+}
+EXPORT_SYMBOL_GPL(lib_ring_buffer_switch_remote);
+
 /*
  * Returns :
  * 0 if ok
@@ -1497,10 +1660,11 @@ int lib_ring_buffer_try_reserve_slow(struct lib_ring_buffer *buf,
                                     struct switch_offsets *offsets,
                                     struct lib_ring_buffer_ctx *ctx)
 {
-       const struct lib_ring_buffer_config *config = chan->backend.config;
-       unsigned long reserve_commit_diff;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
+       unsigned long reserve_commit_diff, offset_cmp;
 
-       offsets->begin = v_read(config, &buf->offset);
+retry:
+       offsets->begin = offset_cmp = v_read(config, &buf->offset);
        offsets->old = offsets->begin;
        offsets->switch_new_start = 0;
        offsets->switch_new_end = 0;
@@ -1532,7 +1696,7 @@ int lib_ring_buffer_try_reserve_slow(struct lib_ring_buffer *buf,
                }
        }
        if (unlikely(offsets->switch_new_start)) {
-               unsigned long sb_index;
+               unsigned long sb_index, commit_count;
 
                /*
                 * We are typically not filling the previous buffer completely.
@@ -1543,12 +1707,31 @@ int lib_ring_buffer_try_reserve_slow(struct lib_ring_buffer *buf,
                                 + config->cb.subbuffer_header_size();
                /* Test new buffer integrity */
                sb_index = subbuf_index(offsets->begin, chan);
+               /*
+                * Read buf->offset before buf->commit_cold[sb_index].cc_sb.
+                * lib_ring_buffer_check_deliver() has the matching
+                * memory barriers required around commit_cold cc_sb
+                * updates to ensure reserve and commit counter updates
+                * are not seen reordered when updated by another CPU.
+                */
+               smp_rmb();
+               commit_count = v_read(config,
+                               &buf->commit_cold[sb_index].cc_sb);
+               /* Read buf->commit_cold[sb_index].cc_sb before buf->offset. */
+               smp_rmb();
+               if (unlikely(offset_cmp != v_read(config, &buf->offset))) {
+                       /*
+                        * The reserve counter have been concurrently updated
+                        * while we read the commit counter. This means the
+                        * commit counter we read might not match buf->offset
+                        * due to concurrent update. We therefore need to retry.
+                        */
+                       goto retry;
+               }
                reserve_commit_diff =
                  (buf_trunc(offsets->begin, chan)
                   >> chan->backend.num_subbuf_order)
-                 - ((unsigned long) v_read(config,
-                                           &buf->commit_cold[sb_index].cc_sb)
-                    & chan->commit_count_mask);
+                 - (commit_count & chan->commit_count_mask);
                if (likely(reserve_commit_diff == 0)) {
                        /* Next subbuffer not being written to. */
                        if (unlikely(config->mode != RING_BUFFER_OVERWRITE &&
@@ -1573,9 +1756,10 @@ int lib_ring_buffer_try_reserve_slow(struct lib_ring_buffer *buf,
                } else {
                        /*
                         * Next subbuffer reserve offset does not match the
-                        * commit offset. Drop record in producer-consumer and
-                        * overwrite mode. Caused by either a writer OOPS or too
-                        * many nested writes over a reserve/commit pair.
+                        * commit offset, and this did not involve update to the
+                        * reserve counter. Drop record in producer-consumer and
+                        * overwrite mode.  Caused by either a writer OOPS or
+                        * too many nested writes over a reserve/commit pair.
                         */
                        v_inc(config, &buf->records_lost_wrap);
                        return -EIO;
@@ -1632,7 +1816,7 @@ int lib_ring_buffer_try_reserve_slow(struct lib_ring_buffer *buf,
 int lib_ring_buffer_reserve_slow(struct lib_ring_buffer_ctx *ctx)
 {
        struct channel *chan = ctx->chan;
-       const struct lib_ring_buffer_config *config = chan->backend.config;
+       const struct lib_ring_buffer_config *config = &chan->backend.config;
        struct lib_ring_buffer *buf;
        struct switch_offsets offsets;
        int ret;
@@ -1697,3 +1881,20 @@ int lib_ring_buffer_reserve_slow(struct lib_ring_buffer_ctx *ctx)
        return 0;
 }
 EXPORT_SYMBOL_GPL(lib_ring_buffer_reserve_slow);
+
+int __init init_lib_ring_buffer_frontend(void)
+{
+       int cpu;
+
+       for_each_possible_cpu(cpu)
+               spin_lock_init(&per_cpu(ring_buffer_nohz_lock, cpu));
+       return 0;
+}
+
+module_init(init_lib_ring_buffer_frontend);
+
+void __exit exit_lib_ring_buffer_frontend(void)
+{
+}
+
+module_exit(exit_lib_ring_buffer_frontend);
This page took 0.034457 seconds and 4 git commands to generate.