Fix: sessiond: size-based rotation threshold exceeded in per-pid tracing (2/2)
[lttng-tools.git] / src / common / consumer / consumer-stream.cpp
CommitLineData
51230d70 1/*
21cf9b6b 2 * Copyright (C) 2011 EfficiOS Inc.
ab5be9fa
MJ
3 * Copyright (C) 2011 Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
4 * Copyright (C) 2013 David Goulet <dgoulet@efficios.com>
51230d70 5 *
ab5be9fa 6 * SPDX-License-Identifier: GPL-2.0-only
51230d70 7 *
51230d70
DG
8 */
9
6c1c0768 10#define _LGPL_SOURCE
10a50311 11#include <inttypes.h>
51230d70
DG
12#include <sys/mman.h>
13#include <unistd.h>
14
c9e313bc
SM
15#include <common/common.hpp>
16#include <common/consumer/consumer-timer.hpp>
17#include <common/consumer/consumer-timer.hpp>
18#include <common/consumer/consumer.hpp>
19#include <common/consumer/consumer.hpp>
20#include <common/consumer/metadata-bucket.hpp>
21#include <common/consumer/metadata-bucket.hpp>
22#include <common/index/index.hpp>
23#include <common/kernel-consumer/kernel-consumer.hpp>
24#include <common/kernel-ctl/kernel-ctl.hpp>
25#include <common/macros.hpp>
26#include <common/relayd/relayd.hpp>
27#include <common/ust-consumer/ust-consumer.hpp>
28#include <common/utils.hpp>
51230d70 29
c9e313bc 30#include "consumer-stream.hpp"
51230d70
DG
31
32/*
33 * RCU call to free stream. MUST only be used with call_rcu().
34 */
35static void free_stream_rcu(struct rcu_head *head)
36{
37 struct lttng_ht_node_u64 *node =
0114db0e 38 lttng::utils::container_of(head, &lttng_ht_node_u64::head);
51230d70 39 struct lttng_consumer_stream *stream =
0114db0e 40 lttng::utils::container_of(node, &lttng_consumer_stream::node);
51230d70
DG
41
42 pthread_mutex_destroy(&stream->lock);
43 free(stream);
44}
45
6f9449c2
JG
46static void consumer_stream_data_lock_all(struct lttng_consumer_stream *stream)
47{
48 pthread_mutex_lock(&stream->chan->lock);
49 pthread_mutex_lock(&stream->lock);
50}
51
52static void consumer_stream_data_unlock_all(struct lttng_consumer_stream *stream)
53{
54 pthread_mutex_unlock(&stream->lock);
55 pthread_mutex_unlock(&stream->chan->lock);
56}
57
947bd097
JR
58static void consumer_stream_data_assert_locked_all(struct lttng_consumer_stream *stream)
59{
60 ASSERT_LOCKED(stream->lock);
61 ASSERT_LOCKED(stream->chan->lock);
62}
63
6f9449c2
JG
64static void consumer_stream_metadata_lock_all(struct lttng_consumer_stream *stream)
65{
66 consumer_stream_data_lock_all(stream);
67 pthread_mutex_lock(&stream->metadata_rdv_lock);
68}
69
70static void consumer_stream_metadata_unlock_all(struct lttng_consumer_stream *stream)
71{
72 pthread_mutex_unlock(&stream->metadata_rdv_lock);
73 consumer_stream_data_unlock_all(stream);
74}
75
947bd097
JR
76static void consumer_stream_metadata_assert_locked_all(struct lttng_consumer_stream *stream)
77{
78 ASSERT_LOCKED(stream->metadata_rdv_lock);
79 consumer_stream_data_assert_locked_all(stream);
80}
81
6f9449c2
JG
82/* Only used for data streams. */
83static int consumer_stream_update_stats(struct lttng_consumer_stream *stream,
84 const struct stream_subbuffer *subbuf)
85{
86 int ret = 0;
87 uint64_t sequence_number;
d9b063d7 88 const uint64_t discarded_events = subbuf->info.data.events_discarded;
6f9449c2
JG
89
90 if (!subbuf->info.data.sequence_number.is_set) {
91 /* Command not supported by the tracer. */
92 sequence_number = -1ULL;
93 stream->sequence_number_unavailable = true;
94 } else {
95 sequence_number = subbuf->info.data.sequence_number.value;
96 }
97
98 /*
99 * Start the sequence when we extract the first packet in case we don't
100 * start at 0 (for example if a consumer is not connected to the
101 * session immediately after the beginning).
102 */
103 if (stream->last_sequence_number == -1ULL) {
104 stream->last_sequence_number = sequence_number;
105 } else if (sequence_number > stream->last_sequence_number) {
106 stream->chan->lost_packets += sequence_number -
107 stream->last_sequence_number - 1;
108 } else {
109 /* seq <= last_sequence_number */
110 ERR("Sequence number inconsistent : prev = %" PRIu64
111 ", current = %" PRIu64,
112 stream->last_sequence_number, sequence_number);
113 ret = -1;
114 goto end;
115 }
116 stream->last_sequence_number = sequence_number;
117
118 if (discarded_events < stream->last_discarded_events) {
119 /*
120 * Overflow has occurred. We assume only one wrap-around
121 * has occurred.
122 */
123 stream->chan->discarded_events +=
124 (1ULL << (CAA_BITS_PER_LONG - 1)) -
125 stream->last_discarded_events +
126 discarded_events;
127 } else {
128 stream->chan->discarded_events += discarded_events -
129 stream->last_discarded_events;
130 }
131 stream->last_discarded_events = discarded_events;
132 ret = 0;
133
134end:
135 return ret;
136}
137
138static
139void ctf_packet_index_populate(struct ctf_packet_index *index,
140 off_t offset, const struct stream_subbuffer *subbuffer)
141{
142 *index = (typeof(*index)){
143 .offset = htobe64(offset),
144 .packet_size = htobe64(subbuffer->info.data.packet_size),
145 .content_size = htobe64(subbuffer->info.data.content_size),
146 .timestamp_begin = htobe64(
147 subbuffer->info.data.timestamp_begin),
148 .timestamp_end = htobe64(
149 subbuffer->info.data.timestamp_end),
150 .events_discarded = htobe64(
151 subbuffer->info.data.events_discarded),
152 .stream_id = htobe64(subbuffer->info.data.stream_id),
153 .stream_instance_id = htobe64(
154 subbuffer->info.data.stream_instance_id.is_set ?
155 subbuffer->info.data.stream_instance_id.value : -1ULL),
156 .packet_seq_num = htobe64(
157 subbuffer->info.data.sequence_number.is_set ?
158 subbuffer->info.data.sequence_number.value : -1ULL),
159 };
160}
161
162static ssize_t consumer_stream_consume_mmap(
f46376a1 163 struct lttng_consumer_local_data *ctx __attribute__((unused)),
6f9449c2
JG
164 struct lttng_consumer_stream *stream,
165 const struct stream_subbuffer *subbuffer)
166{
167 const unsigned long padding_size =
168 subbuffer->info.data.padded_subbuf_size -
169 subbuffer->info.data.subbuf_size;
514775d9 170 const ssize_t written_bytes = lttng_consumer_on_read_subbuffer_mmap(
f5ba75b4 171 stream, &subbuffer->buffer.buffer, padding_size);
514775d9
FD
172
173 if (stream->net_seq_idx == -1ULL) {
174 /*
175 * When writing on disk, check that only the subbuffer (no
176 * padding) was written to disk.
177 */
178 if (written_bytes != subbuffer->info.data.padded_subbuf_size) {
179 DBG("Failed to write the entire padded subbuffer on disk (written_bytes: %zd, padded subbuffer size %lu)",
180 written_bytes,
181 subbuffer->info.data.padded_subbuf_size);
182 }
183 } else {
184 /*
185 * When streaming over the network, check that the entire
186 * subbuffer including padding was successfully written.
187 */
188 if (written_bytes != subbuffer->info.data.subbuf_size) {
189 DBG("Failed to write only the subbuffer over the network (written_bytes: %zd, subbuffer size %lu)",
190 written_bytes,
191 subbuffer->info.data.subbuf_size);
192 }
193 }
194
195 /*
196 * If `lttng_consumer_on_read_subbuffer_mmap()` returned an error, pass
197 * it along to the caller, else return zero.
198 */
199 if (written_bytes < 0) {
200 ERR("Error reading mmap subbuffer: %zd", written_bytes);
201 }
202
203 return written_bytes;
6f9449c2
JG
204}
205
206static ssize_t consumer_stream_consume_splice(
207 struct lttng_consumer_local_data *ctx,
208 struct lttng_consumer_stream *stream,
209 const struct stream_subbuffer *subbuffer)
210{
514775d9
FD
211 const ssize_t written_bytes = lttng_consumer_on_read_subbuffer_splice(
212 ctx, stream, subbuffer->info.data.padded_subbuf_size, 0);
213
214 if (written_bytes != subbuffer->info.data.padded_subbuf_size) {
215 DBG("Failed to write the entire padded subbuffer (written_bytes: %zd, padded subbuffer size %lu)",
216 written_bytes,
217 subbuffer->info.data.padded_subbuf_size);
218 }
219
220 /*
221 * If `lttng_consumer_on_read_subbuffer_splice()` returned an error,
222 * pass it along to the caller, else return zero.
223 */
224 if (written_bytes < 0) {
225 ERR("Error reading splice subbuffer: %zd", written_bytes);
226 }
227
228 return written_bytes;
6f9449c2
JG
229}
230
231static int consumer_stream_send_index(
232 struct lttng_consumer_stream *stream,
233 const struct stream_subbuffer *subbuffer,
f46376a1 234 struct lttng_consumer_local_data *ctx __attribute__((unused)))
6f9449c2
JG
235{
236 off_t packet_offset = 0;
237 struct ctf_packet_index index = {};
238
239 /*
240 * This is called after consuming the sub-buffer; substract the
241 * effect this sub-buffer from the offset.
242 */
243 if (stream->net_seq_idx == (uint64_t) -1ULL) {
244 packet_offset = stream->out_fd_offset -
245 subbuffer->info.data.padded_subbuf_size;
246 }
247
248 ctf_packet_index_populate(&index, packet_offset, subbuffer);
249 return consumer_stream_write_index(stream, &index);
250}
251
252/*
253 * Actually do the metadata sync using the given metadata stream.
254 *
255 * Return 0 on success else a negative value. ENODATA can be returned also
256 * indicating that there is no metadata available for that stream.
257 */
258static int do_sync_metadata(struct lttng_consumer_stream *metadata,
259 struct lttng_consumer_local_data *ctx)
260{
261 int ret;
577eea73 262 enum sync_metadata_status status;
6f9449c2 263
a0377dfe
FD
264 LTTNG_ASSERT(metadata);
265 LTTNG_ASSERT(metadata->metadata_flag);
266 LTTNG_ASSERT(ctx);
6f9449c2
JG
267
268 /*
269 * In UST, since we have to write the metadata from the cache packet
270 * by packet, we might need to start this procedure multiple times
271 * until all the metadata from the cache has been extracted.
272 */
273 do {
274 /*
275 * Steps :
276 * - Lock the metadata stream
277 * - Check if metadata stream node was deleted before locking.
278 * - if yes, release and return success
279 * - Check if new metadata is ready (flush + snapshot pos)
280 * - If nothing : release and return.
281 * - Lock the metadata_rdv_lock
282 * - Unlock the metadata stream
283 * - cond_wait on metadata_rdv to wait the wakeup from the
284 * metadata thread
285 * - Unlock the metadata_rdv_lock
286 */
287 pthread_mutex_lock(&metadata->lock);
288
289 /*
290 * There is a possibility that we were able to acquire a reference on the
291 * stream from the RCU hash table but between then and now, the node might
292 * have been deleted just before the lock is acquired. Thus, after locking,
293 * we make sure the metadata node has not been deleted which means that the
294 * buffers are closed.
295 *
296 * In that case, there is no need to sync the metadata hence returning a
297 * success return code.
298 */
299 ret = cds_lfht_is_node_deleted(&metadata->node.node);
300 if (ret) {
301 ret = 0;
302 goto end_unlock_mutex;
303 }
304
305 switch (ctx->type) {
306 case LTTNG_CONSUMER_KERNEL:
307 /*
308 * Empty the metadata cache and flush the current stream.
309 */
577eea73 310 status = lttng_kconsumer_sync_metadata(metadata);
6f9449c2
JG
311 break;
312 case LTTNG_CONSUMER32_UST:
313 case LTTNG_CONSUMER64_UST:
314 /*
315 * Ask the sessiond if we have new metadata waiting and update the
316 * consumer metadata cache.
317 */
577eea73 318 status = lttng_ustconsumer_sync_metadata(ctx, metadata);
6f9449c2
JG
319 break;
320 default:
577eea73 321 abort();
6f9449c2 322 }
577eea73
JG
323
324 switch (status) {
325 case SYNC_METADATA_STATUS_NEW_DATA:
326 break;
327 case SYNC_METADATA_STATUS_NO_DATA:
328 ret = 0;
6f9449c2 329 goto end_unlock_mutex;
577eea73
JG
330 case SYNC_METADATA_STATUS_ERROR:
331 ret = -1;
332 goto end_unlock_mutex;
333 default:
334 abort();
6f9449c2
JG
335 }
336
337 /*
338 * At this point, new metadata have been flushed, so we wait on the
339 * rendez-vous point for the metadata thread to wake us up when it
340 * finishes consuming the metadata and continue execution.
341 */
342
343 pthread_mutex_lock(&metadata->metadata_rdv_lock);
344
345 /*
346 * Release metadata stream lock so the metadata thread can process it.
347 */
348 pthread_mutex_unlock(&metadata->lock);
349
350 /*
351 * Wait on the rendez-vous point. Once woken up, it means the metadata was
352 * consumed and thus synchronization is achieved.
353 */
354 pthread_cond_wait(&metadata->metadata_rdv, &metadata->metadata_rdv_lock);
355 pthread_mutex_unlock(&metadata->metadata_rdv_lock);
577eea73 356 } while (status == SYNC_METADATA_STATUS_NEW_DATA);
6f9449c2
JG
357
358 /* Success */
359 return 0;
360
361end_unlock_mutex:
362 pthread_mutex_unlock(&metadata->lock);
363 return ret;
364}
365
366/*
367 * Synchronize the metadata using a given session ID. A successful acquisition
368 * of a metadata stream will trigger a request to the session daemon and a
369 * snapshot so the metadata thread can consume it.
370 *
371 * This function call is a rendez-vous point between the metadata thread and
372 * the data thread.
373 *
374 * Return 0 on success or else a negative value.
375 */
376int consumer_stream_sync_metadata(struct lttng_consumer_local_data *ctx,
377 uint64_t session_id)
378{
379 int ret;
380 struct lttng_consumer_stream *stream = NULL;
381 struct lttng_ht_iter iter;
382 struct lttng_ht *ht;
383
a0377dfe 384 LTTNG_ASSERT(ctx);
6f9449c2
JG
385
386 /* Ease our life a bit. */
fa29bfbf 387 ht = the_consumer_data.stream_list_ht;
6f9449c2
JG
388
389 rcu_read_lock();
390
391 /* Search the metadata associated with the session id of the given stream. */
392
393 cds_lfht_for_each_entry_duplicate(ht->ht,
394 ht->hash_fct(&session_id, lttng_ht_seed), ht->match_fct,
395 &session_id, &iter.iter, stream, node_session_id.node) {
396 if (!stream->metadata_flag) {
397 continue;
398 }
399
400 ret = do_sync_metadata(stream, ctx);
401 if (ret < 0) {
402 goto end;
403 }
404 }
405
406 /*
407 * Force return code to 0 (success) since ret might be ENODATA for instance
408 * which is not an error but rather that we should come back.
409 */
410 ret = 0;
411
412end:
413 rcu_read_unlock();
414 return ret;
415}
416
417static int consumer_stream_sync_metadata_index(
418 struct lttng_consumer_stream *stream,
419 const struct stream_subbuffer *subbuffer,
420 struct lttng_consumer_local_data *ctx)
421{
6653fca6 422 bool missed_metadata_flush;
6f9449c2
JG
423 int ret;
424
425 /* Block until all the metadata is sent. */
426 pthread_mutex_lock(&stream->metadata_timer_lock);
a0377dfe 427 LTTNG_ASSERT(!stream->missed_metadata_flush);
6f9449c2
JG
428 stream->waiting_on_metadata = true;
429 pthread_mutex_unlock(&stream->metadata_timer_lock);
430
431 ret = consumer_stream_sync_metadata(ctx, stream->session_id);
432
433 pthread_mutex_lock(&stream->metadata_timer_lock);
434 stream->waiting_on_metadata = false;
6653fca6
MD
435 missed_metadata_flush = stream->missed_metadata_flush;
436 if (missed_metadata_flush) {
6f9449c2 437 stream->missed_metadata_flush = false;
6f9449c2 438 }
6653fca6 439 pthread_mutex_unlock(&stream->metadata_timer_lock);
6f9449c2
JG
440 if (ret < 0) {
441 goto end;
442 }
443
444 ret = consumer_stream_send_index(stream, subbuffer, ctx);
6653fca6
MD
445 /*
446 * Send the live inactivity beacon to handle the situation where
447 * the live timer is prevented from sampling this stream
448 * because the stream lock was being held while this stream is
449 * waiting on metadata. This ensures live viewer progress in the
450 * unlikely scenario where a live timer would be prevented from
451 * locking a stream lock repeatedly due to a steady flow of
452 * incoming metadata, for a stream which is mostly inactive.
453 *
454 * It is important to send the inactivity beacon packet to
455 * relayd _after_ sending the index associated with the data
456 * that was just sent, otherwise this can cause live viewers to
457 * observe timestamps going backwards between an inactivity
458 * beacon and a following trace packet.
459 */
460 if (missed_metadata_flush) {
461 (void) stream->read_subbuffer_ops.send_live_beacon(stream);
462 }
6f9449c2
JG
463end:
464 return ret;
465}
466
467/*
468 * Check if the local version of the metadata stream matches with the version
469 * of the metadata stream in the kernel. If it was updated, set the reset flag
470 * on the stream.
471 */
472static
473int metadata_stream_check_version(struct lttng_consumer_stream *stream,
474 const struct stream_subbuffer *subbuffer)
475{
476 if (stream->metadata_version == subbuffer->info.metadata.version) {
477 goto end;
478 }
479
480 DBG("New metadata version detected");
55954e07
JG
481 consumer_stream_metadata_set_version(stream,
482 subbuffer->info.metadata.version);
f5ba75b4 483
6f9449c2
JG
484 if (stream->read_subbuffer_ops.reset_metadata) {
485 stream->read_subbuffer_ops.reset_metadata(stream);
486 }
487
488end:
489 return 0;
490}
491
503fefca
JG
492static
493bool stream_is_rotating_to_null_chunk(
494 const struct lttng_consumer_stream *stream)
495{
496 bool rotating_to_null_chunk = false;
497
498 if (stream->rotate_position == -1ULL) {
499 /* No rotation ongoing. */
500 goto end;
501 }
502
503 if (stream->trace_chunk == stream->chan->trace_chunk ||
504 !stream->chan->trace_chunk) {
505 rotating_to_null_chunk = true;
506 }
507end:
508 return rotating_to_null_chunk;
509}
510
511enum consumer_stream_open_packet_status consumer_stream_open_packet(
512 struct lttng_consumer_stream *stream)
513{
514 int ret;
515 enum consumer_stream_open_packet_status status;
516 unsigned long produced_pos_before, produced_pos_after;
517
518 ret = lttng_consumer_sample_snapshot_positions(stream);
519 if (ret < 0) {
520 ERR("Failed to snapshot positions before post-rotation empty packet flush: stream id = %" PRIu64
521 ", channel name = %s, session id = %" PRIu64,
522 stream->key, stream->chan->name,
523 stream->chan->session_id);
524 status = CONSUMER_STREAM_OPEN_PACKET_STATUS_ERROR;
525 goto end;
526 }
527
528 ret = lttng_consumer_get_produced_snapshot(
529 stream, &produced_pos_before);
530 if (ret < 0) {
531 ERR("Failed to read produced position before post-rotation empty packet flush: stream id = %" PRIu64
532 ", channel name = %s, session id = %" PRIu64,
533 stream->key, stream->chan->name,
534 stream->chan->session_id);
535 status = CONSUMER_STREAM_OPEN_PACKET_STATUS_ERROR;
536 goto end;
537 }
538
539 ret = consumer_stream_flush_buffer(stream, 0);
540 if (ret) {
541 ERR("Failed to flush an empty packet at rotation point: stream id = %" PRIu64
542 ", channel name = %s, session id = %" PRIu64,
543 stream->key, stream->chan->name,
544 stream->chan->session_id);
545 status = CONSUMER_STREAM_OPEN_PACKET_STATUS_ERROR;
546 goto end;
547 }
548
549 ret = lttng_consumer_sample_snapshot_positions(stream);
550 if (ret < 0) {
551 ERR("Failed to snapshot positions after post-rotation empty packet flush: stream id = %" PRIu64
552 ", channel name = %s, session id = %" PRIu64,
553 stream->key, stream->chan->name,
554 stream->chan->session_id);
555 status = CONSUMER_STREAM_OPEN_PACKET_STATUS_ERROR;
556 goto end;
557 }
558
559 ret = lttng_consumer_get_produced_snapshot(stream, &produced_pos_after);
560 if (ret < 0) {
561 ERR("Failed to read produced position after post-rotation empty packet flush: stream id = %" PRIu64
562 ", channel name = %s, session id = %" PRIu64,
563 stream->key, stream->chan->name,
564 stream->chan->session_id);
565 status = CONSUMER_STREAM_OPEN_PACKET_STATUS_ERROR;
566 goto end;
567 }
568
569 /*
570 * Determine if the flush had an effect by comparing the produced
571 * positons before and after the flush.
572 */
573 status = produced_pos_before != produced_pos_after ?
574 CONSUMER_STREAM_OPEN_PACKET_STATUS_OPENED :
575 CONSUMER_STREAM_OPEN_PACKET_STATUS_NO_SPACE;
576 if (status == CONSUMER_STREAM_OPEN_PACKET_STATUS_OPENED) {
577 stream->opened_packet_in_current_trace_chunk = true;
578 }
579
580end:
581 return status;
582}
583
584/*
585 * An attempt to open a new packet is performed after a rotation completes to
586 * get a begin timestamp as close as possible to the rotation point.
587 *
588 * However, that initial attempt at opening a packet can fail due to a full
589 * ring-buffer. In that case, a second attempt is performed after consuming
590 * a packet since that will have freed enough space in the ring-buffer.
591 */
592static
593int post_consume_open_new_packet(struct lttng_consumer_stream *stream,
f46376a1
MJ
594 const struct stream_subbuffer *subbuffer __attribute__((unused)),
595 struct lttng_consumer_local_data *ctx __attribute__((unused)))
503fefca
JG
596{
597 int ret = 0;
598
599 if (!stream->opened_packet_in_current_trace_chunk &&
600 stream->trace_chunk &&
601 !stream_is_rotating_to_null_chunk(stream)) {
602 const enum consumer_stream_open_packet_status status =
603 consumer_stream_open_packet(stream);
604
605 switch (status) {
606 case CONSUMER_STREAM_OPEN_PACKET_STATUS_OPENED:
607 DBG("Opened a packet after consuming a packet rotation: stream id = %" PRIu64
608 ", channel name = %s, session id = %" PRIu64,
609 stream->key, stream->chan->name,
610 stream->chan->session_id);
611 stream->opened_packet_in_current_trace_chunk = true;
612 break;
613 case CONSUMER_STREAM_OPEN_PACKET_STATUS_NO_SPACE:
614 /*
615 * Can't open a packet as there is no space left.
616 * This means that new events were produced, resulting
617 * in a packet being opened, which is what we want
618 * anyhow.
619 */
620 DBG("No space left to open a packet after consuming a packet: stream id = %" PRIu64
621 ", channel name = %s, session id = %" PRIu64,
622 stream->key, stream->chan->name,
623 stream->chan->session_id);
624 stream->opened_packet_in_current_trace_chunk = true;
625 break;
626 case CONSUMER_STREAM_OPEN_PACKET_STATUS_ERROR:
627 /* Logged by callee. */
628 ret = -1;
629 goto end;
630 default:
631 abort();
632 }
633
634 stream->opened_packet_in_current_trace_chunk = true;
635 }
636
637end:
638 return ret;
639}
640
6f9449c2
JG
641struct lttng_consumer_stream *consumer_stream_create(
642 struct lttng_consumer_channel *channel,
643 uint64_t channel_key,
644 uint64_t stream_key,
645 const char *channel_name,
646 uint64_t relayd_id,
647 uint64_t session_id,
648 struct lttng_trace_chunk *trace_chunk,
649 int cpu,
650 int *alloc_ret,
651 enum consumer_channel_type type,
652 unsigned int monitor)
653{
654 int ret;
655 struct lttng_consumer_stream *stream;
656
64803277 657 stream = zmalloc<lttng_consumer_stream>();
6f9449c2
JG
658 if (stream == NULL) {
659 PERROR("malloc struct lttng_consumer_stream");
660 ret = -ENOMEM;
661 goto end;
662 }
663
63e9caa2
MD
664 rcu_read_lock();
665
6f9449c2
JG
666 if (trace_chunk && !lttng_trace_chunk_get(trace_chunk)) {
667 ERR("Failed to acquire trace chunk reference during the creation of a stream");
668 ret = -1;
669 goto error;
670 }
671
5c5e3d71 672 stream->send_node = CDS_LIST_HEAD_INIT(stream->send_node);
6f9449c2
JG
673 stream->chan = channel;
674 stream->key = stream_key;
675 stream->trace_chunk = trace_chunk;
676 stream->out_fd = -1;
677 stream->out_fd_offset = 0;
678 stream->output_written = 0;
679 stream->net_seq_idx = relayd_id;
680 stream->session_id = session_id;
681 stream->monitor = monitor;
682 stream->endpoint_status = CONSUMER_ENDPOINT_ACTIVE;
683 stream->index_file = NULL;
684 stream->last_sequence_number = -1ULL;
685 stream->rotate_position = -1ULL;
f96af312
JG
686 /* Buffer is created with an open packet. */
687 stream->opened_packet_in_current_trace_chunk = true;
6f9449c2
JG
688 pthread_mutex_init(&stream->lock, NULL);
689 pthread_mutex_init(&stream->metadata_timer_lock, NULL);
690
691 /* If channel is the metadata, flag this stream as metadata. */
692 if (type == CONSUMER_CHANNEL_TYPE_METADATA) {
693 stream->metadata_flag = 1;
694 /* Metadata is flat out. */
695 strncpy(stream->name, DEFAULT_METADATA_NAME, sizeof(stream->name));
696 /* Live rendez-vous point. */
697 pthread_cond_init(&stream->metadata_rdv, NULL);
698 pthread_mutex_init(&stream->metadata_rdv_lock, NULL);
699 } else {
700 /* Format stream name to <channel_name>_<cpu_number> */
701 ret = snprintf(stream->name, sizeof(stream->name), "%s_%d",
702 channel_name, cpu);
703 if (ret < 0) {
704 PERROR("snprintf stream name");
705 goto error;
706 }
707 }
708
709 switch (channel->output) {
710 case CONSUMER_CHANNEL_SPLICE:
711 stream->output = LTTNG_EVENT_SPLICE;
712 ret = utils_create_pipe(stream->splice_pipe);
713 if (ret < 0) {
714 goto error;
715 }
716 break;
717 case CONSUMER_CHANNEL_MMAP:
718 stream->output = LTTNG_EVENT_MMAP;
719 break;
720 default:
721 abort();
722 }
723
724 /* Key is always the wait_fd for streams. */
725 lttng_ht_node_init_u64(&stream->node, stream->key);
726
727 /* Init node per channel id key */
728 lttng_ht_node_init_u64(&stream->node_channel_id, channel_key);
729
730 /* Init session id node with the stream session id */
731 lttng_ht_node_init_u64(&stream->node_session_id, stream->session_id);
732
733 DBG3("Allocated stream %s (key %" PRIu64 ", chan_key %" PRIu64
734 " relayd_id %" PRIu64 ", session_id %" PRIu64,
735 stream->name, stream->key, channel_key,
736 stream->net_seq_idx, stream->session_id);
737
738 rcu_read_unlock();
739
503fefca
JG
740 lttng_dynamic_array_init(&stream->read_subbuffer_ops.post_consume_cbs,
741 sizeof(post_consume_cb), NULL);
742
6f9449c2
JG
743 if (type == CONSUMER_CHANNEL_TYPE_METADATA) {
744 stream->read_subbuffer_ops.lock =
745 consumer_stream_metadata_lock_all;
746 stream->read_subbuffer_ops.unlock =
747 consumer_stream_metadata_unlock_all;
947bd097
JR
748 stream->read_subbuffer_ops.assert_locked =
749 consumer_stream_metadata_assert_locked_all;
6f9449c2
JG
750 stream->read_subbuffer_ops.pre_consume_subbuffer =
751 metadata_stream_check_version;
752 } else {
503fefca
JG
753 const post_consume_cb post_consume_index_op = channel->is_live ?
754 consumer_stream_sync_metadata_index :
755 consumer_stream_send_index;
97535efa
SM
756 const post_consume_cb post_consume_open_new_packet_ =
757 post_consume_open_new_packet;
503fefca
JG
758
759 ret = lttng_dynamic_array_add_element(
760 &stream->read_subbuffer_ops.post_consume_cbs,
761 &post_consume_index_op);
762 if (ret) {
763 PERROR("Failed to add `send index` callback to stream's post consumption callbacks");
764 goto error;
765 }
766
767 ret = lttng_dynamic_array_add_element(
768 &stream->read_subbuffer_ops.post_consume_cbs,
97535efa 769 &post_consume_open_new_packet_);
503fefca
JG
770 if (ret) {
771 PERROR("Failed to add `open new packet` callback to stream's post consumption callbacks");
772 goto error;
773 }
774
6f9449c2
JG
775 stream->read_subbuffer_ops.lock = consumer_stream_data_lock_all;
776 stream->read_subbuffer_ops.unlock =
777 consumer_stream_data_unlock_all;
947bd097
JR
778 stream->read_subbuffer_ops.assert_locked =
779 consumer_stream_data_assert_locked_all;
6f9449c2
JG
780 stream->read_subbuffer_ops.pre_consume_subbuffer =
781 consumer_stream_update_stats;
6f9449c2
JG
782 }
783
784 if (channel->output == CONSUMER_CHANNEL_MMAP) {
785 stream->read_subbuffer_ops.consume_subbuffer =
786 consumer_stream_consume_mmap;
787 } else {
788 stream->read_subbuffer_ops.consume_subbuffer =
789 consumer_stream_consume_splice;
790 }
791
792 return stream;
793
794error:
795 rcu_read_unlock();
796 lttng_trace_chunk_put(stream->trace_chunk);
503fefca 797 lttng_dynamic_array_reset(&stream->read_subbuffer_ops.post_consume_cbs);
6f9449c2
JG
798 free(stream);
799end:
800 if (alloc_ret) {
801 *alloc_ret = ret;
802 }
803 return NULL;
804}
805
51230d70
DG
806/*
807 * Close stream on the relayd side. This call can destroy a relayd if the
808 * conditions are met.
809 *
810 * A RCU read side lock MUST be acquired if the relayd object was looked up in
811 * a hash table before calling this.
812 */
813void consumer_stream_relayd_close(struct lttng_consumer_stream *stream,
814 struct consumer_relayd_sock_pair *relayd)
815{
816 int ret;
817
a0377dfe
FD
818 LTTNG_ASSERT(stream);
819 LTTNG_ASSERT(relayd);
51230d70 820
d01178b6
DG
821 if (stream->sent_to_relayd) {
822 uatomic_dec(&relayd->refcount);
a0377dfe 823 LTTNG_ASSERT(uatomic_read(&relayd->refcount) >= 0);
d01178b6 824 }
51230d70
DG
825
826 /* Closing streams requires to lock the control socket. */
827 pthread_mutex_lock(&relayd->ctrl_sock_mutex);
828 ret = relayd_send_close_stream(&relayd->control_sock,
829 stream->relayd_stream_id,
830 stream->next_net_seq_num - 1);
831 pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
832 if (ret < 0) {
9276e5c8
JR
833 ERR("Relayd send close stream failed. Cleaning up relayd %" PRIu64 ".", relayd->net_seq_idx);
834 lttng_consumer_cleanup_relayd(relayd);
51230d70
DG
835 }
836
837 /* Both conditions are met, we destroy the relayd. */
838 if (uatomic_read(&relayd->refcount) == 0 &&
839 uatomic_read(&relayd->destroy_flag)) {
840 consumer_destroy_relayd(relayd);
841 }
10a50311 842 stream->net_seq_idx = (uint64_t) -1ULL;
d01178b6 843 stream->sent_to_relayd = 0;
51230d70
DG
844}
845
846/*
847 * Close stream's file descriptors and, if needed, close stream also on the
848 * relayd side.
849 *
850 * The consumer data lock MUST be acquired.
851 * The stream lock MUST be acquired.
852 */
853void consumer_stream_close(struct lttng_consumer_stream *stream)
854{
855 int ret;
856 struct consumer_relayd_sock_pair *relayd;
857
a0377dfe 858 LTTNG_ASSERT(stream);
51230d70 859
fa29bfbf 860 switch (the_consumer_data.type) {
51230d70
DG
861 case LTTNG_CONSUMER_KERNEL:
862 if (stream->mmap_base != NULL) {
863 ret = munmap(stream->mmap_base, stream->mmap_len);
864 if (ret != 0) {
865 PERROR("munmap");
866 }
867 }
868
869 if (stream->wait_fd >= 0) {
870 ret = close(stream->wait_fd);
871 if (ret) {
872 PERROR("close");
873 }
10a50311 874 stream->wait_fd = -1;
51230d70 875 }
a2361a61
JD
876 if (stream->chan->output == CONSUMER_CHANNEL_SPLICE) {
877 utils_close_pipe(stream->splice_pipe);
878 }
51230d70
DG
879 break;
880 case LTTNG_CONSUMER32_UST:
881 case LTTNG_CONSUMER64_UST:
6d574024
DG
882 {
883 /*
884 * Special case for the metadata since the wait fd is an internal pipe
885 * polled in the metadata thread.
886 */
887 if (stream->metadata_flag && stream->chan->monitor) {
888 int rpipe = stream->ust_metadata_poll_pipe[0];
889
890 /*
891 * This will stop the channel timer if one and close the write side
892 * of the metadata poll pipe.
893 */
894 lttng_ustconsumer_close_metadata(stream->chan);
895 if (rpipe >= 0) {
896 ret = close(rpipe);
897 if (ret < 0) {
b4a650f3 898 PERROR("closing metadata pipe read side");
6d574024
DG
899 }
900 stream->ust_metadata_poll_pipe[0] = -1;
901 }
902 }
51230d70 903 break;
6d574024 904 }
51230d70
DG
905 default:
906 ERR("Unknown consumer_data type");
a0377dfe 907 abort();
51230d70
DG
908 }
909
910 /* Close output fd. Could be a socket or local file at this point. */
911 if (stream->out_fd >= 0) {
912 ret = close(stream->out_fd);
913 if (ret) {
914 PERROR("close");
915 }
10a50311 916 stream->out_fd = -1;
51230d70
DG
917 }
918
f8f3885c
MD
919 if (stream->index_file) {
920 lttng_index_file_put(stream->index_file);
921 stream->index_file = NULL;
309167d2
JD
922 }
923
d2956687
JG
924 lttng_trace_chunk_put(stream->trace_chunk);
925 stream->trace_chunk = NULL;
926
51230d70
DG
927 /* Check and cleanup relayd if needed. */
928 rcu_read_lock();
929 relayd = consumer_find_relayd(stream->net_seq_idx);
930 if (relayd != NULL) {
931 consumer_stream_relayd_close(stream, relayd);
932 }
933 rcu_read_unlock();
934}
935
936/*
937 * Delete the stream from all possible hash tables.
938 *
939 * The consumer data lock MUST be acquired.
940 * The stream lock MUST be acquired.
941 */
942void consumer_stream_delete(struct lttng_consumer_stream *stream,
943 struct lttng_ht *ht)
944{
945 int ret;
946 struct lttng_ht_iter iter;
947
a0377dfe 948 LTTNG_ASSERT(stream);
10a50311 949 /* Should NEVER be called not in monitor mode. */
a0377dfe 950 LTTNG_ASSERT(stream->chan->monitor);
51230d70
DG
951
952 rcu_read_lock();
953
954 if (ht) {
955 iter.iter.node = &stream->node.node;
956 ret = lttng_ht_del(ht, &iter);
a0377dfe 957 LTTNG_ASSERT(!ret);
51230d70
DG
958 }
959
960 /* Delete from stream per channel ID hash table. */
961 iter.iter.node = &stream->node_channel_id.node;
962 /*
963 * The returned value is of no importance. Even if the node is NOT in the
964 * hash table, we continue since we may have been called by a code path
965 * that did not add the stream to a (all) hash table. Same goes for the
966 * next call ht del call.
967 */
fa29bfbf 968 (void) lttng_ht_del(the_consumer_data.stream_per_chan_id_ht, &iter);
51230d70
DG
969
970 /* Delete from the global stream list. */
971 iter.iter.node = &stream->node_session_id.node;
972 /* See the previous ht del on why we ignore the returned value. */
fa29bfbf 973 (void) lttng_ht_del(the_consumer_data.stream_list_ht, &iter);
51230d70
DG
974
975 rcu_read_unlock();
976
6d574024
DG
977 if (!stream->metadata_flag) {
978 /* Decrement the stream count of the global consumer data. */
a0377dfe 979 LTTNG_ASSERT(the_consumer_data.stream_count > 0);
fa29bfbf 980 the_consumer_data.stream_count--;
6d574024 981 }
51230d70
DG
982}
983
984/*
985 * Free the given stream within a RCU call.
986 */
987void consumer_stream_free(struct lttng_consumer_stream *stream)
988{
a0377dfe 989 LTTNG_ASSERT(stream);
51230d70 990
f5ba75b4 991 metadata_bucket_destroy(stream->metadata_bucket);
51230d70
DG
992 call_rcu(&stream->node.head, free_stream_rcu);
993}
994
995/*
10a50311 996 * Destroy the stream's buffers of the tracer.
51230d70 997 */
10a50311 998void consumer_stream_destroy_buffers(struct lttng_consumer_stream *stream)
51230d70 999{
a0377dfe 1000 LTTNG_ASSERT(stream);
10a50311 1001
fa29bfbf 1002 switch (the_consumer_data.type) {
10a50311
JD
1003 case LTTNG_CONSUMER_KERNEL:
1004 break;
1005 case LTTNG_CONSUMER32_UST:
1006 case LTTNG_CONSUMER64_UST:
1007 lttng_ustconsumer_del_stream(stream);
1008 break;
1009 default:
1010 ERR("Unknown consumer_data type");
a0377dfe 1011 abort();
10a50311
JD
1012 }
1013}
51230d70 1014
10a50311 1015/*
4891ece8 1016 * Destroy and close a already created stream.
10a50311 1017 */
4891ece8 1018static void destroy_close_stream(struct lttng_consumer_stream *stream)
10a50311 1019{
a0377dfe 1020 LTTNG_ASSERT(stream);
51230d70 1021
4891ece8 1022 DBG("Consumer stream destroy monitored key: %" PRIu64, stream->key);
10a50311
JD
1023
1024 /* Destroy tracer buffers of the stream. */
1025 consumer_stream_destroy_buffers(stream);
1026 /* Close down everything including the relayd if one. */
1027 consumer_stream_close(stream);
1028}
51230d70 1029
10a50311 1030/*
4891ece8
DG
1031 * Decrement the stream's channel refcount and if down to 0, return the channel
1032 * pointer so it can be destroyed by the caller or NULL if not.
10a50311 1033 */
4891ece8
DG
1034static struct lttng_consumer_channel *unref_channel(
1035 struct lttng_consumer_stream *stream)
10a50311 1036{
4891ece8
DG
1037 struct lttng_consumer_channel *free_chan = NULL;
1038
a0377dfe
FD
1039 LTTNG_ASSERT(stream);
1040 LTTNG_ASSERT(stream->chan);
10a50311 1041
4891ece8
DG
1042 /* Update refcount of channel and see if we need to destroy it. */
1043 if (!uatomic_sub_return(&stream->chan->refcount, 1)
1044 && !uatomic_read(&stream->chan->nb_init_stream_left)) {
1045 free_chan = stream->chan;
1046 }
51230d70 1047
4891ece8 1048 return free_chan;
10a50311 1049}
51230d70 1050
10a50311
JD
1051/*
1052 * Destroy a stream completely. This will delete, close and free the stream.
1053 * Once return, the stream is NO longer usable. Its channel may get destroyed
1054 * if conditions are met for a monitored stream.
1055 *
1056 * This MUST be called WITHOUT the consumer data and stream lock acquired if
1057 * the stream is in _monitor_ mode else it does not matter.
1058 */
1059void consumer_stream_destroy(struct lttng_consumer_stream *stream,
1060 struct lttng_ht *ht)
1061{
a0377dfe 1062 LTTNG_ASSERT(stream);
10a50311 1063
5c5e3d71
JG
1064 cds_list_del_init(&stream->send_node);
1065
10a50311 1066 /* Stream is in monitor mode. */
4891ece8 1067 if (stream->monitor) {
10a50311 1068 struct lttng_consumer_channel *free_chan = NULL;
51230d70 1069
4891ece8
DG
1070 /*
1071 * This means that the stream was successfully removed from the streams
1072 * list of the channel and sent to the right thread managing this
1073 * stream thus being globally visible.
1074 */
1075 if (stream->globally_visible) {
fa29bfbf 1076 pthread_mutex_lock(&the_consumer_data.lock);
a9838785 1077 pthread_mutex_lock(&stream->chan->lock);
319dcddc 1078
4891ece8
DG
1079 pthread_mutex_lock(&stream->lock);
1080 /* Remove every reference of the stream in the consumer. */
1081 consumer_stream_delete(stream, ht);
1082
319dcddc 1083
4891ece8
DG
1084 destroy_close_stream(stream);
1085
1086 /* Update channel's refcount of the stream. */
1087 free_chan = unref_channel(stream);
1088
1089 /* Indicates that the consumer data state MUST be updated after this. */
fa29bfbf 1090 the_consumer_data.need_update = 1;
4891ece8
DG
1091
1092 pthread_mutex_unlock(&stream->lock);
a9838785 1093 pthread_mutex_unlock(&stream->chan->lock);
fa29bfbf 1094 pthread_mutex_unlock(&the_consumer_data.lock);
4891ece8
DG
1095 } else {
1096 /*
1097 * If the stream is not visible globally, this needs to be done
1098 * outside of the consumer data lock section.
1099 */
38aea171 1100 destroy_close_stream(stream);
4891ece8 1101 free_chan = unref_channel(stream);
10a50311
JD
1102 }
1103
10a50311
JD
1104 if (free_chan) {
1105 consumer_del_channel(free_chan);
1106 }
1107 } else {
4891ece8 1108 destroy_close_stream(stream);
51230d70
DG
1109 }
1110
1111 /* Free stream within a RCU call. */
d2956687
JG
1112 lttng_trace_chunk_put(stream->trace_chunk);
1113 stream->trace_chunk = NULL;
503fefca 1114 lttng_dynamic_array_reset(&stream->read_subbuffer_ops.post_consume_cbs);
51230d70
DG
1115 consumer_stream_free(stream);
1116}
1c20f0e2
JD
1117
1118/*
1119 * Write index of a specific stream either on the relayd or local disk.
1120 *
1121 * Return 0 on success or else a negative value.
1122 */
1123int consumer_stream_write_index(struct lttng_consumer_stream *stream,
f8f3885c 1124 struct ctf_packet_index *element)
1c20f0e2
JD
1125{
1126 int ret;
1c20f0e2 1127
a0377dfe
FD
1128 LTTNG_ASSERT(stream);
1129 LTTNG_ASSERT(element);
1c20f0e2
JD
1130
1131 rcu_read_lock();
23c910e5
JR
1132 if (stream->net_seq_idx != (uint64_t) -1ULL) {
1133 struct consumer_relayd_sock_pair *relayd;
1134 relayd = consumer_find_relayd(stream->net_seq_idx);
1135 if (relayd) {
1136 pthread_mutex_lock(&relayd->ctrl_sock_mutex);
1137 ret = relayd_send_index(&relayd->control_sock, element,
1c20f0e2 1138 stream->relayd_stream_id, stream->next_net_seq_num - 1);
9276e5c8
JR
1139 if (ret < 0) {
1140 /*
1141 * Communication error with lttng-relayd,
1142 * perform cleanup now
1143 */
1144 ERR("Relayd send index failed. Cleaning up relayd %" PRIu64 ".", relayd->net_seq_idx);
1145 lttng_consumer_cleanup_relayd(relayd);
1146 ret = -1;
1147 }
23c910e5
JR
1148 pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
1149 } else {
1150 ERR("Stream %" PRIu64 " relayd ID %" PRIu64 " unknown. Can't write index.",
1151 stream->key, stream->net_seq_idx);
1152 ret = -1;
1153 }
1c20f0e2 1154 } else {
f8f3885c 1155 if (lttng_index_file_write(stream->index_file, element)) {
6cd525e8
MD
1156 ret = -1;
1157 } else {
1158 ret = 0;
1159 }
1c20f0e2
JD
1160 }
1161 if (ret < 0) {
1162 goto error;
1163 }
1164
1165error:
1166 rcu_read_unlock();
1167 return ret;
1168}
94d49140 1169
d2956687
JG
1170int consumer_stream_create_output_files(struct lttng_consumer_stream *stream,
1171 bool create_index)
1172{
1173 int ret;
1174 enum lttng_trace_chunk_status chunk_status;
1175 const int flags = O_WRONLY | O_CREAT | O_TRUNC;
1176 const mode_t mode = S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP;
1177 char stream_path[LTTNG_PATH_MAX];
1178
1179 ASSERT_LOCKED(stream->lock);
a0377dfe 1180 LTTNG_ASSERT(stream->trace_chunk);
d2956687
JG
1181
1182 ret = utils_stream_file_path(stream->chan->pathname, stream->name,
1183 stream->chan->tracefile_size,
3b16476a 1184 stream->tracefile_count_current, NULL,
d2956687
JG
1185 stream_path, sizeof(stream_path));
1186 if (ret < 0) {
1187 goto end;
1188 }
1189
1190 if (stream->out_fd >= 0) {
1191 ret = close(stream->out_fd);
1192 if (ret < 0) {
1193 PERROR("Failed to close stream file \"%s\"",
1194 stream->name);
1195 goto end;
1196 }
1197 stream->out_fd = -1;
2932b393 1198 }
d2956687
JG
1199
1200 DBG("Opening stream output file \"%s\"", stream_path);
1201 chunk_status = lttng_trace_chunk_open_file(stream->trace_chunk, stream_path,
3ff5c5db 1202 flags, mode, &stream->out_fd, false);
2932b393 1203 if (chunk_status != LTTNG_TRACE_CHUNK_STATUS_OK) {
d2956687
JG
1204 ERR("Failed to open stream file \"%s\"", stream->name);
1205 ret = -1;
1206 goto end;
2932b393 1207 }
d2956687
JG
1208
1209 if (!stream->metadata_flag && (create_index || stream->index_file)) {
1210 if (stream->index_file) {
1211 lttng_index_file_put(stream->index_file);
1212 }
3ff5c5db 1213 chunk_status = lttng_index_file_create_from_trace_chunk(
d2956687
JG
1214 stream->trace_chunk,
1215 stream->chan->pathname,
1216 stream->name,
1217 stream->chan->tracefile_size,
1218 stream->tracefile_count_current,
1219 CTF_INDEX_MAJOR, CTF_INDEX_MINOR,
3ff5c5db
MD
1220 false, &stream->index_file);
1221 if (chunk_status != LTTNG_TRACE_CHUNK_STATUS_OK) {
d2956687
JG
1222 ret = -1;
1223 goto end;
1224 }
1225 }
1226
1227 /* Reset current size because we just perform a rotation. */
1228 stream->tracefile_size_current = 0;
1229 stream->out_fd_offset = 0;
1230end:
1231 return ret;
1232}
1233
1234int consumer_stream_rotate_output_files(struct lttng_consumer_stream *stream)
1235{
1236 int ret;
1237
1238 stream->tracefile_count_current++;
1239 if (stream->chan->tracefile_count > 0) {
1240 stream->tracefile_count_current %=
1241 stream->chan->tracefile_count;
1242 }
1243
1244 DBG("Rotating output files of stream \"%s\"", stream->name);
1245 ret = consumer_stream_create_output_files(stream, true);
1246 if (ret) {
1247 goto end;
1248 }
1249
1250end:
1251 return ret;
1252}
cdb72e4e
JG
1253
1254bool consumer_stream_is_deleted(struct lttng_consumer_stream *stream)
1255{
1256 /*
1257 * This function does not take a const stream since
1258 * cds_lfht_is_node_deleted was not const before liburcu 0.12.
1259 */
a0377dfe 1260 LTTNG_ASSERT(stream);
cdb72e4e
JG
1261 return cds_lfht_is_node_deleted(&stream->node.node);
1262}
f5ba75b4
JG
1263
1264static ssize_t metadata_bucket_flush(
1265 const struct stream_subbuffer *buffer, void *data)
1266{
1267 ssize_t ret;
97535efa 1268 struct lttng_consumer_stream *stream = (lttng_consumer_stream *) data;
f5ba75b4
JG
1269
1270 ret = consumer_stream_consume_mmap(NULL, stream, buffer);
1271 if (ret < 0) {
1272 goto end;
1273 }
1274end:
1275 return ret;
1276}
1277
1278static ssize_t metadata_bucket_consume(
f46376a1 1279 struct lttng_consumer_local_data *unused __attribute__((unused)),
f5ba75b4
JG
1280 struct lttng_consumer_stream *stream,
1281 const struct stream_subbuffer *subbuffer)
1282{
1283 ssize_t ret;
1284 enum metadata_bucket_status status;
1285
1286 status = metadata_bucket_fill(stream->metadata_bucket, subbuffer);
1287 switch (status) {
1288 case METADATA_BUCKET_STATUS_OK:
1289 /* Return consumed size. */
1290 ret = subbuffer->buffer.buffer.size;
1291 break;
1292 default:
1293 ret = -1;
1294 }
1295
1296 return ret;
1297}
1298
1299int consumer_stream_enable_metadata_bucketization(
1300 struct lttng_consumer_stream *stream)
1301{
1302 int ret = 0;
1303
a0377dfe
FD
1304 LTTNG_ASSERT(stream->metadata_flag);
1305 LTTNG_ASSERT(!stream->metadata_bucket);
1306 LTTNG_ASSERT(stream->chan->output == CONSUMER_CHANNEL_MMAP);
f5ba75b4
JG
1307
1308 stream->metadata_bucket = metadata_bucket_create(
1309 metadata_bucket_flush, stream);
1310 if (!stream->metadata_bucket) {
1311 ret = -1;
1312 goto end;
1313 }
1314
1315 stream->read_subbuffer_ops.consume_subbuffer = metadata_bucket_consume;
1316end:
1317 return ret;
1318}
55954e07
JG
1319
1320void consumer_stream_metadata_set_version(
1321 struct lttng_consumer_stream *stream, uint64_t new_version)
1322{
a0377dfe 1323 LTTNG_ASSERT(new_version > stream->metadata_version);
55954e07
JG
1324 stream->metadata_version = new_version;
1325 stream->reset_metadata_flag = 1;
1326
1327 if (stream->metadata_bucket) {
1328 metadata_bucket_reset(stream->metadata_bucket);
1329 }
1330}
503fefca
JG
1331
1332int consumer_stream_flush_buffer(struct lttng_consumer_stream *stream,
1333 bool producer_active)
1334{
1335 int ret = 0;
1336
fa29bfbf 1337 switch (the_consumer_data.type) {
503fefca
JG
1338 case LTTNG_CONSUMER_KERNEL:
1339 if (producer_active) {
1340 ret = kernctl_buffer_flush(stream->wait_fd);
1341 if (ret < 0) {
1342 ERR("Failed to flush kernel stream");
1343 goto end;
1344 }
1345 } else {
1346 ret = kernctl_buffer_flush_empty(stream->wait_fd);
1347 if (ret < 0) {
1348 /*
1349 * Doing a buffer flush which does not take into
1350 * account empty packets. This is not perfect,
1351 * but required as a fall-back when
1352 * "flush_empty" is not implemented by
1353 * lttng-modules.
1354 */
1355 ret = kernctl_buffer_flush(stream->wait_fd);
1356 if (ret < 0) {
1357 ERR("Failed to flush kernel stream");
1358 goto end;
1359 }
1360 }
1361 }
1362 break;
1363 case LTTNG_CONSUMER32_UST:
1364 case LTTNG_CONSUMER64_UST:
881fc67f 1365 ret = lttng_ustconsumer_flush_buffer(stream, (int) producer_active);
503fefca
JG
1366 break;
1367 default:
1368 ERR("Unknown consumer_data type");
1369 abort();
1370 }
1371
1372end:
1373 return ret;
1374}
This page took 0.120022 seconds and 4 git commands to generate.