rseq: output whether configure finds rseq syscall
[lttng-ust.git] / liblttng-ust / lttng-ring-buffer-client.h
index 3f1e2656437a13b20d4fcbfd8f2b429622b83c78..1a15cfab0139e68af24baea6b3c0fed67e5e9c61 100644 (file)
 #include "clock.h"
 #include "lttng-tracer.h"
 #include "../libringbuffer/frontend_types.h"
+#include "../libringbuffer/rseq.h"
 
 #define LTTNG_COMPACT_EVENT_BITS       5
 #define LTTNG_COMPACT_TSC_BITS         27
+#define LTTNG_RSEQ_ATTEMPTS            8
+
+enum app_ctx_mode {
+       APP_CTX_DISABLED,
+       APP_CTX_ENABLED,
+};
 
 /*
  * Keep the natural field alignment for _each field_ within this structure if
@@ -45,6 +52,7 @@ struct packet_header {
                                         */
        uint8_t uuid[LTTNG_UST_UUID_LEN];
        uint32_t stream_id;
+       uint64_t stream_instance_id;
 
        struct {
                /* Stream packet context */
@@ -52,6 +60,7 @@ struct packet_header {
                uint64_t timestamp_end;         /* Cycle count at subbuffer end */
                uint64_t content_size;          /* Size of data in subbuffer */
                uint64_t packet_size;           /* Subbuffer size (include padding) */
+               uint64_t packet_seq_num;        /* Packet sequence number */
                unsigned long events_discarded; /*
                                                 * Events lost in this subbuffer since
                                                 * the beginning of the trace.
@@ -69,7 +78,8 @@ static inline uint64_t lib_ring_buffer_clock_read(struct channel *chan)
 }
 
 static inline
-size_t ctx_get_size(size_t offset, struct lttng_ctx *ctx)
+size_t ctx_get_size(size_t offset, struct lttng_ctx *ctx,
+               enum app_ctx_mode mode)
 {
        int i;
        size_t orig_offset = offset;
@@ -77,23 +87,62 @@ size_t ctx_get_size(size_t offset, struct lttng_ctx *ctx)
        if (caa_likely(!ctx))
                return 0;
        offset += lib_ring_buffer_align(offset, ctx->largest_align);
-       for (i = 0; i < ctx->nr_fields; i++)
-               offset += ctx->fields[i].get_size(offset);
+       for (i = 0; i < ctx->nr_fields; i++) {
+               if (mode == APP_CTX_ENABLED) {
+                       offset += ctx->fields[i].get_size(&ctx->fields[i], offset);
+               } else {
+                       if (lttng_context_is_app(ctx->fields[i].event_field.name)) {
+                               /*
+                                * Before UST 2.8, we cannot use the
+                                * application context, because we
+                                * cannot trust that the handler used
+                                * for get_size is the same used for
+                                * ctx_record, which would result in
+                                * corrupted traces when tracing
+                                * concurrently with application context
+                                * register/unregister.
+                                */
+                               offset += lttng_ust_dummy_get_size(&ctx->fields[i], offset);
+                       } else {
+                               offset += ctx->fields[i].get_size(&ctx->fields[i], offset);
+                       }
+               }
+       }
        return offset - orig_offset;
 }
 
 static inline
 void ctx_record(struct lttng_ust_lib_ring_buffer_ctx *bufctx,
                struct lttng_channel *chan,
-               struct lttng_ctx *ctx)
+               struct lttng_ctx *ctx,
+               enum app_ctx_mode mode)
 {
        int i;
 
        if (caa_likely(!ctx))
                return;
        lib_ring_buffer_align_ctx(bufctx, ctx->largest_align);
-       for (i = 0; i < ctx->nr_fields; i++)
-               ctx->fields[i].record(&ctx->fields[i], bufctx, chan);
+       for (i = 0; i < ctx->nr_fields; i++) {
+               if (mode == APP_CTX_ENABLED) {
+                       ctx->fields[i].record(&ctx->fields[i], bufctx, chan);
+               } else {
+                       if (lttng_context_is_app(ctx->fields[i].event_field.name)) {
+                               /*
+                                * Before UST 2.8, we cannot use the
+                                * application context, because we
+                                * cannot trust that the handler used
+                                * for get_size is the same used for
+                                * ctx_record, which would result in
+                                * corrupted traces when tracing
+                                * concurrently with application context
+                                * register/unregister.
+                                */
+                               lttng_ust_dummy_record(&ctx->fields[i], bufctx, chan);
+                       } else {
+                               ctx->fields[i].record(&ctx->fields[i], bufctx, chan);
+                       }
+               }
+       }
 }
 
 /*
@@ -117,6 +166,7 @@ size_t record_header_size(const struct lttng_ust_lib_ring_buffer_config *config,
 {
        struct lttng_channel *lttng_chan = channel_get_private(chan);
        struct lttng_event *event = ctx->priv;
+       struct lttng_stack_ctx *lttng_ctx = ctx->priv2;
        size_t orig_offset = offset;
        size_t padding;
 
@@ -155,9 +205,15 @@ size_t record_header_size(const struct lttng_ust_lib_ring_buffer_config *config,
                padding = 0;
                WARN_ON_ONCE(1);
        }
-       offset += ctx_get_size(offset, event->ctx);
-       offset += ctx_get_size(offset, lttng_chan->ctx);
-
+       if (lttng_ctx) {
+               /* 2.8+ probe ABI. */
+               offset += ctx_get_size(offset, lttng_ctx->chan_ctx, APP_CTX_ENABLED);
+               offset += ctx_get_size(offset, lttng_ctx->event_ctx, APP_CTX_ENABLED);
+       } else {
+               /* Pre 2.8 probe ABI. */
+               offset += ctx_get_size(offset, lttng_chan->ctx, APP_CTX_DISABLED);
+               offset += ctx_get_size(offset, event->ctx, APP_CTX_DISABLED);
+       }
        *pre_header_padding = padding;
        return offset - orig_offset;
 }
@@ -186,6 +242,7 @@ void lttng_write_event_header(const struct lttng_ust_lib_ring_buffer_config *con
 {
        struct lttng_channel *lttng_chan = channel_get_private(ctx->chan);
        struct lttng_event *event = ctx->priv;
+       struct lttng_stack_ctx *lttng_ctx = ctx->priv2;
 
        if (caa_unlikely(ctx->rflags))
                goto slow_path;
@@ -220,8 +277,15 @@ void lttng_write_event_header(const struct lttng_ust_lib_ring_buffer_config *con
                WARN_ON_ONCE(1);
        }
 
-       ctx_record(ctx, lttng_chan, lttng_chan->ctx);
-       ctx_record(ctx, lttng_chan, event->ctx);
+       if (lttng_ctx) {
+               /* 2.8+ probe ABI. */
+               ctx_record(ctx, lttng_chan, lttng_ctx->chan_ctx, APP_CTX_ENABLED);
+               ctx_record(ctx, lttng_chan, lttng_ctx->event_ctx, APP_CTX_ENABLED);
+       } else {
+               /* Pre 2.8 probe ABI. */
+               ctx_record(ctx, lttng_chan, lttng_chan->ctx, APP_CTX_DISABLED);
+               ctx_record(ctx, lttng_chan, event->ctx, APP_CTX_DISABLED);
+       }
        lib_ring_buffer_align_ctx(ctx, ctx->largest_align);
 
        return;
@@ -237,6 +301,7 @@ void lttng_write_event_header_slow(const struct lttng_ust_lib_ring_buffer_config
 {
        struct lttng_channel *lttng_chan = channel_get_private(ctx->chan);
        struct lttng_event *event = ctx->priv;
+       struct lttng_stack_ctx *lttng_ctx = ctx->priv2;
 
        switch (lttng_chan->header_type) {
        case 1: /* compact */
@@ -293,8 +358,15 @@ void lttng_write_event_header_slow(const struct lttng_ust_lib_ring_buffer_config
        default:
                WARN_ON_ONCE(1);
        }
-       ctx_record(ctx, lttng_chan, lttng_chan->ctx);
-       ctx_record(ctx, lttng_chan, event->ctx);
+       if (lttng_ctx) {
+               /* 2.8+ probe ABI. */
+               ctx_record(ctx, lttng_chan, lttng_ctx->chan_ctx, APP_CTX_ENABLED);
+               ctx_record(ctx, lttng_chan, lttng_ctx->event_ctx, APP_CTX_ENABLED);
+       } else {
+               /* Pre 2.8 probe ABI. */
+               ctx_record(ctx, lttng_chan, lttng_chan->ctx, APP_CTX_DISABLED);
+               ctx_record(ctx, lttng_chan, event->ctx, APP_CTX_DISABLED);
+       }
        lib_ring_buffer_align_ctx(ctx, ctx->largest_align);
 }
 
@@ -338,6 +410,7 @@ static void client_buffer_begin(struct lttng_ust_lib_ring_buffer *buf, uint64_t
                                subbuf_idx * chan->backend.subbuf_size,
                                handle);
        struct lttng_channel *lttng_chan = channel_get_private(chan);
+       uint64_t cnt = shmp_index(handle, buf->backend.buf_cnt, subbuf_idx)->seq_cnt;
 
        assert(header);
        if (!header)
@@ -345,10 +418,12 @@ static void client_buffer_begin(struct lttng_ust_lib_ring_buffer *buf, uint64_t
        header->magic = CTF_MAGIC_NUMBER;
        memcpy(header->uuid, lttng_chan->uuid, sizeof(lttng_chan->uuid));
        header->stream_id = lttng_chan->id;
+       header->stream_instance_id = buf->backend.cpu;
        header->ctx.timestamp_begin = tsc;
        header->ctx.timestamp_end = 0;
        header->ctx.content_size = ~0ULL; /* for debugging */
        header->ctx.packet_size = ~0ULL;
+       header->ctx.packet_seq_num = chan->backend.num_subbuf * cnt + subbuf_idx;
        header->ctx.events_discarded = 0;
        header->ctx.cpu_id = buf->backend.cpu;
 }
@@ -504,6 +579,28 @@ static int client_current_timestamp(struct lttng_ust_lib_ring_buffer *buf,
        return 0;
 }
 
+static int client_sequence_number(struct lttng_ust_lib_ring_buffer *buf,
+               struct lttng_ust_shm_handle *handle,
+               uint64_t *seq)
+{
+       struct packet_header *header;
+
+       header = client_packet_header(buf, handle);
+       *seq = header->ctx.packet_seq_num;
+       return 0;
+}
+
+static int client_instance_id(struct lttng_ust_lib_ring_buffer *buf,
+               struct lttng_ust_shm_handle *handle,
+               uint64_t *id)
+{
+       struct packet_header *header;
+
+       header = client_packet_header(buf, handle);
+       *id = header->stream_instance_id;
+       return 0;
+}
+
 static const
 struct lttng_ust_client_lib_ring_buffer_client_cb client_cb = {
        .parent = {
@@ -524,6 +621,8 @@ struct lttng_ust_client_lib_ring_buffer_client_cb client_cb = {
        .packet_size = client_packet_size,
        .stream_id = client_stream_id,
        .current_timestamp = client_current_timestamp,
+       .sequence_number = client_sequence_number,
+       .instance_id = client_instance_id,
 };
 
 static const struct lttng_ust_lib_ring_buffer_config client_config = {
@@ -539,7 +638,7 @@ static const struct lttng_ust_lib_ring_buffer_config client_config = {
 
        .tsc_bits = LTTNG_COMPACT_TSC_BITS,
        .alloc = RING_BUFFER_ALLOC_PER_CPU,
-       .sync = RING_BUFFER_SYNC_GLOBAL,
+       .sync = RING_BUFFER_SYNC_PER_CPU,
        .mode = RING_BUFFER_MODE_TEMPLATE,
        .backend = RING_BUFFER_PAGE,
        .output = RING_BUFFER_MMAP,
@@ -592,16 +691,56 @@ void lttng_channel_destroy(struct lttng_channel *chan)
        channel_destroy(chan->chan, chan->handle, 1);
 }
 
+static
+bool refcount_get_saturate(long *ref)
+{
+       long old, _new, res;
+
+       old = uatomic_read(ref);
+       for (;;) {
+               if (old == LONG_MAX) {
+                       return false;   /* Saturated. */
+               }
+               _new = old + 1;
+               res = uatomic_cmpxchg(ref, old, _new);
+               if (res == old) {
+                       if (_new == LONG_MAX) {
+                               return false; /* Saturation. */
+                       }
+                       return true;    /* Success. */
+               }
+               old = res;
+       }
+}
+
 static
 int lttng_event_reserve(struct lttng_ust_lib_ring_buffer_ctx *ctx,
                      uint32_t event_id)
 {
        struct lttng_channel *lttng_chan = channel_get_private(ctx->chan);
-       int ret, cpu;
+       struct lttng_rseq_state rseq_state;
+       int ret, cpu, attempt = 0;
+       bool put_fallback_ref = false;
 
-       cpu = lib_ring_buffer_get_cpu(&client_config);
-       if (cpu < 0)
+       if (lib_ring_buffer_begin(&client_config))
                return -EPERM;
+retry:
+       rseq_state = rseq_start();
+       if (caa_unlikely(rseq_cpu_at_start(rseq_state) < 0)) {
+               if (caa_unlikely(rseq_cpu_at_start(rseq_state) == -1)) {
+                       if (!rseq_register_current_thread())
+                               goto retry;
+               }
+               /* rseq is unavailable. */
+               cpu = lib_ring_buffer_get_cpu(&client_config);
+               if (caa_unlikely(cpu < 0)) {
+                       ret = -EPERM;
+                       goto end;
+               }
+       } else {
+               cpu = rseq_cpu_at_start(rseq_state);
+       }
+fallback:
        ctx->cpu = cpu;
 
        switch (lttng_chan->header_type) {
@@ -617,13 +756,47 @@ int lttng_event_reserve(struct lttng_ust_lib_ring_buffer_ctx *ctx,
                WARN_ON_ONCE(1);
        }
 
+       if (caa_likely(ctx->ctx_len
+                       >= sizeof(struct lttng_ust_lib_ring_buffer_ctx)))
+               ctx->rseq_state = rseq_state;
+
        ret = lib_ring_buffer_reserve(&client_config, ctx);
-       if (ret)
-               goto put;
+       if (caa_unlikely(ret)) {
+               if (ret == -EAGAIN) {
+                       assert(!put_fallback_ref);
+                       if (++attempt < LTTNG_RSEQ_ATTEMPTS) {
+                               caa_cpu_relax();
+                               goto retry;
+                       }
+                       put_fallback_ref = refcount_get_saturate(
+                               &lttng_chan->chan->u.reserve_fallback_ref);
+                       cpu = lib_ring_buffer_get_cpu(&client_config);
+                       if (caa_unlikely(cpu < 0)) {
+                               ret = -EPERM;
+                               goto end;
+                       }
+                       goto fallback;
+               }
+               goto end;
+       }
+       if (caa_likely(ctx->ctx_len
+                       >= sizeof(struct lttng_ust_lib_ring_buffer_ctx))) {
+               if (lib_ring_buffer_backend_get_pages(&client_config, ctx,
+                               &ctx->backend_pages)) {
+                       ret = -EPERM;
+                       goto end;
+               }
+       }
        lttng_write_event_header(&client_config, ctx, event_id);
+
+       if (caa_unlikely(put_fallback_ref))
+               uatomic_dec(&lttng_chan->chan->u.reserve_fallback_ref);
+
        return 0;
-put:
-       lib_ring_buffer_put_cpu(&client_config);
+end:
+       lib_ring_buffer_end(&client_config);
+       if (put_fallback_ref)
+               uatomic_dec(&lttng_chan->chan->u.reserve_fallback_ref);
        return ret;
 }
 
@@ -631,7 +804,7 @@ static
 void lttng_event_commit(struct lttng_ust_lib_ring_buffer_ctx *ctx)
 {
        lib_ring_buffer_commit(&client_config, ctx);
-       lib_ring_buffer_put_cpu(&client_config);
+       lib_ring_buffer_end(&client_config);
 }
 
 static
This page took 0.028062 seconds and 5 git commands to generate.