Move libconsumer under common/consumer/
authorJonathan Rajotte <jonathan.rajotte-julien@efficios.com>
Thu, 15 Oct 2015 20:47:34 +0000 (16:47 -0400)
committerJérémie Galarneau <jeremie.galarneau@efficios.com>
Mon, 19 Oct 2015 20:02:57 +0000 (16:02 -0400)
Signed-off-by: Jonathan Rajotte <jonathan.rajotte-julien@efficios.com>
Signed-off-by: Jérémie Galarneau <jeremie.galarneau@efficios.com>
31 files changed:
configure.ac
src/bin/lttng-consumerd/Makefile.am
src/bin/lttng-consumerd/health-consumerd.c
src/bin/lttng-consumerd/lttng-consumerd.c
src/bin/lttng-relayd/health-relayd.c
src/bin/lttng-sessiond/consumer.h
src/bin/lttng-sessiond/ust-consumer.c
src/common/Makefile.am
src/common/consumer-metadata-cache.c [deleted file]
src/common/consumer-metadata-cache.h [deleted file]
src/common/consumer-stream.c [deleted file]
src/common/consumer-stream.h [deleted file]
src/common/consumer-testpoint.h [deleted file]
src/common/consumer-timer.c [deleted file]
src/common/consumer-timer.h [deleted file]
src/common/consumer.c [deleted file]
src/common/consumer.h [deleted file]
src/common/consumer/Makefile.am [new file with mode: 0644]
src/common/consumer/consumer-metadata-cache.c [new file with mode: 0644]
src/common/consumer/consumer-metadata-cache.h [new file with mode: 0644]
src/common/consumer/consumer-stream.c [new file with mode: 0644]
src/common/consumer/consumer-stream.h [new file with mode: 0644]
src/common/consumer/consumer-testpoint.h [new file with mode: 0644]
src/common/consumer/consumer-timer.c [new file with mode: 0644]
src/common/consumer/consumer-timer.h [new file with mode: 0644]
src/common/consumer/consumer.c [new file with mode: 0644]
src/common/consumer/consumer.h [new file with mode: 0644]
src/common/kernel-consumer/kernel-consumer.c
src/common/kernel-consumer/kernel-consumer.h
src/common/ust-consumer/ust-consumer.c
src/common/ust-consumer/ust-consumer.h

index eabcff2aeae81c868f509b86c5e10bece5300a42..ceaf3771520674a2fee3f38d38b71f1759d38844 100644 (file)
@@ -604,6 +604,7 @@ AC_CONFIG_FILES([
        src/common/Makefile
        src/common/kernel-ctl/Makefile
        src/common/kernel-consumer/Makefile
+       src/common/consumer/Makefile
        src/common/ust-consumer/Makefile
        src/common/hashtable/Makefile
        src/common/sessiond-comm/Makefile
index 28034ca34e6a31d76404e866e45346cd184ff494..7c6b0275ec823a7eb23674926bc252824ee30b36 100644 (file)
@@ -8,7 +8,7 @@ lttng_consumerd_SOURCES = lttng-consumerd.c \
        health-consumerd.c
 
 lttng_consumerd_LDADD = \
-          $(top_builddir)/src/common/libconsumer.la \
+          $(top_builddir)/src/common/consumer/libconsumer.la \
           $(top_builddir)/src/common/sessiond-comm/libsessiond-comm.la \
           $(top_builddir)/src/common/libcommon.la \
           $(top_builddir)/src/common/index/libindex.la \
index 8d632ef95262256868555e5d2dcb0a6fc7dcb25c..2ea6303f77533fd28bf9684598733bdf4f8cf9c9 100644 (file)
@@ -42,8 +42,8 @@
 
 #include <common/defaults.h>
 #include <common/common.h>
-#include <common/consumer.h>
-#include <common/consumer-timer.h>
+#include <common/consumer/consumer.h>
+#include <common/consumer/consumer-timer.h>
 #include <common/compat/poll.h>
 #include <common/sessiond-comm/sessiond-comm.h>
 #include <common/utils.h>
index cd0fc73b2167781e600348c711307dc4bf4e2bd2..87ffb05dd5c78af8a70472900f985f5fd9dd3f1f 100644 (file)
@@ -42,8 +42,8 @@
 
 #include <common/defaults.h>
 #include <common/common.h>
-#include <common/consumer.h>
-#include <common/consumer-timer.h>
+#include <common/consumer/consumer.h>
+#include <common/consumer/consumer-timer.h>
 #include <common/compat/poll.h>
 #include <common/compat/getenv.h>
 #include <common/sessiond-comm/sessiond-comm.h>
index 8452c1a6c3886f4aaa11b9e71faa7a358d48d254..c6dd2e8523f3466abb81d97356b1c13ce1f71273 100644 (file)
@@ -42,8 +42,8 @@
 
 #include <common/defaults.h>
 #include <common/common.h>
-#include <common/consumer.h>
-#include <common/consumer-timer.h>
+#include <common/consumer/consumer.h>
+#include <common/consumer/consumer-timer.h>
 #include <common/compat/poll.h>
 #include <common/sessiond-comm/sessiond-comm.h>
 #include <common/utils.h>
index 73f113d0aca6fd8f4d33e2ba60cfe9bedd398d9f..47e59c7c28de86a8e550df650bd705b5dc9de9f3 100644 (file)
@@ -18,7 +18,7 @@
 #ifndef _CONSUMER_H
 #define _CONSUMER_H
 
-#include <common/consumer.h>
+#include <common/consumer/consumer.h>
 #include <common/hashtable/hashtable.h>
 #include <lttng/lttng.h>
 #include <urcu/ref.h>
index abb9e103d6c40278e37508506a24eb7fff7ae7e2..3bb54f03983178f035fdcb345f8e1d7670557847 100644 (file)
@@ -24,7 +24,7 @@
 #include <inttypes.h>
 
 #include <common/common.h>
-#include <common/consumer.h>
+#include <common/consumer/consumer.h>
 #include <common/defaults.h>
 
 #include "consumer.h"
index 5b716129cf96156961b666f12dde0913dcd8e211..0b16e4be9f3e7cb7e3b4b9e007f8131a50c7c453 100644 (file)
@@ -3,14 +3,14 @@ AM_CPPFLAGS = -I$(top_srcdir)/include -I$(top_srcdir)/src
 AUTOMAKE_OPTIONS = subdir-objects
 
 SUBDIRS = compat health hashtable kernel-ctl sessiond-comm relayd \
-                 kernel-consumer ust-consumer testpoint index config
+                 kernel-consumer ust-consumer testpoint index config \
+                 consumer
 
 AM_CFLAGS = -fno-strict-aliasing
 
 noinst_HEADERS = lttng-kernel.h defaults.h macros.h error.h futex.h \
                                 uri.h utils.h lttng-kernel-old.h \
-                                consumer-metadata-cache.h consumer-timer.h \
-                                consumer-testpoint.h align.h bitfield.h bug.h
+                                align.h bitfield.h bug.h
 
 # Common library
 noinst_LTLIBRARIES = libcommon.la
@@ -28,24 +28,6 @@ libcommon_la_LIBADD = \
                -lrt \
                $(top_builddir)/src/common/config/libconfig.la
 
-# Consumer library
-noinst_LTLIBRARIES += libconsumer.la
-
-libconsumer_la_SOURCES = consumer.c consumer.h consumer-metadata-cache.c \
-                         consumer-timer.c consumer-stream.c consumer-stream.h
-
-libconsumer_la_LIBADD = \
-               $(top_builddir)/src/common/sessiond-comm/libsessiond-comm.la \
-               $(top_builddir)/src/common/kernel-consumer/libkernel-consumer.la \
-               $(top_builddir)/src/common/hashtable/libhashtable.la \
-               $(top_builddir)/src/common/compat/libcompat.la \
-               $(top_builddir)/src/common/relayd/librelayd.la
-
-if HAVE_LIBLTTNG_UST_CTL
-libconsumer_la_LIBADD += \
-               $(top_builddir)/src/common/ust-consumer/libust-consumer.la
-endif
-
 all-local:
        @if [ x"$(srcdir)" != x"$(builddir)" ]; then \
                for script in $(EXTRA_DIST); do \
diff --git a/src/common/consumer-metadata-cache.c b/src/common/consumer-metadata-cache.c
deleted file mode 100644 (file)
index cbd3ef3..0000000
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * Copyright (C) 2013 - Julien Desfossez <jdesfossez@efficios.com>
- *                      David Goulet <dgoulet@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify
- * it under the terms of the GNU General Public License, version 2 only,
- * as published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along
- * with this program; if not, write to the Free Software Foundation, Inc.,
- * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#define _LGPL_SOURCE
-#include <assert.h>
-#include <pthread.h>
-#include <stdlib.h>
-#include <string.h>
-#include <sys/types.h>
-#include <unistd.h>
-#include <inttypes.h>
-
-#include <common/common.h>
-#include <common/utils.h>
-#include <common/sessiond-comm/sessiond-comm.h>
-#include <common/ust-consumer/ust-consumer.h>
-#include <common/consumer.h>
-
-#include "consumer-metadata-cache.h"
-
-extern struct lttng_consumer_global_data consumer_data;
-
-/*
- * Extend the allocated size of the metadata cache. Called only from
- * lttng_ustconsumer_write_metadata_cache.
- *
- * Return 0 on success, a negative value on error.
- */
-static int extend_metadata_cache(struct lttng_consumer_channel *channel,
-               unsigned int size)
-{
-       int ret = 0;
-       char *tmp_data_ptr;
-       unsigned int new_size, old_size;
-
-       assert(channel);
-       assert(channel->metadata_cache);
-
-       old_size = channel->metadata_cache->cache_alloc_size;
-       new_size = max_t(unsigned int, old_size + size, old_size << 1);
-       DBG("Extending metadata cache to %u", new_size);
-       tmp_data_ptr = realloc(channel->metadata_cache->data, new_size);
-       if (!tmp_data_ptr) {
-               ERR("Reallocating metadata cache");
-               free(channel->metadata_cache->data);
-               ret = -1;
-               goto end;
-       }
-       /* Zero newly allocated memory */
-       memset(tmp_data_ptr + old_size, 0, new_size - old_size);
-       channel->metadata_cache->data = tmp_data_ptr;
-       channel->metadata_cache->cache_alloc_size = new_size;
-
-end:
-       return ret;
-}
-
-/*
- * Write metadata to the cache, extend the cache if necessary. We support
- * overlapping updates, but they need to be contiguous. Send the
- * contiguous metadata in cache to the ring buffer. The metadata cache
- * lock MUST be acquired to write in the cache.
- *
- * Return 0 on success, a negative value on error.
- */
-int consumer_metadata_cache_write(struct lttng_consumer_channel *channel,
-               unsigned int offset, unsigned int len, char *data)
-{
-       int ret = 0;
-       int size_ret;
-       struct consumer_metadata_cache *cache;
-
-       assert(channel);
-       assert(channel->metadata_cache);
-
-       cache = channel->metadata_cache;
-       DBG("Writing %u bytes from offset %u in metadata cache", len, offset);
-
-       if (offset + len > cache->cache_alloc_size) {
-               ret = extend_metadata_cache(channel,
-                               len - cache->cache_alloc_size + offset);
-               if (ret < 0) {
-                       ERR("Extending metadata cache");
-                       goto end;
-               }
-       }
-
-       memcpy(cache->data + offset, data, len);
-       if (offset + len > cache->max_offset) {
-               char dummy = 'c';
-
-               cache->max_offset = offset + len;
-               if (channel->monitor) {
-                       size_ret = lttng_write(channel->metadata_stream->ust_metadata_poll_pipe[1],
-                                       &dummy, 1);
-                       if (size_ret < 1) {
-                               ERR("Wakeup UST metadata pipe");
-                               ret = -1;
-                               goto end;
-                       }
-               }
-       }
-
-end:
-       return ret;
-}
-
-/*
- * Create the metadata cache, original allocated size: max_sb_size
- *
- * Return 0 on success, a negative value on error.
- */
-int consumer_metadata_cache_allocate(struct lttng_consumer_channel *channel)
-{
-       int ret;
-
-       assert(channel);
-
-       channel->metadata_cache = zmalloc(
-                       sizeof(struct consumer_metadata_cache));
-       if (!channel->metadata_cache) {
-               PERROR("zmalloc metadata cache struct");
-               ret = -1;
-               goto end;
-       }
-       ret = pthread_mutex_init(&channel->metadata_cache->lock, NULL);
-       if (ret != 0) {
-               PERROR("mutex init");
-               goto end_free_cache;
-       }
-
-       channel->metadata_cache->cache_alloc_size = DEFAULT_METADATA_CACHE_SIZE;
-       channel->metadata_cache->data = zmalloc(
-                       channel->metadata_cache->cache_alloc_size * sizeof(char));
-       if (!channel->metadata_cache->data) {
-               PERROR("zmalloc metadata cache data");
-               ret = -1;
-               goto end_free_mutex;
-       }
-       DBG("Allocated metadata cache of %" PRIu64 " bytes",
-                       channel->metadata_cache->cache_alloc_size);
-
-       ret = 0;
-       goto end;
-
-end_free_mutex:
-       pthread_mutex_destroy(&channel->metadata_cache->lock);
-end_free_cache:
-       free(channel->metadata_cache);
-end:
-       return ret;
-}
-
-/*
- * Destroy and free the metadata cache
- */
-void consumer_metadata_cache_destroy(struct lttng_consumer_channel *channel)
-{
-       if (!channel || !channel->metadata_cache) {
-               return;
-       }
-
-       DBG("Destroying metadata cache");
-
-       pthread_mutex_destroy(&channel->metadata_cache->lock);
-       free(channel->metadata_cache->data);
-       free(channel->metadata_cache);
-}
-
-/*
- * Check if the cache is flushed up to the offset passed in parameter.
- *
- * Return 0 if everything has been flushed, 1 if there is data not flushed.
- */
-int consumer_metadata_cache_flushed(struct lttng_consumer_channel *channel,
-               uint64_t offset, int timer)
-{
-       int ret = 0;
-       struct lttng_consumer_stream *metadata_stream;
-
-       assert(channel);
-       assert(channel->metadata_cache);
-
-       /*
-        * If not called from a timer handler, we have to take the
-        * channel lock to be mutually exclusive with channel teardown.
-        * Timer handler does not need to take this lock because it is
-        * already synchronized by timer stop (and, more importantly,
-        * taking this lock in a timer handler would cause a deadlock).
-        */
-       if (!timer) {
-               pthread_mutex_lock(&channel->lock);
-       }
-       pthread_mutex_lock(&channel->timer_lock);
-       pthread_mutex_lock(&channel->metadata_cache->lock);
-
-       metadata_stream = channel->metadata_stream;
-
-       if (!metadata_stream) {
-               /*
-                * Having no metadata stream means the channel is being destroyed so there
-                * is no cache to flush anymore.
-                */
-               ret = 0;
-       } else if (metadata_stream->ust_metadata_pushed >= offset) {
-               ret = 0;
-       } else if (channel->metadata_stream->endpoint_status !=
-                       CONSUMER_ENDPOINT_ACTIVE) {
-               /* An inactive endpoint means we don't have to flush anymore. */
-               ret = 0;
-       } else {
-               /* Still not completely flushed. */
-               ret = 1;
-       }
-
-       pthread_mutex_unlock(&channel->metadata_cache->lock);
-       pthread_mutex_unlock(&channel->timer_lock);
-       if (!timer) {
-               pthread_mutex_unlock(&channel->lock);
-       }
-
-       return ret;
-}
diff --git a/src/common/consumer-metadata-cache.h b/src/common/consumer-metadata-cache.h
deleted file mode 100644 (file)
index e7aba4a..0000000
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Copyright (C) 2013 - Julien Desfossez <jdesfossez@efficios.com>
- *                      David Goulet <dgoulet@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify
- * it under the terms of the GNU General Public License, version 2 only,
- * as published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along
- * with this program; if not, write to the Free Software Foundation, Inc.,
- * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#ifndef CONSUMER_METADATA_CACHE_H
-#define CONSUMER_METADATA_CACHE_H
-
-#include <common/consumer.h>
-
-struct consumer_metadata_cache {
-       char *data;
-       uint64_t cache_alloc_size;
-       /*
-        * The upper-limit of data written inside the buffer.
-        *
-        * With the total_bytes_written it allows us to keep track of when the
-        * cache contains contiguous metadata ready to be sent to the RB.
-        * All cached data is contiguous.
-        */
-       uint64_t max_offset;
-       /*
-        * Lock to update the metadata cache and push into the ring_buffer
-        * (ustctl_write_metadata_to_channel).
-        *
-        * This is nested INSIDE the consumer_data lock.
-        */
-       pthread_mutex_t lock;
-};
-
-int consumer_metadata_cache_write(struct lttng_consumer_channel *channel,
-               unsigned int offset, unsigned int len, char *data);
-int consumer_metadata_cache_allocate(struct lttng_consumer_channel *channel);
-void consumer_metadata_cache_destroy(struct lttng_consumer_channel *channel);
-int consumer_metadata_cache_flushed(struct lttng_consumer_channel *channel,
-               uint64_t offset, int timer);
-
-#endif /* CONSUMER_METADATA_CACHE_H */
diff --git a/src/common/consumer-stream.c b/src/common/consumer-stream.c
deleted file mode 100644 (file)
index a62cef2..0000000
+++ /dev/null
@@ -1,554 +0,0 @@
-/*
- * Copyright (C) 2011 - Julien Desfossez <julien.desfossez@polymtl.ca>
- *                      Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
- * Copyright (C) 2013 - David Goulet <dgoulet@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify it
- * under the terms of the GNU General Public License, version 2 only, as
- * published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along with
- * this program; if not, write to the Free Software Foundation, Inc., 51
- * Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#define _LGPL_SOURCE
-#include <assert.h>
-#include <inttypes.h>
-#include <sys/mman.h>
-#include <unistd.h>
-
-#include <common/common.h>
-#include <common/index/index.h>
-#include <common/kernel-consumer/kernel-consumer.h>
-#include <common/relayd/relayd.h>
-#include <common/ust-consumer/ust-consumer.h>
-#include <common/utils.h>
-
-#include "consumer-stream.h"
-
-/*
- * RCU call to free stream. MUST only be used with call_rcu().
- */
-static void free_stream_rcu(struct rcu_head *head)
-{
-       struct lttng_ht_node_u64 *node =
-               caa_container_of(head, struct lttng_ht_node_u64, head);
-       struct lttng_consumer_stream *stream =
-               caa_container_of(node, struct lttng_consumer_stream, node);
-
-       pthread_mutex_destroy(&stream->lock);
-       free(stream);
-}
-
-/*
- * Close stream on the relayd side. This call can destroy a relayd if the
- * conditions are met.
- *
- * A RCU read side lock MUST be acquired if the relayd object was looked up in
- * a hash table before calling this.
- */
-void consumer_stream_relayd_close(struct lttng_consumer_stream *stream,
-               struct consumer_relayd_sock_pair *relayd)
-{
-       int ret;
-
-       assert(stream);
-       assert(relayd);
-
-       if (stream->sent_to_relayd) {
-               uatomic_dec(&relayd->refcount);
-               assert(uatomic_read(&relayd->refcount) >= 0);
-       }
-
-       /* Closing streams requires to lock the control socket. */
-       pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-       ret = relayd_send_close_stream(&relayd->control_sock,
-                       stream->relayd_stream_id,
-                       stream->next_net_seq_num - 1);
-       pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-       if (ret < 0) {
-               DBG("Unable to close stream on the relayd. Continuing");
-               /*
-                * Continue here. There is nothing we can do for the relayd.
-                * Chances are that the relayd has closed the socket so we just
-                * continue cleaning up.
-                */
-       }
-
-       /* Both conditions are met, we destroy the relayd. */
-       if (uatomic_read(&relayd->refcount) == 0 &&
-                       uatomic_read(&relayd->destroy_flag)) {
-               consumer_destroy_relayd(relayd);
-       }
-       stream->net_seq_idx = (uint64_t) -1ULL;
-       stream->sent_to_relayd = 0;
-}
-
-/*
- * Close stream's file descriptors and, if needed, close stream also on the
- * relayd side.
- *
- * The consumer data lock MUST be acquired.
- * The stream lock MUST be acquired.
- */
-void consumer_stream_close(struct lttng_consumer_stream *stream)
-{
-       int ret;
-       struct consumer_relayd_sock_pair *relayd;
-
-       assert(stream);
-
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               if (stream->mmap_base != NULL) {
-                       ret = munmap(stream->mmap_base, stream->mmap_len);
-                       if (ret != 0) {
-                               PERROR("munmap");
-                       }
-               }
-
-               if (stream->wait_fd >= 0) {
-                       ret = close(stream->wait_fd);
-                       if (ret) {
-                               PERROR("close");
-                       }
-                       stream->wait_fd = -1;
-               }
-               if (stream->chan->output == CONSUMER_CHANNEL_SPLICE) {
-                       utils_close_pipe(stream->splice_pipe);
-               }
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-       {
-               /*
-                * Special case for the metadata since the wait fd is an internal pipe
-                * polled in the metadata thread.
-                */
-               if (stream->metadata_flag && stream->chan->monitor) {
-                       int rpipe = stream->ust_metadata_poll_pipe[0];
-
-                       /*
-                        * This will stop the channel timer if one and close the write side
-                        * of the metadata poll pipe.
-                        */
-                       lttng_ustconsumer_close_metadata(stream->chan);
-                       if (rpipe >= 0) {
-                               ret = close(rpipe);
-                               if (ret < 0) {
-                                       PERROR("closing metadata pipe read side");
-                               }
-                               stream->ust_metadata_poll_pipe[0] = -1;
-                       }
-               }
-               break;
-       }
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-       }
-
-       /* Close output fd. Could be a socket or local file at this point. */
-       if (stream->out_fd >= 0) {
-               ret = close(stream->out_fd);
-               if (ret) {
-                       PERROR("close");
-               }
-               stream->out_fd = -1;
-       }
-
-       if (stream->index_fd >= 0) {
-               ret = close(stream->index_fd);
-               if (ret) {
-                       PERROR("close stream index_fd");
-               }
-               stream->index_fd = -1;
-       }
-
-       /* Check and cleanup relayd if needed. */
-       rcu_read_lock();
-       relayd = consumer_find_relayd(stream->net_seq_idx);
-       if (relayd != NULL) {
-               consumer_stream_relayd_close(stream, relayd);
-       }
-       rcu_read_unlock();
-}
-
-/*
- * Delete the stream from all possible hash tables.
- *
- * The consumer data lock MUST be acquired.
- * The stream lock MUST be acquired.
- */
-void consumer_stream_delete(struct lttng_consumer_stream *stream,
-               struct lttng_ht *ht)
-{
-       int ret;
-       struct lttng_ht_iter iter;
-
-       assert(stream);
-       /* Should NEVER be called not in monitor mode. */
-       assert(stream->chan->monitor);
-
-       rcu_read_lock();
-
-       if (ht) {
-               iter.iter.node = &stream->node.node;
-               ret = lttng_ht_del(ht, &iter);
-               assert(!ret);
-       }
-
-       /* Delete from stream per channel ID hash table. */
-       iter.iter.node = &stream->node_channel_id.node;
-       /*
-        * The returned value is of no importance. Even if the node is NOT in the
-        * hash table, we continue since we may have been called by a code path
-        * that did not add the stream to a (all) hash table. Same goes for the
-        * next call ht del call.
-        */
-       (void) lttng_ht_del(consumer_data.stream_per_chan_id_ht, &iter);
-
-       /* Delete from the global stream list. */
-       iter.iter.node = &stream->node_session_id.node;
-       /* See the previous ht del on why we ignore the returned value. */
-       (void) lttng_ht_del(consumer_data.stream_list_ht, &iter);
-
-       rcu_read_unlock();
-
-       if (!stream->metadata_flag) {
-               /* Decrement the stream count of the global consumer data. */
-               assert(consumer_data.stream_count > 0);
-               consumer_data.stream_count--;
-       }
-}
-
-/*
- * Free the given stream within a RCU call.
- */
-void consumer_stream_free(struct lttng_consumer_stream *stream)
-{
-       assert(stream);
-
-       call_rcu(&stream->node.head, free_stream_rcu);
-}
-
-/*
- * Destroy the stream's buffers of the tracer.
- */
-void consumer_stream_destroy_buffers(struct lttng_consumer_stream *stream)
-{
-       assert(stream);
-
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               lttng_ustconsumer_del_stream(stream);
-               break;
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-       }
-}
-
-/*
- * Destroy and close a already created stream.
- */
-static void destroy_close_stream(struct lttng_consumer_stream *stream)
-{
-       assert(stream);
-
-       DBG("Consumer stream destroy monitored key: %" PRIu64, stream->key);
-
-       /* Destroy tracer buffers of the stream. */
-       consumer_stream_destroy_buffers(stream);
-       /* Close down everything including the relayd if one. */
-       consumer_stream_close(stream);
-}
-
-/*
- * Decrement the stream's channel refcount and if down to 0, return the channel
- * pointer so it can be destroyed by the caller or NULL if not.
- */
-static struct lttng_consumer_channel *unref_channel(
-               struct lttng_consumer_stream *stream)
-{
-       struct lttng_consumer_channel *free_chan = NULL;
-
-       assert(stream);
-       assert(stream->chan);
-
-       /* Update refcount of channel and see if we need to destroy it. */
-       if (!uatomic_sub_return(&stream->chan->refcount, 1)
-                       && !uatomic_read(&stream->chan->nb_init_stream_left)) {
-               free_chan = stream->chan;
-       }
-
-       return free_chan;
-}
-
-/*
- * Destroy a stream completely. This will delete, close and free the stream.
- * Once return, the stream is NO longer usable. Its channel may get destroyed
- * if conditions are met for a monitored stream.
- *
- * This MUST be called WITHOUT the consumer data and stream lock acquired if
- * the stream is in _monitor_ mode else it does not matter.
- */
-void consumer_stream_destroy(struct lttng_consumer_stream *stream,
-               struct lttng_ht *ht)
-{
-       assert(stream);
-
-       /* Stream is in monitor mode. */
-       if (stream->monitor) {
-               struct lttng_consumer_channel *free_chan = NULL;
-
-               /*
-                * This means that the stream was successfully removed from the streams
-                * list of the channel and sent to the right thread managing this
-                * stream thus being globally visible.
-                */
-               if (stream->globally_visible) {
-                       pthread_mutex_lock(&consumer_data.lock);
-                       pthread_mutex_lock(&stream->chan->lock);
-                       pthread_mutex_lock(&stream->lock);
-                       /* Remove every reference of the stream in the consumer. */
-                       consumer_stream_delete(stream, ht);
-
-                       destroy_close_stream(stream);
-
-                       /* Update channel's refcount of the stream. */
-                       free_chan = unref_channel(stream);
-
-                       /* Indicates that the consumer data state MUST be updated after this. */
-                       consumer_data.need_update = 1;
-
-                       pthread_mutex_unlock(&stream->lock);
-                       pthread_mutex_unlock(&stream->chan->lock);
-                       pthread_mutex_unlock(&consumer_data.lock);
-               } else {
-                       /*
-                        * If the stream is not visible globally, this needs to be done
-                        * outside of the consumer data lock section.
-                        */
-                       free_chan = unref_channel(stream);
-               }
-
-               if (free_chan) {
-                       consumer_del_channel(free_chan);
-               }
-       } else {
-               destroy_close_stream(stream);
-       }
-
-       /* Free stream within a RCU call. */
-       consumer_stream_free(stream);
-}
-
-/*
- * Write index of a specific stream either on the relayd or local disk.
- *
- * Return 0 on success or else a negative value.
- */
-int consumer_stream_write_index(struct lttng_consumer_stream *stream,
-               struct ctf_packet_index *index)
-{
-       int ret;
-       struct consumer_relayd_sock_pair *relayd;
-
-       assert(stream);
-       assert(index);
-
-       rcu_read_lock();
-       relayd = consumer_find_relayd(stream->net_seq_idx);
-       if (relayd) {
-               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-               ret = relayd_send_index(&relayd->control_sock, index,
-                               stream->relayd_stream_id, stream->next_net_seq_num - 1);
-               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-       } else {
-               ssize_t size_ret;
-
-               size_ret = index_write(stream->index_fd, index,
-                               sizeof(struct ctf_packet_index));
-               if (size_ret < sizeof(struct ctf_packet_index)) {
-                       ret = -1;
-               } else {
-                       ret = 0;
-               }
-       }
-       if (ret < 0) {
-               goto error;
-       }
-
-error:
-       rcu_read_unlock();
-       return ret;
-}
-
-/*
- * Actually do the metadata sync using the given metadata stream.
- *
- * Return 0 on success else a negative value. ENODATA can be returned also
- * indicating that there is no metadata available for that stream.
- */
-static int do_sync_metadata(struct lttng_consumer_stream *metadata,
-               struct lttng_consumer_local_data *ctx)
-{
-       int ret;
-
-       assert(metadata);
-       assert(metadata->metadata_flag);
-       assert(ctx);
-
-       /*
-        * In UST, since we have to write the metadata from the cache packet
-        * by packet, we might need to start this procedure multiple times
-        * until all the metadata from the cache has been extracted.
-        */
-       do {
-               /*
-                * Steps :
-                * - Lock the metadata stream
-                * - Check if metadata stream node was deleted before locking.
-                *   - if yes, release and return success
-                * - Check if new metadata is ready (flush + snapshot pos)
-                * - If nothing : release and return.
-                * - Lock the metadata_rdv_lock
-                * - Unlock the metadata stream
-                * - cond_wait on metadata_rdv to wait the wakeup from the
-                *   metadata thread
-                * - Unlock the metadata_rdv_lock
-                */
-               pthread_mutex_lock(&metadata->lock);
-
-               /*
-                * There is a possibility that we were able to acquire a reference on the
-                * stream from the RCU hash table but between then and now, the node might
-                * have been deleted just before the lock is acquired. Thus, after locking,
-                * we make sure the metadata node has not been deleted which means that the
-                * buffers are closed.
-                *
-                * In that case, there is no need to sync the metadata hence returning a
-                * success return code.
-                */
-               ret = cds_lfht_is_node_deleted(&metadata->node.node);
-               if (ret) {
-                       ret = 0;
-                       goto end_unlock_mutex;
-               }
-
-               switch (ctx->type) {
-               case LTTNG_CONSUMER_KERNEL:
-                       /*
-                        * Empty the metadata cache and flush the current stream.
-                        */
-                       ret = lttng_kconsumer_sync_metadata(metadata);
-                       break;
-               case LTTNG_CONSUMER32_UST:
-               case LTTNG_CONSUMER64_UST:
-                       /*
-                        * Ask the sessiond if we have new metadata waiting and update the
-                        * consumer metadata cache.
-                        */
-                       ret = lttng_ustconsumer_sync_metadata(ctx, metadata);
-                       break;
-               default:
-                       assert(0);
-                       ret = -1;
-                       break;
-               }
-               /*
-                * Error or no new metadata, we exit here.
-                */
-               if (ret <= 0 || ret == ENODATA) {
-                       goto end_unlock_mutex;
-               }
-
-               /*
-                * At this point, new metadata have been flushed, so we wait on the
-                * rendez-vous point for the metadata thread to wake us up when it
-                * finishes consuming the metadata and continue execution.
-                */
-
-               pthread_mutex_lock(&metadata->metadata_rdv_lock);
-
-               /*
-                * Release metadata stream lock so the metadata thread can process it.
-                */
-               pthread_mutex_unlock(&metadata->lock);
-
-               /*
-                * Wait on the rendez-vous point. Once woken up, it means the metadata was
-                * consumed and thus synchronization is achieved.
-                */
-               pthread_cond_wait(&metadata->metadata_rdv, &metadata->metadata_rdv_lock);
-               pthread_mutex_unlock(&metadata->metadata_rdv_lock);
-       } while (ret == EAGAIN);
-
-       /* Success */
-       return 0;
-
-end_unlock_mutex:
-       pthread_mutex_unlock(&metadata->lock);
-       return ret;
-}
-
-/*
- * Synchronize the metadata using a given session ID. A successful acquisition
- * of a metadata stream will trigger a request to the session daemon and a
- * snapshot so the metadata thread can consume it.
- *
- * This function call is a rendez-vous point between the metadata thread and
- * the data thread.
- *
- * Return 0 on success or else a negative value.
- */
-int consumer_stream_sync_metadata(struct lttng_consumer_local_data *ctx,
-               uint64_t session_id)
-{
-       int ret;
-       struct lttng_consumer_stream *stream = NULL;
-       struct lttng_ht_iter iter;
-       struct lttng_ht *ht;
-
-       assert(ctx);
-
-       /* Ease our life a bit. */
-       ht = consumer_data.stream_list_ht;
-
-       rcu_read_lock();
-
-       /* Search the metadata associated with the session id of the given stream. */
-
-       cds_lfht_for_each_entry_duplicate(ht->ht,
-                       ht->hash_fct(&session_id, lttng_ht_seed), ht->match_fct,
-                       &session_id, &iter.iter, stream, node_session_id.node) {
-               if (!stream->metadata_flag) {
-                       continue;
-               }
-
-               ret = do_sync_metadata(stream, ctx);
-               if (ret < 0) {
-                       goto end;
-               }
-       }
-
-       /*
-        * Force return code to 0 (success) since ret might be ENODATA for instance
-        * which is not an error but rather that we should come back.
-        */
-       ret = 0;
-
-end:
-       rcu_read_unlock();
-       return ret;
-}
diff --git a/src/common/consumer-stream.h b/src/common/consumer-stream.h
deleted file mode 100644 (file)
index c5fb097..0000000
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright (C) 2013 - David Goulet <dgoulet@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify it
- * under the terms of the GNU General Public License, version 2 only, as
- * published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along with
- * this program; if not, write to the Free Software Foundation, Inc., 51
- * Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#ifndef LTTNG_CONSUMER_STREAM_H
-#define LTTNG_CONSUMER_STREAM_H
-
-#include "consumer.h"
-
-/*
- * Close stream's file descriptors and, if needed, close stream also on the
- * relayd side.
- *
- * The stream lock MUST be acquired.
- * The consumer data lock MUST be acquired.
- */
-void consumer_stream_close(struct lttng_consumer_stream *stream);
-
-/*
- * Close stream on the relayd side. This call can destroy a relayd if the
- * conditions are met.
- *
- * A RCU read side lock MUST be acquired if the relayd object was looked up in
- * a hash table before calling this.
- */
-void consumer_stream_relayd_close(struct lttng_consumer_stream *stream,
-               struct consumer_relayd_sock_pair *relayd);
-
-/*
- * Delete the stream from all possible hash tables.
- *
- * The consumer data lock MUST be acquired.
- */
-void consumer_stream_delete(struct lttng_consumer_stream *stream,
-               struct lttng_ht *ht);
-
-/*
- * Free the given stream within a RCU call.
- */
-void consumer_stream_free(struct lttng_consumer_stream *stream);
-
-/*
- * Destroy a stream completely. This will delete, close and free the stream.
- * Once return, the stream is NO longer usable. Its channel may get destroyed
- * if conditions are met.
- *
- * This MUST be called WITHOUT the consumer data and stream lock acquired.
- */
-void consumer_stream_destroy(struct lttng_consumer_stream *stream,
-               struct lttng_ht *ht);
-
-/*
- * Destroy the stream's buffers on the tracer side. This is also called in a
- * stream destroy.
- */
-void consumer_stream_destroy_buffers(struct lttng_consumer_stream *stream);
-
-/*
- * Write index of a specific stream either on the relayd or local disk.
- */
-int consumer_stream_write_index(struct lttng_consumer_stream *stream,
-               struct ctf_packet_index *index);
-
-int consumer_stream_sync_metadata(struct lttng_consumer_local_data *ctx,
-               uint64_t session_id);
-
-#endif /* LTTNG_CONSUMER_STREAM_H */
diff --git a/src/common/consumer-testpoint.h b/src/common/consumer-testpoint.h
deleted file mode 100644 (file)
index e94ce57..0000000
+++ /dev/null
@@ -1,31 +0,0 @@
-#ifndef CONSUMERD_TESTPOINT_H
-#define CONSUMERD_TESTPOINT_H
-
-/*
- * Copyright (C) 2012 - Christian Babeux <christian.babeux@efficios.com>
- * Copyright (C) 2014 - Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify it
- * under the terms of the GNU General Public License, version 2 only, as
- * published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along with
- * this program; if not, write to the Free Software Foundation, Inc., 51
- * Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#include <common/testpoint/testpoint.h>
-
-/* Testpoints, internal use only */
-TESTPOINT_DECL(consumerd_thread_channel);
-TESTPOINT_DECL(consumerd_thread_metadata);
-TESTPOINT_DECL(consumerd_thread_data);
-TESTPOINT_DECL(consumerd_thread_sessiond);
-TESTPOINT_DECL(consumerd_thread_metadata_timer);
-
-#endif /* CONSUMERD_TESTPOINT_H */
diff --git a/src/common/consumer-timer.c b/src/common/consumer-timer.c
deleted file mode 100644 (file)
index 5764b13..0000000
+++ /dev/null
@@ -1,607 +0,0 @@
-/*
- * Copyright (C) 2012 - Julien Desfossez <julien.desfossez@efficios.com>
- *                      David Goulet <dgoulet@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify it
- * under the terms of the GNU General Public License, version 2 only, as
- * published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along with
- * this program; if not, write to the Free Software Foundation, Inc., 51
- * Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#define _LGPL_SOURCE
-#include <assert.h>
-#include <inttypes.h>
-#include <signal.h>
-
-#include <bin/lttng-consumerd/health-consumerd.h>
-#include <common/common.h>
-#include <common/compat/endian.h>
-#include <common/kernel-ctl/kernel-ctl.h>
-#include <common/kernel-consumer/kernel-consumer.h>
-#include <common/consumer-stream.h>
-
-#include "consumer-timer.h"
-#include "consumer-testpoint.h"
-#include "ust-consumer/ust-consumer.h"
-
-static struct timer_signal_data timer_signal = {
-       .tid = 0,
-       .setup_done = 0,
-       .qs_done = 0,
-       .lock = PTHREAD_MUTEX_INITIALIZER,
-};
-
-/*
- * Set custom signal mask to current thread.
- */
-static void setmask(sigset_t *mask)
-{
-       int ret;
-
-       ret = sigemptyset(mask);
-       if (ret) {
-               PERROR("sigemptyset");
-       }
-       ret = sigaddset(mask, LTTNG_CONSUMER_SIG_SWITCH);
-       if (ret) {
-               PERROR("sigaddset switch");
-       }
-       ret = sigaddset(mask, LTTNG_CONSUMER_SIG_TEARDOWN);
-       if (ret) {
-               PERROR("sigaddset teardown");
-       }
-       ret = sigaddset(mask, LTTNG_CONSUMER_SIG_LIVE);
-       if (ret) {
-               PERROR("sigaddset live");
-       }
-}
-
-/*
- * Execute action on a timer switch.
- *
- * Beware: metadata_switch_timer() should *never* take a mutex also held
- * while consumer_timer_switch_stop() is called. It would result in
- * deadlocks.
- */
-static void metadata_switch_timer(struct lttng_consumer_local_data *ctx,
-               int sig, siginfo_t *si, void *uc)
-{
-       int ret;
-       struct lttng_consumer_channel *channel;
-
-       channel = si->si_value.sival_ptr;
-       assert(channel);
-
-       if (channel->switch_timer_error) {
-               return;
-       }
-
-       DBG("Switch timer for channel %" PRIu64, channel->key);
-       switch (ctx->type) {
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               /*
-                * Locks taken by lttng_ustconsumer_request_metadata():
-                * - metadata_socket_lock
-                *   - Calling lttng_ustconsumer_recv_metadata():
-                *     - channel->metadata_cache->lock
-                *     - Calling consumer_metadata_cache_flushed():
-                *       - channel->timer_lock
-                *         - channel->metadata_cache->lock
-                *
-                * Ensure that neither consumer_data.lock nor
-                * channel->lock are taken within this function, since
-                * they are held while consumer_timer_switch_stop() is
-                * called.
-                */
-               ret = lttng_ustconsumer_request_metadata(ctx, channel, 1, 1);
-               if (ret < 0) {
-                       channel->switch_timer_error = 1;
-               }
-               break;
-       case LTTNG_CONSUMER_KERNEL:
-       case LTTNG_CONSUMER_UNKNOWN:
-               assert(0);
-               break;
-       }
-}
-
-static int send_empty_index(struct lttng_consumer_stream *stream, uint64_t ts,
-               uint64_t stream_id)
-{
-       int ret;
-       struct ctf_packet_index index;
-
-       memset(&index, 0, sizeof(index));
-       index.stream_id = htobe64(stream_id);
-       index.timestamp_end = htobe64(ts);
-       ret = consumer_stream_write_index(stream, &index);
-       if (ret < 0) {
-               goto error;
-       }
-
-error:
-       return ret;
-}
-
-int consumer_flush_kernel_index(struct lttng_consumer_stream *stream)
-{
-       uint64_t ts, stream_id;
-       int ret;
-
-       ret = kernctl_get_current_timestamp(stream->wait_fd, &ts);
-       if (ret < 0) {
-               ERR("Failed to get the current timestamp");
-               goto end;
-       }
-       ret = kernctl_buffer_flush(stream->wait_fd);
-       if (ret < 0) {
-               ERR("Failed to flush kernel stream");
-               goto end;
-       }
-       ret = kernctl_snapshot(stream->wait_fd);
-       if (ret < 0) {
-               if (errno != EAGAIN && errno != ENODATA) {
-                       PERROR("live timer kernel snapshot");
-                       ret = -1;
-                       goto end;
-               }
-               ret = kernctl_get_stream_id(stream->wait_fd, &stream_id);
-               if (ret < 0) {
-                       PERROR("kernctl_get_stream_id");
-                       goto end;
-               }
-               DBG("Stream %" PRIu64 " empty, sending beacon", stream->key);
-               ret = send_empty_index(stream, ts, stream_id);
-               if (ret < 0) {
-                       goto end;
-               }
-       }
-       ret = 0;
-end:
-       return ret;
-}
-
-static int check_kernel_stream(struct lttng_consumer_stream *stream)
-{
-       int ret;
-
-       /*
-        * While holding the stream mutex, try to take a snapshot, if it
-        * succeeds, it means that data is ready to be sent, just let the data
-        * thread handle that. Otherwise, if the snapshot returns EAGAIN, it
-        * means that there is no data to read after the flush, so we can
-        * safely send the empty index.
-        *
-        * Doing a trylock and checking if waiting on metadata if
-        * trylock fails. Bail out of the stream is indeed waiting for
-        * metadata to be pushed. Busy wait on trylock otherwise.
-        */
-       for (;;) {
-               ret = pthread_mutex_trylock(&stream->lock);
-               switch (ret) {
-               case 0:
-                       break;  /* We have the lock. */
-               case EBUSY:
-                       pthread_mutex_lock(&stream->metadata_timer_lock);
-                       if (stream->waiting_on_metadata) {
-                               ret = 0;
-                               stream->missed_metadata_flush = true;
-                               pthread_mutex_unlock(&stream->metadata_timer_lock);
-                               goto end;       /* Bail out. */
-                       }
-                       pthread_mutex_unlock(&stream->metadata_timer_lock);
-                       /* Try again. */
-                       caa_cpu_relax();
-                       continue;
-               default:
-                       ERR("Unexpected pthread_mutex_trylock error %d", ret);
-                       ret = -1;
-                       goto end;
-               }
-               break;
-       }
-       ret = consumer_flush_kernel_index(stream);
-       pthread_mutex_unlock(&stream->lock);
-end:
-       return ret;
-}
-
-int consumer_flush_ust_index(struct lttng_consumer_stream *stream)
-{
-       uint64_t ts, stream_id;
-       int ret;
-
-       ret = cds_lfht_is_node_deleted(&stream->node.node);
-       if (ret) {
-               goto end;
-       }
-
-       ret = lttng_ustconsumer_get_current_timestamp(stream, &ts);
-       if (ret < 0) {
-               ERR("Failed to get the current timestamp");
-               goto end;
-       }
-       lttng_ustconsumer_flush_buffer(stream, 1);
-       ret = lttng_ustconsumer_take_snapshot(stream);
-       if (ret < 0) {
-               if (ret != -EAGAIN) {
-                       ERR("Taking UST snapshot");
-                       ret = -1;
-                       goto end;
-               }
-               ret = lttng_ustconsumer_get_stream_id(stream, &stream_id);
-               if (ret < 0) {
-                       PERROR("ustctl_get_stream_id");
-                       goto end;
-               }
-               DBG("Stream %" PRIu64 " empty, sending beacon", stream->key);
-               ret = send_empty_index(stream, ts, stream_id);
-               if (ret < 0) {
-                       goto end;
-               }
-       }
-       ret = 0;
-end:
-       return ret;
-}
-
-static int check_ust_stream(struct lttng_consumer_stream *stream)
-{
-       int ret;
-
-       assert(stream);
-       assert(stream->ustream);
-       /*
-        * While holding the stream mutex, try to take a snapshot, if it
-        * succeeds, it means that data is ready to be sent, just let the data
-        * thread handle that. Otherwise, if the snapshot returns EAGAIN, it
-        * means that there is no data to read after the flush, so we can
-        * safely send the empty index.
-        *
-        * Doing a trylock and checking if waiting on metadata if
-        * trylock fails. Bail out of the stream is indeed waiting for
-        * metadata to be pushed. Busy wait on trylock otherwise.
-        */
-       for (;;) {
-               ret = pthread_mutex_trylock(&stream->lock);
-               switch (ret) {
-               case 0:
-                       break;  /* We have the lock. */
-               case EBUSY:
-                       pthread_mutex_lock(&stream->metadata_timer_lock);
-                       if (stream->waiting_on_metadata) {
-                               ret = 0;
-                               stream->missed_metadata_flush = true;
-                               pthread_mutex_unlock(&stream->metadata_timer_lock);
-                               goto end;       /* Bail out. */
-                       }
-                       pthread_mutex_unlock(&stream->metadata_timer_lock);
-                       /* Try again. */
-                       caa_cpu_relax();
-                       continue;
-               default:
-                       ERR("Unexpected pthread_mutex_trylock error %d", ret);
-                       ret = -1;
-                       goto end;
-               }
-               break;
-       }
-       ret = consumer_flush_ust_index(stream);
-       pthread_mutex_unlock(&stream->lock);
-end:
-       return ret;
-}
-
-/*
- * Execute action on a live timer
- */
-static void live_timer(struct lttng_consumer_local_data *ctx,
-               int sig, siginfo_t *si, void *uc)
-{
-       int ret;
-       struct lttng_consumer_channel *channel;
-       struct lttng_consumer_stream *stream;
-       struct lttng_ht *ht;
-       struct lttng_ht_iter iter;
-
-       channel = si->si_value.sival_ptr;
-       assert(channel);
-
-       if (channel->switch_timer_error) {
-               goto error;
-       }
-       ht = consumer_data.stream_per_chan_id_ht;
-
-       DBG("Live timer for channel %" PRIu64, channel->key);
-
-       rcu_read_lock();
-       switch (ctx->type) {
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               cds_lfht_for_each_entry_duplicate(ht->ht,
-                               ht->hash_fct(&channel->key, lttng_ht_seed),
-                               ht->match_fct, &channel->key, &iter.iter,
-                               stream, node_channel_id.node) {
-                       ret = check_ust_stream(stream);
-                       if (ret < 0) {
-                               goto error_unlock;
-                       }
-               }
-               break;
-       case LTTNG_CONSUMER_KERNEL:
-               cds_lfht_for_each_entry_duplicate(ht->ht,
-                               ht->hash_fct(&channel->key, lttng_ht_seed),
-                               ht->match_fct, &channel->key, &iter.iter,
-                               stream, node_channel_id.node) {
-                       ret = check_kernel_stream(stream);
-                       if (ret < 0) {
-                               goto error_unlock;
-                       }
-               }
-               break;
-       case LTTNG_CONSUMER_UNKNOWN:
-               assert(0);
-               break;
-       }
-
-error_unlock:
-       rcu_read_unlock();
-
-error:
-       return;
-}
-
-static
-void consumer_timer_signal_thread_qs(unsigned int signr)
-{
-       sigset_t pending_set;
-       int ret;
-
-       /*
-        * We need to be the only thread interacting with the thread
-        * that manages signals for teardown synchronization.
-        */
-       pthread_mutex_lock(&timer_signal.lock);
-
-       /* Ensure we don't have any signal queued for this channel. */
-       for (;;) {
-               ret = sigemptyset(&pending_set);
-               if (ret == -1) {
-                       PERROR("sigemptyset");
-               }
-               ret = sigpending(&pending_set);
-               if (ret == -1) {
-                       PERROR("sigpending");
-               }
-               if (!sigismember(&pending_set, LTTNG_CONSUMER_SIG_SWITCH)) {
-                       break;
-               }
-               caa_cpu_relax();
-       }
-
-       /*
-        * From this point, no new signal handler will be fired that would try to
-        * access "chan". However, we still need to wait for any currently
-        * executing handler to complete.
-        */
-       cmm_smp_mb();
-       CMM_STORE_SHARED(timer_signal.qs_done, 0);
-       cmm_smp_mb();
-
-       /*
-        * Kill with LTTNG_CONSUMER_SIG_TEARDOWN, so signal management thread wakes
-        * up.
-        */
-       kill(getpid(), LTTNG_CONSUMER_SIG_TEARDOWN);
-
-       while (!CMM_LOAD_SHARED(timer_signal.qs_done)) {
-               caa_cpu_relax();
-       }
-       cmm_smp_mb();
-
-       pthread_mutex_unlock(&timer_signal.lock);
-}
-
-/*
- * Set the timer for periodical metadata flush.
- */
-void consumer_timer_switch_start(struct lttng_consumer_channel *channel,
-               unsigned int switch_timer_interval)
-{
-       int ret;
-       struct sigevent sev;
-       struct itimerspec its;
-
-       assert(channel);
-       assert(channel->key);
-
-       if (switch_timer_interval == 0) {
-               return;
-       }
-
-       sev.sigev_notify = SIGEV_SIGNAL;
-       sev.sigev_signo = LTTNG_CONSUMER_SIG_SWITCH;
-       sev.sigev_value.sival_ptr = channel;
-       ret = timer_create(CLOCKID, &sev, &channel->switch_timer);
-       if (ret == -1) {
-               PERROR("timer_create");
-       }
-       channel->switch_timer_enabled = 1;
-
-       its.it_value.tv_sec = switch_timer_interval / 1000000;
-       its.it_value.tv_nsec = switch_timer_interval % 1000000;
-       its.it_interval.tv_sec = its.it_value.tv_sec;
-       its.it_interval.tv_nsec = its.it_value.tv_nsec;
-
-       ret = timer_settime(channel->switch_timer, 0, &its, NULL);
-       if (ret == -1) {
-               PERROR("timer_settime");
-       }
-}
-
-/*
- * Stop and delete timer.
- */
-void consumer_timer_switch_stop(struct lttng_consumer_channel *channel)
-{
-       int ret;
-
-       assert(channel);
-
-       ret = timer_delete(channel->switch_timer);
-       if (ret == -1) {
-               PERROR("timer_delete");
-       }
-
-       consumer_timer_signal_thread_qs(LTTNG_CONSUMER_SIG_SWITCH);
-
-       channel->switch_timer = 0;
-       channel->switch_timer_enabled = 0;
-}
-
-/*
- * Set the timer for the live mode.
- */
-void consumer_timer_live_start(struct lttng_consumer_channel *channel,
-               int live_timer_interval)
-{
-       int ret;
-       struct sigevent sev;
-       struct itimerspec its;
-
-       assert(channel);
-       assert(channel->key);
-
-       if (live_timer_interval <= 0) {
-               return;
-       }
-
-       sev.sigev_notify = SIGEV_SIGNAL;
-       sev.sigev_signo = LTTNG_CONSUMER_SIG_LIVE;
-       sev.sigev_value.sival_ptr = channel;
-       ret = timer_create(CLOCKID, &sev, &channel->live_timer);
-       if (ret == -1) {
-               PERROR("timer_create");
-       }
-       channel->live_timer_enabled = 1;
-
-       its.it_value.tv_sec = live_timer_interval / 1000000;
-       its.it_value.tv_nsec = live_timer_interval % 1000000;
-       its.it_interval.tv_sec = its.it_value.tv_sec;
-       its.it_interval.tv_nsec = its.it_value.tv_nsec;
-
-       ret = timer_settime(channel->live_timer, 0, &its, NULL);
-       if (ret == -1) {
-               PERROR("timer_settime");
-       }
-}
-
-/*
- * Stop and delete timer.
- */
-void consumer_timer_live_stop(struct lttng_consumer_channel *channel)
-{
-       int ret;
-
-       assert(channel);
-
-       ret = timer_delete(channel->live_timer);
-       if (ret == -1) {
-               PERROR("timer_delete");
-       }
-
-       consumer_timer_signal_thread_qs(LTTNG_CONSUMER_SIG_LIVE);
-
-       channel->live_timer = 0;
-       channel->live_timer_enabled = 0;
-}
-
-/*
- * Block the RT signals for the entire process. It must be called from the
- * consumer main before creating the threads
- */
-int consumer_signal_init(void)
-{
-       int ret;
-       sigset_t mask;
-
-       /* Block signal for entire process, so only our thread processes it. */
-       setmask(&mask);
-       ret = pthread_sigmask(SIG_BLOCK, &mask, NULL);
-       if (ret) {
-               errno = ret;
-               PERROR("pthread_sigmask");
-               return -1;
-       }
-       return 0;
-}
-
-/*
- * This thread is the sighandler for signals LTTNG_CONSUMER_SIG_SWITCH,
- * LTTNG_CONSUMER_SIG_TEARDOWN and LTTNG_CONSUMER_SIG_LIVE.
- */
-void *consumer_timer_thread(void *data)
-{
-       int signr;
-       sigset_t mask;
-       siginfo_t info;
-       struct lttng_consumer_local_data *ctx = data;
-
-       rcu_register_thread();
-
-       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_METADATA_TIMER);
-
-       if (testpoint(consumerd_thread_metadata_timer)) {
-               goto error_testpoint;
-       }
-
-       health_code_update();
-
-       /* Only self thread will receive signal mask. */
-       setmask(&mask);
-       CMM_STORE_SHARED(timer_signal.tid, pthread_self());
-
-       while (1) {
-               health_code_update();
-
-               health_poll_entry();
-               signr = sigwaitinfo(&mask, &info);
-               health_poll_exit();
-               if (signr == -1) {
-                       if (errno != EINTR) {
-                               PERROR("sigwaitinfo");
-                       }
-                       continue;
-               } else if (signr == LTTNG_CONSUMER_SIG_SWITCH) {
-                       metadata_switch_timer(ctx, info.si_signo, &info, NULL);
-               } else if (signr == LTTNG_CONSUMER_SIG_TEARDOWN) {
-                       cmm_smp_mb();
-                       CMM_STORE_SHARED(timer_signal.qs_done, 1);
-                       cmm_smp_mb();
-                       DBG("Signal timer metadata thread teardown");
-               } else if (signr == LTTNG_CONSUMER_SIG_LIVE) {
-                       live_timer(ctx, info.si_signo, &info, NULL);
-               } else {
-                       ERR("Unexpected signal %d\n", info.si_signo);
-               }
-       }
-
-error_testpoint:
-       /* Only reached in testpoint error */
-       health_error();
-       health_unregister(health_consumerd);
-
-       rcu_unregister_thread();
-
-       /* Never return */
-       return NULL;
-}
diff --git a/src/common/consumer-timer.h b/src/common/consumer-timer.h
deleted file mode 100644 (file)
index 22e7457..0000000
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Copyright (C) 2011 - Julien Desfossez <julien.desfossez@polymtl.ca>
- *                      Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
- *               2012 - David Goulet <dgoulet@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify
- * it under the terms of the GNU General Public License, version 2 only,
- * as published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along
- * with this program; if not, write to the Free Software Foundation, Inc.,
- * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#ifndef CONSUMER_TIMER_H
-#define CONSUMER_TIMER_H
-
-#include <pthread.h>
-
-#include "consumer.h"
-
-#define LTTNG_CONSUMER_SIG_SWITCH      SIGRTMIN + 10
-#define LTTNG_CONSUMER_SIG_TEARDOWN    SIGRTMIN + 11
-#define LTTNG_CONSUMER_SIG_LIVE                SIGRTMIN + 12
-
-#define CLOCKID CLOCK_MONOTONIC
-
-/*
- * Handle timer teardown race wrt memory free of private data by consumer
- * signals are handled by a single thread, which permits a synchronization
- * point between handling of each signal. Internal lock ensures mutual
- * exclusion.
- */
-struct timer_signal_data {
-       pthread_t tid;  /* thread id managing signals */
-       int setup_done;
-       int qs_done;
-       pthread_mutex_t lock;
-};
-
-void consumer_timer_switch_start(struct lttng_consumer_channel *channel,
-               unsigned int switch_timer_interval);
-void consumer_timer_switch_stop(struct lttng_consumer_channel *channel);
-void consumer_timer_live_start(struct lttng_consumer_channel *channel,
-               int live_timer_interval);
-void consumer_timer_live_stop(struct lttng_consumer_channel *channel);
-void *consumer_timer_thread(void *data);
-int consumer_signal_init(void);
-
-int consumer_flush_kernel_index(struct lttng_consumer_stream *stream);
-int consumer_flush_ust_index(struct lttng_consumer_stream *stream);
-
-#endif /* CONSUMER_TIMER_H */
diff --git a/src/common/consumer.c b/src/common/consumer.c
deleted file mode 100644 (file)
index 9909fc1..0000000
+++ /dev/null
@@ -1,3713 +0,0 @@
-/*
- * Copyright (C) 2011 - Julien Desfossez <julien.desfossez@polymtl.ca>
- *                      Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
- *               2012 - David Goulet <dgoulet@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify
- * it under the terms of the GNU General Public License, version 2 only,
- * as published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along
- * with this program; if not, write to the Free Software Foundation, Inc.,
- * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#define _LGPL_SOURCE
-#include <assert.h>
-#include <poll.h>
-#include <pthread.h>
-#include <stdlib.h>
-#include <string.h>
-#include <sys/mman.h>
-#include <sys/socket.h>
-#include <sys/types.h>
-#include <unistd.h>
-#include <inttypes.h>
-#include <signal.h>
-
-#include <bin/lttng-consumerd/health-consumerd.h>
-#include <common/common.h>
-#include <common/utils.h>
-#include <common/compat/poll.h>
-#include <common/compat/endian.h>
-#include <common/index/index.h>
-#include <common/kernel-ctl/kernel-ctl.h>
-#include <common/sessiond-comm/relayd.h>
-#include <common/sessiond-comm/sessiond-comm.h>
-#include <common/kernel-consumer/kernel-consumer.h>
-#include <common/relayd/relayd.h>
-#include <common/ust-consumer/ust-consumer.h>
-#include <common/consumer-timer.h>
-
-#include "consumer.h"
-#include "consumer-stream.h"
-#include "consumer-testpoint.h"
-#include "align.h"
-
-struct lttng_consumer_global_data consumer_data = {
-       .stream_count = 0,
-       .need_update = 1,
-       .type = LTTNG_CONSUMER_UNKNOWN,
-};
-
-enum consumer_channel_action {
-       CONSUMER_CHANNEL_ADD,
-       CONSUMER_CHANNEL_DEL,
-       CONSUMER_CHANNEL_QUIT,
-};
-
-struct consumer_channel_msg {
-       enum consumer_channel_action action;
-       struct lttng_consumer_channel *chan;    /* add */
-       uint64_t key;                           /* del */
-};
-
-/*
- * Flag to inform the polling thread to quit when all fd hung up. Updated by
- * the consumer_thread_receive_fds when it notices that all fds has hung up.
- * Also updated by the signal handler (consumer_should_exit()). Read by the
- * polling threads.
- */
-volatile int consumer_quit;
-
-/*
- * Global hash table containing respectively metadata and data streams. The
- * stream element in this ht should only be updated by the metadata poll thread
- * for the metadata and the data poll thread for the data.
- */
-static struct lttng_ht *metadata_ht;
-static struct lttng_ht *data_ht;
-
-/*
- * Notify a thread lttng pipe to poll back again. This usually means that some
- * global state has changed so we just send back the thread in a poll wait
- * call.
- */
-static void notify_thread_lttng_pipe(struct lttng_pipe *pipe)
-{
-       struct lttng_consumer_stream *null_stream = NULL;
-
-       assert(pipe);
-
-       (void) lttng_pipe_write(pipe, &null_stream, sizeof(null_stream));
-}
-
-static void notify_health_quit_pipe(int *pipe)
-{
-       ssize_t ret;
-
-       ret = lttng_write(pipe[1], "4", 1);
-       if (ret < 1) {
-               PERROR("write consumer health quit");
-       }
-}
-
-static void notify_channel_pipe(struct lttng_consumer_local_data *ctx,
-               struct lttng_consumer_channel *chan,
-               uint64_t key,
-               enum consumer_channel_action action)
-{
-       struct consumer_channel_msg msg;
-       ssize_t ret;
-
-       memset(&msg, 0, sizeof(msg));
-
-       msg.action = action;
-       msg.chan = chan;
-       msg.key = key;
-       ret = lttng_write(ctx->consumer_channel_pipe[1], &msg, sizeof(msg));
-       if (ret < sizeof(msg)) {
-               PERROR("notify_channel_pipe write error");
-       }
-}
-
-void notify_thread_del_channel(struct lttng_consumer_local_data *ctx,
-               uint64_t key)
-{
-       notify_channel_pipe(ctx, NULL, key, CONSUMER_CHANNEL_DEL);
-}
-
-static int read_channel_pipe(struct lttng_consumer_local_data *ctx,
-               struct lttng_consumer_channel **chan,
-               uint64_t *key,
-               enum consumer_channel_action *action)
-{
-       struct consumer_channel_msg msg;
-       ssize_t ret;
-
-       ret = lttng_read(ctx->consumer_channel_pipe[0], &msg, sizeof(msg));
-       if (ret < sizeof(msg)) {
-               ret = -1;
-               goto error;
-       }
-       *action = msg.action;
-       *chan = msg.chan;
-       *key = msg.key;
-error:
-       return (int) ret;
-}
-
-/*
- * Cleanup the stream list of a channel. Those streams are not yet globally
- * visible
- */
-static void clean_channel_stream_list(struct lttng_consumer_channel *channel)
-{
-       struct lttng_consumer_stream *stream, *stmp;
-
-       assert(channel);
-
-       /* Delete streams that might have been left in the stream list. */
-       cds_list_for_each_entry_safe(stream, stmp, &channel->streams.head,
-                       send_node) {
-               cds_list_del(&stream->send_node);
-               /*
-                * Once a stream is added to this list, the buffers were created so we
-                * have a guarantee that this call will succeed. Setting the monitor
-                * mode to 0 so we don't lock nor try to delete the stream from the
-                * global hash table.
-                */
-               stream->monitor = 0;
-               consumer_stream_destroy(stream, NULL);
-       }
-}
-
-/*
- * Find a stream. The consumer_data.lock must be locked during this
- * call.
- */
-static struct lttng_consumer_stream *find_stream(uint64_t key,
-               struct lttng_ht *ht)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_ht_node_u64 *node;
-       struct lttng_consumer_stream *stream = NULL;
-
-       assert(ht);
-
-       /* -1ULL keys are lookup failures */
-       if (key == (uint64_t) -1ULL) {
-               return NULL;
-       }
-
-       rcu_read_lock();
-
-       lttng_ht_lookup(ht, &key, &iter);
-       node = lttng_ht_iter_get_node_u64(&iter);
-       if (node != NULL) {
-               stream = caa_container_of(node, struct lttng_consumer_stream, node);
-       }
-
-       rcu_read_unlock();
-
-       return stream;
-}
-
-static void steal_stream_key(uint64_t key, struct lttng_ht *ht)
-{
-       struct lttng_consumer_stream *stream;
-
-       rcu_read_lock();
-       stream = find_stream(key, ht);
-       if (stream) {
-               stream->key = (uint64_t) -1ULL;
-               /*
-                * We don't want the lookup to match, but we still need
-                * to iterate on this stream when iterating over the hash table. Just
-                * change the node key.
-                */
-               stream->node.key = (uint64_t) -1ULL;
-       }
-       rcu_read_unlock();
-}
-
-/*
- * Return a channel object for the given key.
- *
- * RCU read side lock MUST be acquired before calling this function and
- * protects the channel ptr.
- */
-struct lttng_consumer_channel *consumer_find_channel(uint64_t key)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_ht_node_u64 *node;
-       struct lttng_consumer_channel *channel = NULL;
-
-       /* -1ULL keys are lookup failures */
-       if (key == (uint64_t) -1ULL) {
-               return NULL;
-       }
-
-       lttng_ht_lookup(consumer_data.channel_ht, &key, &iter);
-       node = lttng_ht_iter_get_node_u64(&iter);
-       if (node != NULL) {
-               channel = caa_container_of(node, struct lttng_consumer_channel, node);
-       }
-
-       return channel;
-}
-
-/*
- * There is a possibility that the consumer does not have enough time between
- * the close of the channel on the session daemon and the cleanup in here thus
- * once we have a channel add with an existing key, we know for sure that this
- * channel will eventually get cleaned up by all streams being closed.
- *
- * This function just nullifies the already existing channel key.
- */
-static void steal_channel_key(uint64_t key)
-{
-       struct lttng_consumer_channel *channel;
-
-       rcu_read_lock();
-       channel = consumer_find_channel(key);
-       if (channel) {
-               channel->key = (uint64_t) -1ULL;
-               /*
-                * We don't want the lookup to match, but we still need to iterate on
-                * this channel when iterating over the hash table. Just change the
-                * node key.
-                */
-               channel->node.key = (uint64_t) -1ULL;
-       }
-       rcu_read_unlock();
-}
-
-static void free_channel_rcu(struct rcu_head *head)
-{
-       struct lttng_ht_node_u64 *node =
-               caa_container_of(head, struct lttng_ht_node_u64, head);
-       struct lttng_consumer_channel *channel =
-               caa_container_of(node, struct lttng_consumer_channel, node);
-
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               lttng_ustconsumer_free_channel(channel);
-               break;
-       default:
-               ERR("Unknown consumer_data type");
-               abort();
-       }
-       free(channel);
-}
-
-/*
- * RCU protected relayd socket pair free.
- */
-static void free_relayd_rcu(struct rcu_head *head)
-{
-       struct lttng_ht_node_u64 *node =
-               caa_container_of(head, struct lttng_ht_node_u64, head);
-       struct consumer_relayd_sock_pair *relayd =
-               caa_container_of(node, struct consumer_relayd_sock_pair, node);
-
-       /*
-        * Close all sockets. This is done in the call RCU since we don't want the
-        * socket fds to be reassigned thus potentially creating bad state of the
-        * relayd object.
-        *
-        * We do not have to lock the control socket mutex here since at this stage
-        * there is no one referencing to this relayd object.
-        */
-       (void) relayd_close(&relayd->control_sock);
-       (void) relayd_close(&relayd->data_sock);
-
-       free(relayd);
-}
-
-/*
- * Destroy and free relayd socket pair object.
- */
-void consumer_destroy_relayd(struct consumer_relayd_sock_pair *relayd)
-{
-       int ret;
-       struct lttng_ht_iter iter;
-
-       if (relayd == NULL) {
-               return;
-       }
-
-       DBG("Consumer destroy and close relayd socket pair");
-
-       iter.iter.node = &relayd->node.node;
-       ret = lttng_ht_del(consumer_data.relayd_ht, &iter);
-       if (ret != 0) {
-               /* We assume the relayd is being or is destroyed */
-               return;
-       }
-
-       /* RCU free() call */
-       call_rcu(&relayd->node.head, free_relayd_rcu);
-}
-
-/*
- * Remove a channel from the global list protected by a mutex. This function is
- * also responsible for freeing its data structures.
- */
-void consumer_del_channel(struct lttng_consumer_channel *channel)
-{
-       int ret;
-       struct lttng_ht_iter iter;
-
-       DBG("Consumer delete channel key %" PRIu64, channel->key);
-
-       pthread_mutex_lock(&consumer_data.lock);
-       pthread_mutex_lock(&channel->lock);
-
-       /* Destroy streams that might have been left in the stream list. */
-       clean_channel_stream_list(channel);
-
-       if (channel->live_timer_enabled == 1) {
-               consumer_timer_live_stop(channel);
-       }
-
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               lttng_ustconsumer_del_channel(channel);
-               break;
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-               goto end;
-       }
-
-       rcu_read_lock();
-       iter.iter.node = &channel->node.node;
-       ret = lttng_ht_del(consumer_data.channel_ht, &iter);
-       assert(!ret);
-       rcu_read_unlock();
-
-       call_rcu(&channel->node.head, free_channel_rcu);
-end:
-       pthread_mutex_unlock(&channel->lock);
-       pthread_mutex_unlock(&consumer_data.lock);
-}
-
-/*
- * Iterate over the relayd hash table and destroy each element. Finally,
- * destroy the whole hash table.
- */
-static void cleanup_relayd_ht(void)
-{
-       struct lttng_ht_iter iter;
-       struct consumer_relayd_sock_pair *relayd;
-
-       rcu_read_lock();
-
-       cds_lfht_for_each_entry(consumer_data.relayd_ht->ht, &iter.iter, relayd,
-                       node.node) {
-               consumer_destroy_relayd(relayd);
-       }
-
-       rcu_read_unlock();
-
-       lttng_ht_destroy(consumer_data.relayd_ht);
-}
-
-/*
- * Update the end point status of all streams having the given network sequence
- * index (relayd index).
- *
- * It's atomically set without having the stream mutex locked which is fine
- * because we handle the write/read race with a pipe wakeup for each thread.
- */
-static void update_endpoint_status_by_netidx(uint64_t net_seq_idx,
-               enum consumer_endpoint_status status)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_consumer_stream *stream;
-
-       DBG("Consumer set delete flag on stream by idx %" PRIu64, net_seq_idx);
-
-       rcu_read_lock();
-
-       /* Let's begin with metadata */
-       cds_lfht_for_each_entry(metadata_ht->ht, &iter.iter, stream, node.node) {
-               if (stream->net_seq_idx == net_seq_idx) {
-                       uatomic_set(&stream->endpoint_status, status);
-                       DBG("Delete flag set to metadata stream %d", stream->wait_fd);
-               }
-       }
-
-       /* Follow up by the data streams */
-       cds_lfht_for_each_entry(data_ht->ht, &iter.iter, stream, node.node) {
-               if (stream->net_seq_idx == net_seq_idx) {
-                       uatomic_set(&stream->endpoint_status, status);
-                       DBG("Delete flag set to data stream %d", stream->wait_fd);
-               }
-       }
-       rcu_read_unlock();
-}
-
-/*
- * Cleanup a relayd object by flagging every associated streams for deletion,
- * destroying the object meaning removing it from the relayd hash table,
- * closing the sockets and freeing the memory in a RCU call.
- *
- * If a local data context is available, notify the threads that the streams'
- * state have changed.
- */
-static void cleanup_relayd(struct consumer_relayd_sock_pair *relayd,
-               struct lttng_consumer_local_data *ctx)
-{
-       uint64_t netidx;
-
-       assert(relayd);
-
-       DBG("Cleaning up relayd sockets");
-
-       /* Save the net sequence index before destroying the object */
-       netidx = relayd->net_seq_idx;
-
-       /*
-        * Delete the relayd from the relayd hash table, close the sockets and free
-        * the object in a RCU call.
-        */
-       consumer_destroy_relayd(relayd);
-
-       /* Set inactive endpoint to all streams */
-       update_endpoint_status_by_netidx(netidx, CONSUMER_ENDPOINT_INACTIVE);
-
-       /*
-        * With a local data context, notify the threads that the streams' state
-        * have changed. The write() action on the pipe acts as an "implicit"
-        * memory barrier ordering the updates of the end point status from the
-        * read of this status which happens AFTER receiving this notify.
-        */
-       if (ctx) {
-               notify_thread_lttng_pipe(ctx->consumer_data_pipe);
-               notify_thread_lttng_pipe(ctx->consumer_metadata_pipe);
-       }
-}
-
-/*
- * Flag a relayd socket pair for destruction. Destroy it if the refcount
- * reaches zero.
- *
- * RCU read side lock MUST be aquired before calling this function.
- */
-void consumer_flag_relayd_for_destroy(struct consumer_relayd_sock_pair *relayd)
-{
-       assert(relayd);
-
-       /* Set destroy flag for this object */
-       uatomic_set(&relayd->destroy_flag, 1);
-
-       /* Destroy the relayd if refcount is 0 */
-       if (uatomic_read(&relayd->refcount) == 0) {
-               consumer_destroy_relayd(relayd);
-       }
-}
-
-/*
- * Completly destroy stream from every visiable data structure and the given
- * hash table if one.
- *
- * One this call returns, the stream object is not longer usable nor visible.
- */
-void consumer_del_stream(struct lttng_consumer_stream *stream,
-               struct lttng_ht *ht)
-{
-       consumer_stream_destroy(stream, ht);
-}
-
-/*
- * XXX naming of del vs destroy is all mixed up.
- */
-void consumer_del_stream_for_data(struct lttng_consumer_stream *stream)
-{
-       consumer_stream_destroy(stream, data_ht);
-}
-
-void consumer_del_stream_for_metadata(struct lttng_consumer_stream *stream)
-{
-       consumer_stream_destroy(stream, metadata_ht);
-}
-
-struct lttng_consumer_stream *consumer_allocate_stream(uint64_t channel_key,
-               uint64_t stream_key,
-               enum lttng_consumer_stream_state state,
-               const char *channel_name,
-               uid_t uid,
-               gid_t gid,
-               uint64_t relayd_id,
-               uint64_t session_id,
-               int cpu,
-               int *alloc_ret,
-               enum consumer_channel_type type,
-               unsigned int monitor)
-{
-       int ret;
-       struct lttng_consumer_stream *stream;
-
-       stream = zmalloc(sizeof(*stream));
-       if (stream == NULL) {
-               PERROR("malloc struct lttng_consumer_stream");
-               ret = -ENOMEM;
-               goto end;
-       }
-
-       rcu_read_lock();
-
-       stream->key = stream_key;
-       stream->out_fd = -1;
-       stream->out_fd_offset = 0;
-       stream->output_written = 0;
-       stream->state = state;
-       stream->uid = uid;
-       stream->gid = gid;
-       stream->net_seq_idx = relayd_id;
-       stream->session_id = session_id;
-       stream->monitor = monitor;
-       stream->endpoint_status = CONSUMER_ENDPOINT_ACTIVE;
-       stream->index_fd = -1;
-       pthread_mutex_init(&stream->lock, NULL);
-       pthread_mutex_init(&stream->metadata_timer_lock, NULL);
-
-       /* If channel is the metadata, flag this stream as metadata. */
-       if (type == CONSUMER_CHANNEL_TYPE_METADATA) {
-               stream->metadata_flag = 1;
-               /* Metadata is flat out. */
-               strncpy(stream->name, DEFAULT_METADATA_NAME, sizeof(stream->name));
-               /* Live rendez-vous point. */
-               pthread_cond_init(&stream->metadata_rdv, NULL);
-               pthread_mutex_init(&stream->metadata_rdv_lock, NULL);
-       } else {
-               /* Format stream name to <channel_name>_<cpu_number> */
-               ret = snprintf(stream->name, sizeof(stream->name), "%s_%d",
-                               channel_name, cpu);
-               if (ret < 0) {
-                       PERROR("snprintf stream name");
-                       goto error;
-               }
-       }
-
-       /* Key is always the wait_fd for streams. */
-       lttng_ht_node_init_u64(&stream->node, stream->key);
-
-       /* Init node per channel id key */
-       lttng_ht_node_init_u64(&stream->node_channel_id, channel_key);
-
-       /* Init session id node with the stream session id */
-       lttng_ht_node_init_u64(&stream->node_session_id, stream->session_id);
-
-       DBG3("Allocated stream %s (key %" PRIu64 ", chan_key %" PRIu64
-                       " relayd_id %" PRIu64 ", session_id %" PRIu64,
-                       stream->name, stream->key, channel_key,
-                       stream->net_seq_idx, stream->session_id);
-
-       rcu_read_unlock();
-       return stream;
-
-error:
-       rcu_read_unlock();
-       free(stream);
-end:
-       if (alloc_ret) {
-               *alloc_ret = ret;
-       }
-       return NULL;
-}
-
-/*
- * Add a stream to the global list protected by a mutex.
- */
-int consumer_add_data_stream(struct lttng_consumer_stream *stream)
-{
-       struct lttng_ht *ht = data_ht;
-       int ret = 0;
-
-       assert(stream);
-       assert(ht);
-
-       DBG3("Adding consumer stream %" PRIu64, stream->key);
-
-       pthread_mutex_lock(&consumer_data.lock);
-       pthread_mutex_lock(&stream->chan->lock);
-       pthread_mutex_lock(&stream->chan->timer_lock);
-       pthread_mutex_lock(&stream->lock);
-       rcu_read_lock();
-
-       /* Steal stream identifier to avoid having streams with the same key */
-       steal_stream_key(stream->key, ht);
-
-       lttng_ht_add_unique_u64(ht, &stream->node);
-
-       lttng_ht_add_u64(consumer_data.stream_per_chan_id_ht,
-                       &stream->node_channel_id);
-
-       /*
-        * Add stream to the stream_list_ht of the consumer data. No need to steal
-        * the key since the HT does not use it and we allow to add redundant keys
-        * into this table.
-        */
-       lttng_ht_add_u64(consumer_data.stream_list_ht, &stream->node_session_id);
-
-       /*
-        * When nb_init_stream_left reaches 0, we don't need to trigger any action
-        * in terms of destroying the associated channel, because the action that
-        * causes the count to become 0 also causes a stream to be added. The
-        * channel deletion will thus be triggered by the following removal of this
-        * stream.
-        */
-       if (uatomic_read(&stream->chan->nb_init_stream_left) > 0) {
-               /* Increment refcount before decrementing nb_init_stream_left */
-               cmm_smp_wmb();
-               uatomic_dec(&stream->chan->nb_init_stream_left);
-       }
-
-       /* Update consumer data once the node is inserted. */
-       consumer_data.stream_count++;
-       consumer_data.need_update = 1;
-
-       rcu_read_unlock();
-       pthread_mutex_unlock(&stream->lock);
-       pthread_mutex_unlock(&stream->chan->timer_lock);
-       pthread_mutex_unlock(&stream->chan->lock);
-       pthread_mutex_unlock(&consumer_data.lock);
-
-       return ret;
-}
-
-void consumer_del_data_stream(struct lttng_consumer_stream *stream)
-{
-       consumer_del_stream(stream, data_ht);
-}
-
-/*
- * Add relayd socket to global consumer data hashtable. RCU read side lock MUST
- * be acquired before calling this.
- */
-static int add_relayd(struct consumer_relayd_sock_pair *relayd)
-{
-       int ret = 0;
-       struct lttng_ht_node_u64 *node;
-       struct lttng_ht_iter iter;
-
-       assert(relayd);
-
-       lttng_ht_lookup(consumer_data.relayd_ht,
-                       &relayd->net_seq_idx, &iter);
-       node = lttng_ht_iter_get_node_u64(&iter);
-       if (node != NULL) {
-               goto end;
-       }
-       lttng_ht_add_unique_u64(consumer_data.relayd_ht, &relayd->node);
-
-end:
-       return ret;
-}
-
-/*
- * Allocate and return a consumer relayd socket.
- */
-static struct consumer_relayd_sock_pair *consumer_allocate_relayd_sock_pair(
-               uint64_t net_seq_idx)
-{
-       struct consumer_relayd_sock_pair *obj = NULL;
-
-       /* net sequence index of -1 is a failure */
-       if (net_seq_idx == (uint64_t) -1ULL) {
-               goto error;
-       }
-
-       obj = zmalloc(sizeof(struct consumer_relayd_sock_pair));
-       if (obj == NULL) {
-               PERROR("zmalloc relayd sock");
-               goto error;
-       }
-
-       obj->net_seq_idx = net_seq_idx;
-       obj->refcount = 0;
-       obj->destroy_flag = 0;
-       obj->control_sock.sock.fd = -1;
-       obj->data_sock.sock.fd = -1;
-       lttng_ht_node_init_u64(&obj->node, obj->net_seq_idx);
-       pthread_mutex_init(&obj->ctrl_sock_mutex, NULL);
-
-error:
-       return obj;
-}
-
-/*
- * Find a relayd socket pair in the global consumer data.
- *
- * Return the object if found else NULL.
- * RCU read-side lock must be held across this call and while using the
- * returned object.
- */
-struct consumer_relayd_sock_pair *consumer_find_relayd(uint64_t key)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_ht_node_u64 *node;
-       struct consumer_relayd_sock_pair *relayd = NULL;
-
-       /* Negative keys are lookup failures */
-       if (key == (uint64_t) -1ULL) {
-               goto error;
-       }
-
-       lttng_ht_lookup(consumer_data.relayd_ht, &key,
-                       &iter);
-       node = lttng_ht_iter_get_node_u64(&iter);
-       if (node != NULL) {
-               relayd = caa_container_of(node, struct consumer_relayd_sock_pair, node);
-       }
-
-error:
-       return relayd;
-}
-
-/*
- * Find a relayd and send the stream
- *
- * Returns 0 on success, < 0 on error
- */
-int consumer_send_relayd_stream(struct lttng_consumer_stream *stream,
-               char *path)
-{
-       int ret = 0;
-       struct consumer_relayd_sock_pair *relayd;
-
-       assert(stream);
-       assert(stream->net_seq_idx != -1ULL);
-       assert(path);
-
-       /* The stream is not metadata. Get relayd reference if exists. */
-       rcu_read_lock();
-       relayd = consumer_find_relayd(stream->net_seq_idx);
-       if (relayd != NULL) {
-               /* Add stream on the relayd */
-               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-               ret = relayd_add_stream(&relayd->control_sock, stream->name,
-                               path, &stream->relayd_stream_id,
-                               stream->chan->tracefile_size, stream->chan->tracefile_count);
-               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-               if (ret < 0) {
-                       goto end;
-               }
-
-               uatomic_inc(&relayd->refcount);
-               stream->sent_to_relayd = 1;
-       } else {
-               ERR("Stream %" PRIu64 " relayd ID %" PRIu64 " unknown. Can't send it.",
-                               stream->key, stream->net_seq_idx);
-               ret = -1;
-               goto end;
-       }
-
-       DBG("Stream %s with key %" PRIu64 " sent to relayd id %" PRIu64,
-                       stream->name, stream->key, stream->net_seq_idx);
-
-end:
-       rcu_read_unlock();
-       return ret;
-}
-
-/*
- * Find a relayd and send the streams sent message
- *
- * Returns 0 on success, < 0 on error
- */
-int consumer_send_relayd_streams_sent(uint64_t net_seq_idx)
-{
-       int ret = 0;
-       struct consumer_relayd_sock_pair *relayd;
-
-       assert(net_seq_idx != -1ULL);
-
-       /* The stream is not metadata. Get relayd reference if exists. */
-       rcu_read_lock();
-       relayd = consumer_find_relayd(net_seq_idx);
-       if (relayd != NULL) {
-               /* Add stream on the relayd */
-               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-               ret = relayd_streams_sent(&relayd->control_sock);
-               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-               if (ret < 0) {
-                       goto end;
-               }
-       } else {
-               ERR("Relayd ID %" PRIu64 " unknown. Can't send streams_sent.",
-                               net_seq_idx);
-               ret = -1;
-               goto end;
-       }
-
-       ret = 0;
-       DBG("All streams sent relayd id %" PRIu64, net_seq_idx);
-
-end:
-       rcu_read_unlock();
-       return ret;
-}
-
-/*
- * Find a relayd and close the stream
- */
-void close_relayd_stream(struct lttng_consumer_stream *stream)
-{
-       struct consumer_relayd_sock_pair *relayd;
-
-       /* The stream is not metadata. Get relayd reference if exists. */
-       rcu_read_lock();
-       relayd = consumer_find_relayd(stream->net_seq_idx);
-       if (relayd) {
-               consumer_stream_relayd_close(stream, relayd);
-       }
-       rcu_read_unlock();
-}
-
-/*
- * Handle stream for relayd transmission if the stream applies for network
- * streaming where the net sequence index is set.
- *
- * Return destination file descriptor or negative value on error.
- */
-static int write_relayd_stream_header(struct lttng_consumer_stream *stream,
-               size_t data_size, unsigned long padding,
-               struct consumer_relayd_sock_pair *relayd)
-{
-       int outfd = -1, ret;
-       struct lttcomm_relayd_data_hdr data_hdr;
-
-       /* Safety net */
-       assert(stream);
-       assert(relayd);
-
-       /* Reset data header */
-       memset(&data_hdr, 0, sizeof(data_hdr));
-
-       if (stream->metadata_flag) {
-               /* Caller MUST acquire the relayd control socket lock */
-               ret = relayd_send_metadata(&relayd->control_sock, data_size);
-               if (ret < 0) {
-                       goto error;
-               }
-
-               /* Metadata are always sent on the control socket. */
-               outfd = relayd->control_sock.sock.fd;
-       } else {
-               /* Set header with stream information */
-               data_hdr.stream_id = htobe64(stream->relayd_stream_id);
-               data_hdr.data_size = htobe32(data_size);
-               data_hdr.padding_size = htobe32(padding);
-               /*
-                * Note that net_seq_num below is assigned with the *current* value of
-                * next_net_seq_num and only after that the next_net_seq_num will be
-                * increment. This is why when issuing a command on the relayd using
-                * this next value, 1 should always be substracted in order to compare
-                * the last seen sequence number on the relayd side to the last sent.
-                */
-               data_hdr.net_seq_num = htobe64(stream->next_net_seq_num);
-               /* Other fields are zeroed previously */
-
-               ret = relayd_send_data_hdr(&relayd->data_sock, &data_hdr,
-                               sizeof(data_hdr));
-               if (ret < 0) {
-                       goto error;
-               }
-
-               ++stream->next_net_seq_num;
-
-               /* Set to go on data socket */
-               outfd = relayd->data_sock.sock.fd;
-       }
-
-error:
-       return outfd;
-}
-
-/*
- * Allocate and return a new lttng_consumer_channel object using the given key
- * to initialize the hash table node.
- *
- * On error, return NULL.
- */
-struct lttng_consumer_channel *consumer_allocate_channel(uint64_t key,
-               uint64_t session_id,
-               const char *pathname,
-               const char *name,
-               uid_t uid,
-               gid_t gid,
-               uint64_t relayd_id,
-               enum lttng_event_output output,
-               uint64_t tracefile_size,
-               uint64_t tracefile_count,
-               uint64_t session_id_per_pid,
-               unsigned int monitor,
-               unsigned int live_timer_interval,
-               const char *root_shm_path,
-               const char *shm_path)
-{
-       struct lttng_consumer_channel *channel;
-
-       channel = zmalloc(sizeof(*channel));
-       if (channel == NULL) {
-               PERROR("malloc struct lttng_consumer_channel");
-               goto end;
-       }
-
-       channel->key = key;
-       channel->refcount = 0;
-       channel->session_id = session_id;
-       channel->session_id_per_pid = session_id_per_pid;
-       channel->uid = uid;
-       channel->gid = gid;
-       channel->relayd_id = relayd_id;
-       channel->tracefile_size = tracefile_size;
-       channel->tracefile_count = tracefile_count;
-       channel->monitor = monitor;
-       channel->live_timer_interval = live_timer_interval;
-       pthread_mutex_init(&channel->lock, NULL);
-       pthread_mutex_init(&channel->timer_lock, NULL);
-
-       switch (output) {
-       case LTTNG_EVENT_SPLICE:
-               channel->output = CONSUMER_CHANNEL_SPLICE;
-               break;
-       case LTTNG_EVENT_MMAP:
-               channel->output = CONSUMER_CHANNEL_MMAP;
-               break;
-       default:
-               assert(0);
-               free(channel);
-               channel = NULL;
-               goto end;
-       }
-
-       /*
-        * In monitor mode, the streams associated with the channel will be put in
-        * a special list ONLY owned by this channel. So, the refcount is set to 1
-        * here meaning that the channel itself has streams that are referenced.
-        *
-        * On a channel deletion, once the channel is no longer visible, the
-        * refcount is decremented and checked for a zero value to delete it. With
-        * streams in no monitor mode, it will now be safe to destroy the channel.
-        */
-       if (!channel->monitor) {
-               channel->refcount = 1;
-       }
-
-       strncpy(channel->pathname, pathname, sizeof(channel->pathname));
-       channel->pathname[sizeof(channel->pathname) - 1] = '\0';
-
-       strncpy(channel->name, name, sizeof(channel->name));
-       channel->name[sizeof(channel->name) - 1] = '\0';
-
-       if (root_shm_path) {
-               strncpy(channel->root_shm_path, root_shm_path, sizeof(channel->root_shm_path));
-               channel->root_shm_path[sizeof(channel->root_shm_path) - 1] = '\0';
-       }
-       if (shm_path) {
-               strncpy(channel->shm_path, shm_path, sizeof(channel->shm_path));
-               channel->shm_path[sizeof(channel->shm_path) - 1] = '\0';
-       }
-
-       lttng_ht_node_init_u64(&channel->node, channel->key);
-
-       channel->wait_fd = -1;
-
-       CDS_INIT_LIST_HEAD(&channel->streams.head);
-
-       DBG("Allocated channel (key %" PRIu64 ")", channel->key)
-
-end:
-       return channel;
-}
-
-/*
- * Add a channel to the global list protected by a mutex.
- *
- * Always return 0 indicating success.
- */
-int consumer_add_channel(struct lttng_consumer_channel *channel,
-               struct lttng_consumer_local_data *ctx)
-{
-       pthread_mutex_lock(&consumer_data.lock);
-       pthread_mutex_lock(&channel->lock);
-       pthread_mutex_lock(&channel->timer_lock);
-
-       /*
-        * This gives us a guarantee that the channel we are about to add to the
-        * channel hash table will be unique. See this function comment on the why
-        * we need to steel the channel key at this stage.
-        */
-       steal_channel_key(channel->key);
-
-       rcu_read_lock();
-       lttng_ht_add_unique_u64(consumer_data.channel_ht, &channel->node);
-       rcu_read_unlock();
-
-       pthread_mutex_unlock(&channel->timer_lock);
-       pthread_mutex_unlock(&channel->lock);
-       pthread_mutex_unlock(&consumer_data.lock);
-
-       if (channel->wait_fd != -1 && channel->type == CONSUMER_CHANNEL_TYPE_DATA) {
-               notify_channel_pipe(ctx, channel, -1, CONSUMER_CHANNEL_ADD);
-       }
-
-       return 0;
-}
-
-/*
- * Allocate the pollfd structure and the local view of the out fds to avoid
- * doing a lookup in the linked list and concurrency issues when writing is
- * needed. Called with consumer_data.lock held.
- *
- * Returns the number of fds in the structures.
- */
-static int update_poll_array(struct lttng_consumer_local_data *ctx,
-               struct pollfd **pollfd, struct lttng_consumer_stream **local_stream,
-               struct lttng_ht *ht)
-{
-       int i = 0;
-       struct lttng_ht_iter iter;
-       struct lttng_consumer_stream *stream;
-
-       assert(ctx);
-       assert(ht);
-       assert(pollfd);
-       assert(local_stream);
-
-       DBG("Updating poll fd array");
-       rcu_read_lock();
-       cds_lfht_for_each_entry(ht->ht, &iter.iter, stream, node.node) {
-               /*
-                * Only active streams with an active end point can be added to the
-                * poll set and local stream storage of the thread.
-                *
-                * There is a potential race here for endpoint_status to be updated
-                * just after the check. However, this is OK since the stream(s) will
-                * be deleted once the thread is notified that the end point state has
-                * changed where this function will be called back again.
-                */
-               if (stream->state != LTTNG_CONSUMER_ACTIVE_STREAM ||
-                               stream->endpoint_status == CONSUMER_ENDPOINT_INACTIVE) {
-                       continue;
-               }
-               /*
-                * This clobbers way too much the debug output. Uncomment that if you
-                * need it for debugging purposes.
-                *
-                * DBG("Active FD %d", stream->wait_fd);
-                */
-               (*pollfd)[i].fd = stream->wait_fd;
-               (*pollfd)[i].events = POLLIN | POLLPRI;
-               local_stream[i] = stream;
-               i++;
-       }
-       rcu_read_unlock();
-
-       /*
-        * Insert the consumer_data_pipe at the end of the array and don't
-        * increment i so nb_fd is the number of real FD.
-        */
-       (*pollfd)[i].fd = lttng_pipe_get_readfd(ctx->consumer_data_pipe);
-       (*pollfd)[i].events = POLLIN | POLLPRI;
-
-       (*pollfd)[i + 1].fd = lttng_pipe_get_readfd(ctx->consumer_wakeup_pipe);
-       (*pollfd)[i + 1].events = POLLIN | POLLPRI;
-       return i;
-}
-
-/*
- * Poll on the should_quit pipe and the command socket return -1 on
- * error, 1 if should exit, 0 if data is available on the command socket
- */
-int lttng_consumer_poll_socket(struct pollfd *consumer_sockpoll)
-{
-       int num_rdy;
-
-restart:
-       num_rdy = poll(consumer_sockpoll, 2, -1);
-       if (num_rdy == -1) {
-               /*
-                * Restart interrupted system call.
-                */
-               if (errno == EINTR) {
-                       goto restart;
-               }
-               PERROR("Poll error");
-               return -1;
-       }
-       if (consumer_sockpoll[0].revents & (POLLIN | POLLPRI)) {
-               DBG("consumer_should_quit wake up");
-               return 1;
-       }
-       return 0;
-}
-
-/*
- * Set the error socket.
- */
-void lttng_consumer_set_error_sock(struct lttng_consumer_local_data *ctx,
-               int sock)
-{
-       ctx->consumer_error_socket = sock;
-}
-
-/*
- * Set the command socket path.
- */
-void lttng_consumer_set_command_sock_path(
-               struct lttng_consumer_local_data *ctx, char *sock)
-{
-       ctx->consumer_command_sock_path = sock;
-}
-
-/*
- * Send return code to the session daemon.
- * If the socket is not defined, we return 0, it is not a fatal error
- */
-int lttng_consumer_send_error(struct lttng_consumer_local_data *ctx, int cmd)
-{
-       if (ctx->consumer_error_socket > 0) {
-               return lttcomm_send_unix_sock(ctx->consumer_error_socket, &cmd,
-                               sizeof(enum lttcomm_sessiond_command));
-       }
-
-       return 0;
-}
-
-/*
- * Close all the tracefiles and stream fds and MUST be called when all
- * instances are destroyed i.e. when all threads were joined and are ended.
- */
-void lttng_consumer_cleanup(void)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_consumer_channel *channel;
-
-       rcu_read_lock();
-
-       cds_lfht_for_each_entry(consumer_data.channel_ht->ht, &iter.iter, channel,
-                       node.node) {
-               consumer_del_channel(channel);
-       }
-
-       rcu_read_unlock();
-
-       lttng_ht_destroy(consumer_data.channel_ht);
-
-       cleanup_relayd_ht();
-
-       lttng_ht_destroy(consumer_data.stream_per_chan_id_ht);
-
-       /*
-        * This HT contains streams that are freed by either the metadata thread or
-        * the data thread so we do *nothing* on the hash table and simply destroy
-        * it.
-        */
-       lttng_ht_destroy(consumer_data.stream_list_ht);
-}
-
-/*
- * Called from signal handler.
- */
-void lttng_consumer_should_exit(struct lttng_consumer_local_data *ctx)
-{
-       ssize_t ret;
-
-       consumer_quit = 1;
-       ret = lttng_write(ctx->consumer_should_quit[1], "4", 1);
-       if (ret < 1) {
-               PERROR("write consumer quit");
-       }
-
-       DBG("Consumer flag that it should quit");
-}
-
-void lttng_consumer_sync_trace_file(struct lttng_consumer_stream *stream,
-               off_t orig_offset)
-{
-       int outfd = stream->out_fd;
-
-       /*
-        * This does a blocking write-and-wait on any page that belongs to the
-        * subbuffer prior to the one we just wrote.
-        * Don't care about error values, as these are just hints and ways to
-        * limit the amount of page cache used.
-        */
-       if (orig_offset < stream->max_sb_size) {
-               return;
-       }
-       lttng_sync_file_range(outfd, orig_offset - stream->max_sb_size,
-                       stream->max_sb_size,
-                       SYNC_FILE_RANGE_WAIT_BEFORE
-                       | SYNC_FILE_RANGE_WRITE
-                       | SYNC_FILE_RANGE_WAIT_AFTER);
-       /*
-        * Give hints to the kernel about how we access the file:
-        * POSIX_FADV_DONTNEED : we won't re-access data in a near future after
-        * we write it.
-        *
-        * We need to call fadvise again after the file grows because the
-        * kernel does not seem to apply fadvise to non-existing parts of the
-        * file.
-        *
-        * Call fadvise _after_ having waited for the page writeback to
-        * complete because the dirty page writeback semantic is not well
-        * defined. So it can be expected to lead to lower throughput in
-        * streaming.
-        */
-       posix_fadvise(outfd, orig_offset - stream->max_sb_size,
-                       stream->max_sb_size, POSIX_FADV_DONTNEED);
-}
-
-/*
- * Initialise the necessary environnement :
- * - create a new context
- * - create the poll_pipe
- * - create the should_quit pipe (for signal handler)
- * - create the thread pipe (for splice)
- *
- * Takes a function pointer as argument, this function is called when data is
- * available on a buffer. This function is responsible to do the
- * kernctl_get_next_subbuf, read the data with mmap or splice depending on the
- * buffer configuration and then kernctl_put_next_subbuf at the end.
- *
- * Returns a pointer to the new context or NULL on error.
- */
-struct lttng_consumer_local_data *lttng_consumer_create(
-               enum lttng_consumer_type type,
-               ssize_t (*buffer_ready)(struct lttng_consumer_stream *stream,
-                       struct lttng_consumer_local_data *ctx),
-               int (*recv_channel)(struct lttng_consumer_channel *channel),
-               int (*recv_stream)(struct lttng_consumer_stream *stream),
-               int (*update_stream)(uint64_t stream_key, uint32_t state))
-{
-       int ret;
-       struct lttng_consumer_local_data *ctx;
-
-       assert(consumer_data.type == LTTNG_CONSUMER_UNKNOWN ||
-               consumer_data.type == type);
-       consumer_data.type = type;
-
-       ctx = zmalloc(sizeof(struct lttng_consumer_local_data));
-       if (ctx == NULL) {
-               PERROR("allocating context");
-               goto error;
-       }
-
-       ctx->consumer_error_socket = -1;
-       ctx->consumer_metadata_socket = -1;
-       pthread_mutex_init(&ctx->metadata_socket_lock, NULL);
-       /* assign the callbacks */
-       ctx->on_buffer_ready = buffer_ready;
-       ctx->on_recv_channel = recv_channel;
-       ctx->on_recv_stream = recv_stream;
-       ctx->on_update_stream = update_stream;
-
-       ctx->consumer_data_pipe = lttng_pipe_open(0);
-       if (!ctx->consumer_data_pipe) {
-               goto error_poll_pipe;
-       }
-
-       ctx->consumer_wakeup_pipe = lttng_pipe_open(0);
-       if (!ctx->consumer_wakeup_pipe) {
-               goto error_wakeup_pipe;
-       }
-
-       ret = pipe(ctx->consumer_should_quit);
-       if (ret < 0) {
-               PERROR("Error creating recv pipe");
-               goto error_quit_pipe;
-       }
-
-       ret = pipe(ctx->consumer_channel_pipe);
-       if (ret < 0) {
-               PERROR("Error creating channel pipe");
-               goto error_channel_pipe;
-       }
-
-       ctx->consumer_metadata_pipe = lttng_pipe_open(0);
-       if (!ctx->consumer_metadata_pipe) {
-               goto error_metadata_pipe;
-       }
-
-       return ctx;
-
-error_metadata_pipe:
-       utils_close_pipe(ctx->consumer_channel_pipe);
-error_channel_pipe:
-       utils_close_pipe(ctx->consumer_should_quit);
-error_quit_pipe:
-       lttng_pipe_destroy(ctx->consumer_wakeup_pipe);
-error_wakeup_pipe:
-       lttng_pipe_destroy(ctx->consumer_data_pipe);
-error_poll_pipe:
-       free(ctx);
-error:
-       return NULL;
-}
-
-/*
- * Iterate over all streams of the hashtable and free them properly.
- */
-static void destroy_data_stream_ht(struct lttng_ht *ht)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_consumer_stream *stream;
-
-       if (ht == NULL) {
-               return;
-       }
-
-       rcu_read_lock();
-       cds_lfht_for_each_entry(ht->ht, &iter.iter, stream, node.node) {
-               /*
-                * Ignore return value since we are currently cleaning up so any error
-                * can't be handled.
-                */
-               (void) consumer_del_stream(stream, ht);
-       }
-       rcu_read_unlock();
-
-       lttng_ht_destroy(ht);
-}
-
-/*
- * Iterate over all streams of the metadata hashtable and free them
- * properly.
- */
-static void destroy_metadata_stream_ht(struct lttng_ht *ht)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_consumer_stream *stream;
-
-       if (ht == NULL) {
-               return;
-       }
-
-       rcu_read_lock();
-       cds_lfht_for_each_entry(ht->ht, &iter.iter, stream, node.node) {
-               /*
-                * Ignore return value since we are currently cleaning up so any error
-                * can't be handled.
-                */
-               (void) consumer_del_metadata_stream(stream, ht);
-       }
-       rcu_read_unlock();
-
-       lttng_ht_destroy(ht);
-}
-
-/*
- * Close all fds associated with the instance and free the context.
- */
-void lttng_consumer_destroy(struct lttng_consumer_local_data *ctx)
-{
-       int ret;
-
-       DBG("Consumer destroying it. Closing everything.");
-
-       if (!ctx) {
-               return;
-       }
-
-       destroy_data_stream_ht(data_ht);
-       destroy_metadata_stream_ht(metadata_ht);
-
-       ret = close(ctx->consumer_error_socket);
-       if (ret) {
-               PERROR("close");
-       }
-       ret = close(ctx->consumer_metadata_socket);
-       if (ret) {
-               PERROR("close");
-       }
-       utils_close_pipe(ctx->consumer_channel_pipe);
-       lttng_pipe_destroy(ctx->consumer_data_pipe);
-       lttng_pipe_destroy(ctx->consumer_metadata_pipe);
-       lttng_pipe_destroy(ctx->consumer_wakeup_pipe);
-       utils_close_pipe(ctx->consumer_should_quit);
-
-       unlink(ctx->consumer_command_sock_path);
-       free(ctx);
-}
-
-/*
- * Write the metadata stream id on the specified file descriptor.
- */
-static int write_relayd_metadata_id(int fd,
-               struct lttng_consumer_stream *stream,
-               struct consumer_relayd_sock_pair *relayd, unsigned long padding)
-{
-       ssize_t ret;
-       struct lttcomm_relayd_metadata_payload hdr;
-
-       hdr.stream_id = htobe64(stream->relayd_stream_id);
-       hdr.padding_size = htobe32(padding);
-       ret = lttng_write(fd, (void *) &hdr, sizeof(hdr));
-       if (ret < sizeof(hdr)) {
-               /*
-                * This error means that the fd's end is closed so ignore the PERROR
-                * not to clubber the error output since this can happen in a normal
-                * code path.
-                */
-               if (errno != EPIPE) {
-                       PERROR("write metadata stream id");
-               }
-               DBG3("Consumer failed to write relayd metadata id (errno: %d)", errno);
-               /*
-                * Set ret to a negative value because if ret != sizeof(hdr), we don't
-                * handle writting the missing part so report that as an error and
-                * don't lie to the caller.
-                */
-               ret = -1;
-               goto end;
-       }
-       DBG("Metadata stream id %" PRIu64 " with padding %lu written before data",
-                       stream->relayd_stream_id, padding);
-
-end:
-       return (int) ret;
-}
-
-/*
- * Mmap the ring buffer, read it and write the data to the tracefile. This is a
- * core function for writing trace buffers to either the local filesystem or
- * the network.
- *
- * It must be called with the stream lock held.
- *
- * Careful review MUST be put if any changes occur!
- *
- * Returns the number of bytes written
- */
-ssize_t lttng_consumer_on_read_subbuffer_mmap(
-               struct lttng_consumer_local_data *ctx,
-               struct lttng_consumer_stream *stream, unsigned long len,
-               unsigned long padding,
-               struct ctf_packet_index *index)
-{
-       unsigned long mmap_offset;
-       void *mmap_base;
-       ssize_t ret = 0;
-       off_t orig_offset = stream->out_fd_offset;
-       /* Default is on the disk */
-       int outfd = stream->out_fd;
-       struct consumer_relayd_sock_pair *relayd = NULL;
-       unsigned int relayd_hang_up = 0;
-
-       /* RCU lock for the relayd pointer */
-       rcu_read_lock();
-
-       /* Flag that the current stream if set for network streaming. */
-       if (stream->net_seq_idx != (uint64_t) -1ULL) {
-               relayd = consumer_find_relayd(stream->net_seq_idx);
-               if (relayd == NULL) {
-                       ret = -EPIPE;
-                       goto end;
-               }
-       }
-
-       /* get the offset inside the fd to mmap */
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               mmap_base = stream->mmap_base;
-               ret = kernctl_get_mmap_read_offset(stream->wait_fd, &mmap_offset);
-               if (ret < 0) {
-                       ret = -errno;
-                       PERROR("tracer ctl get_mmap_read_offset");
-                       goto end;
-               }
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               mmap_base = lttng_ustctl_get_mmap_base(stream);
-               if (!mmap_base) {
-                       ERR("read mmap get mmap base for stream %s", stream->name);
-                       ret = -EPERM;
-                       goto end;
-               }
-               ret = lttng_ustctl_get_mmap_read_offset(stream, &mmap_offset);
-               if (ret != 0) {
-                       PERROR("tracer ctl get_mmap_read_offset");
-                       ret = -EINVAL;
-                       goto end;
-               }
-               break;
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-       }
-
-       /* Handle stream on the relayd if the output is on the network */
-       if (relayd) {
-               unsigned long netlen = len;
-
-               /*
-                * Lock the control socket for the complete duration of the function
-                * since from this point on we will use the socket.
-                */
-               if (stream->metadata_flag) {
-                       /* Metadata requires the control socket. */
-                       pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-                       netlen += sizeof(struct lttcomm_relayd_metadata_payload);
-               }
-
-               ret = write_relayd_stream_header(stream, netlen, padding, relayd);
-               if (ret < 0) {
-                       relayd_hang_up = 1;
-                       goto write_error;
-               }
-               /* Use the returned socket. */
-               outfd = ret;
-
-               /* Write metadata stream id before payload */
-               if (stream->metadata_flag) {
-                       ret = write_relayd_metadata_id(outfd, stream, relayd, padding);
-                       if (ret < 0) {
-                               relayd_hang_up = 1;
-                               goto write_error;
-                       }
-               }
-       } else {
-               /* No streaming, we have to set the len with the full padding */
-               len += padding;
-
-               /*
-                * Check if we need to change the tracefile before writing the packet.
-                */
-               if (stream->chan->tracefile_size > 0 &&
-                               (stream->tracefile_size_current + len) >
-                               stream->chan->tracefile_size) {
-                       ret = utils_rotate_stream_file(stream->chan->pathname,
-                                       stream->name, stream->chan->tracefile_size,
-                                       stream->chan->tracefile_count, stream->uid, stream->gid,
-                                       stream->out_fd, &(stream->tracefile_count_current),
-                                       &stream->out_fd);
-                       if (ret < 0) {
-                               ERR("Rotating output file");
-                               goto end;
-                       }
-                       outfd = stream->out_fd;
-
-                       if (stream->index_fd >= 0) {
-                               ret = index_create_file(stream->chan->pathname,
-                                               stream->name, stream->uid, stream->gid,
-                                               stream->chan->tracefile_size,
-                                               stream->tracefile_count_current);
-                               if (ret < 0) {
-                                       goto end;
-                               }
-                               stream->index_fd = ret;
-                       }
-
-                       /* Reset current size because we just perform a rotation. */
-                       stream->tracefile_size_current = 0;
-                       stream->out_fd_offset = 0;
-                       orig_offset = 0;
-               }
-               stream->tracefile_size_current += len;
-               if (index) {
-                       index->offset = htobe64(stream->out_fd_offset);
-               }
-       }
-
-       /*
-        * This call guarantee that len or less is returned. It's impossible to
-        * receive a ret value that is bigger than len.
-        */
-       ret = lttng_write(outfd, mmap_base + mmap_offset, len);
-       DBG("Consumer mmap write() ret %zd (len %lu)", ret, len);
-       if (ret < 0 || ((size_t) ret != len)) {
-               /*
-                * Report error to caller if nothing was written else at least send the
-                * amount written.
-                */
-               if (ret < 0) {
-                       ret = -errno;
-               }
-               relayd_hang_up = 1;
-
-               /* Socket operation failed. We consider the relayd dead */
-               if (errno == EPIPE || errno == EINVAL || errno == EBADF) {
-                       /*
-                        * This is possible if the fd is closed on the other side
-                        * (outfd) or any write problem. It can be verbose a bit for a
-                        * normal execution if for instance the relayd is stopped
-                        * abruptly. This can happen so set this to a DBG statement.
-                        */
-                       DBG("Consumer mmap write detected relayd hang up");
-               } else {
-                       /* Unhandled error, print it and stop function right now. */
-                       PERROR("Error in write mmap (ret %zd != len %lu)", ret, len);
-               }
-               goto write_error;
-       }
-       stream->output_written += ret;
-
-       /* This call is useless on a socket so better save a syscall. */
-       if (!relayd) {
-               /* This won't block, but will start writeout asynchronously */
-               lttng_sync_file_range(outfd, stream->out_fd_offset, len,
-                               SYNC_FILE_RANGE_WRITE);
-               stream->out_fd_offset += len;
-       }
-       lttng_consumer_sync_trace_file(stream, orig_offset);
-
-write_error:
-       /*
-        * This is a special case that the relayd has closed its socket. Let's
-        * cleanup the relayd object and all associated streams.
-        */
-       if (relayd && relayd_hang_up) {
-               cleanup_relayd(relayd, ctx);
-       }
-
-end:
-       /* Unlock only if ctrl socket used */
-       if (relayd && stream->metadata_flag) {
-               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-       }
-
-       rcu_read_unlock();
-       return ret;
-}
-
-/*
- * Splice the data from the ring buffer to the tracefile.
- *
- * It must be called with the stream lock held.
- *
- * Returns the number of bytes spliced.
- */
-ssize_t lttng_consumer_on_read_subbuffer_splice(
-               struct lttng_consumer_local_data *ctx,
-               struct lttng_consumer_stream *stream, unsigned long len,
-               unsigned long padding,
-               struct ctf_packet_index *index)
-{
-       ssize_t ret = 0, written = 0, ret_splice = 0;
-       loff_t offset = 0;
-       off_t orig_offset = stream->out_fd_offset;
-       int fd = stream->wait_fd;
-       /* Default is on the disk */
-       int outfd = stream->out_fd;
-       struct consumer_relayd_sock_pair *relayd = NULL;
-       int *splice_pipe;
-       unsigned int relayd_hang_up = 0;
-
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               /* Not supported for user space tracing */
-               return -ENOSYS;
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-       }
-
-       /* RCU lock for the relayd pointer */
-       rcu_read_lock();
-
-       /* Flag that the current stream if set for network streaming. */
-       if (stream->net_seq_idx != (uint64_t) -1ULL) {
-               relayd = consumer_find_relayd(stream->net_seq_idx);
-               if (relayd == NULL) {
-                       written = -ret;
-                       goto end;
-               }
-       }
-       splice_pipe = stream->splice_pipe;
-
-       /* Write metadata stream id before payload */
-       if (relayd) {
-               unsigned long total_len = len;
-
-               if (stream->metadata_flag) {
-                       /*
-                        * Lock the control socket for the complete duration of the function
-                        * since from this point on we will use the socket.
-                        */
-                       pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-
-                       ret = write_relayd_metadata_id(splice_pipe[1], stream, relayd,
-                                       padding);
-                       if (ret < 0) {
-                               written = ret;
-                               relayd_hang_up = 1;
-                               goto write_error;
-                       }
-
-                       total_len += sizeof(struct lttcomm_relayd_metadata_payload);
-               }
-
-               ret = write_relayd_stream_header(stream, total_len, padding, relayd);
-               if (ret < 0) {
-                       written = ret;
-                       relayd_hang_up = 1;
-                       goto write_error;
-               }
-               /* Use the returned socket. */
-               outfd = ret;
-       } else {
-               /* No streaming, we have to set the len with the full padding */
-               len += padding;
-
-               /*
-                * Check if we need to change the tracefile before writing the packet.
-                */
-               if (stream->chan->tracefile_size > 0 &&
-                               (stream->tracefile_size_current + len) >
-                               stream->chan->tracefile_size) {
-                       ret = utils_rotate_stream_file(stream->chan->pathname,
-                                       stream->name, stream->chan->tracefile_size,
-                                       stream->chan->tracefile_count, stream->uid, stream->gid,
-                                       stream->out_fd, &(stream->tracefile_count_current),
-                                       &stream->out_fd);
-                       if (ret < 0) {
-                               written = ret;
-                               ERR("Rotating output file");
-                               goto end;
-                       }
-                       outfd = stream->out_fd;
-
-                       if (stream->index_fd >= 0) {
-                               ret = index_create_file(stream->chan->pathname,
-                                               stream->name, stream->uid, stream->gid,
-                                               stream->chan->tracefile_size,
-                                               stream->tracefile_count_current);
-                               if (ret < 0) {
-                                       written = ret;
-                                       goto end;
-                               }
-                               stream->index_fd = ret;
-                       }
-
-                       /* Reset current size because we just perform a rotation. */
-                       stream->tracefile_size_current = 0;
-                       stream->out_fd_offset = 0;
-                       orig_offset = 0;
-               }
-               stream->tracefile_size_current += len;
-               index->offset = htobe64(stream->out_fd_offset);
-       }
-
-       while (len > 0) {
-               DBG("splice chan to pipe offset %lu of len %lu (fd : %d, pipe: %d)",
-                               (unsigned long)offset, len, fd, splice_pipe[1]);
-               ret_splice = splice(fd, &offset, splice_pipe[1], NULL, len,
-                               SPLICE_F_MOVE | SPLICE_F_MORE);
-               DBG("splice chan to pipe, ret %zd", ret_splice);
-               if (ret_splice < 0) {
-                       ret = errno;
-                       written = -ret;
-                       PERROR("Error in relay splice");
-                       goto splice_error;
-               }
-
-               /* Handle stream on the relayd if the output is on the network */
-               if (relayd && stream->metadata_flag) {
-                       size_t metadata_payload_size =
-                               sizeof(struct lttcomm_relayd_metadata_payload);
-
-                       /* Update counter to fit the spliced data */
-                       ret_splice += metadata_payload_size;
-                       len += metadata_payload_size;
-                       /*
-                        * We do this so the return value can match the len passed as
-                        * argument to this function.
-                        */
-                       written -= metadata_payload_size;
-               }
-
-               /* Splice data out */
-               ret_splice = splice(splice_pipe[0], NULL, outfd, NULL,
-                               ret_splice, SPLICE_F_MOVE | SPLICE_F_MORE);
-               DBG("Consumer splice pipe to file (out_fd: %d), ret %zd",
-                               outfd, ret_splice);
-               if (ret_splice < 0) {
-                       ret = errno;
-                       written = -ret;
-                       relayd_hang_up = 1;
-                       goto write_error;
-               } else if (ret_splice > len) {
-                       /*
-                        * We don't expect this code path to be executed but you never know
-                        * so this is an extra protection agains a buggy splice().
-                        */
-                       ret = errno;
-                       written += ret_splice;
-                       PERROR("Wrote more data than requested %zd (len: %lu)", ret_splice,
-                                       len);
-                       goto splice_error;
-               } else {
-                       /* All good, update current len and continue. */
-                       len -= ret_splice;
-               }
-
-               /* This call is useless on a socket so better save a syscall. */
-               if (!relayd) {
-                       /* This won't block, but will start writeout asynchronously */
-                       lttng_sync_file_range(outfd, stream->out_fd_offset, ret_splice,
-                                       SYNC_FILE_RANGE_WRITE);
-                       stream->out_fd_offset += ret_splice;
-               }
-               stream->output_written += ret_splice;
-               written += ret_splice;
-       }
-       lttng_consumer_sync_trace_file(stream, orig_offset);
-       goto end;
-
-write_error:
-       /*
-        * This is a special case that the relayd has closed its socket. Let's
-        * cleanup the relayd object and all associated streams.
-        */
-       if (relayd && relayd_hang_up) {
-               cleanup_relayd(relayd, ctx);
-               /* Skip splice error so the consumer does not fail */
-               goto end;
-       }
-
-splice_error:
-       /* send the appropriate error description to sessiond */
-       switch (ret) {
-       case EINVAL:
-               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_SPLICE_EINVAL);
-               break;
-       case ENOMEM:
-               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_SPLICE_ENOMEM);
-               break;
-       case ESPIPE:
-               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_SPLICE_ESPIPE);
-               break;
-       }
-
-end:
-       if (relayd && stream->metadata_flag) {
-               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-       }
-
-       rcu_read_unlock();
-       return written;
-}
-
-/*
- * Take a snapshot for a specific fd
- *
- * Returns 0 on success, < 0 on error
- */
-int lttng_consumer_take_snapshot(struct lttng_consumer_stream *stream)
-{
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               return lttng_kconsumer_take_snapshot(stream);
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               return lttng_ustconsumer_take_snapshot(stream);
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-               return -ENOSYS;
-       }
-}
-
-/*
- * Get the produced position
- *
- * Returns 0 on success, < 0 on error
- */
-int lttng_consumer_get_produced_snapshot(struct lttng_consumer_stream *stream,
-               unsigned long *pos)
-{
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               return lttng_kconsumer_get_produced_snapshot(stream, pos);
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               return lttng_ustconsumer_get_produced_snapshot(stream, pos);
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-               return -ENOSYS;
-       }
-}
-
-int lttng_consumer_recv_cmd(struct lttng_consumer_local_data *ctx,
-               int sock, struct pollfd *consumer_sockpoll)
-{
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               return lttng_kconsumer_recv_cmd(ctx, sock, consumer_sockpoll);
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               return lttng_ustconsumer_recv_cmd(ctx, sock, consumer_sockpoll);
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-               return -ENOSYS;
-       }
-}
-
-void lttng_consumer_close_all_metadata(void)
-{
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               /*
-                * The Kernel consumer has a different metadata scheme so we don't
-                * close anything because the stream will be closed by the session
-                * daemon.
-                */
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               /*
-                * Close all metadata streams. The metadata hash table is passed and
-                * this call iterates over it by closing all wakeup fd. This is safe
-                * because at this point we are sure that the metadata producer is
-                * either dead or blocked.
-                */
-               lttng_ustconsumer_close_all_metadata(metadata_ht);
-               break;
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-       }
-}
-
-/*
- * Clean up a metadata stream and free its memory.
- */
-void consumer_del_metadata_stream(struct lttng_consumer_stream *stream,
-               struct lttng_ht *ht)
-{
-       struct lttng_consumer_channel *free_chan = NULL;
-
-       assert(stream);
-       /*
-        * This call should NEVER receive regular stream. It must always be
-        * metadata stream and this is crucial for data structure synchronization.
-        */
-       assert(stream->metadata_flag);
-
-       DBG3("Consumer delete metadata stream %d", stream->wait_fd);
-
-       pthread_mutex_lock(&consumer_data.lock);
-       pthread_mutex_lock(&stream->chan->lock);
-       pthread_mutex_lock(&stream->lock);
-
-       /* Remove any reference to that stream. */
-       consumer_stream_delete(stream, ht);
-
-       /* Close down everything including the relayd if one. */
-       consumer_stream_close(stream);
-       /* Destroy tracer buffers of the stream. */
-       consumer_stream_destroy_buffers(stream);
-
-       /* Atomically decrement channel refcount since other threads can use it. */
-       if (!uatomic_sub_return(&stream->chan->refcount, 1)
-                       && !uatomic_read(&stream->chan->nb_init_stream_left)) {
-               /* Go for channel deletion! */
-               free_chan = stream->chan;
-       }
-
-       /*
-        * Nullify the stream reference so it is not used after deletion. The
-        * channel lock MUST be acquired before being able to check for a NULL
-        * pointer value.
-        */
-       stream->chan->metadata_stream = NULL;
-
-       pthread_mutex_unlock(&stream->lock);
-       pthread_mutex_unlock(&stream->chan->lock);
-       pthread_mutex_unlock(&consumer_data.lock);
-
-       if (free_chan) {
-               consumer_del_channel(free_chan);
-       }
-
-       consumer_stream_free(stream);
-}
-
-/*
- * Action done with the metadata stream when adding it to the consumer internal
- * data structures to handle it.
- */
-int consumer_add_metadata_stream(struct lttng_consumer_stream *stream)
-{
-       struct lttng_ht *ht = metadata_ht;
-       int ret = 0;
-       struct lttng_ht_iter iter;
-       struct lttng_ht_node_u64 *node;
-
-       assert(stream);
-       assert(ht);
-
-       DBG3("Adding metadata stream %" PRIu64 " to hash table", stream->key);
-
-       pthread_mutex_lock(&consumer_data.lock);
-       pthread_mutex_lock(&stream->chan->lock);
-       pthread_mutex_lock(&stream->chan->timer_lock);
-       pthread_mutex_lock(&stream->lock);
-
-       /*
-        * From here, refcounts are updated so be _careful_ when returning an error
-        * after this point.
-        */
-
-       rcu_read_lock();
-
-       /*
-        * Lookup the stream just to make sure it does not exist in our internal
-        * state. This should NEVER happen.
-        */
-       lttng_ht_lookup(ht, &stream->key, &iter);
-       node = lttng_ht_iter_get_node_u64(&iter);
-       assert(!node);
-
-       /*
-        * When nb_init_stream_left reaches 0, we don't need to trigger any action
-        * in terms of destroying the associated channel, because the action that
-        * causes the count to become 0 also causes a stream to be added. The
-        * channel deletion will thus be triggered by the following removal of this
-        * stream.
-        */
-       if (uatomic_read(&stream->chan->nb_init_stream_left) > 0) {
-               /* Increment refcount before decrementing nb_init_stream_left */
-               cmm_smp_wmb();
-               uatomic_dec(&stream->chan->nb_init_stream_left);
-       }
-
-       lttng_ht_add_unique_u64(ht, &stream->node);
-
-       lttng_ht_add_unique_u64(consumer_data.stream_per_chan_id_ht,
-               &stream->node_channel_id);
-
-       /*
-        * Add stream to the stream_list_ht of the consumer data. No need to steal
-        * the key since the HT does not use it and we allow to add redundant keys
-        * into this table.
-        */
-       lttng_ht_add_u64(consumer_data.stream_list_ht, &stream->node_session_id);
-
-       rcu_read_unlock();
-
-       pthread_mutex_unlock(&stream->lock);
-       pthread_mutex_unlock(&stream->chan->lock);
-       pthread_mutex_unlock(&stream->chan->timer_lock);
-       pthread_mutex_unlock(&consumer_data.lock);
-       return ret;
-}
-
-/*
- * Delete data stream that are flagged for deletion (endpoint_status).
- */
-static void validate_endpoint_status_data_stream(void)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_consumer_stream *stream;
-
-       DBG("Consumer delete flagged data stream");
-
-       rcu_read_lock();
-       cds_lfht_for_each_entry(data_ht->ht, &iter.iter, stream, node.node) {
-               /* Validate delete flag of the stream */
-               if (stream->endpoint_status == CONSUMER_ENDPOINT_ACTIVE) {
-                       continue;
-               }
-               /* Delete it right now */
-               consumer_del_stream(stream, data_ht);
-       }
-       rcu_read_unlock();
-}
-
-/*
- * Delete metadata stream that are flagged for deletion (endpoint_status).
- */
-static void validate_endpoint_status_metadata_stream(
-               struct lttng_poll_event *pollset)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_consumer_stream *stream;
-
-       DBG("Consumer delete flagged metadata stream");
-
-       assert(pollset);
-
-       rcu_read_lock();
-       cds_lfht_for_each_entry(metadata_ht->ht, &iter.iter, stream, node.node) {
-               /* Validate delete flag of the stream */
-               if (stream->endpoint_status == CONSUMER_ENDPOINT_ACTIVE) {
-                       continue;
-               }
-               /*
-                * Remove from pollset so the metadata thread can continue without
-                * blocking on a deleted stream.
-                */
-               lttng_poll_del(pollset, stream->wait_fd);
-
-               /* Delete it right now */
-               consumer_del_metadata_stream(stream, metadata_ht);
-       }
-       rcu_read_unlock();
-}
-
-/*
- * Thread polls on metadata file descriptor and write them on disk or on the
- * network.
- */
-void *consumer_thread_metadata_poll(void *data)
-{
-       int ret, i, pollfd, err = -1;
-       uint32_t revents, nb_fd;
-       struct lttng_consumer_stream *stream = NULL;
-       struct lttng_ht_iter iter;
-       struct lttng_ht_node_u64 *node;
-       struct lttng_poll_event events;
-       struct lttng_consumer_local_data *ctx = data;
-       ssize_t len;
-
-       rcu_register_thread();
-
-       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_METADATA);
-
-       if (testpoint(consumerd_thread_metadata)) {
-               goto error_testpoint;
-       }
-
-       health_code_update();
-
-       DBG("Thread metadata poll started");
-
-       /* Size is set to 1 for the consumer_metadata pipe */
-       ret = lttng_poll_create(&events, 2, LTTNG_CLOEXEC);
-       if (ret < 0) {
-               ERR("Poll set creation failed");
-               goto end_poll;
-       }
-
-       ret = lttng_poll_add(&events,
-                       lttng_pipe_get_readfd(ctx->consumer_metadata_pipe), LPOLLIN);
-       if (ret < 0) {
-               goto end;
-       }
-
-       /* Main loop */
-       DBG("Metadata main loop started");
-
-       while (1) {
-restart:
-               health_code_update();
-               health_poll_entry();
-               DBG("Metadata poll wait");
-               ret = lttng_poll_wait(&events, -1);
-               DBG("Metadata poll return from wait with %d fd(s)",
-                               LTTNG_POLL_GETNB(&events));
-               health_poll_exit();
-               DBG("Metadata event catched in thread");
-               if (ret < 0) {
-                       if (errno == EINTR) {
-                               ERR("Poll EINTR catched");
-                               goto restart;
-                       }
-                       if (LTTNG_POLL_GETNB(&events) == 0) {
-                               err = 0;        /* All is OK */
-                       }
-                       goto end;
-               }
-
-               nb_fd = ret;
-
-               /* From here, the event is a metadata wait fd */
-               for (i = 0; i < nb_fd; i++) {
-                       health_code_update();
-
-                       revents = LTTNG_POLL_GETEV(&events, i);
-                       pollfd = LTTNG_POLL_GETFD(&events, i);
-
-                       if (!revents) {
-                               /* No activity for this FD (poll implementation). */
-                               continue;
-                       }
-
-                       if (pollfd == lttng_pipe_get_readfd(ctx->consumer_metadata_pipe)) {
-                               if (revents & LPOLLIN) {
-                                       ssize_t pipe_len;
-
-                                       pipe_len = lttng_pipe_read(ctx->consumer_metadata_pipe,
-                                                       &stream, sizeof(stream));
-                                       if (pipe_len < sizeof(stream)) {
-                                               if (pipe_len < 0) {
-                                                       PERROR("read metadata stream");
-                                               }
-                                               /*
-                                                * Remove the pipe from the poll set and continue the loop
-                                                * since their might be data to consume.
-                                                */
-                                               lttng_poll_del(&events,
-                                                               lttng_pipe_get_readfd(ctx->consumer_metadata_pipe));
-                                               lttng_pipe_read_close(ctx->consumer_metadata_pipe);
-                                               continue;
-                                       }
-
-                                       /* A NULL stream means that the state has changed. */
-                                       if (stream == NULL) {
-                                               /* Check for deleted streams. */
-                                               validate_endpoint_status_metadata_stream(&events);
-                                               goto restart;
-                                       }
-
-                                       DBG("Adding metadata stream %d to poll set",
-                                                       stream->wait_fd);
-
-                                       /* Add metadata stream to the global poll events list */
-                                       lttng_poll_add(&events, stream->wait_fd,
-                                                       LPOLLIN | LPOLLPRI | LPOLLHUP);
-                               } else if (revents & (LPOLLERR | LPOLLHUP)) {
-                                       DBG("Metadata thread pipe hung up");
-                                       /*
-                                        * Remove the pipe from the poll set and continue the loop
-                                        * since their might be data to consume.
-                                        */
-                                       lttng_poll_del(&events,
-                                                       lttng_pipe_get_readfd(ctx->consumer_metadata_pipe));
-                                       lttng_pipe_read_close(ctx->consumer_metadata_pipe);
-                                       continue;
-                               } else {
-                                       ERR("Unexpected poll events %u for sock %d", revents, pollfd);
-                                       goto end;
-                               }
-
-                               /* Handle other stream */
-                               continue;
-                       }
-
-                       rcu_read_lock();
-                       {
-                               uint64_t tmp_id = (uint64_t) pollfd;
-
-                               lttng_ht_lookup(metadata_ht, &tmp_id, &iter);
-                       }
-                       node = lttng_ht_iter_get_node_u64(&iter);
-                       assert(node);
-
-                       stream = caa_container_of(node, struct lttng_consumer_stream,
-                                       node);
-
-                       if (revents & (LPOLLIN | LPOLLPRI)) {
-                               /* Get the data out of the metadata file descriptor */
-                               DBG("Metadata available on fd %d", pollfd);
-                               assert(stream->wait_fd == pollfd);
-
-                               do {
-                                       health_code_update();
-
-                                       len = ctx->on_buffer_ready(stream, ctx);
-                                       /*
-                                        * We don't check the return value here since if we get
-                                        * a negative len, it means an error occured thus we
-                                        * simply remove it from the poll set and free the
-                                        * stream.
-                                        */
-                               } while (len > 0);
-
-                               /* It's ok to have an unavailable sub-buffer */
-                               if (len < 0 && len != -EAGAIN && len != -ENODATA) {
-                                       /* Clean up stream from consumer and free it. */
-                                       lttng_poll_del(&events, stream->wait_fd);
-                                       consumer_del_metadata_stream(stream, metadata_ht);
-                               }
-                       } else if (revents & (LPOLLERR | LPOLLHUP)) {
-                               DBG("Metadata fd %d is hup|err.", pollfd);
-                               if (!stream->hangup_flush_done
-                                               && (consumer_data.type == LTTNG_CONSUMER32_UST
-                                                       || consumer_data.type == LTTNG_CONSUMER64_UST)) {
-                                       DBG("Attempting to flush and consume the UST buffers");
-                                       lttng_ustconsumer_on_stream_hangup(stream);
-
-                                       /* We just flushed the stream now read it. */
-                                       do {
-                                               health_code_update();
-
-                                               len = ctx->on_buffer_ready(stream, ctx);
-                                               /*
-                                                * We don't check the return value here since if we get
-                                                * a negative len, it means an error occured thus we
-                                                * simply remove it from the poll set and free the
-                                                * stream.
-                                                */
-                                       } while (len > 0);
-                               }
-
-                               lttng_poll_del(&events, stream->wait_fd);
-                               /*
-                                * This call update the channel states, closes file descriptors
-                                * and securely free the stream.
-                                */
-                               consumer_del_metadata_stream(stream, metadata_ht);
-                       } else {
-                               ERR("Unexpected poll events %u for sock %d", revents, pollfd);
-                               rcu_read_unlock();
-                               goto end;
-                       }
-                       /* Release RCU lock for the stream looked up */
-                       rcu_read_unlock();
-               }
-       }
-
-       /* All is OK */
-       err = 0;
-end:
-       DBG("Metadata poll thread exiting");
-
-       lttng_poll_clean(&events);
-end_poll:
-error_testpoint:
-       if (err) {
-               health_error();
-               ERR("Health error occurred in %s", __func__);
-       }
-       health_unregister(health_consumerd);
-       rcu_unregister_thread();
-       return NULL;
-}
-
-/*
- * This thread polls the fds in the set to consume the data and write
- * it to tracefile if necessary.
- */
-void *consumer_thread_data_poll(void *data)
-{
-       int num_rdy, num_hup, high_prio, ret, i, err = -1;
-       struct pollfd *pollfd = NULL;
-       /* local view of the streams */
-       struct lttng_consumer_stream **local_stream = NULL, *new_stream = NULL;
-       /* local view of consumer_data.fds_count */
-       int nb_fd = 0;
-       struct lttng_consumer_local_data *ctx = data;
-       ssize_t len;
-
-       rcu_register_thread();
-
-       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_DATA);
-
-       if (testpoint(consumerd_thread_data)) {
-               goto error_testpoint;
-       }
-
-       health_code_update();
-
-       local_stream = zmalloc(sizeof(struct lttng_consumer_stream *));
-       if (local_stream == NULL) {
-               PERROR("local_stream malloc");
-               goto end;
-       }
-
-       while (1) {
-               health_code_update();
-
-               high_prio = 0;
-               num_hup = 0;
-
-               /*
-                * the fds set has been updated, we need to update our
-                * local array as well
-                */
-               pthread_mutex_lock(&consumer_data.lock);
-               if (consumer_data.need_update) {
-                       free(pollfd);
-                       pollfd = NULL;
-
-                       free(local_stream);
-                       local_stream = NULL;
-
-                       /*
-                        * Allocate for all fds +1 for the consumer_data_pipe and +1 for
-                        * wake up pipe.
-                        */
-                       pollfd = zmalloc((consumer_data.stream_count + 2) * sizeof(struct pollfd));
-                       if (pollfd == NULL) {
-                               PERROR("pollfd malloc");
-                               pthread_mutex_unlock(&consumer_data.lock);
-                               goto end;
-                       }
-
-                       local_stream = zmalloc((consumer_data.stream_count + 2) *
-                                       sizeof(struct lttng_consumer_stream *));
-                       if (local_stream == NULL) {
-                               PERROR("local_stream malloc");
-                               pthread_mutex_unlock(&consumer_data.lock);
-                               goto end;
-                       }
-                       ret = update_poll_array(ctx, &pollfd, local_stream,
-                                       data_ht);
-                       if (ret < 0) {
-                               ERR("Error in allocating pollfd or local_outfds");
-                               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_POLL_ERROR);
-                               pthread_mutex_unlock(&consumer_data.lock);
-                               goto end;
-                       }
-                       nb_fd = ret;
-                       consumer_data.need_update = 0;
-               }
-               pthread_mutex_unlock(&consumer_data.lock);
-
-               /* No FDs and consumer_quit, consumer_cleanup the thread */
-               if (nb_fd == 0 && consumer_quit == 1) {
-                       err = 0;        /* All is OK */
-                       goto end;
-               }
-               /* poll on the array of fds */
-       restart:
-               DBG("polling on %d fd", nb_fd + 2);
-               health_poll_entry();
-               num_rdy = poll(pollfd, nb_fd + 2, -1);
-               health_poll_exit();
-               DBG("poll num_rdy : %d", num_rdy);
-               if (num_rdy == -1) {
-                       /*
-                        * Restart interrupted system call.
-                        */
-                       if (errno == EINTR) {
-                               goto restart;
-                       }
-                       PERROR("Poll error");
-                       lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_POLL_ERROR);
-                       goto end;
-               } else if (num_rdy == 0) {
-                       DBG("Polling thread timed out");
-                       goto end;
-               }
-
-               /*
-                * If the consumer_data_pipe triggered poll go directly to the
-                * beginning of the loop to update the array. We want to prioritize
-                * array update over low-priority reads.
-                */
-               if (pollfd[nb_fd].revents & (POLLIN | POLLPRI)) {
-                       ssize_t pipe_readlen;
-
-                       DBG("consumer_data_pipe wake up");
-                       pipe_readlen = lttng_pipe_read(ctx->consumer_data_pipe,
-                                       &new_stream, sizeof(new_stream));
-                       if (pipe_readlen < sizeof(new_stream)) {
-                               PERROR("Consumer data pipe");
-                               /* Continue so we can at least handle the current stream(s). */
-                               continue;
-                       }
-
-                       /*
-                        * If the stream is NULL, just ignore it. It's also possible that
-                        * the sessiond poll thread changed the consumer_quit state and is
-                        * waking us up to test it.
-                        */
-                       if (new_stream == NULL) {
-                               validate_endpoint_status_data_stream();
-                               continue;
-                       }
-
-                       /* Continue to update the local streams and handle prio ones */
-                       continue;
-               }
-
-               /* Handle wakeup pipe. */
-               if (pollfd[nb_fd + 1].revents & (POLLIN | POLLPRI)) {
-                       char dummy;
-                       ssize_t pipe_readlen;
-
-                       pipe_readlen = lttng_pipe_read(ctx->consumer_wakeup_pipe, &dummy,
-                                       sizeof(dummy));
-                       if (pipe_readlen < 0) {
-                               PERROR("Consumer data wakeup pipe");
-                       }
-                       /* We've been awakened to handle stream(s). */
-                       ctx->has_wakeup = 0;
-               }
-
-               /* Take care of high priority channels first. */
-               for (i = 0; i < nb_fd; i++) {
-                       health_code_update();
-
-                       if (local_stream[i] == NULL) {
-                               continue;
-                       }
-                       if (pollfd[i].revents & POLLPRI) {
-                               DBG("Urgent read on fd %d", pollfd[i].fd);
-                               high_prio = 1;
-                               len = ctx->on_buffer_ready(local_stream[i], ctx);
-                               /* it's ok to have an unavailable sub-buffer */
-                               if (len < 0 && len != -EAGAIN && len != -ENODATA) {
-                                       /* Clean the stream and free it. */
-                                       consumer_del_stream(local_stream[i], data_ht);
-                                       local_stream[i] = NULL;
-                               } else if (len > 0) {
-                                       local_stream[i]->data_read = 1;
-                               }
-                       }
-               }
-
-               /*
-                * If we read high prio channel in this loop, try again
-                * for more high prio data.
-                */
-               if (high_prio) {
-                       continue;
-               }
-
-               /* Take care of low priority channels. */
-               for (i = 0; i < nb_fd; i++) {
-                       health_code_update();
-
-                       if (local_stream[i] == NULL) {
-                               continue;
-                       }
-                       if ((pollfd[i].revents & POLLIN) ||
-                                       local_stream[i]->hangup_flush_done ||
-                                       local_stream[i]->has_data) {
-                               DBG("Normal read on fd %d", pollfd[i].fd);
-                               len = ctx->on_buffer_ready(local_stream[i], ctx);
-                               /* it's ok to have an unavailable sub-buffer */
-                               if (len < 0 && len != -EAGAIN && len != -ENODATA) {
-                                       /* Clean the stream and free it. */
-                                       consumer_del_stream(local_stream[i], data_ht);
-                                       local_stream[i] = NULL;
-                               } else if (len > 0) {
-                                       local_stream[i]->data_read = 1;
-                               }
-                       }
-               }
-
-               /* Handle hangup and errors */
-               for (i = 0; i < nb_fd; i++) {
-                       health_code_update();
-
-                       if (local_stream[i] == NULL) {
-                               continue;
-                       }
-                       if (!local_stream[i]->hangup_flush_done
-                                       && (pollfd[i].revents & (POLLHUP | POLLERR | POLLNVAL))
-                                       && (consumer_data.type == LTTNG_CONSUMER32_UST
-                                               || consumer_data.type == LTTNG_CONSUMER64_UST)) {
-                               DBG("fd %d is hup|err|nval. Attempting flush and read.",
-                                               pollfd[i].fd);
-                               lttng_ustconsumer_on_stream_hangup(local_stream[i]);
-                               /* Attempt read again, for the data we just flushed. */
-                               local_stream[i]->data_read = 1;
-                       }
-                       /*
-                        * If the poll flag is HUP/ERR/NVAL and we have
-                        * read no data in this pass, we can remove the
-                        * stream from its hash table.
-                        */
-                       if ((pollfd[i].revents & POLLHUP)) {
-                               DBG("Polling fd %d tells it has hung up.", pollfd[i].fd);
-                               if (!local_stream[i]->data_read) {
-                                       consumer_del_stream(local_stream[i], data_ht);
-                                       local_stream[i] = NULL;
-                                       num_hup++;
-                               }
-                       } else if (pollfd[i].revents & POLLERR) {
-                               ERR("Error returned in polling fd %d.", pollfd[i].fd);
-                               if (!local_stream[i]->data_read) {
-                                       consumer_del_stream(local_stream[i], data_ht);
-                                       local_stream[i] = NULL;
-                                       num_hup++;
-                               }
-                       } else if (pollfd[i].revents & POLLNVAL) {
-                               ERR("Polling fd %d tells fd is not open.", pollfd[i].fd);
-                               if (!local_stream[i]->data_read) {
-                                       consumer_del_stream(local_stream[i], data_ht);
-                                       local_stream[i] = NULL;
-                                       num_hup++;
-                               }
-                       }
-                       if (local_stream[i] != NULL) {
-                               local_stream[i]->data_read = 0;
-                       }
-               }
-       }
-       /* All is OK */
-       err = 0;
-end:
-       DBG("polling thread exiting");
-       free(pollfd);
-       free(local_stream);
-
-       /*
-        * Close the write side of the pipe so epoll_wait() in
-        * consumer_thread_metadata_poll can catch it. The thread is monitoring the
-        * read side of the pipe. If we close them both, epoll_wait strangely does
-        * not return and could create a endless wait period if the pipe is the
-        * only tracked fd in the poll set. The thread will take care of closing
-        * the read side.
-        */
-       (void) lttng_pipe_write_close(ctx->consumer_metadata_pipe);
-
-error_testpoint:
-       if (err) {
-               health_error();
-               ERR("Health error occurred in %s", __func__);
-       }
-       health_unregister(health_consumerd);
-
-       rcu_unregister_thread();
-       return NULL;
-}
-
-/*
- * Close wake-up end of each stream belonging to the channel. This will
- * allow the poll() on the stream read-side to detect when the
- * write-side (application) finally closes them.
- */
-static
-void consumer_close_channel_streams(struct lttng_consumer_channel *channel)
-{
-       struct lttng_ht *ht;
-       struct lttng_consumer_stream *stream;
-       struct lttng_ht_iter iter;
-
-       ht = consumer_data.stream_per_chan_id_ht;
-
-       rcu_read_lock();
-       cds_lfht_for_each_entry_duplicate(ht->ht,
-                       ht->hash_fct(&channel->key, lttng_ht_seed),
-                       ht->match_fct, &channel->key,
-                       &iter.iter, stream, node_channel_id.node) {
-               /*
-                * Protect against teardown with mutex.
-                */
-               pthread_mutex_lock(&stream->lock);
-               if (cds_lfht_is_node_deleted(&stream->node.node)) {
-                       goto next;
-               }
-               switch (consumer_data.type) {
-               case LTTNG_CONSUMER_KERNEL:
-                       break;
-               case LTTNG_CONSUMER32_UST:
-               case LTTNG_CONSUMER64_UST:
-                       if (stream->metadata_flag) {
-                               /* Safe and protected by the stream lock. */
-                               lttng_ustconsumer_close_metadata(stream->chan);
-                       } else {
-                               /*
-                                * Note: a mutex is taken internally within
-                                * liblttng-ust-ctl to protect timer wakeup_fd
-                                * use from concurrent close.
-                                */
-                               lttng_ustconsumer_close_stream_wakeup(stream);
-                       }
-                       break;
-               default:
-                       ERR("Unknown consumer_data type");
-                       assert(0);
-               }
-       next:
-               pthread_mutex_unlock(&stream->lock);
-       }
-       rcu_read_unlock();
-}
-
-static void destroy_channel_ht(struct lttng_ht *ht)
-{
-       struct lttng_ht_iter iter;
-       struct lttng_consumer_channel *channel;
-       int ret;
-
-       if (ht == NULL) {
-               return;
-       }
-
-       rcu_read_lock();
-       cds_lfht_for_each_entry(ht->ht, &iter.iter, channel, wait_fd_node.node) {
-               ret = lttng_ht_del(ht, &iter);
-               assert(ret != 0);
-       }
-       rcu_read_unlock();
-
-       lttng_ht_destroy(ht);
-}
-
-/*
- * This thread polls the channel fds to detect when they are being
- * closed. It closes all related streams if the channel is detected as
- * closed. It is currently only used as a shim layer for UST because the
- * consumerd needs to keep the per-stream wakeup end of pipes open for
- * periodical flush.
- */
-void *consumer_thread_channel_poll(void *data)
-{
-       int ret, i, pollfd, err = -1;
-       uint32_t revents, nb_fd;
-       struct lttng_consumer_channel *chan = NULL;
-       struct lttng_ht_iter iter;
-       struct lttng_ht_node_u64 *node;
-       struct lttng_poll_event events;
-       struct lttng_consumer_local_data *ctx = data;
-       struct lttng_ht *channel_ht;
-
-       rcu_register_thread();
-
-       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_CHANNEL);
-
-       if (testpoint(consumerd_thread_channel)) {
-               goto error_testpoint;
-       }
-
-       health_code_update();
-
-       channel_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
-       if (!channel_ht) {
-               /* ENOMEM at this point. Better to bail out. */
-               goto end_ht;
-       }
-
-       DBG("Thread channel poll started");
-
-       /* Size is set to 1 for the consumer_channel pipe */
-       ret = lttng_poll_create(&events, 2, LTTNG_CLOEXEC);
-       if (ret < 0) {
-               ERR("Poll set creation failed");
-               goto end_poll;
-       }
-
-       ret = lttng_poll_add(&events, ctx->consumer_channel_pipe[0], LPOLLIN);
-       if (ret < 0) {
-               goto end;
-       }
-
-       /* Main loop */
-       DBG("Channel main loop started");
-
-       while (1) {
-restart:
-               health_code_update();
-               DBG("Channel poll wait");
-               health_poll_entry();
-               ret = lttng_poll_wait(&events, -1);
-               DBG("Channel poll return from wait with %d fd(s)",
-                               LTTNG_POLL_GETNB(&events));
-               health_poll_exit();
-               DBG("Channel event catched in thread");
-               if (ret < 0) {
-                       if (errno == EINTR) {
-                               ERR("Poll EINTR catched");
-                               goto restart;
-                       }
-                       if (LTTNG_POLL_GETNB(&events) == 0) {
-                               err = 0;        /* All is OK */
-                       }
-                       goto end;
-               }
-
-               nb_fd = ret;
-
-               /* From here, the event is a channel wait fd */
-               for (i = 0; i < nb_fd; i++) {
-                       health_code_update();
-
-                       revents = LTTNG_POLL_GETEV(&events, i);
-                       pollfd = LTTNG_POLL_GETFD(&events, i);
-
-                       if (!revents) {
-                               /* No activity for this FD (poll implementation). */
-                               continue;
-                       }
-
-                       if (pollfd == ctx->consumer_channel_pipe[0]) {
-                               if (revents & LPOLLIN) {
-                                       enum consumer_channel_action action;
-                                       uint64_t key;
-
-                                       ret = read_channel_pipe(ctx, &chan, &key, &action);
-                                       if (ret <= 0) {
-                                               if (ret < 0) {
-                                                       ERR("Error reading channel pipe");
-                                               }
-                                               lttng_poll_del(&events, ctx->consumer_channel_pipe[0]);
-                                               continue;
-                                       }
-
-                                       switch (action) {
-                                       case CONSUMER_CHANNEL_ADD:
-                                               DBG("Adding channel %d to poll set",
-                                                       chan->wait_fd);
-
-                                               lttng_ht_node_init_u64(&chan->wait_fd_node,
-                                                       chan->wait_fd);
-                                               rcu_read_lock();
-                                               lttng_ht_add_unique_u64(channel_ht,
-                                                               &chan->wait_fd_node);
-                                               rcu_read_unlock();
-                                               /* Add channel to the global poll events list */
-                                               lttng_poll_add(&events, chan->wait_fd,
-                                                               LPOLLERR | LPOLLHUP);
-                                               break;
-                                       case CONSUMER_CHANNEL_DEL:
-                                       {
-                                               /*
-                                                * This command should never be called if the channel
-                                                * has streams monitored by either the data or metadata
-                                                * thread. The consumer only notify this thread with a
-                                                * channel del. command if it receives a destroy
-                                                * channel command from the session daemon that send it
-                                                * if a command prior to the GET_CHANNEL failed.
-                                                */
-
-                                               rcu_read_lock();
-                                               chan = consumer_find_channel(key);
-                                               if (!chan) {
-                                                       rcu_read_unlock();
-                                                       ERR("UST consumer get channel key %" PRIu64 " not found for del channel", key);
-                                                       break;
-                                               }
-                                               lttng_poll_del(&events, chan->wait_fd);
-                                               iter.iter.node = &chan->wait_fd_node.node;
-                                               ret = lttng_ht_del(channel_ht, &iter);
-                                               assert(ret == 0);
-
-                                               switch (consumer_data.type) {
-                                               case LTTNG_CONSUMER_KERNEL:
-                                                       break;
-                                               case LTTNG_CONSUMER32_UST:
-                                               case LTTNG_CONSUMER64_UST:
-                                                       health_code_update();
-                                                       /* Destroy streams that might have been left in the stream list. */
-                                                       clean_channel_stream_list(chan);
-                                                       break;
-                                               default:
-                                                       ERR("Unknown consumer_data type");
-                                                       assert(0);
-                                               }
-
-                                               /*
-                                                * Release our own refcount. Force channel deletion even if
-                                                * streams were not initialized.
-                                                */
-                                               if (!uatomic_sub_return(&chan->refcount, 1)) {
-                                                       consumer_del_channel(chan);
-                                               }
-                                               rcu_read_unlock();
-                                               goto restart;
-                                       }
-                                       case CONSUMER_CHANNEL_QUIT:
-                                               /*
-                                                * Remove the pipe from the poll set and continue the loop
-                                                * since their might be data to consume.
-                                                */
-                                               lttng_poll_del(&events, ctx->consumer_channel_pipe[0]);
-                                               continue;
-                                       default:
-                                               ERR("Unknown action");
-                                               break;
-                                       }
-                               } else if (revents & (LPOLLERR | LPOLLHUP)) {
-                                       DBG("Channel thread pipe hung up");
-                                       /*
-                                        * Remove the pipe from the poll set and continue the loop
-                                        * since their might be data to consume.
-                                        */
-                                       lttng_poll_del(&events, ctx->consumer_channel_pipe[0]);
-                                       continue;
-                               } else {
-                                       ERR("Unexpected poll events %u for sock %d", revents, pollfd);
-                                       goto end;
-                               }
-
-                               /* Handle other stream */
-                               continue;
-                       }
-
-                       rcu_read_lock();
-                       {
-                               uint64_t tmp_id = (uint64_t) pollfd;
-
-                               lttng_ht_lookup(channel_ht, &tmp_id, &iter);
-                       }
-                       node = lttng_ht_iter_get_node_u64(&iter);
-                       assert(node);
-
-                       chan = caa_container_of(node, struct lttng_consumer_channel,
-                                       wait_fd_node);
-
-                       /* Check for error event */
-                       if (revents & (LPOLLERR | LPOLLHUP)) {
-                               DBG("Channel fd %d is hup|err.", pollfd);
-
-                               lttng_poll_del(&events, chan->wait_fd);
-                               ret = lttng_ht_del(channel_ht, &iter);
-                               assert(ret == 0);
-
-                               /*
-                                * This will close the wait fd for each stream associated to
-                                * this channel AND monitored by the data/metadata thread thus
-                                * will be clean by the right thread.
-                                */
-                               consumer_close_channel_streams(chan);
-
-                               /* Release our own refcount */
-                               if (!uatomic_sub_return(&chan->refcount, 1)
-                                               && !uatomic_read(&chan->nb_init_stream_left)) {
-                                       consumer_del_channel(chan);
-                               }
-                       } else {
-                               ERR("Unexpected poll events %u for sock %d", revents, pollfd);
-                               rcu_read_unlock();
-                               goto end;
-                       }
-
-                       /* Release RCU lock for the channel looked up */
-                       rcu_read_unlock();
-               }
-       }
-
-       /* All is OK */
-       err = 0;
-end:
-       lttng_poll_clean(&events);
-end_poll:
-       destroy_channel_ht(channel_ht);
-end_ht:
-error_testpoint:
-       DBG("Channel poll thread exiting");
-       if (err) {
-               health_error();
-               ERR("Health error occurred in %s", __func__);
-       }
-       health_unregister(health_consumerd);
-       rcu_unregister_thread();
-       return NULL;
-}
-
-static int set_metadata_socket(struct lttng_consumer_local_data *ctx,
-               struct pollfd *sockpoll, int client_socket)
-{
-       int ret;
-
-       assert(ctx);
-       assert(sockpoll);
-
-       ret = lttng_consumer_poll_socket(sockpoll);
-       if (ret) {
-               goto error;
-       }
-       DBG("Metadata connection on client_socket");
-
-       /* Blocking call, waiting for transmission */
-       ctx->consumer_metadata_socket = lttcomm_accept_unix_sock(client_socket);
-       if (ctx->consumer_metadata_socket < 0) {
-               WARN("On accept metadata");
-               ret = -1;
-               goto error;
-       }
-       ret = 0;
-
-error:
-       return ret;
-}
-
-/*
- * This thread listens on the consumerd socket and receives the file
- * descriptors from the session daemon.
- */
-void *consumer_thread_sessiond_poll(void *data)
-{
-       int sock = -1, client_socket, ret, err = -1;
-       /*
-        * structure to poll for incoming data on communication socket avoids
-        * making blocking sockets.
-        */
-       struct pollfd consumer_sockpoll[2];
-       struct lttng_consumer_local_data *ctx = data;
-
-       rcu_register_thread();
-
-       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_SESSIOND);
-
-       if (testpoint(consumerd_thread_sessiond)) {
-               goto error_testpoint;
-       }
-
-       health_code_update();
-
-       DBG("Creating command socket %s", ctx->consumer_command_sock_path);
-       unlink(ctx->consumer_command_sock_path);
-       client_socket = lttcomm_create_unix_sock(ctx->consumer_command_sock_path);
-       if (client_socket < 0) {
-               ERR("Cannot create command socket");
-               goto end;
-       }
-
-       ret = lttcomm_listen_unix_sock(client_socket);
-       if (ret < 0) {
-               goto end;
-       }
-
-       DBG("Sending ready command to lttng-sessiond");
-       ret = lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_COMMAND_SOCK_READY);
-       /* return < 0 on error, but == 0 is not fatal */
-       if (ret < 0) {
-               ERR("Error sending ready command to lttng-sessiond");
-               goto end;
-       }
-
-       /* prepare the FDs to poll : to client socket and the should_quit pipe */
-       consumer_sockpoll[0].fd = ctx->consumer_should_quit[0];
-       consumer_sockpoll[0].events = POLLIN | POLLPRI;
-       consumer_sockpoll[1].fd = client_socket;
-       consumer_sockpoll[1].events = POLLIN | POLLPRI;
-
-       ret = lttng_consumer_poll_socket(consumer_sockpoll);
-       if (ret) {
-               if (ret > 0) {
-                       /* should exit */
-                       err = 0;
-               }
-               goto end;
-       }
-       DBG("Connection on client_socket");
-
-       /* Blocking call, waiting for transmission */
-       sock = lttcomm_accept_unix_sock(client_socket);
-       if (sock < 0) {
-               WARN("On accept");
-               goto end;
-       }
-
-       /*
-        * Setup metadata socket which is the second socket connection on the
-        * command unix socket.
-        */
-       ret = set_metadata_socket(ctx, consumer_sockpoll, client_socket);
-       if (ret) {
-               if (ret > 0) {
-                       /* should exit */
-                       err = 0;
-               }
-               goto end;
-       }
-
-       /* This socket is not useful anymore. */
-       ret = close(client_socket);
-       if (ret < 0) {
-               PERROR("close client_socket");
-       }
-       client_socket = -1;
-
-       /* update the polling structure to poll on the established socket */
-       consumer_sockpoll[1].fd = sock;
-       consumer_sockpoll[1].events = POLLIN | POLLPRI;
-
-       while (1) {
-               health_code_update();
-
-               health_poll_entry();
-               ret = lttng_consumer_poll_socket(consumer_sockpoll);
-               health_poll_exit();
-               if (ret) {
-                       if (ret > 0) {
-                               /* should exit */
-                               err = 0;
-                       }
-                       goto end;
-               }
-               DBG("Incoming command on sock");
-               ret = lttng_consumer_recv_cmd(ctx, sock, consumer_sockpoll);
-               if (ret <= 0) {
-                       /*
-                        * This could simply be a session daemon quitting. Don't output
-                        * ERR() here.
-                        */
-                       DBG("Communication interrupted on command socket");
-                       err = 0;
-                       goto end;
-               }
-               if (consumer_quit) {
-                       DBG("consumer_thread_receive_fds received quit from signal");
-                       err = 0;        /* All is OK */
-                       goto end;
-               }
-               DBG("received command on sock");
-       }
-       /* All is OK */
-       err = 0;
-
-end:
-       DBG("Consumer thread sessiond poll exiting");
-
-       /*
-        * Close metadata streams since the producer is the session daemon which
-        * just died.
-        *
-        * NOTE: for now, this only applies to the UST tracer.
-        */
-       lttng_consumer_close_all_metadata();
-
-       /*
-        * when all fds have hung up, the polling thread
-        * can exit cleanly
-        */
-       consumer_quit = 1;
-
-       /*
-        * Notify the data poll thread to poll back again and test the
-        * consumer_quit state that we just set so to quit gracefully.
-        */
-       notify_thread_lttng_pipe(ctx->consumer_data_pipe);
-
-       notify_channel_pipe(ctx, NULL, -1, CONSUMER_CHANNEL_QUIT);
-
-       notify_health_quit_pipe(health_quit_pipe);
-
-       /* Cleaning up possibly open sockets. */
-       if (sock >= 0) {
-               ret = close(sock);
-               if (ret < 0) {
-                       PERROR("close sock sessiond poll");
-               }
-       }
-       if (client_socket >= 0) {
-               ret = close(client_socket);
-               if (ret < 0) {
-                       PERROR("close client_socket sessiond poll");
-               }
-       }
-
-error_testpoint:
-       if (err) {
-               health_error();
-               ERR("Health error occurred in %s", __func__);
-       }
-       health_unregister(health_consumerd);
-
-       rcu_unregister_thread();
-       return NULL;
-}
-
-ssize_t lttng_consumer_read_subbuffer(struct lttng_consumer_stream *stream,
-               struct lttng_consumer_local_data *ctx)
-{
-       ssize_t ret;
-
-       pthread_mutex_lock(&stream->lock);
-       if (stream->metadata_flag) {
-               pthread_mutex_lock(&stream->metadata_rdv_lock);
-       }
-
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               ret = lttng_kconsumer_read_subbuffer(stream, ctx);
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               ret = lttng_ustconsumer_read_subbuffer(stream, ctx);
-               break;
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-               ret = -ENOSYS;
-               break;
-       }
-
-       if (stream->metadata_flag) {
-               pthread_cond_broadcast(&stream->metadata_rdv);
-               pthread_mutex_unlock(&stream->metadata_rdv_lock);
-       }
-       pthread_mutex_unlock(&stream->lock);
-       return ret;
-}
-
-int lttng_consumer_on_recv_stream(struct lttng_consumer_stream *stream)
-{
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               return lttng_kconsumer_on_recv_stream(stream);
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               return lttng_ustconsumer_on_recv_stream(stream);
-       default:
-               ERR("Unknown consumer_data type");
-               assert(0);
-               return -ENOSYS;
-       }
-}
-
-/*
- * Allocate and set consumer data hash tables.
- */
-int lttng_consumer_init(void)
-{
-       consumer_data.channel_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
-       if (!consumer_data.channel_ht) {
-               goto error;
-       }
-
-       consumer_data.relayd_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
-       if (!consumer_data.relayd_ht) {
-               goto error;
-       }
-
-       consumer_data.stream_list_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
-       if (!consumer_data.stream_list_ht) {
-               goto error;
-       }
-
-       consumer_data.stream_per_chan_id_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
-       if (!consumer_data.stream_per_chan_id_ht) {
-               goto error;
-       }
-
-       data_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
-       if (!data_ht) {
-               goto error;
-       }
-
-       metadata_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
-       if (!metadata_ht) {
-               goto error;
-       }
-
-       return 0;
-
-error:
-       return -1;
-}
-
-/*
- * Process the ADD_RELAYD command receive by a consumer.
- *
- * This will create a relayd socket pair and add it to the relayd hash table.
- * The caller MUST acquire a RCU read side lock before calling it.
- */
-int consumer_add_relayd_socket(uint64_t net_seq_idx, int sock_type,
-               struct lttng_consumer_local_data *ctx, int sock,
-               struct pollfd *consumer_sockpoll,
-               struct lttcomm_relayd_sock *relayd_sock, uint64_t sessiond_id,
-               uint64_t relayd_session_id)
-{
-       int fd = -1, ret = -1, relayd_created = 0;
-       enum lttcomm_return_code ret_code = LTTCOMM_CONSUMERD_SUCCESS;
-       struct consumer_relayd_sock_pair *relayd = NULL;
-
-       assert(ctx);
-       assert(relayd_sock);
-
-       DBG("Consumer adding relayd socket (idx: %" PRIu64 ")", net_seq_idx);
-
-       /* Get relayd reference if exists. */
-       relayd = consumer_find_relayd(net_seq_idx);
-       if (relayd == NULL) {
-               assert(sock_type == LTTNG_STREAM_CONTROL);
-               /* Not found. Allocate one. */
-               relayd = consumer_allocate_relayd_sock_pair(net_seq_idx);
-               if (relayd == NULL) {
-                       ret = -ENOMEM;
-                       ret_code = LTTCOMM_CONSUMERD_ENOMEM;
-                       goto error;
-               } else {
-                       relayd->sessiond_session_id = sessiond_id;
-                       relayd_created = 1;
-               }
-
-               /*
-                * This code path MUST continue to the consumer send status message to
-                * we can notify the session daemon and continue our work without
-                * killing everything.
-                */
-       } else {
-               /*
-                * relayd key should never be found for control socket.
-                */
-               assert(sock_type != LTTNG_STREAM_CONTROL);
-       }
-
-       /* First send a status message before receiving the fds. */
-       ret = consumer_send_status_msg(sock, LTTCOMM_CONSUMERD_SUCCESS);
-       if (ret < 0) {
-               /* Somehow, the session daemon is not responding anymore. */
-               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_FATAL);
-               goto error_nosignal;
-       }
-
-       /* Poll on consumer socket. */
-       ret = lttng_consumer_poll_socket(consumer_sockpoll);
-       if (ret) {
-               /* Needing to exit in the middle of a command: error. */
-               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_POLL_ERROR);
-               ret = -EINTR;
-               goto error_nosignal;
-       }
-
-       /* Get relayd socket from session daemon */
-       ret = lttcomm_recv_fds_unix_sock(sock, &fd, 1);
-       if (ret != sizeof(fd)) {
-               ret = -1;
-               fd = -1;        /* Just in case it gets set with an invalid value. */
-
-               /*
-                * Failing to receive FDs might indicate a major problem such as
-                * reaching a fd limit during the receive where the kernel returns a
-                * MSG_CTRUNC and fails to cleanup the fd in the queue. Any case, we
-                * don't take any chances and stop everything.
-                *
-                * XXX: Feature request #558 will fix that and avoid this possible
-                * issue when reaching the fd limit.
-                */
-               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_ERROR_RECV_FD);
-               ret_code = LTTCOMM_CONSUMERD_ERROR_RECV_FD;
-               goto error;
-       }
-
-       /* Copy socket information and received FD */
-       switch (sock_type) {
-       case LTTNG_STREAM_CONTROL:
-               /* Copy received lttcomm socket */
-               lttcomm_copy_sock(&relayd->control_sock.sock, &relayd_sock->sock);
-               ret = lttcomm_create_sock(&relayd->control_sock.sock);
-               /* Handle create_sock error. */
-               if (ret < 0) {
-                       ret_code = LTTCOMM_CONSUMERD_ENOMEM;
-                       goto error;
-               }
-               /*
-                * Close the socket created internally by
-                * lttcomm_create_sock, so we can replace it by the one
-                * received from sessiond.
-                */
-               if (close(relayd->control_sock.sock.fd)) {
-                       PERROR("close");
-               }
-
-               /* Assign new file descriptor */
-               relayd->control_sock.sock.fd = fd;
-               fd = -1;        /* For error path */
-               /* Assign version values. */
-               relayd->control_sock.major = relayd_sock->major;
-               relayd->control_sock.minor = relayd_sock->minor;
-
-               relayd->relayd_session_id = relayd_session_id;
-
-               break;
-       case LTTNG_STREAM_DATA:
-               /* Copy received lttcomm socket */
-               lttcomm_copy_sock(&relayd->data_sock.sock, &relayd_sock->sock);
-               ret = lttcomm_create_sock(&relayd->data_sock.sock);
-               /* Handle create_sock error. */
-               if (ret < 0) {
-                       ret_code = LTTCOMM_CONSUMERD_ENOMEM;
-                       goto error;
-               }
-               /*
-                * Close the socket created internally by
-                * lttcomm_create_sock, so we can replace it by the one
-                * received from sessiond.
-                */
-               if (close(relayd->data_sock.sock.fd)) {
-                       PERROR("close");
-               }
-
-               /* Assign new file descriptor */
-               relayd->data_sock.sock.fd = fd;
-               fd = -1;        /* for eventual error paths */
-               /* Assign version values. */
-               relayd->data_sock.major = relayd_sock->major;
-               relayd->data_sock.minor = relayd_sock->minor;
-               break;
-       default:
-               ERR("Unknown relayd socket type (%d)", sock_type);
-               ret = -1;
-               ret_code = LTTCOMM_CONSUMERD_FATAL;
-               goto error;
-       }
-
-       DBG("Consumer %s socket created successfully with net idx %" PRIu64 " (fd: %d)",
-                       sock_type == LTTNG_STREAM_CONTROL ? "control" : "data",
-                       relayd->net_seq_idx, fd);
-
-       /* We successfully added the socket. Send status back. */
-       ret = consumer_send_status_msg(sock, ret_code);
-       if (ret < 0) {
-               /* Somehow, the session daemon is not responding anymore. */
-               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_FATAL);
-               goto error_nosignal;
-       }
-
-       /*
-        * Add relayd socket pair to consumer data hashtable. If object already
-        * exists or on error, the function gracefully returns.
-        */
-       add_relayd(relayd);
-
-       /* All good! */
-       return 0;
-
-error:
-       if (consumer_send_status_msg(sock, ret_code) < 0) {
-               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_FATAL);
-       }
-
-error_nosignal:
-       /* Close received socket if valid. */
-       if (fd >= 0) {
-               if (close(fd)) {
-                       PERROR("close received socket");
-               }
-       }
-
-       if (relayd_created) {
-               free(relayd);
-       }
-
-       return ret;
-}
-
-/*
- * Try to lock the stream mutex.
- *
- * On success, 1 is returned else 0 indicating that the mutex is NOT lock.
- */
-static int stream_try_lock(struct lttng_consumer_stream *stream)
-{
-       int ret;
-
-       assert(stream);
-
-       /*
-        * Try to lock the stream mutex. On failure, we know that the stream is
-        * being used else where hence there is data still being extracted.
-        */
-       ret = pthread_mutex_trylock(&stream->lock);
-       if (ret) {
-               /* For both EBUSY and EINVAL error, the mutex is NOT locked. */
-               ret = 0;
-               goto end;
-       }
-
-       ret = 1;
-
-end:
-       return ret;
-}
-
-/*
- * Search for a relayd associated to the session id and return the reference.
- *
- * A rcu read side lock MUST be acquire before calling this function and locked
- * until the relayd object is no longer necessary.
- */
-static struct consumer_relayd_sock_pair *find_relayd_by_session_id(uint64_t id)
-{
-       struct lttng_ht_iter iter;
-       struct consumer_relayd_sock_pair *relayd = NULL;
-
-       /* Iterate over all relayd since they are indexed by net_seq_idx. */
-       cds_lfht_for_each_entry(consumer_data.relayd_ht->ht, &iter.iter, relayd,
-                       node.node) {
-               /*
-                * Check by sessiond id which is unique here where the relayd session
-                * id might not be when having multiple relayd.
-                */
-               if (relayd->sessiond_session_id == id) {
-                       /* Found the relayd. There can be only one per id. */
-                       goto found;
-               }
-       }
-
-       return NULL;
-
-found:
-       return relayd;
-}
-
-/*
- * Check if for a given session id there is still data needed to be extract
- * from the buffers.
- *
- * Return 1 if data is pending or else 0 meaning ready to be read.
- */
-int consumer_data_pending(uint64_t id)
-{
-       int ret;
-       struct lttng_ht_iter iter;
-       struct lttng_ht *ht;
-       struct lttng_consumer_stream *stream;
-       struct consumer_relayd_sock_pair *relayd = NULL;
-       int (*data_pending)(struct lttng_consumer_stream *);
-
-       DBG("Consumer data pending command on session id %" PRIu64, id);
-
-       rcu_read_lock();
-       pthread_mutex_lock(&consumer_data.lock);
-
-       switch (consumer_data.type) {
-       case LTTNG_CONSUMER_KERNEL:
-               data_pending = lttng_kconsumer_data_pending;
-               break;
-       case LTTNG_CONSUMER32_UST:
-       case LTTNG_CONSUMER64_UST:
-               data_pending = lttng_ustconsumer_data_pending;
-               break;
-       default:
-               ERR("Unknown consumer data type");
-               assert(0);
-       }
-
-       /* Ease our life a bit */
-       ht = consumer_data.stream_list_ht;
-
-       relayd = find_relayd_by_session_id(id);
-       if (relayd) {
-               /* Send init command for data pending. */
-               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-               ret = relayd_begin_data_pending(&relayd->control_sock,
-                               relayd->relayd_session_id);
-               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-               if (ret < 0) {
-                       /* Communication error thus the relayd so no data pending. */
-                       goto data_not_pending;
-               }
-       }
-
-       cds_lfht_for_each_entry_duplicate(ht->ht,
-                       ht->hash_fct(&id, lttng_ht_seed),
-                       ht->match_fct, &id,
-                       &iter.iter, stream, node_session_id.node) {
-               /* If this call fails, the stream is being used hence data pending. */
-               ret = stream_try_lock(stream);
-               if (!ret) {
-                       goto data_pending;
-               }
-
-               /*
-                * A removed node from the hash table indicates that the stream has
-                * been deleted thus having a guarantee that the buffers are closed
-                * on the consumer side. However, data can still be transmitted
-                * over the network so don't skip the relayd check.
-                */
-               ret = cds_lfht_is_node_deleted(&stream->node.node);
-               if (!ret) {
-                       /* Check the stream if there is data in the buffers. */
-                       ret = data_pending(stream);
-                       if (ret == 1) {
-                               pthread_mutex_unlock(&stream->lock);
-                               goto data_pending;
-                       }
-               }
-
-               /* Relayd check */
-               if (relayd) {
-                       pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-                       if (stream->metadata_flag) {
-                               ret = relayd_quiescent_control(&relayd->control_sock,
-                                               stream->relayd_stream_id);
-                       } else {
-                               ret = relayd_data_pending(&relayd->control_sock,
-                                               stream->relayd_stream_id,
-                                               stream->next_net_seq_num - 1);
-                       }
-                       pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-                       if (ret == 1) {
-                               pthread_mutex_unlock(&stream->lock);
-                               goto data_pending;
-                       }
-               }
-               pthread_mutex_unlock(&stream->lock);
-       }
-
-       if (relayd) {
-               unsigned int is_data_inflight = 0;
-
-               /* Send init command for data pending. */
-               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
-               ret = relayd_end_data_pending(&relayd->control_sock,
-                               relayd->relayd_session_id, &is_data_inflight);
-               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
-               if (ret < 0) {
-                       goto data_not_pending;
-               }
-               if (is_data_inflight) {
-                       goto data_pending;
-               }
-       }
-
-       /*
-        * Finding _no_ node in the hash table and no inflight data means that the
-        * stream(s) have been removed thus data is guaranteed to be available for
-        * analysis from the trace files.
-        */
-
-data_not_pending:
-       /* Data is available to be read by a viewer. */
-       pthread_mutex_unlock(&consumer_data.lock);
-       rcu_read_unlock();
-       return 0;
-
-data_pending:
-       /* Data is still being extracted from buffers. */
-       pthread_mutex_unlock(&consumer_data.lock);
-       rcu_read_unlock();
-       return 1;
-}
-
-/*
- * Send a ret code status message to the sessiond daemon.
- *
- * Return the sendmsg() return value.
- */
-int consumer_send_status_msg(int sock, int ret_code)
-{
-       struct lttcomm_consumer_status_msg msg;
-
-       memset(&msg, 0, sizeof(msg));
-       msg.ret_code = ret_code;
-
-       return lttcomm_send_unix_sock(sock, &msg, sizeof(msg));
-}
-
-/*
- * Send a channel status message to the sessiond daemon.
- *
- * Return the sendmsg() return value.
- */
-int consumer_send_status_channel(int sock,
-               struct lttng_consumer_channel *channel)
-{
-       struct lttcomm_consumer_status_channel msg;
-
-       assert(sock >= 0);
-
-       memset(&msg, 0, sizeof(msg));
-       if (!channel) {
-               msg.ret_code = LTTCOMM_CONSUMERD_CHANNEL_FAIL;
-       } else {
-               msg.ret_code = LTTCOMM_CONSUMERD_SUCCESS;
-               msg.key = channel->key;
-               msg.stream_count = channel->streams.count;
-       }
-
-       return lttcomm_send_unix_sock(sock, &msg, sizeof(msg));
-}
-
-unsigned long consumer_get_consume_start_pos(unsigned long consumed_pos,
-               unsigned long produced_pos, uint64_t nb_packets_per_stream,
-               uint64_t max_sb_size)
-{
-       unsigned long start_pos;
-
-       if (!nb_packets_per_stream) {
-               return consumed_pos;    /* Grab everything */
-       }
-       start_pos = produced_pos - offset_align_floor(produced_pos, max_sb_size);
-       start_pos -= max_sb_size * nb_packets_per_stream;
-       if ((long) (start_pos - consumed_pos) < 0) {
-               return consumed_pos;    /* Grab everything */
-       }
-       return start_pos;
-}
diff --git a/src/common/consumer.h b/src/common/consumer.h
deleted file mode 100644 (file)
index c7ef3fb..0000000
+++ /dev/null
@@ -1,698 +0,0 @@
-/*
- * Copyright (C) 2011 - Julien Desfossez <julien.desfossez@polymtl.ca>
- *                      Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
- *               2012 - David Goulet <dgoulet@efficios.com>
- *
- * This program is free software; you can redistribute it and/or modify
- * it under the terms of the GNU General Public License, version 2 only,
- * as published by the Free Software Foundation.
- *
- * This program 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 General Public License for
- * more details.
- *
- * You should have received a copy of the GNU General Public License along
- * with this program; if not, write to the Free Software Foundation, Inc.,
- * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
- */
-
-#ifndef LIB_CONSUMER_H
-#define LIB_CONSUMER_H
-
-#include <limits.h>
-#include <poll.h>
-#include <unistd.h>
-#include <urcu/list.h>
-
-#include <lttng/lttng.h>
-
-#include <common/hashtable/hashtable.h>
-#include <common/compat/fcntl.h>
-#include <common/compat/uuid.h>
-#include <common/sessiond-comm/sessiond-comm.h>
-#include <common/pipe.h>
-#include <common/index/ctf-index.h>
-
-/* Commands for consumer */
-enum lttng_consumer_command {
-       LTTNG_CONSUMER_ADD_CHANNEL,
-       LTTNG_CONSUMER_ADD_STREAM,
-       /* pause, delete, active depending on fd state */
-       LTTNG_CONSUMER_UPDATE_STREAM,
-       /* inform the consumer to quit when all fd has hang up */
-       LTTNG_CONSUMER_STOP,    /* deprecated */
-       LTTNG_CONSUMER_ADD_RELAYD_SOCKET,
-       /* Inform the consumer to kill a specific relayd connection */
-       LTTNG_CONSUMER_DESTROY_RELAYD,
-       /* Return to the sessiond if there is data pending for a session */
-       LTTNG_CONSUMER_DATA_PENDING,
-       /* Consumer creates a channel and returns it to sessiond. */
-       LTTNG_CONSUMER_ASK_CHANNEL_CREATION,
-       LTTNG_CONSUMER_GET_CHANNEL,
-       LTTNG_CONSUMER_DESTROY_CHANNEL,
-       LTTNG_CONSUMER_PUSH_METADATA,
-       LTTNG_CONSUMER_CLOSE_METADATA,
-       LTTNG_CONSUMER_SETUP_METADATA,
-       LTTNG_CONSUMER_FLUSH_CHANNEL,
-       LTTNG_CONSUMER_SNAPSHOT_CHANNEL,
-       LTTNG_CONSUMER_SNAPSHOT_METADATA,
-       LTTNG_CONSUMER_STREAMS_SENT,
-};
-
-/* State of each fd in consumer */
-enum lttng_consumer_stream_state {
-       LTTNG_CONSUMER_ACTIVE_STREAM,
-       LTTNG_CONSUMER_PAUSE_STREAM,
-       LTTNG_CONSUMER_DELETE_STREAM,
-};
-
-enum lttng_consumer_type {
-       LTTNG_CONSUMER_UNKNOWN = 0,
-       LTTNG_CONSUMER_KERNEL,
-       LTTNG_CONSUMER64_UST,
-       LTTNG_CONSUMER32_UST,
-};
-
-enum consumer_endpoint_status {
-       CONSUMER_ENDPOINT_ACTIVE,
-       CONSUMER_ENDPOINT_INACTIVE,
-};
-
-enum consumer_channel_output {
-       CONSUMER_CHANNEL_MMAP   = 0,
-       CONSUMER_CHANNEL_SPLICE = 1,
-};
-
-enum consumer_channel_type {
-       CONSUMER_CHANNEL_TYPE_METADATA  = 0,
-       CONSUMER_CHANNEL_TYPE_DATA      = 1,
-};
-
-extern struct lttng_consumer_global_data consumer_data;
-
-struct stream_list {
-       struct cds_list_head head;
-       unsigned int count;
-};
-
-/* Stub. */
-struct consumer_metadata_cache;
-
-struct lttng_consumer_channel {
-       /* HT node used for consumer_data.channel_ht */
-       struct lttng_ht_node_u64 node;
-       /* Indexed key. Incremented value in the consumer. */
-       uint64_t key;
-       /* Number of streams referencing this channel */
-       int refcount;
-       /* Tracing session id on the session daemon side. */
-       uint64_t session_id;
-       /*
-        * Session id when requesting metadata to the session daemon for
-        * a session with per-PID buffers.
-        */
-       uint64_t session_id_per_pid;
-       /* Channel trace file path name. */
-       char pathname[PATH_MAX];
-       /* Channel name. */
-       char name[LTTNG_SYMBOL_NAME_LEN];
-       /* UID and GID of the session owning this channel. */
-       uid_t uid;
-       gid_t gid;
-       /* Relayd id of the channel. -1ULL if it does not apply. */
-       uint64_t relayd_id;
-       /*
-        * Number of streams NOT initialized yet. This is used in order to not
-        * delete this channel if streams are getting initialized.
-        */
-       unsigned int nb_init_stream_left;
-       /* Output type (mmap or splice). */
-       enum consumer_channel_output output;
-       /* Channel type for stream */
-       enum consumer_channel_type type;
-
-       /* For UST */
-       uid_t ust_app_uid;      /* Application UID. */
-       struct ustctl_consumer_channel *uchan;
-       unsigned char uuid[UUID_STR_LEN];
-       /*
-        * Temporary stream list used to store the streams once created and waiting
-        * to be sent to the session daemon by receiving the
-        * LTTNG_CONSUMER_GET_CHANNEL.
-        */
-       struct stream_list streams;
-
-       /*
-        * Set if the channel is metadata. We keep a reference to the stream
-        * because we have to flush data once pushed by the session daemon. For a
-        * regular channel, this is always set to NULL.
-        */
-       struct lttng_consumer_stream *metadata_stream;
-
-       /* for UST */
-       int wait_fd;
-       /* Node within channel thread ht */
-       struct lttng_ht_node_u64 wait_fd_node;
-
-       /* Metadata cache is metadata channel */
-       struct consumer_metadata_cache *metadata_cache;
-       /* For UST metadata periodical flush */
-       int switch_timer_enabled;
-       timer_t switch_timer;
-       int switch_timer_error;
-
-       /* For the live mode */
-       int live_timer_enabled;
-       timer_t live_timer;
-       int live_timer_error;
-
-       /* On-disk circular buffer */
-       uint64_t tracefile_size;
-       uint64_t tracefile_count;
-       /*
-        * Monitor or not the streams of this channel meaning this indicates if the
-        * streams should be sent to the data/metadata thread or added to the no
-        * monitor list of the channel.
-        */
-       unsigned int monitor;
-
-       /*
-        * Channel lock.
-        *
-        * This lock protects against concurrent update of channel.
-        *
-        * This is nested INSIDE the consumer data lock.
-        * This is nested OUTSIDE the channel timer lock.
-        * This is nested OUTSIDE the metadata cache lock.
-        * This is nested OUTSIDE stream lock.
-        * This is nested OUTSIDE consumer_relayd_sock_pair lock.
-        */
-       pthread_mutex_t lock;
-
-       /*
-        * Channel teardown lock.
-        *
-        * This lock protect against teardown of channel. It is _never_
-        * taken by the timer handler.
-        *
-        * This is nested INSIDE the consumer data lock.
-        * This is nested INSIDE the channel lock.
-        * This is nested OUTSIDE the metadata cache lock.
-        * This is nested OUTSIDE stream lock.
-        * This is nested OUTSIDE consumer_relayd_sock_pair lock.
-        */
-       pthread_mutex_t timer_lock;
-
-       /* Timer value in usec for live streaming. */
-       unsigned int live_timer_interval;
-
-       int *stream_fds;
-       int nr_stream_fds;
-       char root_shm_path[PATH_MAX];
-       char shm_path[PATH_MAX];
-};
-
-/*
- * Internal representation of the streams, sessiond_key is used to identify
- * uniquely a stream.
- */
-struct lttng_consumer_stream {
-       /* HT node used by the data_ht and metadata_ht */
-       struct lttng_ht_node_u64 node;
-       /* stream indexed per channel key node */
-       struct lttng_ht_node_u64 node_channel_id;
-       /* HT node used in consumer_data.stream_list_ht */
-       struct lttng_ht_node_u64 node_session_id;
-       /* Pointer to associated channel. */
-       struct lttng_consumer_channel *chan;
-
-       /* Key by which the stream is indexed for 'node'. */
-       uint64_t key;
-       /*
-        * File descriptor of the data output file. This can be either a file or a
-        * socket fd for relayd streaming.
-        */
-       int out_fd; /* output file to write the data */
-       /* Write position in the output file descriptor */
-       off_t out_fd_offset;
-       /* Amount of bytes written to the output */
-       uint64_t output_written;
-       enum lttng_consumer_stream_state state;
-       int shm_fd_is_copy;
-       int data_read;
-       int hangup_flush_done;
-
-       /*
-        * metadata_timer_lock protects flags waiting_on_metadata and
-        * missed_metadata_flush.
-        */
-       pthread_mutex_t metadata_timer_lock;
-       /*
-        * Flag set when awaiting metadata to be pushed. Used in the
-        * timer thread to skip waiting on the stream (and stream lock) to
-        * ensure we can proceed to flushing metadata in live mode.
-        */
-       bool waiting_on_metadata;
-       /* Raised when a timer misses a metadata flush. */
-       bool missed_metadata_flush;
-
-       enum lttng_event_output output;
-       /* Maximum subbuffer size. */
-       unsigned long max_sb_size;
-
-       /*
-        * Still used by the kernel for MMAP output. For UST, the ustctl getter is
-        * used for the mmap base and offset.
-        */
-       void *mmap_base;
-       unsigned long mmap_len;
-
-       /* For UST */
-
-       int wait_fd;
-       /* UID/GID of the user owning the session to which stream belongs */
-       uid_t uid;
-       gid_t gid;
-       /* Network sequence number. Indicating on which relayd socket it goes. */
-       uint64_t net_seq_idx;
-       /*
-        * Indicate if this stream was successfully sent to a relayd. This is set
-        * after the refcount of the relayd is incremented and is checked when the
-        * stream is closed before decrementing the refcount in order to avoid an
-        * unbalanced state.
-        */
-       unsigned int sent_to_relayd;
-
-       /* Identify if the stream is the metadata */
-       unsigned int metadata_flag;
-       /* Used when the stream is set for network streaming */
-       uint64_t relayd_stream_id;
-       /*
-        * When sending a stream packet to a relayd, this number is used to track
-        * the packet sent by the consumer and seen by the relayd. When sending the
-        * data header to the relayd, this number is sent and if the transmission
-        * was successful, it is incremented.
-        *
-        * Even if the full data is not fully transmitted it won't matter since
-        * only two possible error can happen after that where either the relayd
-        * died or a read error is detected on the stream making this value useless
-        * after that.
-        *
-        * This value SHOULD be read/updated atomically or with the lock acquired.
-        */
-       uint64_t next_net_seq_num;
-       /*
-        * Lock to use the stream FDs since they are used between threads.
-        *
-        * This is nested INSIDE the consumer_data lock.
-        * This is nested INSIDE the metadata cache lock.
-        * This is nested INSIDE the channel lock.
-        * This is nested INSIDE the channel timer lock.
-        * This is nested OUTSIDE consumer_relayd_sock_pair lock.
-        */
-       pthread_mutex_t lock;
-       /* Tracing session id */
-       uint64_t session_id;
-       /*
-        * Indicates if the stream end point is still active or not (network
-        * streaming or local file system). The thread "owning" the stream is
-        * handling this status and can be notified of a state change through the
-        * consumer data appropriate pipe.
-        */
-       enum consumer_endpoint_status endpoint_status;
-       /* Stream name. Format is: <channel_name>_<cpu_number> */
-       char name[LTTNG_SYMBOL_NAME_LEN];
-       /* Internal state of libustctl. */
-       struct ustctl_consumer_stream *ustream;
-       struct cds_list_head send_node;
-       /* On-disk circular buffer */
-       uint64_t tracefile_size_current;
-       uint64_t tracefile_count_current;
-       /*
-        * Monitor or not the streams of this channel meaning this indicates if the
-        * streams should be sent to the data/metadata thread or added to the no
-        * monitor list of the channel.
-        */
-       unsigned int monitor;
-       /*
-        * Indicate if the stream is globally visible meaning that it has been
-        * added to the multiple hash tables. If *not* set, NO lock should be
-        * acquired in the destroy path.
-        */
-       unsigned int globally_visible;
-       /*
-        * Pipe to wake up the metadata poll thread when the UST metadata
-        * cache is updated.
-        */
-       int ust_metadata_poll_pipe[2];
-       /*
-        * How much metadata was read from the metadata cache and sent
-        * to the channel.
-        */
-       uint64_t ust_metadata_pushed;
-       /*
-        * FD of the index file for this stream.
-        */
-       int index_fd;
-
-       /*
-        * Local pipe to extract data when using splice.
-        */
-       int splice_pipe[2];
-
-       /*
-        * Rendez-vous point between data and metadata stream in live mode.
-        */
-       pthread_cond_t metadata_rdv;
-       pthread_mutex_t metadata_rdv_lock;
-
-       /* Indicate if the stream still has some data to be read. */
-       unsigned int has_data:1;
-};
-
-/*
- * Internal representation of a relayd socket pair.
- */
-struct consumer_relayd_sock_pair {
-       /* Network sequence number. */
-       uint64_t net_seq_idx;
-       /* Number of stream associated with this relayd */
-       unsigned int refcount;
-
-       /*
-        * This flag indicates whether or not we should destroy this object. The
-        * destruction should ONLY occurs when this flag is set and the refcount is
-        * set to zero.
-        */
-       unsigned int destroy_flag;
-
-       /*
-        * Mutex protecting the control socket to avoid out of order packets
-        * between threads sending data to the relayd. Since metadata data is sent
-        * over that socket, at least two sendmsg() are needed (header + data)
-        * creating a race for packets to overlap between threads using it.
-        *
-        * This is nested INSIDE the consumer_data lock.
-        * This is nested INSIDE the stream lock.
-        */
-       pthread_mutex_t ctrl_sock_mutex;
-
-       /* Control socket. Command and metadata are passed over it */
-       struct lttcomm_relayd_sock control_sock;
-
-       /*
-        * We don't need a mutex at this point since we only splice or write single
-        * large chunk of data with a header appended at the begining. Moreover,
-        * this socket is for now only used in a single thread.
-        */
-       struct lttcomm_relayd_sock data_sock;
-       struct lttng_ht_node_u64 node;
-
-       /* Session id on both sides for the sockets. */
-       uint64_t relayd_session_id;
-       uint64_t sessiond_session_id;
-};
-
-/*
- * UST consumer local data to the program. One or more instance per
- * process.
- */
-struct lttng_consumer_local_data {
-       /*
-        * Function to call when data is available on a buffer.
-        * Returns the number of bytes read, or negative error value.
-        */
-       ssize_t (*on_buffer_ready)(struct lttng_consumer_stream *stream,
-                       struct lttng_consumer_local_data *ctx);
-       /*
-        * function to call when we receive a new channel, it receives a
-        * newly allocated channel, depending on the return code of this
-        * function, the new channel will be handled by the application
-        * or the library.
-        *
-        * Returns:
-        *    > 0 (success, FD is kept by application)
-        *   == 0 (success, FD is left to library)
-        *    < 0 (error)
-        */
-       int (*on_recv_channel)(struct lttng_consumer_channel *channel);
-       /*
-        * function to call when we receive a new stream, it receives a
-        * newly allocated stream, depending on the return code of this
-        * function, the new stream will be handled by the application
-        * or the library.
-        *
-        * Returns:
-        *    > 0 (success, FD is kept by application)
-        *   == 0 (success, FD is left to library)
-        *    < 0 (error)
-        */
-       int (*on_recv_stream)(struct lttng_consumer_stream *stream);
-       /*
-        * function to call when a stream is getting updated by the session
-        * daemon, this function receives the sessiond key and the new
-        * state, depending on the return code of this function the
-        * update of state for the stream is handled by the application
-        * or the library.
-        *
-        * Returns:
-        *    > 0 (success, FD is kept by application)
-        *   == 0 (success, FD is left to library)
-        *    < 0 (error)
-        */
-       int (*on_update_stream)(uint64_t sessiond_key, uint32_t state);
-       enum lttng_consumer_type type;
-       /* socket to communicate errors with sessiond */
-       int consumer_error_socket;
-       /* socket to ask metadata to sessiond. */
-       int consumer_metadata_socket;
-       /*
-        * Protect consumer_metadata_socket.
-        *
-        * This is nested OUTSIDE the metadata cache lock.
-        */
-       pthread_mutex_t metadata_socket_lock;
-       /* socket to exchange commands with sessiond */
-       char *consumer_command_sock_path;
-       /* communication with splice */
-       int consumer_channel_pipe[2];
-       /* Data stream poll thread pipe. To transfer data stream to the thread */
-       struct lttng_pipe *consumer_data_pipe;
-
-       /*
-        * Data thread use that pipe to catch wakeup from read subbuffer that
-        * detects that there is still data to be read for the stream encountered.
-        * Before doing so, the stream is flagged to indicate that there is still
-        * data to be read.
-        *
-        * Both pipes (read/write) are owned and used inside the data thread.
-        */
-       struct lttng_pipe *consumer_wakeup_pipe;
-       /* Indicate if the wakeup thread has been notified. */
-       unsigned int has_wakeup:1;
-
-       /* to let the signal handler wake up the fd receiver thread */
-       int consumer_should_quit[2];
-       /* Metadata poll thread pipe. Transfer metadata stream to it */
-       struct lttng_pipe *consumer_metadata_pipe;
-};
-
-/*
- * Library-level data. One instance per process.
- */
-struct lttng_consumer_global_data {
-       /*
-        * At this time, this lock is used to ensure coherence between the count
-        * and number of element in the hash table. It's also a protection for
-        * concurrent read/write between threads.
-        *
-        * This is nested OUTSIDE the stream lock.
-        * This is nested OUTSIDE the consumer_relayd_sock_pair lock.
-        */
-       pthread_mutex_t lock;
-
-       /*
-        * Number of streams in the data stream hash table declared outside.
-        * Protected by consumer_data.lock.
-        */
-       int stream_count;
-
-       /* Channel hash table protected by consumer_data.lock. */
-       struct lttng_ht *channel_ht;
-       /*
-        * Flag specifying if the local array of FDs needs update in the
-        * poll function. Protected by consumer_data.lock.
-        */
-       unsigned int need_update;
-       enum lttng_consumer_type type;
-
-       /*
-        * Relayd socket(s) hashtable indexed by network sequence number. Each
-        * stream has an index which associate the right relayd socket to use.
-        */
-       struct lttng_ht *relayd_ht;
-
-       /*
-        * This hash table contains all streams (metadata and data) indexed by
-        * session id. In other words, the ht is indexed by session id and each
-        * bucket contains the list of associated streams.
-        *
-        * This HT uses the "node_session_id" of the consumer stream.
-        */
-       struct lttng_ht *stream_list_ht;
-
-       /*
-        * This HT uses the "node_channel_id" of the consumer stream.
-        */
-       struct lttng_ht *stream_per_chan_id_ht;
-};
-
-/*
- * Init consumer data structures.
- */
-int lttng_consumer_init(void);
-
-/*
- * Set the error socket for communication with a session daemon.
- */
-void lttng_consumer_set_error_sock(struct lttng_consumer_local_data *ctx,
-               int sock);
-
-/*
- * Set the command socket path for communication with a session daemon.
- */
-void lttng_consumer_set_command_sock_path(
-               struct lttng_consumer_local_data *ctx, char *sock);
-
-/*
- * Send return code to session daemon.
- *
- * Returns the return code of sendmsg : the number of bytes transmitted or -1
- * on error.
- */
-int lttng_consumer_send_error(struct lttng_consumer_local_data *ctx, int cmd);
-
-/*
- * Called from signal handler to ensure a clean exit.
- */
-void lttng_consumer_should_exit(struct lttng_consumer_local_data *ctx);
-
-/*
- * Cleanup the daemon's socket on exit.
- */
-void lttng_consumer_cleanup(void);
-
-/*
- * Flush pending writes to trace output disk file.
- */
-void lttng_consumer_sync_trace_file(struct lttng_consumer_stream *stream,
-               off_t orig_offset);
-
-/*
- * Poll on the should_quit pipe and the command socket return -1 on error and
- * should exit, 0 if data is available on the command socket
- */
-int lttng_consumer_poll_socket(struct pollfd *kconsumer_sockpoll);
-
-struct lttng_consumer_stream *consumer_allocate_stream(uint64_t channel_key,
-               uint64_t stream_key,
-               enum lttng_consumer_stream_state state,
-               const char *channel_name,
-               uid_t uid,
-               gid_t gid,
-               uint64_t relayd_id,
-               uint64_t session_id,
-               int cpu,
-               int *alloc_ret,
-               enum consumer_channel_type type,
-               unsigned int monitor);
-struct lttng_consumer_channel *consumer_allocate_channel(uint64_t key,
-               uint64_t session_id,
-               const char *pathname,
-               const char *name,
-               uid_t uid,
-               gid_t gid,
-               uint64_t relayd_id,
-               enum lttng_event_output output,
-               uint64_t tracefile_size,
-               uint64_t tracefile_count,
-               uint64_t session_id_per_pid,
-               unsigned int monitor,
-               unsigned int live_timer_interval,
-               const char *root_shm_path,
-               const char *shm_path);
-void consumer_del_stream(struct lttng_consumer_stream *stream,
-               struct lttng_ht *ht);
-void consumer_del_metadata_stream(struct lttng_consumer_stream *stream,
-               struct lttng_ht *ht);
-int consumer_add_channel(struct lttng_consumer_channel *channel,
-               struct lttng_consumer_local_data *ctx);
-void consumer_del_channel(struct lttng_consumer_channel *channel);
-
-/* lttng-relayd consumer command */
-struct consumer_relayd_sock_pair *consumer_find_relayd(uint64_t key);
-int consumer_send_relayd_stream(struct lttng_consumer_stream *stream, char *path);
-int consumer_send_relayd_streams_sent(uint64_t net_seq_idx);
-void close_relayd_stream(struct lttng_consumer_stream *stream);
-struct lttng_consumer_channel *consumer_find_channel(uint64_t key);
-int consumer_handle_stream_before_relayd(struct lttng_consumer_stream *stream,
-               size_t data_size);
-void consumer_steal_stream_key(int key, struct lttng_ht *ht);
-
-struct lttng_consumer_local_data *lttng_consumer_create(
-               enum lttng_consumer_type type,
-               ssize_t (*buffer_ready)(struct lttng_consumer_stream *stream,
-                       struct lttng_consumer_local_data *ctx),
-               int (*recv_channel)(struct lttng_consumer_channel *channel),
-               int (*recv_stream)(struct lttng_consumer_stream *stream),
-               int (*update_stream)(uint64_t sessiond_key, uint32_t state));
-void lttng_consumer_destroy(struct lttng_consumer_local_data *ctx);
-ssize_t lttng_consumer_on_read_subbuffer_mmap(
-               struct lttng_consumer_local_data *ctx,
-               struct lttng_consumer_stream *stream, unsigned long len,
-               unsigned long padding,
-               struct ctf_packet_index *index);
-ssize_t lttng_consumer_on_read_subbuffer_splice(
-               struct lttng_consumer_local_data *ctx,
-               struct lttng_consumer_stream *stream, unsigned long len,
-               unsigned long padding,
-               struct ctf_packet_index *index);
-int lttng_consumer_take_snapshot(struct lttng_consumer_stream *stream);
-int lttng_consumer_get_produced_snapshot(struct lttng_consumer_stream *stream,
-               unsigned long *pos);
-int lttng_ustconsumer_get_wakeup_fd(struct lttng_consumer_stream *stream);
-int lttng_ustconsumer_close_wakeup_fd(struct lttng_consumer_stream *stream);
-void *consumer_thread_metadata_poll(void *data);
-void *consumer_thread_data_poll(void *data);
-void *consumer_thread_sessiond_poll(void *data);
-void *consumer_thread_channel_poll(void *data);
-int lttng_consumer_recv_cmd(struct lttng_consumer_local_data *ctx,
-               int sock, struct pollfd *consumer_sockpoll);
-
-ssize_t lttng_consumer_read_subbuffer(struct lttng_consumer_stream *stream,
-               struct lttng_consumer_local_data *ctx);
-int lttng_consumer_on_recv_stream(struct lttng_consumer_stream *stream);
-int consumer_add_relayd_socket(uint64_t net_seq_idx, int sock_type,
-               struct lttng_consumer_local_data *ctx, int sock,
-               struct pollfd *consumer_sockpoll, struct lttcomm_relayd_sock *relayd_sock,
-               uint64_t sessiond_id, uint64_t relayd_session_id);
-void consumer_flag_relayd_for_destroy(
-               struct consumer_relayd_sock_pair *relayd);
-int consumer_data_pending(uint64_t id);
-int consumer_send_status_msg(int sock, int ret_code);
-int consumer_send_status_channel(int sock,
-               struct lttng_consumer_channel *channel);
-void notify_thread_del_channel(struct lttng_consumer_local_data *ctx,
-               uint64_t key);
-void consumer_destroy_relayd(struct consumer_relayd_sock_pair *relayd);
-unsigned long consumer_get_consume_start_pos(unsigned long consumed_pos,
-               unsigned long produced_pos, uint64_t nb_packets_per_stream,
-               uint64_t max_sb_size);
-int consumer_add_data_stream(struct lttng_consumer_stream *stream);
-void consumer_del_stream_for_data(struct lttng_consumer_stream *stream);
-int consumer_add_metadata_stream(struct lttng_consumer_stream *stream);
-void consumer_del_stream_for_metadata(struct lttng_consumer_stream *stream);
-int consumer_create_index_file(struct lttng_consumer_stream *stream);
-
-#endif /* LIB_CONSUMER_H */
diff --git a/src/common/consumer/Makefile.am b/src/common/consumer/Makefile.am
new file mode 100644 (file)
index 0000000..029fb1d
--- /dev/null
@@ -0,0 +1,22 @@
+AM_CPPFLAGS = -I$(top_srcdir)/include -I$(top_srcdir)/src
+
+# Consumer library
+noinst_LTLIBRARIES = libconsumer.la
+
+noinst_HEADERS = consumer-metadata-cache.h consumer-timer.h \
+                consumer-testpoint.h
+
+libconsumer_la_SOURCES = consumer.c consumer.h consumer-metadata-cache.c \
+                         consumer-timer.c consumer-stream.c consumer-stream.h
+
+libconsumer_la_LIBADD = \
+               $(top_builddir)/src/common/sessiond-comm/libsessiond-comm.la \
+               $(top_builddir)/src/common/kernel-consumer/libkernel-consumer.la \
+               $(top_builddir)/src/common/hashtable/libhashtable.la \
+               $(top_builddir)/src/common/compat/libcompat.la \
+               $(top_builddir)/src/common/relayd/librelayd.la
+
+if HAVE_LIBLTTNG_UST_CTL
+libconsumer_la_LIBADD += \
+               $(top_builddir)/src/common/ust-consumer/libust-consumer.la
+endif
diff --git a/src/common/consumer/consumer-metadata-cache.c b/src/common/consumer/consumer-metadata-cache.c
new file mode 100644 (file)
index 0000000..48257c2
--- /dev/null
@@ -0,0 +1,238 @@
+/*
+ * Copyright (C) 2013 - Julien Desfossez <jdesfossez@efficios.com>
+ *                      David Goulet <dgoulet@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License, version 2 only,
+ * as published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along
+ * with this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#define _LGPL_SOURCE
+#include <assert.h>
+#include <pthread.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/types.h>
+#include <unistd.h>
+#include <inttypes.h>
+
+#include <common/common.h>
+#include <common/utils.h>
+#include <common/sessiond-comm/sessiond-comm.h>
+#include <common/ust-consumer/ust-consumer.h>
+#include <common/consumer/consumer.h>
+
+#include "consumer-metadata-cache.h"
+
+extern struct lttng_consumer_global_data consumer_data;
+
+/*
+ * Extend the allocated size of the metadata cache. Called only from
+ * lttng_ustconsumer_write_metadata_cache.
+ *
+ * Return 0 on success, a negative value on error.
+ */
+static int extend_metadata_cache(struct lttng_consumer_channel *channel,
+               unsigned int size)
+{
+       int ret = 0;
+       char *tmp_data_ptr;
+       unsigned int new_size, old_size;
+
+       assert(channel);
+       assert(channel->metadata_cache);
+
+       old_size = channel->metadata_cache->cache_alloc_size;
+       new_size = max_t(unsigned int, old_size + size, old_size << 1);
+       DBG("Extending metadata cache to %u", new_size);
+       tmp_data_ptr = realloc(channel->metadata_cache->data, new_size);
+       if (!tmp_data_ptr) {
+               ERR("Reallocating metadata cache");
+               free(channel->metadata_cache->data);
+               ret = -1;
+               goto end;
+       }
+       /* Zero newly allocated memory */
+       memset(tmp_data_ptr + old_size, 0, new_size - old_size);
+       channel->metadata_cache->data = tmp_data_ptr;
+       channel->metadata_cache->cache_alloc_size = new_size;
+
+end:
+       return ret;
+}
+
+/*
+ * Write metadata to the cache, extend the cache if necessary. We support
+ * overlapping updates, but they need to be contiguous. Send the
+ * contiguous metadata in cache to the ring buffer. The metadata cache
+ * lock MUST be acquired to write in the cache.
+ *
+ * Return 0 on success, a negative value on error.
+ */
+int consumer_metadata_cache_write(struct lttng_consumer_channel *channel,
+               unsigned int offset, unsigned int len, char *data)
+{
+       int ret = 0;
+       int size_ret;
+       struct consumer_metadata_cache *cache;
+
+       assert(channel);
+       assert(channel->metadata_cache);
+
+       cache = channel->metadata_cache;
+       DBG("Writing %u bytes from offset %u in metadata cache", len, offset);
+
+       if (offset + len > cache->cache_alloc_size) {
+               ret = extend_metadata_cache(channel,
+                               len - cache->cache_alloc_size + offset);
+               if (ret < 0) {
+                       ERR("Extending metadata cache");
+                       goto end;
+               }
+       }
+
+       memcpy(cache->data + offset, data, len);
+       if (offset + len > cache->max_offset) {
+               char dummy = 'c';
+
+               cache->max_offset = offset + len;
+               if (channel->monitor) {
+                       size_ret = lttng_write(channel->metadata_stream->ust_metadata_poll_pipe[1],
+                                       &dummy, 1);
+                       if (size_ret < 1) {
+                               ERR("Wakeup UST metadata pipe");
+                               ret = -1;
+                               goto end;
+                       }
+               }
+       }
+
+end:
+       return ret;
+}
+
+/*
+ * Create the metadata cache, original allocated size: max_sb_size
+ *
+ * Return 0 on success, a negative value on error.
+ */
+int consumer_metadata_cache_allocate(struct lttng_consumer_channel *channel)
+{
+       int ret;
+
+       assert(channel);
+
+       channel->metadata_cache = zmalloc(
+                       sizeof(struct consumer_metadata_cache));
+       if (!channel->metadata_cache) {
+               PERROR("zmalloc metadata cache struct");
+               ret = -1;
+               goto end;
+       }
+       ret = pthread_mutex_init(&channel->metadata_cache->lock, NULL);
+       if (ret != 0) {
+               PERROR("mutex init");
+               goto end_free_cache;
+       }
+
+       channel->metadata_cache->cache_alloc_size = DEFAULT_METADATA_CACHE_SIZE;
+       channel->metadata_cache->data = zmalloc(
+                       channel->metadata_cache->cache_alloc_size * sizeof(char));
+       if (!channel->metadata_cache->data) {
+               PERROR("zmalloc metadata cache data");
+               ret = -1;
+               goto end_free_mutex;
+       }
+       DBG("Allocated metadata cache of %" PRIu64 " bytes",
+                       channel->metadata_cache->cache_alloc_size);
+
+       ret = 0;
+       goto end;
+
+end_free_mutex:
+       pthread_mutex_destroy(&channel->metadata_cache->lock);
+end_free_cache:
+       free(channel->metadata_cache);
+end:
+       return ret;
+}
+
+/*
+ * Destroy and free the metadata cache
+ */
+void consumer_metadata_cache_destroy(struct lttng_consumer_channel *channel)
+{
+       if (!channel || !channel->metadata_cache) {
+               return;
+       }
+
+       DBG("Destroying metadata cache");
+
+       pthread_mutex_destroy(&channel->metadata_cache->lock);
+       free(channel->metadata_cache->data);
+       free(channel->metadata_cache);
+}
+
+/*
+ * Check if the cache is flushed up to the offset passed in parameter.
+ *
+ * Return 0 if everything has been flushed, 1 if there is data not flushed.
+ */
+int consumer_metadata_cache_flushed(struct lttng_consumer_channel *channel,
+               uint64_t offset, int timer)
+{
+       int ret = 0;
+       struct lttng_consumer_stream *metadata_stream;
+
+       assert(channel);
+       assert(channel->metadata_cache);
+
+       /*
+        * If not called from a timer handler, we have to take the
+        * channel lock to be mutually exclusive with channel teardown.
+        * Timer handler does not need to take this lock because it is
+        * already synchronized by timer stop (and, more importantly,
+        * taking this lock in a timer handler would cause a deadlock).
+        */
+       if (!timer) {
+               pthread_mutex_lock(&channel->lock);
+       }
+       pthread_mutex_lock(&channel->timer_lock);
+       pthread_mutex_lock(&channel->metadata_cache->lock);
+
+       metadata_stream = channel->metadata_stream;
+
+       if (!metadata_stream) {
+               /*
+                * Having no metadata stream means the channel is being destroyed so there
+                * is no cache to flush anymore.
+                */
+               ret = 0;
+       } else if (metadata_stream->ust_metadata_pushed >= offset) {
+               ret = 0;
+       } else if (channel->metadata_stream->endpoint_status !=
+                       CONSUMER_ENDPOINT_ACTIVE) {
+               /* An inactive endpoint means we don't have to flush anymore. */
+               ret = 0;
+       } else {
+               /* Still not completely flushed. */
+               ret = 1;
+       }
+
+       pthread_mutex_unlock(&channel->metadata_cache->lock);
+       pthread_mutex_unlock(&channel->timer_lock);
+       if (!timer) {
+               pthread_mutex_unlock(&channel->lock);
+       }
+
+       return ret;
+}
diff --git a/src/common/consumer/consumer-metadata-cache.h b/src/common/consumer/consumer-metadata-cache.h
new file mode 100644 (file)
index 0000000..8ce7f30
--- /dev/null
@@ -0,0 +1,51 @@
+/*
+ * Copyright (C) 2013 - Julien Desfossez <jdesfossez@efficios.com>
+ *                      David Goulet <dgoulet@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License, version 2 only,
+ * as published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along
+ * with this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#ifndef CONSUMER_METADATA_CACHE_H
+#define CONSUMER_METADATA_CACHE_H
+
+#include <common/consumer/consumer.h>
+
+struct consumer_metadata_cache {
+       char *data;
+       uint64_t cache_alloc_size;
+       /*
+        * The upper-limit of data written inside the buffer.
+        *
+        * With the total_bytes_written it allows us to keep track of when the
+        * cache contains contiguous metadata ready to be sent to the RB.
+        * All cached data is contiguous.
+        */
+       uint64_t max_offset;
+       /*
+        * Lock to update the metadata cache and push into the ring_buffer
+        * (ustctl_write_metadata_to_channel).
+        *
+        * This is nested INSIDE the consumer_data lock.
+        */
+       pthread_mutex_t lock;
+};
+
+int consumer_metadata_cache_write(struct lttng_consumer_channel *channel,
+               unsigned int offset, unsigned int len, char *data);
+int consumer_metadata_cache_allocate(struct lttng_consumer_channel *channel);
+void consumer_metadata_cache_destroy(struct lttng_consumer_channel *channel);
+int consumer_metadata_cache_flushed(struct lttng_consumer_channel *channel,
+               uint64_t offset, int timer);
+
+#endif /* CONSUMER_METADATA_CACHE_H */
diff --git a/src/common/consumer/consumer-stream.c b/src/common/consumer/consumer-stream.c
new file mode 100644 (file)
index 0000000..a62cef2
--- /dev/null
@@ -0,0 +1,554 @@
+/*
+ * Copyright (C) 2011 - Julien Desfossez <julien.desfossez@polymtl.ca>
+ *                      Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
+ * Copyright (C) 2013 - David Goulet <dgoulet@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License, version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along with
+ * this program; if not, write to the Free Software Foundation, Inc., 51
+ * Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#define _LGPL_SOURCE
+#include <assert.h>
+#include <inttypes.h>
+#include <sys/mman.h>
+#include <unistd.h>
+
+#include <common/common.h>
+#include <common/index/index.h>
+#include <common/kernel-consumer/kernel-consumer.h>
+#include <common/relayd/relayd.h>
+#include <common/ust-consumer/ust-consumer.h>
+#include <common/utils.h>
+
+#include "consumer-stream.h"
+
+/*
+ * RCU call to free stream. MUST only be used with call_rcu().
+ */
+static void free_stream_rcu(struct rcu_head *head)
+{
+       struct lttng_ht_node_u64 *node =
+               caa_container_of(head, struct lttng_ht_node_u64, head);
+       struct lttng_consumer_stream *stream =
+               caa_container_of(node, struct lttng_consumer_stream, node);
+
+       pthread_mutex_destroy(&stream->lock);
+       free(stream);
+}
+
+/*
+ * Close stream on the relayd side. This call can destroy a relayd if the
+ * conditions are met.
+ *
+ * A RCU read side lock MUST be acquired if the relayd object was looked up in
+ * a hash table before calling this.
+ */
+void consumer_stream_relayd_close(struct lttng_consumer_stream *stream,
+               struct consumer_relayd_sock_pair *relayd)
+{
+       int ret;
+
+       assert(stream);
+       assert(relayd);
+
+       if (stream->sent_to_relayd) {
+               uatomic_dec(&relayd->refcount);
+               assert(uatomic_read(&relayd->refcount) >= 0);
+       }
+
+       /* Closing streams requires to lock the control socket. */
+       pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+       ret = relayd_send_close_stream(&relayd->control_sock,
+                       stream->relayd_stream_id,
+                       stream->next_net_seq_num - 1);
+       pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+       if (ret < 0) {
+               DBG("Unable to close stream on the relayd. Continuing");
+               /*
+                * Continue here. There is nothing we can do for the relayd.
+                * Chances are that the relayd has closed the socket so we just
+                * continue cleaning up.
+                */
+       }
+
+       /* Both conditions are met, we destroy the relayd. */
+       if (uatomic_read(&relayd->refcount) == 0 &&
+                       uatomic_read(&relayd->destroy_flag)) {
+               consumer_destroy_relayd(relayd);
+       }
+       stream->net_seq_idx = (uint64_t) -1ULL;
+       stream->sent_to_relayd = 0;
+}
+
+/*
+ * Close stream's file descriptors and, if needed, close stream also on the
+ * relayd side.
+ *
+ * The consumer data lock MUST be acquired.
+ * The stream lock MUST be acquired.
+ */
+void consumer_stream_close(struct lttng_consumer_stream *stream)
+{
+       int ret;
+       struct consumer_relayd_sock_pair *relayd;
+
+       assert(stream);
+
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               if (stream->mmap_base != NULL) {
+                       ret = munmap(stream->mmap_base, stream->mmap_len);
+                       if (ret != 0) {
+                               PERROR("munmap");
+                       }
+               }
+
+               if (stream->wait_fd >= 0) {
+                       ret = close(stream->wait_fd);
+                       if (ret) {
+                               PERROR("close");
+                       }
+                       stream->wait_fd = -1;
+               }
+               if (stream->chan->output == CONSUMER_CHANNEL_SPLICE) {
+                       utils_close_pipe(stream->splice_pipe);
+               }
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+       {
+               /*
+                * Special case for the metadata since the wait fd is an internal pipe
+                * polled in the metadata thread.
+                */
+               if (stream->metadata_flag && stream->chan->monitor) {
+                       int rpipe = stream->ust_metadata_poll_pipe[0];
+
+                       /*
+                        * This will stop the channel timer if one and close the write side
+                        * of the metadata poll pipe.
+                        */
+                       lttng_ustconsumer_close_metadata(stream->chan);
+                       if (rpipe >= 0) {
+                               ret = close(rpipe);
+                               if (ret < 0) {
+                                       PERROR("closing metadata pipe read side");
+                               }
+                               stream->ust_metadata_poll_pipe[0] = -1;
+                       }
+               }
+               break;
+       }
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+       }
+
+       /* Close output fd. Could be a socket or local file at this point. */
+       if (stream->out_fd >= 0) {
+               ret = close(stream->out_fd);
+               if (ret) {
+                       PERROR("close");
+               }
+               stream->out_fd = -1;
+       }
+
+       if (stream->index_fd >= 0) {
+               ret = close(stream->index_fd);
+               if (ret) {
+                       PERROR("close stream index_fd");
+               }
+               stream->index_fd = -1;
+       }
+
+       /* Check and cleanup relayd if needed. */
+       rcu_read_lock();
+       relayd = consumer_find_relayd(stream->net_seq_idx);
+       if (relayd != NULL) {
+               consumer_stream_relayd_close(stream, relayd);
+       }
+       rcu_read_unlock();
+}
+
+/*
+ * Delete the stream from all possible hash tables.
+ *
+ * The consumer data lock MUST be acquired.
+ * The stream lock MUST be acquired.
+ */
+void consumer_stream_delete(struct lttng_consumer_stream *stream,
+               struct lttng_ht *ht)
+{
+       int ret;
+       struct lttng_ht_iter iter;
+
+       assert(stream);
+       /* Should NEVER be called not in monitor mode. */
+       assert(stream->chan->monitor);
+
+       rcu_read_lock();
+
+       if (ht) {
+               iter.iter.node = &stream->node.node;
+               ret = lttng_ht_del(ht, &iter);
+               assert(!ret);
+       }
+
+       /* Delete from stream per channel ID hash table. */
+       iter.iter.node = &stream->node_channel_id.node;
+       /*
+        * The returned value is of no importance. Even if the node is NOT in the
+        * hash table, we continue since we may have been called by a code path
+        * that did not add the stream to a (all) hash table. Same goes for the
+        * next call ht del call.
+        */
+       (void) lttng_ht_del(consumer_data.stream_per_chan_id_ht, &iter);
+
+       /* Delete from the global stream list. */
+       iter.iter.node = &stream->node_session_id.node;
+       /* See the previous ht del on why we ignore the returned value. */
+       (void) lttng_ht_del(consumer_data.stream_list_ht, &iter);
+
+       rcu_read_unlock();
+
+       if (!stream->metadata_flag) {
+               /* Decrement the stream count of the global consumer data. */
+               assert(consumer_data.stream_count > 0);
+               consumer_data.stream_count--;
+       }
+}
+
+/*
+ * Free the given stream within a RCU call.
+ */
+void consumer_stream_free(struct lttng_consumer_stream *stream)
+{
+       assert(stream);
+
+       call_rcu(&stream->node.head, free_stream_rcu);
+}
+
+/*
+ * Destroy the stream's buffers of the tracer.
+ */
+void consumer_stream_destroy_buffers(struct lttng_consumer_stream *stream)
+{
+       assert(stream);
+
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               lttng_ustconsumer_del_stream(stream);
+               break;
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+       }
+}
+
+/*
+ * Destroy and close a already created stream.
+ */
+static void destroy_close_stream(struct lttng_consumer_stream *stream)
+{
+       assert(stream);
+
+       DBG("Consumer stream destroy monitored key: %" PRIu64, stream->key);
+
+       /* Destroy tracer buffers of the stream. */
+       consumer_stream_destroy_buffers(stream);
+       /* Close down everything including the relayd if one. */
+       consumer_stream_close(stream);
+}
+
+/*
+ * Decrement the stream's channel refcount and if down to 0, return the channel
+ * pointer so it can be destroyed by the caller or NULL if not.
+ */
+static struct lttng_consumer_channel *unref_channel(
+               struct lttng_consumer_stream *stream)
+{
+       struct lttng_consumer_channel *free_chan = NULL;
+
+       assert(stream);
+       assert(stream->chan);
+
+       /* Update refcount of channel and see if we need to destroy it. */
+       if (!uatomic_sub_return(&stream->chan->refcount, 1)
+                       && !uatomic_read(&stream->chan->nb_init_stream_left)) {
+               free_chan = stream->chan;
+       }
+
+       return free_chan;
+}
+
+/*
+ * Destroy a stream completely. This will delete, close and free the stream.
+ * Once return, the stream is NO longer usable. Its channel may get destroyed
+ * if conditions are met for a monitored stream.
+ *
+ * This MUST be called WITHOUT the consumer data and stream lock acquired if
+ * the stream is in _monitor_ mode else it does not matter.
+ */
+void consumer_stream_destroy(struct lttng_consumer_stream *stream,
+               struct lttng_ht *ht)
+{
+       assert(stream);
+
+       /* Stream is in monitor mode. */
+       if (stream->monitor) {
+               struct lttng_consumer_channel *free_chan = NULL;
+
+               /*
+                * This means that the stream was successfully removed from the streams
+                * list of the channel and sent to the right thread managing this
+                * stream thus being globally visible.
+                */
+               if (stream->globally_visible) {
+                       pthread_mutex_lock(&consumer_data.lock);
+                       pthread_mutex_lock(&stream->chan->lock);
+                       pthread_mutex_lock(&stream->lock);
+                       /* Remove every reference of the stream in the consumer. */
+                       consumer_stream_delete(stream, ht);
+
+                       destroy_close_stream(stream);
+
+                       /* Update channel's refcount of the stream. */
+                       free_chan = unref_channel(stream);
+
+                       /* Indicates that the consumer data state MUST be updated after this. */
+                       consumer_data.need_update = 1;
+
+                       pthread_mutex_unlock(&stream->lock);
+                       pthread_mutex_unlock(&stream->chan->lock);
+                       pthread_mutex_unlock(&consumer_data.lock);
+               } else {
+                       /*
+                        * If the stream is not visible globally, this needs to be done
+                        * outside of the consumer data lock section.
+                        */
+                       free_chan = unref_channel(stream);
+               }
+
+               if (free_chan) {
+                       consumer_del_channel(free_chan);
+               }
+       } else {
+               destroy_close_stream(stream);
+       }
+
+       /* Free stream within a RCU call. */
+       consumer_stream_free(stream);
+}
+
+/*
+ * Write index of a specific stream either on the relayd or local disk.
+ *
+ * Return 0 on success or else a negative value.
+ */
+int consumer_stream_write_index(struct lttng_consumer_stream *stream,
+               struct ctf_packet_index *index)
+{
+       int ret;
+       struct consumer_relayd_sock_pair *relayd;
+
+       assert(stream);
+       assert(index);
+
+       rcu_read_lock();
+       relayd = consumer_find_relayd(stream->net_seq_idx);
+       if (relayd) {
+               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+               ret = relayd_send_index(&relayd->control_sock, index,
+                               stream->relayd_stream_id, stream->next_net_seq_num - 1);
+               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+       } else {
+               ssize_t size_ret;
+
+               size_ret = index_write(stream->index_fd, index,
+                               sizeof(struct ctf_packet_index));
+               if (size_ret < sizeof(struct ctf_packet_index)) {
+                       ret = -1;
+               } else {
+                       ret = 0;
+               }
+       }
+       if (ret < 0) {
+               goto error;
+       }
+
+error:
+       rcu_read_unlock();
+       return ret;
+}
+
+/*
+ * Actually do the metadata sync using the given metadata stream.
+ *
+ * Return 0 on success else a negative value. ENODATA can be returned also
+ * indicating that there is no metadata available for that stream.
+ */
+static int do_sync_metadata(struct lttng_consumer_stream *metadata,
+               struct lttng_consumer_local_data *ctx)
+{
+       int ret;
+
+       assert(metadata);
+       assert(metadata->metadata_flag);
+       assert(ctx);
+
+       /*
+        * In UST, since we have to write the metadata from the cache packet
+        * by packet, we might need to start this procedure multiple times
+        * until all the metadata from the cache has been extracted.
+        */
+       do {
+               /*
+                * Steps :
+                * - Lock the metadata stream
+                * - Check if metadata stream node was deleted before locking.
+                *   - if yes, release and return success
+                * - Check if new metadata is ready (flush + snapshot pos)
+                * - If nothing : release and return.
+                * - Lock the metadata_rdv_lock
+                * - Unlock the metadata stream
+                * - cond_wait on metadata_rdv to wait the wakeup from the
+                *   metadata thread
+                * - Unlock the metadata_rdv_lock
+                */
+               pthread_mutex_lock(&metadata->lock);
+
+               /*
+                * There is a possibility that we were able to acquire a reference on the
+                * stream from the RCU hash table but between then and now, the node might
+                * have been deleted just before the lock is acquired. Thus, after locking,
+                * we make sure the metadata node has not been deleted which means that the
+                * buffers are closed.
+                *
+                * In that case, there is no need to sync the metadata hence returning a
+                * success return code.
+                */
+               ret = cds_lfht_is_node_deleted(&metadata->node.node);
+               if (ret) {
+                       ret = 0;
+                       goto end_unlock_mutex;
+               }
+
+               switch (ctx->type) {
+               case LTTNG_CONSUMER_KERNEL:
+                       /*
+                        * Empty the metadata cache and flush the current stream.
+                        */
+                       ret = lttng_kconsumer_sync_metadata(metadata);
+                       break;
+               case LTTNG_CONSUMER32_UST:
+               case LTTNG_CONSUMER64_UST:
+                       /*
+                        * Ask the sessiond if we have new metadata waiting and update the
+                        * consumer metadata cache.
+                        */
+                       ret = lttng_ustconsumer_sync_metadata(ctx, metadata);
+                       break;
+               default:
+                       assert(0);
+                       ret = -1;
+                       break;
+               }
+               /*
+                * Error or no new metadata, we exit here.
+                */
+               if (ret <= 0 || ret == ENODATA) {
+                       goto end_unlock_mutex;
+               }
+
+               /*
+                * At this point, new metadata have been flushed, so we wait on the
+                * rendez-vous point for the metadata thread to wake us up when it
+                * finishes consuming the metadata and continue execution.
+                */
+
+               pthread_mutex_lock(&metadata->metadata_rdv_lock);
+
+               /*
+                * Release metadata stream lock so the metadata thread can process it.
+                */
+               pthread_mutex_unlock(&metadata->lock);
+
+               /*
+                * Wait on the rendez-vous point. Once woken up, it means the metadata was
+                * consumed and thus synchronization is achieved.
+                */
+               pthread_cond_wait(&metadata->metadata_rdv, &metadata->metadata_rdv_lock);
+               pthread_mutex_unlock(&metadata->metadata_rdv_lock);
+       } while (ret == EAGAIN);
+
+       /* Success */
+       return 0;
+
+end_unlock_mutex:
+       pthread_mutex_unlock(&metadata->lock);
+       return ret;
+}
+
+/*
+ * Synchronize the metadata using a given session ID. A successful acquisition
+ * of a metadata stream will trigger a request to the session daemon and a
+ * snapshot so the metadata thread can consume it.
+ *
+ * This function call is a rendez-vous point between the metadata thread and
+ * the data thread.
+ *
+ * Return 0 on success or else a negative value.
+ */
+int consumer_stream_sync_metadata(struct lttng_consumer_local_data *ctx,
+               uint64_t session_id)
+{
+       int ret;
+       struct lttng_consumer_stream *stream = NULL;
+       struct lttng_ht_iter iter;
+       struct lttng_ht *ht;
+
+       assert(ctx);
+
+       /* Ease our life a bit. */
+       ht = consumer_data.stream_list_ht;
+
+       rcu_read_lock();
+
+       /* Search the metadata associated with the session id of the given stream. */
+
+       cds_lfht_for_each_entry_duplicate(ht->ht,
+                       ht->hash_fct(&session_id, lttng_ht_seed), ht->match_fct,
+                       &session_id, &iter.iter, stream, node_session_id.node) {
+               if (!stream->metadata_flag) {
+                       continue;
+               }
+
+               ret = do_sync_metadata(stream, ctx);
+               if (ret < 0) {
+                       goto end;
+               }
+       }
+
+       /*
+        * Force return code to 0 (success) since ret might be ENODATA for instance
+        * which is not an error but rather that we should come back.
+        */
+       ret = 0;
+
+end:
+       rcu_read_unlock();
+       return ret;
+}
diff --git a/src/common/consumer/consumer-stream.h b/src/common/consumer/consumer-stream.h
new file mode 100644 (file)
index 0000000..c5fb097
--- /dev/null
@@ -0,0 +1,80 @@
+/*
+ * Copyright (C) 2013 - David Goulet <dgoulet@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License, version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along with
+ * this program; if not, write to the Free Software Foundation, Inc., 51
+ * Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#ifndef LTTNG_CONSUMER_STREAM_H
+#define LTTNG_CONSUMER_STREAM_H
+
+#include "consumer.h"
+
+/*
+ * Close stream's file descriptors and, if needed, close stream also on the
+ * relayd side.
+ *
+ * The stream lock MUST be acquired.
+ * The consumer data lock MUST be acquired.
+ */
+void consumer_stream_close(struct lttng_consumer_stream *stream);
+
+/*
+ * Close stream on the relayd side. This call can destroy a relayd if the
+ * conditions are met.
+ *
+ * A RCU read side lock MUST be acquired if the relayd object was looked up in
+ * a hash table before calling this.
+ */
+void consumer_stream_relayd_close(struct lttng_consumer_stream *stream,
+               struct consumer_relayd_sock_pair *relayd);
+
+/*
+ * Delete the stream from all possible hash tables.
+ *
+ * The consumer data lock MUST be acquired.
+ */
+void consumer_stream_delete(struct lttng_consumer_stream *stream,
+               struct lttng_ht *ht);
+
+/*
+ * Free the given stream within a RCU call.
+ */
+void consumer_stream_free(struct lttng_consumer_stream *stream);
+
+/*
+ * Destroy a stream completely. This will delete, close and free the stream.
+ * Once return, the stream is NO longer usable. Its channel may get destroyed
+ * if conditions are met.
+ *
+ * This MUST be called WITHOUT the consumer data and stream lock acquired.
+ */
+void consumer_stream_destroy(struct lttng_consumer_stream *stream,
+               struct lttng_ht *ht);
+
+/*
+ * Destroy the stream's buffers on the tracer side. This is also called in a
+ * stream destroy.
+ */
+void consumer_stream_destroy_buffers(struct lttng_consumer_stream *stream);
+
+/*
+ * Write index of a specific stream either on the relayd or local disk.
+ */
+int consumer_stream_write_index(struct lttng_consumer_stream *stream,
+               struct ctf_packet_index *index);
+
+int consumer_stream_sync_metadata(struct lttng_consumer_local_data *ctx,
+               uint64_t session_id);
+
+#endif /* LTTNG_CONSUMER_STREAM_H */
diff --git a/src/common/consumer/consumer-testpoint.h b/src/common/consumer/consumer-testpoint.h
new file mode 100644 (file)
index 0000000..e94ce57
--- /dev/null
@@ -0,0 +1,31 @@
+#ifndef CONSUMERD_TESTPOINT_H
+#define CONSUMERD_TESTPOINT_H
+
+/*
+ * Copyright (C) 2012 - Christian Babeux <christian.babeux@efficios.com>
+ * Copyright (C) 2014 - Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License, version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along with
+ * this program; if not, write to the Free Software Foundation, Inc., 51
+ * Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#include <common/testpoint/testpoint.h>
+
+/* Testpoints, internal use only */
+TESTPOINT_DECL(consumerd_thread_channel);
+TESTPOINT_DECL(consumerd_thread_metadata);
+TESTPOINT_DECL(consumerd_thread_data);
+TESTPOINT_DECL(consumerd_thread_sessiond);
+TESTPOINT_DECL(consumerd_thread_metadata_timer);
+
+#endif /* CONSUMERD_TESTPOINT_H */
diff --git a/src/common/consumer/consumer-timer.c b/src/common/consumer/consumer-timer.c
new file mode 100644 (file)
index 0000000..3c9b3fe
--- /dev/null
@@ -0,0 +1,606 @@
+/*
+ * Copyright (C) 2012 - Julien Desfossez <julien.desfossez@efficios.com>
+ *                      David Goulet <dgoulet@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU General Public License, version 2 only, as
+ * published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along with
+ * this program; if not, write to the Free Software Foundation, Inc., 51
+ * Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#define _LGPL_SOURCE
+#include <assert.h>
+#include <inttypes.h>
+#include <signal.h>
+
+#include <bin/lttng-consumerd/health-consumerd.h>
+#include <common/common.h>
+#include <common/compat/endian.h>
+#include <common/kernel-ctl/kernel-ctl.h>
+#include <common/kernel-consumer/kernel-consumer.h>
+#include <common/consumer/consumer-stream.h>
+#include <common/consumer/consumer-timer.h>
+#include <common/consumer/consumer-testpoint.h>
+#include <common/ust-consumer/ust-consumer.h>
+
+static struct timer_signal_data timer_signal = {
+       .tid = 0,
+       .setup_done = 0,
+       .qs_done = 0,
+       .lock = PTHREAD_MUTEX_INITIALIZER,
+};
+
+/*
+ * Set custom signal mask to current thread.
+ */
+static void setmask(sigset_t *mask)
+{
+       int ret;
+
+       ret = sigemptyset(mask);
+       if (ret) {
+               PERROR("sigemptyset");
+       }
+       ret = sigaddset(mask, LTTNG_CONSUMER_SIG_SWITCH);
+       if (ret) {
+               PERROR("sigaddset switch");
+       }
+       ret = sigaddset(mask, LTTNG_CONSUMER_SIG_TEARDOWN);
+       if (ret) {
+               PERROR("sigaddset teardown");
+       }
+       ret = sigaddset(mask, LTTNG_CONSUMER_SIG_LIVE);
+       if (ret) {
+               PERROR("sigaddset live");
+       }
+}
+
+/*
+ * Execute action on a timer switch.
+ *
+ * Beware: metadata_switch_timer() should *never* take a mutex also held
+ * while consumer_timer_switch_stop() is called. It would result in
+ * deadlocks.
+ */
+static void metadata_switch_timer(struct lttng_consumer_local_data *ctx,
+               int sig, siginfo_t *si, void *uc)
+{
+       int ret;
+       struct lttng_consumer_channel *channel;
+
+       channel = si->si_value.sival_ptr;
+       assert(channel);
+
+       if (channel->switch_timer_error) {
+               return;
+       }
+
+       DBG("Switch timer for channel %" PRIu64, channel->key);
+       switch (ctx->type) {
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               /*
+                * Locks taken by lttng_ustconsumer_request_metadata():
+                * - metadata_socket_lock
+                *   - Calling lttng_ustconsumer_recv_metadata():
+                *     - channel->metadata_cache->lock
+                *     - Calling consumer_metadata_cache_flushed():
+                *       - channel->timer_lock
+                *         - channel->metadata_cache->lock
+                *
+                * Ensure that neither consumer_data.lock nor
+                * channel->lock are taken within this function, since
+                * they are held while consumer_timer_switch_stop() is
+                * called.
+                */
+               ret = lttng_ustconsumer_request_metadata(ctx, channel, 1, 1);
+               if (ret < 0) {
+                       channel->switch_timer_error = 1;
+               }
+               break;
+       case LTTNG_CONSUMER_KERNEL:
+       case LTTNG_CONSUMER_UNKNOWN:
+               assert(0);
+               break;
+       }
+}
+
+static int send_empty_index(struct lttng_consumer_stream *stream, uint64_t ts,
+               uint64_t stream_id)
+{
+       int ret;
+       struct ctf_packet_index index;
+
+       memset(&index, 0, sizeof(index));
+       index.stream_id = htobe64(stream_id);
+       index.timestamp_end = htobe64(ts);
+       ret = consumer_stream_write_index(stream, &index);
+       if (ret < 0) {
+               goto error;
+       }
+
+error:
+       return ret;
+}
+
+int consumer_flush_kernel_index(struct lttng_consumer_stream *stream)
+{
+       uint64_t ts, stream_id;
+       int ret;
+
+       ret = kernctl_get_current_timestamp(stream->wait_fd, &ts);
+       if (ret < 0) {
+               ERR("Failed to get the current timestamp");
+               goto end;
+       }
+       ret = kernctl_buffer_flush(stream->wait_fd);
+       if (ret < 0) {
+               ERR("Failed to flush kernel stream");
+               goto end;
+       }
+       ret = kernctl_snapshot(stream->wait_fd);
+       if (ret < 0) {
+               if (errno != EAGAIN && errno != ENODATA) {
+                       PERROR("live timer kernel snapshot");
+                       ret = -1;
+                       goto end;
+               }
+               ret = kernctl_get_stream_id(stream->wait_fd, &stream_id);
+               if (ret < 0) {
+                       PERROR("kernctl_get_stream_id");
+                       goto end;
+               }
+               DBG("Stream %" PRIu64 " empty, sending beacon", stream->key);
+               ret = send_empty_index(stream, ts, stream_id);
+               if (ret < 0) {
+                       goto end;
+               }
+       }
+       ret = 0;
+end:
+       return ret;
+}
+
+static int check_kernel_stream(struct lttng_consumer_stream *stream)
+{
+       int ret;
+
+       /*
+        * While holding the stream mutex, try to take a snapshot, if it
+        * succeeds, it means that data is ready to be sent, just let the data
+        * thread handle that. Otherwise, if the snapshot returns EAGAIN, it
+        * means that there is no data to read after the flush, so we can
+        * safely send the empty index.
+        *
+        * Doing a trylock and checking if waiting on metadata if
+        * trylock fails. Bail out of the stream is indeed waiting for
+        * metadata to be pushed. Busy wait on trylock otherwise.
+        */
+       for (;;) {
+               ret = pthread_mutex_trylock(&stream->lock);
+               switch (ret) {
+               case 0:
+                       break;  /* We have the lock. */
+               case EBUSY:
+                       pthread_mutex_lock(&stream->metadata_timer_lock);
+                       if (stream->waiting_on_metadata) {
+                               ret = 0;
+                               stream->missed_metadata_flush = true;
+                               pthread_mutex_unlock(&stream->metadata_timer_lock);
+                               goto end;       /* Bail out. */
+                       }
+                       pthread_mutex_unlock(&stream->metadata_timer_lock);
+                       /* Try again. */
+                       caa_cpu_relax();
+                       continue;
+               default:
+                       ERR("Unexpected pthread_mutex_trylock error %d", ret);
+                       ret = -1;
+                       goto end;
+               }
+               break;
+       }
+       ret = consumer_flush_kernel_index(stream);
+       pthread_mutex_unlock(&stream->lock);
+end:
+       return ret;
+}
+
+int consumer_flush_ust_index(struct lttng_consumer_stream *stream)
+{
+       uint64_t ts, stream_id;
+       int ret;
+
+       ret = cds_lfht_is_node_deleted(&stream->node.node);
+       if (ret) {
+               goto end;
+       }
+
+       ret = lttng_ustconsumer_get_current_timestamp(stream, &ts);
+       if (ret < 0) {
+               ERR("Failed to get the current timestamp");
+               goto end;
+       }
+       lttng_ustconsumer_flush_buffer(stream, 1);
+       ret = lttng_ustconsumer_take_snapshot(stream);
+       if (ret < 0) {
+               if (ret != -EAGAIN) {
+                       ERR("Taking UST snapshot");
+                       ret = -1;
+                       goto end;
+               }
+               ret = lttng_ustconsumer_get_stream_id(stream, &stream_id);
+               if (ret < 0) {
+                       PERROR("ustctl_get_stream_id");
+                       goto end;
+               }
+               DBG("Stream %" PRIu64 " empty, sending beacon", stream->key);
+               ret = send_empty_index(stream, ts, stream_id);
+               if (ret < 0) {
+                       goto end;
+               }
+       }
+       ret = 0;
+end:
+       return ret;
+}
+
+static int check_ust_stream(struct lttng_consumer_stream *stream)
+{
+       int ret;
+
+       assert(stream);
+       assert(stream->ustream);
+       /*
+        * While holding the stream mutex, try to take a snapshot, if it
+        * succeeds, it means that data is ready to be sent, just let the data
+        * thread handle that. Otherwise, if the snapshot returns EAGAIN, it
+        * means that there is no data to read after the flush, so we can
+        * safely send the empty index.
+        *
+        * Doing a trylock and checking if waiting on metadata if
+        * trylock fails. Bail out of the stream is indeed waiting for
+        * metadata to be pushed. Busy wait on trylock otherwise.
+        */
+       for (;;) {
+               ret = pthread_mutex_trylock(&stream->lock);
+               switch (ret) {
+               case 0:
+                       break;  /* We have the lock. */
+               case EBUSY:
+                       pthread_mutex_lock(&stream->metadata_timer_lock);
+                       if (stream->waiting_on_metadata) {
+                               ret = 0;
+                               stream->missed_metadata_flush = true;
+                               pthread_mutex_unlock(&stream->metadata_timer_lock);
+                               goto end;       /* Bail out. */
+                       }
+                       pthread_mutex_unlock(&stream->metadata_timer_lock);
+                       /* Try again. */
+                       caa_cpu_relax();
+                       continue;
+               default:
+                       ERR("Unexpected pthread_mutex_trylock error %d", ret);
+                       ret = -1;
+                       goto end;
+               }
+               break;
+       }
+       ret = consumer_flush_ust_index(stream);
+       pthread_mutex_unlock(&stream->lock);
+end:
+       return ret;
+}
+
+/*
+ * Execute action on a live timer
+ */
+static void live_timer(struct lttng_consumer_local_data *ctx,
+               int sig, siginfo_t *si, void *uc)
+{
+       int ret;
+       struct lttng_consumer_channel *channel;
+       struct lttng_consumer_stream *stream;
+       struct lttng_ht *ht;
+       struct lttng_ht_iter iter;
+
+       channel = si->si_value.sival_ptr;
+       assert(channel);
+
+       if (channel->switch_timer_error) {
+               goto error;
+       }
+       ht = consumer_data.stream_per_chan_id_ht;
+
+       DBG("Live timer for channel %" PRIu64, channel->key);
+
+       rcu_read_lock();
+       switch (ctx->type) {
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               cds_lfht_for_each_entry_duplicate(ht->ht,
+                               ht->hash_fct(&channel->key, lttng_ht_seed),
+                               ht->match_fct, &channel->key, &iter.iter,
+                               stream, node_channel_id.node) {
+                       ret = check_ust_stream(stream);
+                       if (ret < 0) {
+                               goto error_unlock;
+                       }
+               }
+               break;
+       case LTTNG_CONSUMER_KERNEL:
+               cds_lfht_for_each_entry_duplicate(ht->ht,
+                               ht->hash_fct(&channel->key, lttng_ht_seed),
+                               ht->match_fct, &channel->key, &iter.iter,
+                               stream, node_channel_id.node) {
+                       ret = check_kernel_stream(stream);
+                       if (ret < 0) {
+                               goto error_unlock;
+                       }
+               }
+               break;
+       case LTTNG_CONSUMER_UNKNOWN:
+               assert(0);
+               break;
+       }
+
+error_unlock:
+       rcu_read_unlock();
+
+error:
+       return;
+}
+
+static
+void consumer_timer_signal_thread_qs(unsigned int signr)
+{
+       sigset_t pending_set;
+       int ret;
+
+       /*
+        * We need to be the only thread interacting with the thread
+        * that manages signals for teardown synchronization.
+        */
+       pthread_mutex_lock(&timer_signal.lock);
+
+       /* Ensure we don't have any signal queued for this channel. */
+       for (;;) {
+               ret = sigemptyset(&pending_set);
+               if (ret == -1) {
+                       PERROR("sigemptyset");
+               }
+               ret = sigpending(&pending_set);
+               if (ret == -1) {
+                       PERROR("sigpending");
+               }
+               if (!sigismember(&pending_set, LTTNG_CONSUMER_SIG_SWITCH)) {
+                       break;
+               }
+               caa_cpu_relax();
+       }
+
+       /*
+        * From this point, no new signal handler will be fired that would try to
+        * access "chan". However, we still need to wait for any currently
+        * executing handler to complete.
+        */
+       cmm_smp_mb();
+       CMM_STORE_SHARED(timer_signal.qs_done, 0);
+       cmm_smp_mb();
+
+       /*
+        * Kill with LTTNG_CONSUMER_SIG_TEARDOWN, so signal management thread wakes
+        * up.
+        */
+       kill(getpid(), LTTNG_CONSUMER_SIG_TEARDOWN);
+
+       while (!CMM_LOAD_SHARED(timer_signal.qs_done)) {
+               caa_cpu_relax();
+       }
+       cmm_smp_mb();
+
+       pthread_mutex_unlock(&timer_signal.lock);
+}
+
+/*
+ * Set the timer for periodical metadata flush.
+ */
+void consumer_timer_switch_start(struct lttng_consumer_channel *channel,
+               unsigned int switch_timer_interval)
+{
+       int ret;
+       struct sigevent sev;
+       struct itimerspec its;
+
+       assert(channel);
+       assert(channel->key);
+
+       if (switch_timer_interval == 0) {
+               return;
+       }
+
+       sev.sigev_notify = SIGEV_SIGNAL;
+       sev.sigev_signo = LTTNG_CONSUMER_SIG_SWITCH;
+       sev.sigev_value.sival_ptr = channel;
+       ret = timer_create(CLOCKID, &sev, &channel->switch_timer);
+       if (ret == -1) {
+               PERROR("timer_create");
+       }
+       channel->switch_timer_enabled = 1;
+
+       its.it_value.tv_sec = switch_timer_interval / 1000000;
+       its.it_value.tv_nsec = switch_timer_interval % 1000000;
+       its.it_interval.tv_sec = its.it_value.tv_sec;
+       its.it_interval.tv_nsec = its.it_value.tv_nsec;
+
+       ret = timer_settime(channel->switch_timer, 0, &its, NULL);
+       if (ret == -1) {
+               PERROR("timer_settime");
+       }
+}
+
+/*
+ * Stop and delete timer.
+ */
+void consumer_timer_switch_stop(struct lttng_consumer_channel *channel)
+{
+       int ret;
+
+       assert(channel);
+
+       ret = timer_delete(channel->switch_timer);
+       if (ret == -1) {
+               PERROR("timer_delete");
+       }
+
+       consumer_timer_signal_thread_qs(LTTNG_CONSUMER_SIG_SWITCH);
+
+       channel->switch_timer = 0;
+       channel->switch_timer_enabled = 0;
+}
+
+/*
+ * Set the timer for the live mode.
+ */
+void consumer_timer_live_start(struct lttng_consumer_channel *channel,
+               int live_timer_interval)
+{
+       int ret;
+       struct sigevent sev;
+       struct itimerspec its;
+
+       assert(channel);
+       assert(channel->key);
+
+       if (live_timer_interval <= 0) {
+               return;
+       }
+
+       sev.sigev_notify = SIGEV_SIGNAL;
+       sev.sigev_signo = LTTNG_CONSUMER_SIG_LIVE;
+       sev.sigev_value.sival_ptr = channel;
+       ret = timer_create(CLOCKID, &sev, &channel->live_timer);
+       if (ret == -1) {
+               PERROR("timer_create");
+       }
+       channel->live_timer_enabled = 1;
+
+       its.it_value.tv_sec = live_timer_interval / 1000000;
+       its.it_value.tv_nsec = live_timer_interval % 1000000;
+       its.it_interval.tv_sec = its.it_value.tv_sec;
+       its.it_interval.tv_nsec = its.it_value.tv_nsec;
+
+       ret = timer_settime(channel->live_timer, 0, &its, NULL);
+       if (ret == -1) {
+               PERROR("timer_settime");
+       }
+}
+
+/*
+ * Stop and delete timer.
+ */
+void consumer_timer_live_stop(struct lttng_consumer_channel *channel)
+{
+       int ret;
+
+       assert(channel);
+
+       ret = timer_delete(channel->live_timer);
+       if (ret == -1) {
+               PERROR("timer_delete");
+       }
+
+       consumer_timer_signal_thread_qs(LTTNG_CONSUMER_SIG_LIVE);
+
+       channel->live_timer = 0;
+       channel->live_timer_enabled = 0;
+}
+
+/*
+ * Block the RT signals for the entire process. It must be called from the
+ * consumer main before creating the threads
+ */
+int consumer_signal_init(void)
+{
+       int ret;
+       sigset_t mask;
+
+       /* Block signal for entire process, so only our thread processes it. */
+       setmask(&mask);
+       ret = pthread_sigmask(SIG_BLOCK, &mask, NULL);
+       if (ret) {
+               errno = ret;
+               PERROR("pthread_sigmask");
+               return -1;
+       }
+       return 0;
+}
+
+/*
+ * This thread is the sighandler for signals LTTNG_CONSUMER_SIG_SWITCH,
+ * LTTNG_CONSUMER_SIG_TEARDOWN and LTTNG_CONSUMER_SIG_LIVE.
+ */
+void *consumer_timer_thread(void *data)
+{
+       int signr;
+       sigset_t mask;
+       siginfo_t info;
+       struct lttng_consumer_local_data *ctx = data;
+
+       rcu_register_thread();
+
+       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_METADATA_TIMER);
+
+       if (testpoint(consumerd_thread_metadata_timer)) {
+               goto error_testpoint;
+       }
+
+       health_code_update();
+
+       /* Only self thread will receive signal mask. */
+       setmask(&mask);
+       CMM_STORE_SHARED(timer_signal.tid, pthread_self());
+
+       while (1) {
+               health_code_update();
+
+               health_poll_entry();
+               signr = sigwaitinfo(&mask, &info);
+               health_poll_exit();
+               if (signr == -1) {
+                       if (errno != EINTR) {
+                               PERROR("sigwaitinfo");
+                       }
+                       continue;
+               } else if (signr == LTTNG_CONSUMER_SIG_SWITCH) {
+                       metadata_switch_timer(ctx, info.si_signo, &info, NULL);
+               } else if (signr == LTTNG_CONSUMER_SIG_TEARDOWN) {
+                       cmm_smp_mb();
+                       CMM_STORE_SHARED(timer_signal.qs_done, 1);
+                       cmm_smp_mb();
+                       DBG("Signal timer metadata thread teardown");
+               } else if (signr == LTTNG_CONSUMER_SIG_LIVE) {
+                       live_timer(ctx, info.si_signo, &info, NULL);
+               } else {
+                       ERR("Unexpected signal %d\n", info.si_signo);
+               }
+       }
+
+error_testpoint:
+       /* Only reached in testpoint error */
+       health_error();
+       health_unregister(health_consumerd);
+
+       rcu_unregister_thread();
+
+       /* Never return */
+       return NULL;
+}
diff --git a/src/common/consumer/consumer-timer.h b/src/common/consumer/consumer-timer.h
new file mode 100644 (file)
index 0000000..22e7457
--- /dev/null
@@ -0,0 +1,58 @@
+/*
+ * Copyright (C) 2011 - Julien Desfossez <julien.desfossez@polymtl.ca>
+ *                      Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
+ *               2012 - David Goulet <dgoulet@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License, version 2 only,
+ * as published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along
+ * with this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#ifndef CONSUMER_TIMER_H
+#define CONSUMER_TIMER_H
+
+#include <pthread.h>
+
+#include "consumer.h"
+
+#define LTTNG_CONSUMER_SIG_SWITCH      SIGRTMIN + 10
+#define LTTNG_CONSUMER_SIG_TEARDOWN    SIGRTMIN + 11
+#define LTTNG_CONSUMER_SIG_LIVE                SIGRTMIN + 12
+
+#define CLOCKID CLOCK_MONOTONIC
+
+/*
+ * Handle timer teardown race wrt memory free of private data by consumer
+ * signals are handled by a single thread, which permits a synchronization
+ * point between handling of each signal. Internal lock ensures mutual
+ * exclusion.
+ */
+struct timer_signal_data {
+       pthread_t tid;  /* thread id managing signals */
+       int setup_done;
+       int qs_done;
+       pthread_mutex_t lock;
+};
+
+void consumer_timer_switch_start(struct lttng_consumer_channel *channel,
+               unsigned int switch_timer_interval);
+void consumer_timer_switch_stop(struct lttng_consumer_channel *channel);
+void consumer_timer_live_start(struct lttng_consumer_channel *channel,
+               int live_timer_interval);
+void consumer_timer_live_stop(struct lttng_consumer_channel *channel);
+void *consumer_timer_thread(void *data);
+int consumer_signal_init(void);
+
+int consumer_flush_kernel_index(struct lttng_consumer_stream *stream);
+int consumer_flush_ust_index(struct lttng_consumer_stream *stream);
+
+#endif /* CONSUMER_TIMER_H */
diff --git a/src/common/consumer/consumer.c b/src/common/consumer/consumer.c
new file mode 100644 (file)
index 0000000..2897fb8
--- /dev/null
@@ -0,0 +1,3712 @@
+/*
+ * Copyright (C) 2011 - Julien Desfossez <julien.desfossez@polymtl.ca>
+ *                      Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
+ *               2012 - David Goulet <dgoulet@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License, version 2 only,
+ * as published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along
+ * with this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#define _LGPL_SOURCE
+#include <assert.h>
+#include <poll.h>
+#include <pthread.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/mman.h>
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <unistd.h>
+#include <inttypes.h>
+#include <signal.h>
+
+#include <bin/lttng-consumerd/health-consumerd.h>
+#include <common/common.h>
+#include <common/utils.h>
+#include <common/compat/poll.h>
+#include <common/compat/endian.h>
+#include <common/index/index.h>
+#include <common/kernel-ctl/kernel-ctl.h>
+#include <common/sessiond-comm/relayd.h>
+#include <common/sessiond-comm/sessiond-comm.h>
+#include <common/kernel-consumer/kernel-consumer.h>
+#include <common/relayd/relayd.h>
+#include <common/ust-consumer/ust-consumer.h>
+#include <common/consumer/consumer-timer.h>
+#include <common/consumer/consumer.h>
+#include <common/consumer/consumer-stream.h>
+#include <common/consumer/consumer-testpoint.h>
+#include <common/align.h>
+
+struct lttng_consumer_global_data consumer_data = {
+       .stream_count = 0,
+       .need_update = 1,
+       .type = LTTNG_CONSUMER_UNKNOWN,
+};
+
+enum consumer_channel_action {
+       CONSUMER_CHANNEL_ADD,
+       CONSUMER_CHANNEL_DEL,
+       CONSUMER_CHANNEL_QUIT,
+};
+
+struct consumer_channel_msg {
+       enum consumer_channel_action action;
+       struct lttng_consumer_channel *chan;    /* add */
+       uint64_t key;                           /* del */
+};
+
+/*
+ * Flag to inform the polling thread to quit when all fd hung up. Updated by
+ * the consumer_thread_receive_fds when it notices that all fds has hung up.
+ * Also updated by the signal handler (consumer_should_exit()). Read by the
+ * polling threads.
+ */
+volatile int consumer_quit;
+
+/*
+ * Global hash table containing respectively metadata and data streams. The
+ * stream element in this ht should only be updated by the metadata poll thread
+ * for the metadata and the data poll thread for the data.
+ */
+static struct lttng_ht *metadata_ht;
+static struct lttng_ht *data_ht;
+
+/*
+ * Notify a thread lttng pipe to poll back again. This usually means that some
+ * global state has changed so we just send back the thread in a poll wait
+ * call.
+ */
+static void notify_thread_lttng_pipe(struct lttng_pipe *pipe)
+{
+       struct lttng_consumer_stream *null_stream = NULL;
+
+       assert(pipe);
+
+       (void) lttng_pipe_write(pipe, &null_stream, sizeof(null_stream));
+}
+
+static void notify_health_quit_pipe(int *pipe)
+{
+       ssize_t ret;
+
+       ret = lttng_write(pipe[1], "4", 1);
+       if (ret < 1) {
+               PERROR("write consumer health quit");
+       }
+}
+
+static void notify_channel_pipe(struct lttng_consumer_local_data *ctx,
+               struct lttng_consumer_channel *chan,
+               uint64_t key,
+               enum consumer_channel_action action)
+{
+       struct consumer_channel_msg msg;
+       ssize_t ret;
+
+       memset(&msg, 0, sizeof(msg));
+
+       msg.action = action;
+       msg.chan = chan;
+       msg.key = key;
+       ret = lttng_write(ctx->consumer_channel_pipe[1], &msg, sizeof(msg));
+       if (ret < sizeof(msg)) {
+               PERROR("notify_channel_pipe write error");
+       }
+}
+
+void notify_thread_del_channel(struct lttng_consumer_local_data *ctx,
+               uint64_t key)
+{
+       notify_channel_pipe(ctx, NULL, key, CONSUMER_CHANNEL_DEL);
+}
+
+static int read_channel_pipe(struct lttng_consumer_local_data *ctx,
+               struct lttng_consumer_channel **chan,
+               uint64_t *key,
+               enum consumer_channel_action *action)
+{
+       struct consumer_channel_msg msg;
+       ssize_t ret;
+
+       ret = lttng_read(ctx->consumer_channel_pipe[0], &msg, sizeof(msg));
+       if (ret < sizeof(msg)) {
+               ret = -1;
+               goto error;
+       }
+       *action = msg.action;
+       *chan = msg.chan;
+       *key = msg.key;
+error:
+       return (int) ret;
+}
+
+/*
+ * Cleanup the stream list of a channel. Those streams are not yet globally
+ * visible
+ */
+static void clean_channel_stream_list(struct lttng_consumer_channel *channel)
+{
+       struct lttng_consumer_stream *stream, *stmp;
+
+       assert(channel);
+
+       /* Delete streams that might have been left in the stream list. */
+       cds_list_for_each_entry_safe(stream, stmp, &channel->streams.head,
+                       send_node) {
+               cds_list_del(&stream->send_node);
+               /*
+                * Once a stream is added to this list, the buffers were created so we
+                * have a guarantee that this call will succeed. Setting the monitor
+                * mode to 0 so we don't lock nor try to delete the stream from the
+                * global hash table.
+                */
+               stream->monitor = 0;
+               consumer_stream_destroy(stream, NULL);
+       }
+}
+
+/*
+ * Find a stream. The consumer_data.lock must be locked during this
+ * call.
+ */
+static struct lttng_consumer_stream *find_stream(uint64_t key,
+               struct lttng_ht *ht)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_ht_node_u64 *node;
+       struct lttng_consumer_stream *stream = NULL;
+
+       assert(ht);
+
+       /* -1ULL keys are lookup failures */
+       if (key == (uint64_t) -1ULL) {
+               return NULL;
+       }
+
+       rcu_read_lock();
+
+       lttng_ht_lookup(ht, &key, &iter);
+       node = lttng_ht_iter_get_node_u64(&iter);
+       if (node != NULL) {
+               stream = caa_container_of(node, struct lttng_consumer_stream, node);
+       }
+
+       rcu_read_unlock();
+
+       return stream;
+}
+
+static void steal_stream_key(uint64_t key, struct lttng_ht *ht)
+{
+       struct lttng_consumer_stream *stream;
+
+       rcu_read_lock();
+       stream = find_stream(key, ht);
+       if (stream) {
+               stream->key = (uint64_t) -1ULL;
+               /*
+                * We don't want the lookup to match, but we still need
+                * to iterate on this stream when iterating over the hash table. Just
+                * change the node key.
+                */
+               stream->node.key = (uint64_t) -1ULL;
+       }
+       rcu_read_unlock();
+}
+
+/*
+ * Return a channel object for the given key.
+ *
+ * RCU read side lock MUST be acquired before calling this function and
+ * protects the channel ptr.
+ */
+struct lttng_consumer_channel *consumer_find_channel(uint64_t key)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_ht_node_u64 *node;
+       struct lttng_consumer_channel *channel = NULL;
+
+       /* -1ULL keys are lookup failures */
+       if (key == (uint64_t) -1ULL) {
+               return NULL;
+       }
+
+       lttng_ht_lookup(consumer_data.channel_ht, &key, &iter);
+       node = lttng_ht_iter_get_node_u64(&iter);
+       if (node != NULL) {
+               channel = caa_container_of(node, struct lttng_consumer_channel, node);
+       }
+
+       return channel;
+}
+
+/*
+ * There is a possibility that the consumer does not have enough time between
+ * the close of the channel on the session daemon and the cleanup in here thus
+ * once we have a channel add with an existing key, we know for sure that this
+ * channel will eventually get cleaned up by all streams being closed.
+ *
+ * This function just nullifies the already existing channel key.
+ */
+static void steal_channel_key(uint64_t key)
+{
+       struct lttng_consumer_channel *channel;
+
+       rcu_read_lock();
+       channel = consumer_find_channel(key);
+       if (channel) {
+               channel->key = (uint64_t) -1ULL;
+               /*
+                * We don't want the lookup to match, but we still need to iterate on
+                * this channel when iterating over the hash table. Just change the
+                * node key.
+                */
+               channel->node.key = (uint64_t) -1ULL;
+       }
+       rcu_read_unlock();
+}
+
+static void free_channel_rcu(struct rcu_head *head)
+{
+       struct lttng_ht_node_u64 *node =
+               caa_container_of(head, struct lttng_ht_node_u64, head);
+       struct lttng_consumer_channel *channel =
+               caa_container_of(node, struct lttng_consumer_channel, node);
+
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               lttng_ustconsumer_free_channel(channel);
+               break;
+       default:
+               ERR("Unknown consumer_data type");
+               abort();
+       }
+       free(channel);
+}
+
+/*
+ * RCU protected relayd socket pair free.
+ */
+static void free_relayd_rcu(struct rcu_head *head)
+{
+       struct lttng_ht_node_u64 *node =
+               caa_container_of(head, struct lttng_ht_node_u64, head);
+       struct consumer_relayd_sock_pair *relayd =
+               caa_container_of(node, struct consumer_relayd_sock_pair, node);
+
+       /*
+        * Close all sockets. This is done in the call RCU since we don't want the
+        * socket fds to be reassigned thus potentially creating bad state of the
+        * relayd object.
+        *
+        * We do not have to lock the control socket mutex here since at this stage
+        * there is no one referencing to this relayd object.
+        */
+       (void) relayd_close(&relayd->control_sock);
+       (void) relayd_close(&relayd->data_sock);
+
+       free(relayd);
+}
+
+/*
+ * Destroy and free relayd socket pair object.
+ */
+void consumer_destroy_relayd(struct consumer_relayd_sock_pair *relayd)
+{
+       int ret;
+       struct lttng_ht_iter iter;
+
+       if (relayd == NULL) {
+               return;
+       }
+
+       DBG("Consumer destroy and close relayd socket pair");
+
+       iter.iter.node = &relayd->node.node;
+       ret = lttng_ht_del(consumer_data.relayd_ht, &iter);
+       if (ret != 0) {
+               /* We assume the relayd is being or is destroyed */
+               return;
+       }
+
+       /* RCU free() call */
+       call_rcu(&relayd->node.head, free_relayd_rcu);
+}
+
+/*
+ * Remove a channel from the global list protected by a mutex. This function is
+ * also responsible for freeing its data structures.
+ */
+void consumer_del_channel(struct lttng_consumer_channel *channel)
+{
+       int ret;
+       struct lttng_ht_iter iter;
+
+       DBG("Consumer delete channel key %" PRIu64, channel->key);
+
+       pthread_mutex_lock(&consumer_data.lock);
+       pthread_mutex_lock(&channel->lock);
+
+       /* Destroy streams that might have been left in the stream list. */
+       clean_channel_stream_list(channel);
+
+       if (channel->live_timer_enabled == 1) {
+               consumer_timer_live_stop(channel);
+       }
+
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               lttng_ustconsumer_del_channel(channel);
+               break;
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+               goto end;
+       }
+
+       rcu_read_lock();
+       iter.iter.node = &channel->node.node;
+       ret = lttng_ht_del(consumer_data.channel_ht, &iter);
+       assert(!ret);
+       rcu_read_unlock();
+
+       call_rcu(&channel->node.head, free_channel_rcu);
+end:
+       pthread_mutex_unlock(&channel->lock);
+       pthread_mutex_unlock(&consumer_data.lock);
+}
+
+/*
+ * Iterate over the relayd hash table and destroy each element. Finally,
+ * destroy the whole hash table.
+ */
+static void cleanup_relayd_ht(void)
+{
+       struct lttng_ht_iter iter;
+       struct consumer_relayd_sock_pair *relayd;
+
+       rcu_read_lock();
+
+       cds_lfht_for_each_entry(consumer_data.relayd_ht->ht, &iter.iter, relayd,
+                       node.node) {
+               consumer_destroy_relayd(relayd);
+       }
+
+       rcu_read_unlock();
+
+       lttng_ht_destroy(consumer_data.relayd_ht);
+}
+
+/*
+ * Update the end point status of all streams having the given network sequence
+ * index (relayd index).
+ *
+ * It's atomically set without having the stream mutex locked which is fine
+ * because we handle the write/read race with a pipe wakeup for each thread.
+ */
+static void update_endpoint_status_by_netidx(uint64_t net_seq_idx,
+               enum consumer_endpoint_status status)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_consumer_stream *stream;
+
+       DBG("Consumer set delete flag on stream by idx %" PRIu64, net_seq_idx);
+
+       rcu_read_lock();
+
+       /* Let's begin with metadata */
+       cds_lfht_for_each_entry(metadata_ht->ht, &iter.iter, stream, node.node) {
+               if (stream->net_seq_idx == net_seq_idx) {
+                       uatomic_set(&stream->endpoint_status, status);
+                       DBG("Delete flag set to metadata stream %d", stream->wait_fd);
+               }
+       }
+
+       /* Follow up by the data streams */
+       cds_lfht_for_each_entry(data_ht->ht, &iter.iter, stream, node.node) {
+               if (stream->net_seq_idx == net_seq_idx) {
+                       uatomic_set(&stream->endpoint_status, status);
+                       DBG("Delete flag set to data stream %d", stream->wait_fd);
+               }
+       }
+       rcu_read_unlock();
+}
+
+/*
+ * Cleanup a relayd object by flagging every associated streams for deletion,
+ * destroying the object meaning removing it from the relayd hash table,
+ * closing the sockets and freeing the memory in a RCU call.
+ *
+ * If a local data context is available, notify the threads that the streams'
+ * state have changed.
+ */
+static void cleanup_relayd(struct consumer_relayd_sock_pair *relayd,
+               struct lttng_consumer_local_data *ctx)
+{
+       uint64_t netidx;
+
+       assert(relayd);
+
+       DBG("Cleaning up relayd sockets");
+
+       /* Save the net sequence index before destroying the object */
+       netidx = relayd->net_seq_idx;
+
+       /*
+        * Delete the relayd from the relayd hash table, close the sockets and free
+        * the object in a RCU call.
+        */
+       consumer_destroy_relayd(relayd);
+
+       /* Set inactive endpoint to all streams */
+       update_endpoint_status_by_netidx(netidx, CONSUMER_ENDPOINT_INACTIVE);
+
+       /*
+        * With a local data context, notify the threads that the streams' state
+        * have changed. The write() action on the pipe acts as an "implicit"
+        * memory barrier ordering the updates of the end point status from the
+        * read of this status which happens AFTER receiving this notify.
+        */
+       if (ctx) {
+               notify_thread_lttng_pipe(ctx->consumer_data_pipe);
+               notify_thread_lttng_pipe(ctx->consumer_metadata_pipe);
+       }
+}
+
+/*
+ * Flag a relayd socket pair for destruction. Destroy it if the refcount
+ * reaches zero.
+ *
+ * RCU read side lock MUST be aquired before calling this function.
+ */
+void consumer_flag_relayd_for_destroy(struct consumer_relayd_sock_pair *relayd)
+{
+       assert(relayd);
+
+       /* Set destroy flag for this object */
+       uatomic_set(&relayd->destroy_flag, 1);
+
+       /* Destroy the relayd if refcount is 0 */
+       if (uatomic_read(&relayd->refcount) == 0) {
+               consumer_destroy_relayd(relayd);
+       }
+}
+
+/*
+ * Completly destroy stream from every visiable data structure and the given
+ * hash table if one.
+ *
+ * One this call returns, the stream object is not longer usable nor visible.
+ */
+void consumer_del_stream(struct lttng_consumer_stream *stream,
+               struct lttng_ht *ht)
+{
+       consumer_stream_destroy(stream, ht);
+}
+
+/*
+ * XXX naming of del vs destroy is all mixed up.
+ */
+void consumer_del_stream_for_data(struct lttng_consumer_stream *stream)
+{
+       consumer_stream_destroy(stream, data_ht);
+}
+
+void consumer_del_stream_for_metadata(struct lttng_consumer_stream *stream)
+{
+       consumer_stream_destroy(stream, metadata_ht);
+}
+
+struct lttng_consumer_stream *consumer_allocate_stream(uint64_t channel_key,
+               uint64_t stream_key,
+               enum lttng_consumer_stream_state state,
+               const char *channel_name,
+               uid_t uid,
+               gid_t gid,
+               uint64_t relayd_id,
+               uint64_t session_id,
+               int cpu,
+               int *alloc_ret,
+               enum consumer_channel_type type,
+               unsigned int monitor)
+{
+       int ret;
+       struct lttng_consumer_stream *stream;
+
+       stream = zmalloc(sizeof(*stream));
+       if (stream == NULL) {
+               PERROR("malloc struct lttng_consumer_stream");
+               ret = -ENOMEM;
+               goto end;
+       }
+
+       rcu_read_lock();
+
+       stream->key = stream_key;
+       stream->out_fd = -1;
+       stream->out_fd_offset = 0;
+       stream->output_written = 0;
+       stream->state = state;
+       stream->uid = uid;
+       stream->gid = gid;
+       stream->net_seq_idx = relayd_id;
+       stream->session_id = session_id;
+       stream->monitor = monitor;
+       stream->endpoint_status = CONSUMER_ENDPOINT_ACTIVE;
+       stream->index_fd = -1;
+       pthread_mutex_init(&stream->lock, NULL);
+       pthread_mutex_init(&stream->metadata_timer_lock, NULL);
+
+       /* If channel is the metadata, flag this stream as metadata. */
+       if (type == CONSUMER_CHANNEL_TYPE_METADATA) {
+               stream->metadata_flag = 1;
+               /* Metadata is flat out. */
+               strncpy(stream->name, DEFAULT_METADATA_NAME, sizeof(stream->name));
+               /* Live rendez-vous point. */
+               pthread_cond_init(&stream->metadata_rdv, NULL);
+               pthread_mutex_init(&stream->metadata_rdv_lock, NULL);
+       } else {
+               /* Format stream name to <channel_name>_<cpu_number> */
+               ret = snprintf(stream->name, sizeof(stream->name), "%s_%d",
+                               channel_name, cpu);
+               if (ret < 0) {
+                       PERROR("snprintf stream name");
+                       goto error;
+               }
+       }
+
+       /* Key is always the wait_fd for streams. */
+       lttng_ht_node_init_u64(&stream->node, stream->key);
+
+       /* Init node per channel id key */
+       lttng_ht_node_init_u64(&stream->node_channel_id, channel_key);
+
+       /* Init session id node with the stream session id */
+       lttng_ht_node_init_u64(&stream->node_session_id, stream->session_id);
+
+       DBG3("Allocated stream %s (key %" PRIu64 ", chan_key %" PRIu64
+                       " relayd_id %" PRIu64 ", session_id %" PRIu64,
+                       stream->name, stream->key, channel_key,
+                       stream->net_seq_idx, stream->session_id);
+
+       rcu_read_unlock();
+       return stream;
+
+error:
+       rcu_read_unlock();
+       free(stream);
+end:
+       if (alloc_ret) {
+               *alloc_ret = ret;
+       }
+       return NULL;
+}
+
+/*
+ * Add a stream to the global list protected by a mutex.
+ */
+int consumer_add_data_stream(struct lttng_consumer_stream *stream)
+{
+       struct lttng_ht *ht = data_ht;
+       int ret = 0;
+
+       assert(stream);
+       assert(ht);
+
+       DBG3("Adding consumer stream %" PRIu64, stream->key);
+
+       pthread_mutex_lock(&consumer_data.lock);
+       pthread_mutex_lock(&stream->chan->lock);
+       pthread_mutex_lock(&stream->chan->timer_lock);
+       pthread_mutex_lock(&stream->lock);
+       rcu_read_lock();
+
+       /* Steal stream identifier to avoid having streams with the same key */
+       steal_stream_key(stream->key, ht);
+
+       lttng_ht_add_unique_u64(ht, &stream->node);
+
+       lttng_ht_add_u64(consumer_data.stream_per_chan_id_ht,
+                       &stream->node_channel_id);
+
+       /*
+        * Add stream to the stream_list_ht of the consumer data. No need to steal
+        * the key since the HT does not use it and we allow to add redundant keys
+        * into this table.
+        */
+       lttng_ht_add_u64(consumer_data.stream_list_ht, &stream->node_session_id);
+
+       /*
+        * When nb_init_stream_left reaches 0, we don't need to trigger any action
+        * in terms of destroying the associated channel, because the action that
+        * causes the count to become 0 also causes a stream to be added. The
+        * channel deletion will thus be triggered by the following removal of this
+        * stream.
+        */
+       if (uatomic_read(&stream->chan->nb_init_stream_left) > 0) {
+               /* Increment refcount before decrementing nb_init_stream_left */
+               cmm_smp_wmb();
+               uatomic_dec(&stream->chan->nb_init_stream_left);
+       }
+
+       /* Update consumer data once the node is inserted. */
+       consumer_data.stream_count++;
+       consumer_data.need_update = 1;
+
+       rcu_read_unlock();
+       pthread_mutex_unlock(&stream->lock);
+       pthread_mutex_unlock(&stream->chan->timer_lock);
+       pthread_mutex_unlock(&stream->chan->lock);
+       pthread_mutex_unlock(&consumer_data.lock);
+
+       return ret;
+}
+
+void consumer_del_data_stream(struct lttng_consumer_stream *stream)
+{
+       consumer_del_stream(stream, data_ht);
+}
+
+/*
+ * Add relayd socket to global consumer data hashtable. RCU read side lock MUST
+ * be acquired before calling this.
+ */
+static int add_relayd(struct consumer_relayd_sock_pair *relayd)
+{
+       int ret = 0;
+       struct lttng_ht_node_u64 *node;
+       struct lttng_ht_iter iter;
+
+       assert(relayd);
+
+       lttng_ht_lookup(consumer_data.relayd_ht,
+                       &relayd->net_seq_idx, &iter);
+       node = lttng_ht_iter_get_node_u64(&iter);
+       if (node != NULL) {
+               goto end;
+       }
+       lttng_ht_add_unique_u64(consumer_data.relayd_ht, &relayd->node);
+
+end:
+       return ret;
+}
+
+/*
+ * Allocate and return a consumer relayd socket.
+ */
+static struct consumer_relayd_sock_pair *consumer_allocate_relayd_sock_pair(
+               uint64_t net_seq_idx)
+{
+       struct consumer_relayd_sock_pair *obj = NULL;
+
+       /* net sequence index of -1 is a failure */
+       if (net_seq_idx == (uint64_t) -1ULL) {
+               goto error;
+       }
+
+       obj = zmalloc(sizeof(struct consumer_relayd_sock_pair));
+       if (obj == NULL) {
+               PERROR("zmalloc relayd sock");
+               goto error;
+       }
+
+       obj->net_seq_idx = net_seq_idx;
+       obj->refcount = 0;
+       obj->destroy_flag = 0;
+       obj->control_sock.sock.fd = -1;
+       obj->data_sock.sock.fd = -1;
+       lttng_ht_node_init_u64(&obj->node, obj->net_seq_idx);
+       pthread_mutex_init(&obj->ctrl_sock_mutex, NULL);
+
+error:
+       return obj;
+}
+
+/*
+ * Find a relayd socket pair in the global consumer data.
+ *
+ * Return the object if found else NULL.
+ * RCU read-side lock must be held across this call and while using the
+ * returned object.
+ */
+struct consumer_relayd_sock_pair *consumer_find_relayd(uint64_t key)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_ht_node_u64 *node;
+       struct consumer_relayd_sock_pair *relayd = NULL;
+
+       /* Negative keys are lookup failures */
+       if (key == (uint64_t) -1ULL) {
+               goto error;
+       }
+
+       lttng_ht_lookup(consumer_data.relayd_ht, &key,
+                       &iter);
+       node = lttng_ht_iter_get_node_u64(&iter);
+       if (node != NULL) {
+               relayd = caa_container_of(node, struct consumer_relayd_sock_pair, node);
+       }
+
+error:
+       return relayd;
+}
+
+/*
+ * Find a relayd and send the stream
+ *
+ * Returns 0 on success, < 0 on error
+ */
+int consumer_send_relayd_stream(struct lttng_consumer_stream *stream,
+               char *path)
+{
+       int ret = 0;
+       struct consumer_relayd_sock_pair *relayd;
+
+       assert(stream);
+       assert(stream->net_seq_idx != -1ULL);
+       assert(path);
+
+       /* The stream is not metadata. Get relayd reference if exists. */
+       rcu_read_lock();
+       relayd = consumer_find_relayd(stream->net_seq_idx);
+       if (relayd != NULL) {
+               /* Add stream on the relayd */
+               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+               ret = relayd_add_stream(&relayd->control_sock, stream->name,
+                               path, &stream->relayd_stream_id,
+                               stream->chan->tracefile_size, stream->chan->tracefile_count);
+               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+               if (ret < 0) {
+                       goto end;
+               }
+
+               uatomic_inc(&relayd->refcount);
+               stream->sent_to_relayd = 1;
+       } else {
+               ERR("Stream %" PRIu64 " relayd ID %" PRIu64 " unknown. Can't send it.",
+                               stream->key, stream->net_seq_idx);
+               ret = -1;
+               goto end;
+       }
+
+       DBG("Stream %s with key %" PRIu64 " sent to relayd id %" PRIu64,
+                       stream->name, stream->key, stream->net_seq_idx);
+
+end:
+       rcu_read_unlock();
+       return ret;
+}
+
+/*
+ * Find a relayd and send the streams sent message
+ *
+ * Returns 0 on success, < 0 on error
+ */
+int consumer_send_relayd_streams_sent(uint64_t net_seq_idx)
+{
+       int ret = 0;
+       struct consumer_relayd_sock_pair *relayd;
+
+       assert(net_seq_idx != -1ULL);
+
+       /* The stream is not metadata. Get relayd reference if exists. */
+       rcu_read_lock();
+       relayd = consumer_find_relayd(net_seq_idx);
+       if (relayd != NULL) {
+               /* Add stream on the relayd */
+               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+               ret = relayd_streams_sent(&relayd->control_sock);
+               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+               if (ret < 0) {
+                       goto end;
+               }
+       } else {
+               ERR("Relayd ID %" PRIu64 " unknown. Can't send streams_sent.",
+                               net_seq_idx);
+               ret = -1;
+               goto end;
+       }
+
+       ret = 0;
+       DBG("All streams sent relayd id %" PRIu64, net_seq_idx);
+
+end:
+       rcu_read_unlock();
+       return ret;
+}
+
+/*
+ * Find a relayd and close the stream
+ */
+void close_relayd_stream(struct lttng_consumer_stream *stream)
+{
+       struct consumer_relayd_sock_pair *relayd;
+
+       /* The stream is not metadata. Get relayd reference if exists. */
+       rcu_read_lock();
+       relayd = consumer_find_relayd(stream->net_seq_idx);
+       if (relayd) {
+               consumer_stream_relayd_close(stream, relayd);
+       }
+       rcu_read_unlock();
+}
+
+/*
+ * Handle stream for relayd transmission if the stream applies for network
+ * streaming where the net sequence index is set.
+ *
+ * Return destination file descriptor or negative value on error.
+ */
+static int write_relayd_stream_header(struct lttng_consumer_stream *stream,
+               size_t data_size, unsigned long padding,
+               struct consumer_relayd_sock_pair *relayd)
+{
+       int outfd = -1, ret;
+       struct lttcomm_relayd_data_hdr data_hdr;
+
+       /* Safety net */
+       assert(stream);
+       assert(relayd);
+
+       /* Reset data header */
+       memset(&data_hdr, 0, sizeof(data_hdr));
+
+       if (stream->metadata_flag) {
+               /* Caller MUST acquire the relayd control socket lock */
+               ret = relayd_send_metadata(&relayd->control_sock, data_size);
+               if (ret < 0) {
+                       goto error;
+               }
+
+               /* Metadata are always sent on the control socket. */
+               outfd = relayd->control_sock.sock.fd;
+       } else {
+               /* Set header with stream information */
+               data_hdr.stream_id = htobe64(stream->relayd_stream_id);
+               data_hdr.data_size = htobe32(data_size);
+               data_hdr.padding_size = htobe32(padding);
+               /*
+                * Note that net_seq_num below is assigned with the *current* value of
+                * next_net_seq_num and only after that the next_net_seq_num will be
+                * increment. This is why when issuing a command on the relayd using
+                * this next value, 1 should always be substracted in order to compare
+                * the last seen sequence number on the relayd side to the last sent.
+                */
+               data_hdr.net_seq_num = htobe64(stream->next_net_seq_num);
+               /* Other fields are zeroed previously */
+
+               ret = relayd_send_data_hdr(&relayd->data_sock, &data_hdr,
+                               sizeof(data_hdr));
+               if (ret < 0) {
+                       goto error;
+               }
+
+               ++stream->next_net_seq_num;
+
+               /* Set to go on data socket */
+               outfd = relayd->data_sock.sock.fd;
+       }
+
+error:
+       return outfd;
+}
+
+/*
+ * Allocate and return a new lttng_consumer_channel object using the given key
+ * to initialize the hash table node.
+ *
+ * On error, return NULL.
+ */
+struct lttng_consumer_channel *consumer_allocate_channel(uint64_t key,
+               uint64_t session_id,
+               const char *pathname,
+               const char *name,
+               uid_t uid,
+               gid_t gid,
+               uint64_t relayd_id,
+               enum lttng_event_output output,
+               uint64_t tracefile_size,
+               uint64_t tracefile_count,
+               uint64_t session_id_per_pid,
+               unsigned int monitor,
+               unsigned int live_timer_interval,
+               const char *root_shm_path,
+               const char *shm_path)
+{
+       struct lttng_consumer_channel *channel;
+
+       channel = zmalloc(sizeof(*channel));
+       if (channel == NULL) {
+               PERROR("malloc struct lttng_consumer_channel");
+               goto end;
+       }
+
+       channel->key = key;
+       channel->refcount = 0;
+       channel->session_id = session_id;
+       channel->session_id_per_pid = session_id_per_pid;
+       channel->uid = uid;
+       channel->gid = gid;
+       channel->relayd_id = relayd_id;
+       channel->tracefile_size = tracefile_size;
+       channel->tracefile_count = tracefile_count;
+       channel->monitor = monitor;
+       channel->live_timer_interval = live_timer_interval;
+       pthread_mutex_init(&channel->lock, NULL);
+       pthread_mutex_init(&channel->timer_lock, NULL);
+
+       switch (output) {
+       case LTTNG_EVENT_SPLICE:
+               channel->output = CONSUMER_CHANNEL_SPLICE;
+               break;
+       case LTTNG_EVENT_MMAP:
+               channel->output = CONSUMER_CHANNEL_MMAP;
+               break;
+       default:
+               assert(0);
+               free(channel);
+               channel = NULL;
+               goto end;
+       }
+
+       /*
+        * In monitor mode, the streams associated with the channel will be put in
+        * a special list ONLY owned by this channel. So, the refcount is set to 1
+        * here meaning that the channel itself has streams that are referenced.
+        *
+        * On a channel deletion, once the channel is no longer visible, the
+        * refcount is decremented and checked for a zero value to delete it. With
+        * streams in no monitor mode, it will now be safe to destroy the channel.
+        */
+       if (!channel->monitor) {
+               channel->refcount = 1;
+       }
+
+       strncpy(channel->pathname, pathname, sizeof(channel->pathname));
+       channel->pathname[sizeof(channel->pathname) - 1] = '\0';
+
+       strncpy(channel->name, name, sizeof(channel->name));
+       channel->name[sizeof(channel->name) - 1] = '\0';
+
+       if (root_shm_path) {
+               strncpy(channel->root_shm_path, root_shm_path, sizeof(channel->root_shm_path));
+               channel->root_shm_path[sizeof(channel->root_shm_path) - 1] = '\0';
+       }
+       if (shm_path) {
+               strncpy(channel->shm_path, shm_path, sizeof(channel->shm_path));
+               channel->shm_path[sizeof(channel->shm_path) - 1] = '\0';
+       }
+
+       lttng_ht_node_init_u64(&channel->node, channel->key);
+
+       channel->wait_fd = -1;
+
+       CDS_INIT_LIST_HEAD(&channel->streams.head);
+
+       DBG("Allocated channel (key %" PRIu64 ")", channel->key)
+
+end:
+       return channel;
+}
+
+/*
+ * Add a channel to the global list protected by a mutex.
+ *
+ * Always return 0 indicating success.
+ */
+int consumer_add_channel(struct lttng_consumer_channel *channel,
+               struct lttng_consumer_local_data *ctx)
+{
+       pthread_mutex_lock(&consumer_data.lock);
+       pthread_mutex_lock(&channel->lock);
+       pthread_mutex_lock(&channel->timer_lock);
+
+       /*
+        * This gives us a guarantee that the channel we are about to add to the
+        * channel hash table will be unique. See this function comment on the why
+        * we need to steel the channel key at this stage.
+        */
+       steal_channel_key(channel->key);
+
+       rcu_read_lock();
+       lttng_ht_add_unique_u64(consumer_data.channel_ht, &channel->node);
+       rcu_read_unlock();
+
+       pthread_mutex_unlock(&channel->timer_lock);
+       pthread_mutex_unlock(&channel->lock);
+       pthread_mutex_unlock(&consumer_data.lock);
+
+       if (channel->wait_fd != -1 && channel->type == CONSUMER_CHANNEL_TYPE_DATA) {
+               notify_channel_pipe(ctx, channel, -1, CONSUMER_CHANNEL_ADD);
+       }
+
+       return 0;
+}
+
+/*
+ * Allocate the pollfd structure and the local view of the out fds to avoid
+ * doing a lookup in the linked list and concurrency issues when writing is
+ * needed. Called with consumer_data.lock held.
+ *
+ * Returns the number of fds in the structures.
+ */
+static int update_poll_array(struct lttng_consumer_local_data *ctx,
+               struct pollfd **pollfd, struct lttng_consumer_stream **local_stream,
+               struct lttng_ht *ht)
+{
+       int i = 0;
+       struct lttng_ht_iter iter;
+       struct lttng_consumer_stream *stream;
+
+       assert(ctx);
+       assert(ht);
+       assert(pollfd);
+       assert(local_stream);
+
+       DBG("Updating poll fd array");
+       rcu_read_lock();
+       cds_lfht_for_each_entry(ht->ht, &iter.iter, stream, node.node) {
+               /*
+                * Only active streams with an active end point can be added to the
+                * poll set and local stream storage of the thread.
+                *
+                * There is a potential race here for endpoint_status to be updated
+                * just after the check. However, this is OK since the stream(s) will
+                * be deleted once the thread is notified that the end point state has
+                * changed where this function will be called back again.
+                */
+               if (stream->state != LTTNG_CONSUMER_ACTIVE_STREAM ||
+                               stream->endpoint_status == CONSUMER_ENDPOINT_INACTIVE) {
+                       continue;
+               }
+               /*
+                * This clobbers way too much the debug output. Uncomment that if you
+                * need it for debugging purposes.
+                *
+                * DBG("Active FD %d", stream->wait_fd);
+                */
+               (*pollfd)[i].fd = stream->wait_fd;
+               (*pollfd)[i].events = POLLIN | POLLPRI;
+               local_stream[i] = stream;
+               i++;
+       }
+       rcu_read_unlock();
+
+       /*
+        * Insert the consumer_data_pipe at the end of the array and don't
+        * increment i so nb_fd is the number of real FD.
+        */
+       (*pollfd)[i].fd = lttng_pipe_get_readfd(ctx->consumer_data_pipe);
+       (*pollfd)[i].events = POLLIN | POLLPRI;
+
+       (*pollfd)[i + 1].fd = lttng_pipe_get_readfd(ctx->consumer_wakeup_pipe);
+       (*pollfd)[i + 1].events = POLLIN | POLLPRI;
+       return i;
+}
+
+/*
+ * Poll on the should_quit pipe and the command socket return -1 on
+ * error, 1 if should exit, 0 if data is available on the command socket
+ */
+int lttng_consumer_poll_socket(struct pollfd *consumer_sockpoll)
+{
+       int num_rdy;
+
+restart:
+       num_rdy = poll(consumer_sockpoll, 2, -1);
+       if (num_rdy == -1) {
+               /*
+                * Restart interrupted system call.
+                */
+               if (errno == EINTR) {
+                       goto restart;
+               }
+               PERROR("Poll error");
+               return -1;
+       }
+       if (consumer_sockpoll[0].revents & (POLLIN | POLLPRI)) {
+               DBG("consumer_should_quit wake up");
+               return 1;
+       }
+       return 0;
+}
+
+/*
+ * Set the error socket.
+ */
+void lttng_consumer_set_error_sock(struct lttng_consumer_local_data *ctx,
+               int sock)
+{
+       ctx->consumer_error_socket = sock;
+}
+
+/*
+ * Set the command socket path.
+ */
+void lttng_consumer_set_command_sock_path(
+               struct lttng_consumer_local_data *ctx, char *sock)
+{
+       ctx->consumer_command_sock_path = sock;
+}
+
+/*
+ * Send return code to the session daemon.
+ * If the socket is not defined, we return 0, it is not a fatal error
+ */
+int lttng_consumer_send_error(struct lttng_consumer_local_data *ctx, int cmd)
+{
+       if (ctx->consumer_error_socket > 0) {
+               return lttcomm_send_unix_sock(ctx->consumer_error_socket, &cmd,
+                               sizeof(enum lttcomm_sessiond_command));
+       }
+
+       return 0;
+}
+
+/*
+ * Close all the tracefiles and stream fds and MUST be called when all
+ * instances are destroyed i.e. when all threads were joined and are ended.
+ */
+void lttng_consumer_cleanup(void)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_consumer_channel *channel;
+
+       rcu_read_lock();
+
+       cds_lfht_for_each_entry(consumer_data.channel_ht->ht, &iter.iter, channel,
+                       node.node) {
+               consumer_del_channel(channel);
+       }
+
+       rcu_read_unlock();
+
+       lttng_ht_destroy(consumer_data.channel_ht);
+
+       cleanup_relayd_ht();
+
+       lttng_ht_destroy(consumer_data.stream_per_chan_id_ht);
+
+       /*
+        * This HT contains streams that are freed by either the metadata thread or
+        * the data thread so we do *nothing* on the hash table and simply destroy
+        * it.
+        */
+       lttng_ht_destroy(consumer_data.stream_list_ht);
+}
+
+/*
+ * Called from signal handler.
+ */
+void lttng_consumer_should_exit(struct lttng_consumer_local_data *ctx)
+{
+       ssize_t ret;
+
+       consumer_quit = 1;
+       ret = lttng_write(ctx->consumer_should_quit[1], "4", 1);
+       if (ret < 1) {
+               PERROR("write consumer quit");
+       }
+
+       DBG("Consumer flag that it should quit");
+}
+
+void lttng_consumer_sync_trace_file(struct lttng_consumer_stream *stream,
+               off_t orig_offset)
+{
+       int outfd = stream->out_fd;
+
+       /*
+        * This does a blocking write-and-wait on any page that belongs to the
+        * subbuffer prior to the one we just wrote.
+        * Don't care about error values, as these are just hints and ways to
+        * limit the amount of page cache used.
+        */
+       if (orig_offset < stream->max_sb_size) {
+               return;
+       }
+       lttng_sync_file_range(outfd, orig_offset - stream->max_sb_size,
+                       stream->max_sb_size,
+                       SYNC_FILE_RANGE_WAIT_BEFORE
+                       | SYNC_FILE_RANGE_WRITE
+                       | SYNC_FILE_RANGE_WAIT_AFTER);
+       /*
+        * Give hints to the kernel about how we access the file:
+        * POSIX_FADV_DONTNEED : we won't re-access data in a near future after
+        * we write it.
+        *
+        * We need to call fadvise again after the file grows because the
+        * kernel does not seem to apply fadvise to non-existing parts of the
+        * file.
+        *
+        * Call fadvise _after_ having waited for the page writeback to
+        * complete because the dirty page writeback semantic is not well
+        * defined. So it can be expected to lead to lower throughput in
+        * streaming.
+        */
+       posix_fadvise(outfd, orig_offset - stream->max_sb_size,
+                       stream->max_sb_size, POSIX_FADV_DONTNEED);
+}
+
+/*
+ * Initialise the necessary environnement :
+ * - create a new context
+ * - create the poll_pipe
+ * - create the should_quit pipe (for signal handler)
+ * - create the thread pipe (for splice)
+ *
+ * Takes a function pointer as argument, this function is called when data is
+ * available on a buffer. This function is responsible to do the
+ * kernctl_get_next_subbuf, read the data with mmap or splice depending on the
+ * buffer configuration and then kernctl_put_next_subbuf at the end.
+ *
+ * Returns a pointer to the new context or NULL on error.
+ */
+struct lttng_consumer_local_data *lttng_consumer_create(
+               enum lttng_consumer_type type,
+               ssize_t (*buffer_ready)(struct lttng_consumer_stream *stream,
+                       struct lttng_consumer_local_data *ctx),
+               int (*recv_channel)(struct lttng_consumer_channel *channel),
+               int (*recv_stream)(struct lttng_consumer_stream *stream),
+               int (*update_stream)(uint64_t stream_key, uint32_t state))
+{
+       int ret;
+       struct lttng_consumer_local_data *ctx;
+
+       assert(consumer_data.type == LTTNG_CONSUMER_UNKNOWN ||
+               consumer_data.type == type);
+       consumer_data.type = type;
+
+       ctx = zmalloc(sizeof(struct lttng_consumer_local_data));
+       if (ctx == NULL) {
+               PERROR("allocating context");
+               goto error;
+       }
+
+       ctx->consumer_error_socket = -1;
+       ctx->consumer_metadata_socket = -1;
+       pthread_mutex_init(&ctx->metadata_socket_lock, NULL);
+       /* assign the callbacks */
+       ctx->on_buffer_ready = buffer_ready;
+       ctx->on_recv_channel = recv_channel;
+       ctx->on_recv_stream = recv_stream;
+       ctx->on_update_stream = update_stream;
+
+       ctx->consumer_data_pipe = lttng_pipe_open(0);
+       if (!ctx->consumer_data_pipe) {
+               goto error_poll_pipe;
+       }
+
+       ctx->consumer_wakeup_pipe = lttng_pipe_open(0);
+       if (!ctx->consumer_wakeup_pipe) {
+               goto error_wakeup_pipe;
+       }
+
+       ret = pipe(ctx->consumer_should_quit);
+       if (ret < 0) {
+               PERROR("Error creating recv pipe");
+               goto error_quit_pipe;
+       }
+
+       ret = pipe(ctx->consumer_channel_pipe);
+       if (ret < 0) {
+               PERROR("Error creating channel pipe");
+               goto error_channel_pipe;
+       }
+
+       ctx->consumer_metadata_pipe = lttng_pipe_open(0);
+       if (!ctx->consumer_metadata_pipe) {
+               goto error_metadata_pipe;
+       }
+
+       return ctx;
+
+error_metadata_pipe:
+       utils_close_pipe(ctx->consumer_channel_pipe);
+error_channel_pipe:
+       utils_close_pipe(ctx->consumer_should_quit);
+error_quit_pipe:
+       lttng_pipe_destroy(ctx->consumer_wakeup_pipe);
+error_wakeup_pipe:
+       lttng_pipe_destroy(ctx->consumer_data_pipe);
+error_poll_pipe:
+       free(ctx);
+error:
+       return NULL;
+}
+
+/*
+ * Iterate over all streams of the hashtable and free them properly.
+ */
+static void destroy_data_stream_ht(struct lttng_ht *ht)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_consumer_stream *stream;
+
+       if (ht == NULL) {
+               return;
+       }
+
+       rcu_read_lock();
+       cds_lfht_for_each_entry(ht->ht, &iter.iter, stream, node.node) {
+               /*
+                * Ignore return value since we are currently cleaning up so any error
+                * can't be handled.
+                */
+               (void) consumer_del_stream(stream, ht);
+       }
+       rcu_read_unlock();
+
+       lttng_ht_destroy(ht);
+}
+
+/*
+ * Iterate over all streams of the metadata hashtable and free them
+ * properly.
+ */
+static void destroy_metadata_stream_ht(struct lttng_ht *ht)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_consumer_stream *stream;
+
+       if (ht == NULL) {
+               return;
+       }
+
+       rcu_read_lock();
+       cds_lfht_for_each_entry(ht->ht, &iter.iter, stream, node.node) {
+               /*
+                * Ignore return value since we are currently cleaning up so any error
+                * can't be handled.
+                */
+               (void) consumer_del_metadata_stream(stream, ht);
+       }
+       rcu_read_unlock();
+
+       lttng_ht_destroy(ht);
+}
+
+/*
+ * Close all fds associated with the instance and free the context.
+ */
+void lttng_consumer_destroy(struct lttng_consumer_local_data *ctx)
+{
+       int ret;
+
+       DBG("Consumer destroying it. Closing everything.");
+
+       if (!ctx) {
+               return;
+       }
+
+       destroy_data_stream_ht(data_ht);
+       destroy_metadata_stream_ht(metadata_ht);
+
+       ret = close(ctx->consumer_error_socket);
+       if (ret) {
+               PERROR("close");
+       }
+       ret = close(ctx->consumer_metadata_socket);
+       if (ret) {
+               PERROR("close");
+       }
+       utils_close_pipe(ctx->consumer_channel_pipe);
+       lttng_pipe_destroy(ctx->consumer_data_pipe);
+       lttng_pipe_destroy(ctx->consumer_metadata_pipe);
+       lttng_pipe_destroy(ctx->consumer_wakeup_pipe);
+       utils_close_pipe(ctx->consumer_should_quit);
+
+       unlink(ctx->consumer_command_sock_path);
+       free(ctx);
+}
+
+/*
+ * Write the metadata stream id on the specified file descriptor.
+ */
+static int write_relayd_metadata_id(int fd,
+               struct lttng_consumer_stream *stream,
+               struct consumer_relayd_sock_pair *relayd, unsigned long padding)
+{
+       ssize_t ret;
+       struct lttcomm_relayd_metadata_payload hdr;
+
+       hdr.stream_id = htobe64(stream->relayd_stream_id);
+       hdr.padding_size = htobe32(padding);
+       ret = lttng_write(fd, (void *) &hdr, sizeof(hdr));
+       if (ret < sizeof(hdr)) {
+               /*
+                * This error means that the fd's end is closed so ignore the PERROR
+                * not to clubber the error output since this can happen in a normal
+                * code path.
+                */
+               if (errno != EPIPE) {
+                       PERROR("write metadata stream id");
+               }
+               DBG3("Consumer failed to write relayd metadata id (errno: %d)", errno);
+               /*
+                * Set ret to a negative value because if ret != sizeof(hdr), we don't
+                * handle writting the missing part so report that as an error and
+                * don't lie to the caller.
+                */
+               ret = -1;
+               goto end;
+       }
+       DBG("Metadata stream id %" PRIu64 " with padding %lu written before data",
+                       stream->relayd_stream_id, padding);
+
+end:
+       return (int) ret;
+}
+
+/*
+ * Mmap the ring buffer, read it and write the data to the tracefile. This is a
+ * core function for writing trace buffers to either the local filesystem or
+ * the network.
+ *
+ * It must be called with the stream lock held.
+ *
+ * Careful review MUST be put if any changes occur!
+ *
+ * Returns the number of bytes written
+ */
+ssize_t lttng_consumer_on_read_subbuffer_mmap(
+               struct lttng_consumer_local_data *ctx,
+               struct lttng_consumer_stream *stream, unsigned long len,
+               unsigned long padding,
+               struct ctf_packet_index *index)
+{
+       unsigned long mmap_offset;
+       void *mmap_base;
+       ssize_t ret = 0;
+       off_t orig_offset = stream->out_fd_offset;
+       /* Default is on the disk */
+       int outfd = stream->out_fd;
+       struct consumer_relayd_sock_pair *relayd = NULL;
+       unsigned int relayd_hang_up = 0;
+
+       /* RCU lock for the relayd pointer */
+       rcu_read_lock();
+
+       /* Flag that the current stream if set for network streaming. */
+       if (stream->net_seq_idx != (uint64_t) -1ULL) {
+               relayd = consumer_find_relayd(stream->net_seq_idx);
+               if (relayd == NULL) {
+                       ret = -EPIPE;
+                       goto end;
+               }
+       }
+
+       /* get the offset inside the fd to mmap */
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               mmap_base = stream->mmap_base;
+               ret = kernctl_get_mmap_read_offset(stream->wait_fd, &mmap_offset);
+               if (ret < 0) {
+                       ret = -errno;
+                       PERROR("tracer ctl get_mmap_read_offset");
+                       goto end;
+               }
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               mmap_base = lttng_ustctl_get_mmap_base(stream);
+               if (!mmap_base) {
+                       ERR("read mmap get mmap base for stream %s", stream->name);
+                       ret = -EPERM;
+                       goto end;
+               }
+               ret = lttng_ustctl_get_mmap_read_offset(stream, &mmap_offset);
+               if (ret != 0) {
+                       PERROR("tracer ctl get_mmap_read_offset");
+                       ret = -EINVAL;
+                       goto end;
+               }
+               break;
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+       }
+
+       /* Handle stream on the relayd if the output is on the network */
+       if (relayd) {
+               unsigned long netlen = len;
+
+               /*
+                * Lock the control socket for the complete duration of the function
+                * since from this point on we will use the socket.
+                */
+               if (stream->metadata_flag) {
+                       /* Metadata requires the control socket. */
+                       pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+                       netlen += sizeof(struct lttcomm_relayd_metadata_payload);
+               }
+
+               ret = write_relayd_stream_header(stream, netlen, padding, relayd);
+               if (ret < 0) {
+                       relayd_hang_up = 1;
+                       goto write_error;
+               }
+               /* Use the returned socket. */
+               outfd = ret;
+
+               /* Write metadata stream id before payload */
+               if (stream->metadata_flag) {
+                       ret = write_relayd_metadata_id(outfd, stream, relayd, padding);
+                       if (ret < 0) {
+                               relayd_hang_up = 1;
+                               goto write_error;
+                       }
+               }
+       } else {
+               /* No streaming, we have to set the len with the full padding */
+               len += padding;
+
+               /*
+                * Check if we need to change the tracefile before writing the packet.
+                */
+               if (stream->chan->tracefile_size > 0 &&
+                               (stream->tracefile_size_current + len) >
+                               stream->chan->tracefile_size) {
+                       ret = utils_rotate_stream_file(stream->chan->pathname,
+                                       stream->name, stream->chan->tracefile_size,
+                                       stream->chan->tracefile_count, stream->uid, stream->gid,
+                                       stream->out_fd, &(stream->tracefile_count_current),
+                                       &stream->out_fd);
+                       if (ret < 0) {
+                               ERR("Rotating output file");
+                               goto end;
+                       }
+                       outfd = stream->out_fd;
+
+                       if (stream->index_fd >= 0) {
+                               ret = index_create_file(stream->chan->pathname,
+                                               stream->name, stream->uid, stream->gid,
+                                               stream->chan->tracefile_size,
+                                               stream->tracefile_count_current);
+                               if (ret < 0) {
+                                       goto end;
+                               }
+                               stream->index_fd = ret;
+                       }
+
+                       /* Reset current size because we just perform a rotation. */
+                       stream->tracefile_size_current = 0;
+                       stream->out_fd_offset = 0;
+                       orig_offset = 0;
+               }
+               stream->tracefile_size_current += len;
+               if (index) {
+                       index->offset = htobe64(stream->out_fd_offset);
+               }
+       }
+
+       /*
+        * This call guarantee that len or less is returned. It's impossible to
+        * receive a ret value that is bigger than len.
+        */
+       ret = lttng_write(outfd, mmap_base + mmap_offset, len);
+       DBG("Consumer mmap write() ret %zd (len %lu)", ret, len);
+       if (ret < 0 || ((size_t) ret != len)) {
+               /*
+                * Report error to caller if nothing was written else at least send the
+                * amount written.
+                */
+               if (ret < 0) {
+                       ret = -errno;
+               }
+               relayd_hang_up = 1;
+
+               /* Socket operation failed. We consider the relayd dead */
+               if (errno == EPIPE || errno == EINVAL || errno == EBADF) {
+                       /*
+                        * This is possible if the fd is closed on the other side
+                        * (outfd) or any write problem. It can be verbose a bit for a
+                        * normal execution if for instance the relayd is stopped
+                        * abruptly. This can happen so set this to a DBG statement.
+                        */
+                       DBG("Consumer mmap write detected relayd hang up");
+               } else {
+                       /* Unhandled error, print it and stop function right now. */
+                       PERROR("Error in write mmap (ret %zd != len %lu)", ret, len);
+               }
+               goto write_error;
+       }
+       stream->output_written += ret;
+
+       /* This call is useless on a socket so better save a syscall. */
+       if (!relayd) {
+               /* This won't block, but will start writeout asynchronously */
+               lttng_sync_file_range(outfd, stream->out_fd_offset, len,
+                               SYNC_FILE_RANGE_WRITE);
+               stream->out_fd_offset += len;
+       }
+       lttng_consumer_sync_trace_file(stream, orig_offset);
+
+write_error:
+       /*
+        * This is a special case that the relayd has closed its socket. Let's
+        * cleanup the relayd object and all associated streams.
+        */
+       if (relayd && relayd_hang_up) {
+               cleanup_relayd(relayd, ctx);
+       }
+
+end:
+       /* Unlock only if ctrl socket used */
+       if (relayd && stream->metadata_flag) {
+               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+       }
+
+       rcu_read_unlock();
+       return ret;
+}
+
+/*
+ * Splice the data from the ring buffer to the tracefile.
+ *
+ * It must be called with the stream lock held.
+ *
+ * Returns the number of bytes spliced.
+ */
+ssize_t lttng_consumer_on_read_subbuffer_splice(
+               struct lttng_consumer_local_data *ctx,
+               struct lttng_consumer_stream *stream, unsigned long len,
+               unsigned long padding,
+               struct ctf_packet_index *index)
+{
+       ssize_t ret = 0, written = 0, ret_splice = 0;
+       loff_t offset = 0;
+       off_t orig_offset = stream->out_fd_offset;
+       int fd = stream->wait_fd;
+       /* Default is on the disk */
+       int outfd = stream->out_fd;
+       struct consumer_relayd_sock_pair *relayd = NULL;
+       int *splice_pipe;
+       unsigned int relayd_hang_up = 0;
+
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               /* Not supported for user space tracing */
+               return -ENOSYS;
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+       }
+
+       /* RCU lock for the relayd pointer */
+       rcu_read_lock();
+
+       /* Flag that the current stream if set for network streaming. */
+       if (stream->net_seq_idx != (uint64_t) -1ULL) {
+               relayd = consumer_find_relayd(stream->net_seq_idx);
+               if (relayd == NULL) {
+                       written = -ret;
+                       goto end;
+               }
+       }
+       splice_pipe = stream->splice_pipe;
+
+       /* Write metadata stream id before payload */
+       if (relayd) {
+               unsigned long total_len = len;
+
+               if (stream->metadata_flag) {
+                       /*
+                        * Lock the control socket for the complete duration of the function
+                        * since from this point on we will use the socket.
+                        */
+                       pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+
+                       ret = write_relayd_metadata_id(splice_pipe[1], stream, relayd,
+                                       padding);
+                       if (ret < 0) {
+                               written = ret;
+                               relayd_hang_up = 1;
+                               goto write_error;
+                       }
+
+                       total_len += sizeof(struct lttcomm_relayd_metadata_payload);
+               }
+
+               ret = write_relayd_stream_header(stream, total_len, padding, relayd);
+               if (ret < 0) {
+                       written = ret;
+                       relayd_hang_up = 1;
+                       goto write_error;
+               }
+               /* Use the returned socket. */
+               outfd = ret;
+       } else {
+               /* No streaming, we have to set the len with the full padding */
+               len += padding;
+
+               /*
+                * Check if we need to change the tracefile before writing the packet.
+                */
+               if (stream->chan->tracefile_size > 0 &&
+                               (stream->tracefile_size_current + len) >
+                               stream->chan->tracefile_size) {
+                       ret = utils_rotate_stream_file(stream->chan->pathname,
+                                       stream->name, stream->chan->tracefile_size,
+                                       stream->chan->tracefile_count, stream->uid, stream->gid,
+                                       stream->out_fd, &(stream->tracefile_count_current),
+                                       &stream->out_fd);
+                       if (ret < 0) {
+                               written = ret;
+                               ERR("Rotating output file");
+                               goto end;
+                       }
+                       outfd = stream->out_fd;
+
+                       if (stream->index_fd >= 0) {
+                               ret = index_create_file(stream->chan->pathname,
+                                               stream->name, stream->uid, stream->gid,
+                                               stream->chan->tracefile_size,
+                                               stream->tracefile_count_current);
+                               if (ret < 0) {
+                                       written = ret;
+                                       goto end;
+                               }
+                               stream->index_fd = ret;
+                       }
+
+                       /* Reset current size because we just perform a rotation. */
+                       stream->tracefile_size_current = 0;
+                       stream->out_fd_offset = 0;
+                       orig_offset = 0;
+               }
+               stream->tracefile_size_current += len;
+               index->offset = htobe64(stream->out_fd_offset);
+       }
+
+       while (len > 0) {
+               DBG("splice chan to pipe offset %lu of len %lu (fd : %d, pipe: %d)",
+                               (unsigned long)offset, len, fd, splice_pipe[1]);
+               ret_splice = splice(fd, &offset, splice_pipe[1], NULL, len,
+                               SPLICE_F_MOVE | SPLICE_F_MORE);
+               DBG("splice chan to pipe, ret %zd", ret_splice);
+               if (ret_splice < 0) {
+                       ret = errno;
+                       written = -ret;
+                       PERROR("Error in relay splice");
+                       goto splice_error;
+               }
+
+               /* Handle stream on the relayd if the output is on the network */
+               if (relayd && stream->metadata_flag) {
+                       size_t metadata_payload_size =
+                               sizeof(struct lttcomm_relayd_metadata_payload);
+
+                       /* Update counter to fit the spliced data */
+                       ret_splice += metadata_payload_size;
+                       len += metadata_payload_size;
+                       /*
+                        * We do this so the return value can match the len passed as
+                        * argument to this function.
+                        */
+                       written -= metadata_payload_size;
+               }
+
+               /* Splice data out */
+               ret_splice = splice(splice_pipe[0], NULL, outfd, NULL,
+                               ret_splice, SPLICE_F_MOVE | SPLICE_F_MORE);
+               DBG("Consumer splice pipe to file (out_fd: %d), ret %zd",
+                               outfd, ret_splice);
+               if (ret_splice < 0) {
+                       ret = errno;
+                       written = -ret;
+                       relayd_hang_up = 1;
+                       goto write_error;
+               } else if (ret_splice > len) {
+                       /*
+                        * We don't expect this code path to be executed but you never know
+                        * so this is an extra protection agains a buggy splice().
+                        */
+                       ret = errno;
+                       written += ret_splice;
+                       PERROR("Wrote more data than requested %zd (len: %lu)", ret_splice,
+                                       len);
+                       goto splice_error;
+               } else {
+                       /* All good, update current len and continue. */
+                       len -= ret_splice;
+               }
+
+               /* This call is useless on a socket so better save a syscall. */
+               if (!relayd) {
+                       /* This won't block, but will start writeout asynchronously */
+                       lttng_sync_file_range(outfd, stream->out_fd_offset, ret_splice,
+                                       SYNC_FILE_RANGE_WRITE);
+                       stream->out_fd_offset += ret_splice;
+               }
+               stream->output_written += ret_splice;
+               written += ret_splice;
+       }
+       lttng_consumer_sync_trace_file(stream, orig_offset);
+       goto end;
+
+write_error:
+       /*
+        * This is a special case that the relayd has closed its socket. Let's
+        * cleanup the relayd object and all associated streams.
+        */
+       if (relayd && relayd_hang_up) {
+               cleanup_relayd(relayd, ctx);
+               /* Skip splice error so the consumer does not fail */
+               goto end;
+       }
+
+splice_error:
+       /* send the appropriate error description to sessiond */
+       switch (ret) {
+       case EINVAL:
+               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_SPLICE_EINVAL);
+               break;
+       case ENOMEM:
+               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_SPLICE_ENOMEM);
+               break;
+       case ESPIPE:
+               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_SPLICE_ESPIPE);
+               break;
+       }
+
+end:
+       if (relayd && stream->metadata_flag) {
+               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+       }
+
+       rcu_read_unlock();
+       return written;
+}
+
+/*
+ * Take a snapshot for a specific fd
+ *
+ * Returns 0 on success, < 0 on error
+ */
+int lttng_consumer_take_snapshot(struct lttng_consumer_stream *stream)
+{
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               return lttng_kconsumer_take_snapshot(stream);
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               return lttng_ustconsumer_take_snapshot(stream);
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+               return -ENOSYS;
+       }
+}
+
+/*
+ * Get the produced position
+ *
+ * Returns 0 on success, < 0 on error
+ */
+int lttng_consumer_get_produced_snapshot(struct lttng_consumer_stream *stream,
+               unsigned long *pos)
+{
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               return lttng_kconsumer_get_produced_snapshot(stream, pos);
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               return lttng_ustconsumer_get_produced_snapshot(stream, pos);
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+               return -ENOSYS;
+       }
+}
+
+int lttng_consumer_recv_cmd(struct lttng_consumer_local_data *ctx,
+               int sock, struct pollfd *consumer_sockpoll)
+{
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               return lttng_kconsumer_recv_cmd(ctx, sock, consumer_sockpoll);
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               return lttng_ustconsumer_recv_cmd(ctx, sock, consumer_sockpoll);
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+               return -ENOSYS;
+       }
+}
+
+void lttng_consumer_close_all_metadata(void)
+{
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               /*
+                * The Kernel consumer has a different metadata scheme so we don't
+                * close anything because the stream will be closed by the session
+                * daemon.
+                */
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               /*
+                * Close all metadata streams. The metadata hash table is passed and
+                * this call iterates over it by closing all wakeup fd. This is safe
+                * because at this point we are sure that the metadata producer is
+                * either dead or blocked.
+                */
+               lttng_ustconsumer_close_all_metadata(metadata_ht);
+               break;
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+       }
+}
+
+/*
+ * Clean up a metadata stream and free its memory.
+ */
+void consumer_del_metadata_stream(struct lttng_consumer_stream *stream,
+               struct lttng_ht *ht)
+{
+       struct lttng_consumer_channel *free_chan = NULL;
+
+       assert(stream);
+       /*
+        * This call should NEVER receive regular stream. It must always be
+        * metadata stream and this is crucial for data structure synchronization.
+        */
+       assert(stream->metadata_flag);
+
+       DBG3("Consumer delete metadata stream %d", stream->wait_fd);
+
+       pthread_mutex_lock(&consumer_data.lock);
+       pthread_mutex_lock(&stream->chan->lock);
+       pthread_mutex_lock(&stream->lock);
+
+       /* Remove any reference to that stream. */
+       consumer_stream_delete(stream, ht);
+
+       /* Close down everything including the relayd if one. */
+       consumer_stream_close(stream);
+       /* Destroy tracer buffers of the stream. */
+       consumer_stream_destroy_buffers(stream);
+
+       /* Atomically decrement channel refcount since other threads can use it. */
+       if (!uatomic_sub_return(&stream->chan->refcount, 1)
+                       && !uatomic_read(&stream->chan->nb_init_stream_left)) {
+               /* Go for channel deletion! */
+               free_chan = stream->chan;
+       }
+
+       /*
+        * Nullify the stream reference so it is not used after deletion. The
+        * channel lock MUST be acquired before being able to check for a NULL
+        * pointer value.
+        */
+       stream->chan->metadata_stream = NULL;
+
+       pthread_mutex_unlock(&stream->lock);
+       pthread_mutex_unlock(&stream->chan->lock);
+       pthread_mutex_unlock(&consumer_data.lock);
+
+       if (free_chan) {
+               consumer_del_channel(free_chan);
+       }
+
+       consumer_stream_free(stream);
+}
+
+/*
+ * Action done with the metadata stream when adding it to the consumer internal
+ * data structures to handle it.
+ */
+int consumer_add_metadata_stream(struct lttng_consumer_stream *stream)
+{
+       struct lttng_ht *ht = metadata_ht;
+       int ret = 0;
+       struct lttng_ht_iter iter;
+       struct lttng_ht_node_u64 *node;
+
+       assert(stream);
+       assert(ht);
+
+       DBG3("Adding metadata stream %" PRIu64 " to hash table", stream->key);
+
+       pthread_mutex_lock(&consumer_data.lock);
+       pthread_mutex_lock(&stream->chan->lock);
+       pthread_mutex_lock(&stream->chan->timer_lock);
+       pthread_mutex_lock(&stream->lock);
+
+       /*
+        * From here, refcounts are updated so be _careful_ when returning an error
+        * after this point.
+        */
+
+       rcu_read_lock();
+
+       /*
+        * Lookup the stream just to make sure it does not exist in our internal
+        * state. This should NEVER happen.
+        */
+       lttng_ht_lookup(ht, &stream->key, &iter);
+       node = lttng_ht_iter_get_node_u64(&iter);
+       assert(!node);
+
+       /*
+        * When nb_init_stream_left reaches 0, we don't need to trigger any action
+        * in terms of destroying the associated channel, because the action that
+        * causes the count to become 0 also causes a stream to be added. The
+        * channel deletion will thus be triggered by the following removal of this
+        * stream.
+        */
+       if (uatomic_read(&stream->chan->nb_init_stream_left) > 0) {
+               /* Increment refcount before decrementing nb_init_stream_left */
+               cmm_smp_wmb();
+               uatomic_dec(&stream->chan->nb_init_stream_left);
+       }
+
+       lttng_ht_add_unique_u64(ht, &stream->node);
+
+       lttng_ht_add_unique_u64(consumer_data.stream_per_chan_id_ht,
+               &stream->node_channel_id);
+
+       /*
+        * Add stream to the stream_list_ht of the consumer data. No need to steal
+        * the key since the HT does not use it and we allow to add redundant keys
+        * into this table.
+        */
+       lttng_ht_add_u64(consumer_data.stream_list_ht, &stream->node_session_id);
+
+       rcu_read_unlock();
+
+       pthread_mutex_unlock(&stream->lock);
+       pthread_mutex_unlock(&stream->chan->lock);
+       pthread_mutex_unlock(&stream->chan->timer_lock);
+       pthread_mutex_unlock(&consumer_data.lock);
+       return ret;
+}
+
+/*
+ * Delete data stream that are flagged for deletion (endpoint_status).
+ */
+static void validate_endpoint_status_data_stream(void)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_consumer_stream *stream;
+
+       DBG("Consumer delete flagged data stream");
+
+       rcu_read_lock();
+       cds_lfht_for_each_entry(data_ht->ht, &iter.iter, stream, node.node) {
+               /* Validate delete flag of the stream */
+               if (stream->endpoint_status == CONSUMER_ENDPOINT_ACTIVE) {
+                       continue;
+               }
+               /* Delete it right now */
+               consumer_del_stream(stream, data_ht);
+       }
+       rcu_read_unlock();
+}
+
+/*
+ * Delete metadata stream that are flagged for deletion (endpoint_status).
+ */
+static void validate_endpoint_status_metadata_stream(
+               struct lttng_poll_event *pollset)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_consumer_stream *stream;
+
+       DBG("Consumer delete flagged metadata stream");
+
+       assert(pollset);
+
+       rcu_read_lock();
+       cds_lfht_for_each_entry(metadata_ht->ht, &iter.iter, stream, node.node) {
+               /* Validate delete flag of the stream */
+               if (stream->endpoint_status == CONSUMER_ENDPOINT_ACTIVE) {
+                       continue;
+               }
+               /*
+                * Remove from pollset so the metadata thread can continue without
+                * blocking on a deleted stream.
+                */
+               lttng_poll_del(pollset, stream->wait_fd);
+
+               /* Delete it right now */
+               consumer_del_metadata_stream(stream, metadata_ht);
+       }
+       rcu_read_unlock();
+}
+
+/*
+ * Thread polls on metadata file descriptor and write them on disk or on the
+ * network.
+ */
+void *consumer_thread_metadata_poll(void *data)
+{
+       int ret, i, pollfd, err = -1;
+       uint32_t revents, nb_fd;
+       struct lttng_consumer_stream *stream = NULL;
+       struct lttng_ht_iter iter;
+       struct lttng_ht_node_u64 *node;
+       struct lttng_poll_event events;
+       struct lttng_consumer_local_data *ctx = data;
+       ssize_t len;
+
+       rcu_register_thread();
+
+       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_METADATA);
+
+       if (testpoint(consumerd_thread_metadata)) {
+               goto error_testpoint;
+       }
+
+       health_code_update();
+
+       DBG("Thread metadata poll started");
+
+       /* Size is set to 1 for the consumer_metadata pipe */
+       ret = lttng_poll_create(&events, 2, LTTNG_CLOEXEC);
+       if (ret < 0) {
+               ERR("Poll set creation failed");
+               goto end_poll;
+       }
+
+       ret = lttng_poll_add(&events,
+                       lttng_pipe_get_readfd(ctx->consumer_metadata_pipe), LPOLLIN);
+       if (ret < 0) {
+               goto end;
+       }
+
+       /* Main loop */
+       DBG("Metadata main loop started");
+
+       while (1) {
+restart:
+               health_code_update();
+               health_poll_entry();
+               DBG("Metadata poll wait");
+               ret = lttng_poll_wait(&events, -1);
+               DBG("Metadata poll return from wait with %d fd(s)",
+                               LTTNG_POLL_GETNB(&events));
+               health_poll_exit();
+               DBG("Metadata event catched in thread");
+               if (ret < 0) {
+                       if (errno == EINTR) {
+                               ERR("Poll EINTR catched");
+                               goto restart;
+                       }
+                       if (LTTNG_POLL_GETNB(&events) == 0) {
+                               err = 0;        /* All is OK */
+                       }
+                       goto end;
+               }
+
+               nb_fd = ret;
+
+               /* From here, the event is a metadata wait fd */
+               for (i = 0; i < nb_fd; i++) {
+                       health_code_update();
+
+                       revents = LTTNG_POLL_GETEV(&events, i);
+                       pollfd = LTTNG_POLL_GETFD(&events, i);
+
+                       if (!revents) {
+                               /* No activity for this FD (poll implementation). */
+                               continue;
+                       }
+
+                       if (pollfd == lttng_pipe_get_readfd(ctx->consumer_metadata_pipe)) {
+                               if (revents & LPOLLIN) {
+                                       ssize_t pipe_len;
+
+                                       pipe_len = lttng_pipe_read(ctx->consumer_metadata_pipe,
+                                                       &stream, sizeof(stream));
+                                       if (pipe_len < sizeof(stream)) {
+                                               if (pipe_len < 0) {
+                                                       PERROR("read metadata stream");
+                                               }
+                                               /*
+                                                * Remove the pipe from the poll set and continue the loop
+                                                * since their might be data to consume.
+                                                */
+                                               lttng_poll_del(&events,
+                                                               lttng_pipe_get_readfd(ctx->consumer_metadata_pipe));
+                                               lttng_pipe_read_close(ctx->consumer_metadata_pipe);
+                                               continue;
+                                       }
+
+                                       /* A NULL stream means that the state has changed. */
+                                       if (stream == NULL) {
+                                               /* Check for deleted streams. */
+                                               validate_endpoint_status_metadata_stream(&events);
+                                               goto restart;
+                                       }
+
+                                       DBG("Adding metadata stream %d to poll set",
+                                                       stream->wait_fd);
+
+                                       /* Add metadata stream to the global poll events list */
+                                       lttng_poll_add(&events, stream->wait_fd,
+                                                       LPOLLIN | LPOLLPRI | LPOLLHUP);
+                               } else if (revents & (LPOLLERR | LPOLLHUP)) {
+                                       DBG("Metadata thread pipe hung up");
+                                       /*
+                                        * Remove the pipe from the poll set and continue the loop
+                                        * since their might be data to consume.
+                                        */
+                                       lttng_poll_del(&events,
+                                                       lttng_pipe_get_readfd(ctx->consumer_metadata_pipe));
+                                       lttng_pipe_read_close(ctx->consumer_metadata_pipe);
+                                       continue;
+                               } else {
+                                       ERR("Unexpected poll events %u for sock %d", revents, pollfd);
+                                       goto end;
+                               }
+
+                               /* Handle other stream */
+                               continue;
+                       }
+
+                       rcu_read_lock();
+                       {
+                               uint64_t tmp_id = (uint64_t) pollfd;
+
+                               lttng_ht_lookup(metadata_ht, &tmp_id, &iter);
+                       }
+                       node = lttng_ht_iter_get_node_u64(&iter);
+                       assert(node);
+
+                       stream = caa_container_of(node, struct lttng_consumer_stream,
+                                       node);
+
+                       if (revents & (LPOLLIN | LPOLLPRI)) {
+                               /* Get the data out of the metadata file descriptor */
+                               DBG("Metadata available on fd %d", pollfd);
+                               assert(stream->wait_fd == pollfd);
+
+                               do {
+                                       health_code_update();
+
+                                       len = ctx->on_buffer_ready(stream, ctx);
+                                       /*
+                                        * We don't check the return value here since if we get
+                                        * a negative len, it means an error occured thus we
+                                        * simply remove it from the poll set and free the
+                                        * stream.
+                                        */
+                               } while (len > 0);
+
+                               /* It's ok to have an unavailable sub-buffer */
+                               if (len < 0 && len != -EAGAIN && len != -ENODATA) {
+                                       /* Clean up stream from consumer and free it. */
+                                       lttng_poll_del(&events, stream->wait_fd);
+                                       consumer_del_metadata_stream(stream, metadata_ht);
+                               }
+                       } else if (revents & (LPOLLERR | LPOLLHUP)) {
+                               DBG("Metadata fd %d is hup|err.", pollfd);
+                               if (!stream->hangup_flush_done
+                                               && (consumer_data.type == LTTNG_CONSUMER32_UST
+                                                       || consumer_data.type == LTTNG_CONSUMER64_UST)) {
+                                       DBG("Attempting to flush and consume the UST buffers");
+                                       lttng_ustconsumer_on_stream_hangup(stream);
+
+                                       /* We just flushed the stream now read it. */
+                                       do {
+                                               health_code_update();
+
+                                               len = ctx->on_buffer_ready(stream, ctx);
+                                               /*
+                                                * We don't check the return value here since if we get
+                                                * a negative len, it means an error occured thus we
+                                                * simply remove it from the poll set and free the
+                                                * stream.
+                                                */
+                                       } while (len > 0);
+                               }
+
+                               lttng_poll_del(&events, stream->wait_fd);
+                               /*
+                                * This call update the channel states, closes file descriptors
+                                * and securely free the stream.
+                                */
+                               consumer_del_metadata_stream(stream, metadata_ht);
+                       } else {
+                               ERR("Unexpected poll events %u for sock %d", revents, pollfd);
+                               rcu_read_unlock();
+                               goto end;
+                       }
+                       /* Release RCU lock for the stream looked up */
+                       rcu_read_unlock();
+               }
+       }
+
+       /* All is OK */
+       err = 0;
+end:
+       DBG("Metadata poll thread exiting");
+
+       lttng_poll_clean(&events);
+end_poll:
+error_testpoint:
+       if (err) {
+               health_error();
+               ERR("Health error occurred in %s", __func__);
+       }
+       health_unregister(health_consumerd);
+       rcu_unregister_thread();
+       return NULL;
+}
+
+/*
+ * This thread polls the fds in the set to consume the data and write
+ * it to tracefile if necessary.
+ */
+void *consumer_thread_data_poll(void *data)
+{
+       int num_rdy, num_hup, high_prio, ret, i, err = -1;
+       struct pollfd *pollfd = NULL;
+       /* local view of the streams */
+       struct lttng_consumer_stream **local_stream = NULL, *new_stream = NULL;
+       /* local view of consumer_data.fds_count */
+       int nb_fd = 0;
+       struct lttng_consumer_local_data *ctx = data;
+       ssize_t len;
+
+       rcu_register_thread();
+
+       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_DATA);
+
+       if (testpoint(consumerd_thread_data)) {
+               goto error_testpoint;
+       }
+
+       health_code_update();
+
+       local_stream = zmalloc(sizeof(struct lttng_consumer_stream *));
+       if (local_stream == NULL) {
+               PERROR("local_stream malloc");
+               goto end;
+       }
+
+       while (1) {
+               health_code_update();
+
+               high_prio = 0;
+               num_hup = 0;
+
+               /*
+                * the fds set has been updated, we need to update our
+                * local array as well
+                */
+               pthread_mutex_lock(&consumer_data.lock);
+               if (consumer_data.need_update) {
+                       free(pollfd);
+                       pollfd = NULL;
+
+                       free(local_stream);
+                       local_stream = NULL;
+
+                       /*
+                        * Allocate for all fds +1 for the consumer_data_pipe and +1 for
+                        * wake up pipe.
+                        */
+                       pollfd = zmalloc((consumer_data.stream_count + 2) * sizeof(struct pollfd));
+                       if (pollfd == NULL) {
+                               PERROR("pollfd malloc");
+                               pthread_mutex_unlock(&consumer_data.lock);
+                               goto end;
+                       }
+
+                       local_stream = zmalloc((consumer_data.stream_count + 2) *
+                                       sizeof(struct lttng_consumer_stream *));
+                       if (local_stream == NULL) {
+                               PERROR("local_stream malloc");
+                               pthread_mutex_unlock(&consumer_data.lock);
+                               goto end;
+                       }
+                       ret = update_poll_array(ctx, &pollfd, local_stream,
+                                       data_ht);
+                       if (ret < 0) {
+                               ERR("Error in allocating pollfd or local_outfds");
+                               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_POLL_ERROR);
+                               pthread_mutex_unlock(&consumer_data.lock);
+                               goto end;
+                       }
+                       nb_fd = ret;
+                       consumer_data.need_update = 0;
+               }
+               pthread_mutex_unlock(&consumer_data.lock);
+
+               /* No FDs and consumer_quit, consumer_cleanup the thread */
+               if (nb_fd == 0 && consumer_quit == 1) {
+                       err = 0;        /* All is OK */
+                       goto end;
+               }
+               /* poll on the array of fds */
+       restart:
+               DBG("polling on %d fd", nb_fd + 2);
+               health_poll_entry();
+               num_rdy = poll(pollfd, nb_fd + 2, -1);
+               health_poll_exit();
+               DBG("poll num_rdy : %d", num_rdy);
+               if (num_rdy == -1) {
+                       /*
+                        * Restart interrupted system call.
+                        */
+                       if (errno == EINTR) {
+                               goto restart;
+                       }
+                       PERROR("Poll error");
+                       lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_POLL_ERROR);
+                       goto end;
+               } else if (num_rdy == 0) {
+                       DBG("Polling thread timed out");
+                       goto end;
+               }
+
+               /*
+                * If the consumer_data_pipe triggered poll go directly to the
+                * beginning of the loop to update the array. We want to prioritize
+                * array update over low-priority reads.
+                */
+               if (pollfd[nb_fd].revents & (POLLIN | POLLPRI)) {
+                       ssize_t pipe_readlen;
+
+                       DBG("consumer_data_pipe wake up");
+                       pipe_readlen = lttng_pipe_read(ctx->consumer_data_pipe,
+                                       &new_stream, sizeof(new_stream));
+                       if (pipe_readlen < sizeof(new_stream)) {
+                               PERROR("Consumer data pipe");
+                               /* Continue so we can at least handle the current stream(s). */
+                               continue;
+                       }
+
+                       /*
+                        * If the stream is NULL, just ignore it. It's also possible that
+                        * the sessiond poll thread changed the consumer_quit state and is
+                        * waking us up to test it.
+                        */
+                       if (new_stream == NULL) {
+                               validate_endpoint_status_data_stream();
+                               continue;
+                       }
+
+                       /* Continue to update the local streams and handle prio ones */
+                       continue;
+               }
+
+               /* Handle wakeup pipe. */
+               if (pollfd[nb_fd + 1].revents & (POLLIN | POLLPRI)) {
+                       char dummy;
+                       ssize_t pipe_readlen;
+
+                       pipe_readlen = lttng_pipe_read(ctx->consumer_wakeup_pipe, &dummy,
+                                       sizeof(dummy));
+                       if (pipe_readlen < 0) {
+                               PERROR("Consumer data wakeup pipe");
+                       }
+                       /* We've been awakened to handle stream(s). */
+                       ctx->has_wakeup = 0;
+               }
+
+               /* Take care of high priority channels first. */
+               for (i = 0; i < nb_fd; i++) {
+                       health_code_update();
+
+                       if (local_stream[i] == NULL) {
+                               continue;
+                       }
+                       if (pollfd[i].revents & POLLPRI) {
+                               DBG("Urgent read on fd %d", pollfd[i].fd);
+                               high_prio = 1;
+                               len = ctx->on_buffer_ready(local_stream[i], ctx);
+                               /* it's ok to have an unavailable sub-buffer */
+                               if (len < 0 && len != -EAGAIN && len != -ENODATA) {
+                                       /* Clean the stream and free it. */
+                                       consumer_del_stream(local_stream[i], data_ht);
+                                       local_stream[i] = NULL;
+                               } else if (len > 0) {
+                                       local_stream[i]->data_read = 1;
+                               }
+                       }
+               }
+
+               /*
+                * If we read high prio channel in this loop, try again
+                * for more high prio data.
+                */
+               if (high_prio) {
+                       continue;
+               }
+
+               /* Take care of low priority channels. */
+               for (i = 0; i < nb_fd; i++) {
+                       health_code_update();
+
+                       if (local_stream[i] == NULL) {
+                               continue;
+                       }
+                       if ((pollfd[i].revents & POLLIN) ||
+                                       local_stream[i]->hangup_flush_done ||
+                                       local_stream[i]->has_data) {
+                               DBG("Normal read on fd %d", pollfd[i].fd);
+                               len = ctx->on_buffer_ready(local_stream[i], ctx);
+                               /* it's ok to have an unavailable sub-buffer */
+                               if (len < 0 && len != -EAGAIN && len != -ENODATA) {
+                                       /* Clean the stream and free it. */
+                                       consumer_del_stream(local_stream[i], data_ht);
+                                       local_stream[i] = NULL;
+                               } else if (len > 0) {
+                                       local_stream[i]->data_read = 1;
+                               }
+                       }
+               }
+
+               /* Handle hangup and errors */
+               for (i = 0; i < nb_fd; i++) {
+                       health_code_update();
+
+                       if (local_stream[i] == NULL) {
+                               continue;
+                       }
+                       if (!local_stream[i]->hangup_flush_done
+                                       && (pollfd[i].revents & (POLLHUP | POLLERR | POLLNVAL))
+                                       && (consumer_data.type == LTTNG_CONSUMER32_UST
+                                               || consumer_data.type == LTTNG_CONSUMER64_UST)) {
+                               DBG("fd %d is hup|err|nval. Attempting flush and read.",
+                                               pollfd[i].fd);
+                               lttng_ustconsumer_on_stream_hangup(local_stream[i]);
+                               /* Attempt read again, for the data we just flushed. */
+                               local_stream[i]->data_read = 1;
+                       }
+                       /*
+                        * If the poll flag is HUP/ERR/NVAL and we have
+                        * read no data in this pass, we can remove the
+                        * stream from its hash table.
+                        */
+                       if ((pollfd[i].revents & POLLHUP)) {
+                               DBG("Polling fd %d tells it has hung up.", pollfd[i].fd);
+                               if (!local_stream[i]->data_read) {
+                                       consumer_del_stream(local_stream[i], data_ht);
+                                       local_stream[i] = NULL;
+                                       num_hup++;
+                               }
+                       } else if (pollfd[i].revents & POLLERR) {
+                               ERR("Error returned in polling fd %d.", pollfd[i].fd);
+                               if (!local_stream[i]->data_read) {
+                                       consumer_del_stream(local_stream[i], data_ht);
+                                       local_stream[i] = NULL;
+                                       num_hup++;
+                               }
+                       } else if (pollfd[i].revents & POLLNVAL) {
+                               ERR("Polling fd %d tells fd is not open.", pollfd[i].fd);
+                               if (!local_stream[i]->data_read) {
+                                       consumer_del_stream(local_stream[i], data_ht);
+                                       local_stream[i] = NULL;
+                                       num_hup++;
+                               }
+                       }
+                       if (local_stream[i] != NULL) {
+                               local_stream[i]->data_read = 0;
+                       }
+               }
+       }
+       /* All is OK */
+       err = 0;
+end:
+       DBG("polling thread exiting");
+       free(pollfd);
+       free(local_stream);
+
+       /*
+        * Close the write side of the pipe so epoll_wait() in
+        * consumer_thread_metadata_poll can catch it. The thread is monitoring the
+        * read side of the pipe. If we close them both, epoll_wait strangely does
+        * not return and could create a endless wait period if the pipe is the
+        * only tracked fd in the poll set. The thread will take care of closing
+        * the read side.
+        */
+       (void) lttng_pipe_write_close(ctx->consumer_metadata_pipe);
+
+error_testpoint:
+       if (err) {
+               health_error();
+               ERR("Health error occurred in %s", __func__);
+       }
+       health_unregister(health_consumerd);
+
+       rcu_unregister_thread();
+       return NULL;
+}
+
+/*
+ * Close wake-up end of each stream belonging to the channel. This will
+ * allow the poll() on the stream read-side to detect when the
+ * write-side (application) finally closes them.
+ */
+static
+void consumer_close_channel_streams(struct lttng_consumer_channel *channel)
+{
+       struct lttng_ht *ht;
+       struct lttng_consumer_stream *stream;
+       struct lttng_ht_iter iter;
+
+       ht = consumer_data.stream_per_chan_id_ht;
+
+       rcu_read_lock();
+       cds_lfht_for_each_entry_duplicate(ht->ht,
+                       ht->hash_fct(&channel->key, lttng_ht_seed),
+                       ht->match_fct, &channel->key,
+                       &iter.iter, stream, node_channel_id.node) {
+               /*
+                * Protect against teardown with mutex.
+                */
+               pthread_mutex_lock(&stream->lock);
+               if (cds_lfht_is_node_deleted(&stream->node.node)) {
+                       goto next;
+               }
+               switch (consumer_data.type) {
+               case LTTNG_CONSUMER_KERNEL:
+                       break;
+               case LTTNG_CONSUMER32_UST:
+               case LTTNG_CONSUMER64_UST:
+                       if (stream->metadata_flag) {
+                               /* Safe and protected by the stream lock. */
+                               lttng_ustconsumer_close_metadata(stream->chan);
+                       } else {
+                               /*
+                                * Note: a mutex is taken internally within
+                                * liblttng-ust-ctl to protect timer wakeup_fd
+                                * use from concurrent close.
+                                */
+                               lttng_ustconsumer_close_stream_wakeup(stream);
+                       }
+                       break;
+               default:
+                       ERR("Unknown consumer_data type");
+                       assert(0);
+               }
+       next:
+               pthread_mutex_unlock(&stream->lock);
+       }
+       rcu_read_unlock();
+}
+
+static void destroy_channel_ht(struct lttng_ht *ht)
+{
+       struct lttng_ht_iter iter;
+       struct lttng_consumer_channel *channel;
+       int ret;
+
+       if (ht == NULL) {
+               return;
+       }
+
+       rcu_read_lock();
+       cds_lfht_for_each_entry(ht->ht, &iter.iter, channel, wait_fd_node.node) {
+               ret = lttng_ht_del(ht, &iter);
+               assert(ret != 0);
+       }
+       rcu_read_unlock();
+
+       lttng_ht_destroy(ht);
+}
+
+/*
+ * This thread polls the channel fds to detect when they are being
+ * closed. It closes all related streams if the channel is detected as
+ * closed. It is currently only used as a shim layer for UST because the
+ * consumerd needs to keep the per-stream wakeup end of pipes open for
+ * periodical flush.
+ */
+void *consumer_thread_channel_poll(void *data)
+{
+       int ret, i, pollfd, err = -1;
+       uint32_t revents, nb_fd;
+       struct lttng_consumer_channel *chan = NULL;
+       struct lttng_ht_iter iter;
+       struct lttng_ht_node_u64 *node;
+       struct lttng_poll_event events;
+       struct lttng_consumer_local_data *ctx = data;
+       struct lttng_ht *channel_ht;
+
+       rcu_register_thread();
+
+       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_CHANNEL);
+
+       if (testpoint(consumerd_thread_channel)) {
+               goto error_testpoint;
+       }
+
+       health_code_update();
+
+       channel_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
+       if (!channel_ht) {
+               /* ENOMEM at this point. Better to bail out. */
+               goto end_ht;
+       }
+
+       DBG("Thread channel poll started");
+
+       /* Size is set to 1 for the consumer_channel pipe */
+       ret = lttng_poll_create(&events, 2, LTTNG_CLOEXEC);
+       if (ret < 0) {
+               ERR("Poll set creation failed");
+               goto end_poll;
+       }
+
+       ret = lttng_poll_add(&events, ctx->consumer_channel_pipe[0], LPOLLIN);
+       if (ret < 0) {
+               goto end;
+       }
+
+       /* Main loop */
+       DBG("Channel main loop started");
+
+       while (1) {
+restart:
+               health_code_update();
+               DBG("Channel poll wait");
+               health_poll_entry();
+               ret = lttng_poll_wait(&events, -1);
+               DBG("Channel poll return from wait with %d fd(s)",
+                               LTTNG_POLL_GETNB(&events));
+               health_poll_exit();
+               DBG("Channel event catched in thread");
+               if (ret < 0) {
+                       if (errno == EINTR) {
+                               ERR("Poll EINTR catched");
+                               goto restart;
+                       }
+                       if (LTTNG_POLL_GETNB(&events) == 0) {
+                               err = 0;        /* All is OK */
+                       }
+                       goto end;
+               }
+
+               nb_fd = ret;
+
+               /* From here, the event is a channel wait fd */
+               for (i = 0; i < nb_fd; i++) {
+                       health_code_update();
+
+                       revents = LTTNG_POLL_GETEV(&events, i);
+                       pollfd = LTTNG_POLL_GETFD(&events, i);
+
+                       if (!revents) {
+                               /* No activity for this FD (poll implementation). */
+                               continue;
+                       }
+
+                       if (pollfd == ctx->consumer_channel_pipe[0]) {
+                               if (revents & LPOLLIN) {
+                                       enum consumer_channel_action action;
+                                       uint64_t key;
+
+                                       ret = read_channel_pipe(ctx, &chan, &key, &action);
+                                       if (ret <= 0) {
+                                               if (ret < 0) {
+                                                       ERR("Error reading channel pipe");
+                                               }
+                                               lttng_poll_del(&events, ctx->consumer_channel_pipe[0]);
+                                               continue;
+                                       }
+
+                                       switch (action) {
+                                       case CONSUMER_CHANNEL_ADD:
+                                               DBG("Adding channel %d to poll set",
+                                                       chan->wait_fd);
+
+                                               lttng_ht_node_init_u64(&chan->wait_fd_node,
+                                                       chan->wait_fd);
+                                               rcu_read_lock();
+                                               lttng_ht_add_unique_u64(channel_ht,
+                                                               &chan->wait_fd_node);
+                                               rcu_read_unlock();
+                                               /* Add channel to the global poll events list */
+                                               lttng_poll_add(&events, chan->wait_fd,
+                                                               LPOLLERR | LPOLLHUP);
+                                               break;
+                                       case CONSUMER_CHANNEL_DEL:
+                                       {
+                                               /*
+                                                * This command should never be called if the channel
+                                                * has streams monitored by either the data or metadata
+                                                * thread. The consumer only notify this thread with a
+                                                * channel del. command if it receives a destroy
+                                                * channel command from the session daemon that send it
+                                                * if a command prior to the GET_CHANNEL failed.
+                                                */
+
+                                               rcu_read_lock();
+                                               chan = consumer_find_channel(key);
+                                               if (!chan) {
+                                                       rcu_read_unlock();
+                                                       ERR("UST consumer get channel key %" PRIu64 " not found for del channel", key);
+                                                       break;
+                                               }
+                                               lttng_poll_del(&events, chan->wait_fd);
+                                               iter.iter.node = &chan->wait_fd_node.node;
+                                               ret = lttng_ht_del(channel_ht, &iter);
+                                               assert(ret == 0);
+
+                                               switch (consumer_data.type) {
+                                               case LTTNG_CONSUMER_KERNEL:
+                                                       break;
+                                               case LTTNG_CONSUMER32_UST:
+                                               case LTTNG_CONSUMER64_UST:
+                                                       health_code_update();
+                                                       /* Destroy streams that might have been left in the stream list. */
+                                                       clean_channel_stream_list(chan);
+                                                       break;
+                                               default:
+                                                       ERR("Unknown consumer_data type");
+                                                       assert(0);
+                                               }
+
+                                               /*
+                                                * Release our own refcount. Force channel deletion even if
+                                                * streams were not initialized.
+                                                */
+                                               if (!uatomic_sub_return(&chan->refcount, 1)) {
+                                                       consumer_del_channel(chan);
+                                               }
+                                               rcu_read_unlock();
+                                               goto restart;
+                                       }
+                                       case CONSUMER_CHANNEL_QUIT:
+                                               /*
+                                                * Remove the pipe from the poll set and continue the loop
+                                                * since their might be data to consume.
+                                                */
+                                               lttng_poll_del(&events, ctx->consumer_channel_pipe[0]);
+                                               continue;
+                                       default:
+                                               ERR("Unknown action");
+                                               break;
+                                       }
+                               } else if (revents & (LPOLLERR | LPOLLHUP)) {
+                                       DBG("Channel thread pipe hung up");
+                                       /*
+                                        * Remove the pipe from the poll set and continue the loop
+                                        * since their might be data to consume.
+                                        */
+                                       lttng_poll_del(&events, ctx->consumer_channel_pipe[0]);
+                                       continue;
+                               } else {
+                                       ERR("Unexpected poll events %u for sock %d", revents, pollfd);
+                                       goto end;
+                               }
+
+                               /* Handle other stream */
+                               continue;
+                       }
+
+                       rcu_read_lock();
+                       {
+                               uint64_t tmp_id = (uint64_t) pollfd;
+
+                               lttng_ht_lookup(channel_ht, &tmp_id, &iter);
+                       }
+                       node = lttng_ht_iter_get_node_u64(&iter);
+                       assert(node);
+
+                       chan = caa_container_of(node, struct lttng_consumer_channel,
+                                       wait_fd_node);
+
+                       /* Check for error event */
+                       if (revents & (LPOLLERR | LPOLLHUP)) {
+                               DBG("Channel fd %d is hup|err.", pollfd);
+
+                               lttng_poll_del(&events, chan->wait_fd);
+                               ret = lttng_ht_del(channel_ht, &iter);
+                               assert(ret == 0);
+
+                               /*
+                                * This will close the wait fd for each stream associated to
+                                * this channel AND monitored by the data/metadata thread thus
+                                * will be clean by the right thread.
+                                */
+                               consumer_close_channel_streams(chan);
+
+                               /* Release our own refcount */
+                               if (!uatomic_sub_return(&chan->refcount, 1)
+                                               && !uatomic_read(&chan->nb_init_stream_left)) {
+                                       consumer_del_channel(chan);
+                               }
+                       } else {
+                               ERR("Unexpected poll events %u for sock %d", revents, pollfd);
+                               rcu_read_unlock();
+                               goto end;
+                       }
+
+                       /* Release RCU lock for the channel looked up */
+                       rcu_read_unlock();
+               }
+       }
+
+       /* All is OK */
+       err = 0;
+end:
+       lttng_poll_clean(&events);
+end_poll:
+       destroy_channel_ht(channel_ht);
+end_ht:
+error_testpoint:
+       DBG("Channel poll thread exiting");
+       if (err) {
+               health_error();
+               ERR("Health error occurred in %s", __func__);
+       }
+       health_unregister(health_consumerd);
+       rcu_unregister_thread();
+       return NULL;
+}
+
+static int set_metadata_socket(struct lttng_consumer_local_data *ctx,
+               struct pollfd *sockpoll, int client_socket)
+{
+       int ret;
+
+       assert(ctx);
+       assert(sockpoll);
+
+       ret = lttng_consumer_poll_socket(sockpoll);
+       if (ret) {
+               goto error;
+       }
+       DBG("Metadata connection on client_socket");
+
+       /* Blocking call, waiting for transmission */
+       ctx->consumer_metadata_socket = lttcomm_accept_unix_sock(client_socket);
+       if (ctx->consumer_metadata_socket < 0) {
+               WARN("On accept metadata");
+               ret = -1;
+               goto error;
+       }
+       ret = 0;
+
+error:
+       return ret;
+}
+
+/*
+ * This thread listens on the consumerd socket and receives the file
+ * descriptors from the session daemon.
+ */
+void *consumer_thread_sessiond_poll(void *data)
+{
+       int sock = -1, client_socket, ret, err = -1;
+       /*
+        * structure to poll for incoming data on communication socket avoids
+        * making blocking sockets.
+        */
+       struct pollfd consumer_sockpoll[2];
+       struct lttng_consumer_local_data *ctx = data;
+
+       rcu_register_thread();
+
+       health_register(health_consumerd, HEALTH_CONSUMERD_TYPE_SESSIOND);
+
+       if (testpoint(consumerd_thread_sessiond)) {
+               goto error_testpoint;
+       }
+
+       health_code_update();
+
+       DBG("Creating command socket %s", ctx->consumer_command_sock_path);
+       unlink(ctx->consumer_command_sock_path);
+       client_socket = lttcomm_create_unix_sock(ctx->consumer_command_sock_path);
+       if (client_socket < 0) {
+               ERR("Cannot create command socket");
+               goto end;
+       }
+
+       ret = lttcomm_listen_unix_sock(client_socket);
+       if (ret < 0) {
+               goto end;
+       }
+
+       DBG("Sending ready command to lttng-sessiond");
+       ret = lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_COMMAND_SOCK_READY);
+       /* return < 0 on error, but == 0 is not fatal */
+       if (ret < 0) {
+               ERR("Error sending ready command to lttng-sessiond");
+               goto end;
+       }
+
+       /* prepare the FDs to poll : to client socket and the should_quit pipe */
+       consumer_sockpoll[0].fd = ctx->consumer_should_quit[0];
+       consumer_sockpoll[0].events = POLLIN | POLLPRI;
+       consumer_sockpoll[1].fd = client_socket;
+       consumer_sockpoll[1].events = POLLIN | POLLPRI;
+
+       ret = lttng_consumer_poll_socket(consumer_sockpoll);
+       if (ret) {
+               if (ret > 0) {
+                       /* should exit */
+                       err = 0;
+               }
+               goto end;
+       }
+       DBG("Connection on client_socket");
+
+       /* Blocking call, waiting for transmission */
+       sock = lttcomm_accept_unix_sock(client_socket);
+       if (sock < 0) {
+               WARN("On accept");
+               goto end;
+       }
+
+       /*
+        * Setup metadata socket which is the second socket connection on the
+        * command unix socket.
+        */
+       ret = set_metadata_socket(ctx, consumer_sockpoll, client_socket);
+       if (ret) {
+               if (ret > 0) {
+                       /* should exit */
+                       err = 0;
+               }
+               goto end;
+       }
+
+       /* This socket is not useful anymore. */
+       ret = close(client_socket);
+       if (ret < 0) {
+               PERROR("close client_socket");
+       }
+       client_socket = -1;
+
+       /* update the polling structure to poll on the established socket */
+       consumer_sockpoll[1].fd = sock;
+       consumer_sockpoll[1].events = POLLIN | POLLPRI;
+
+       while (1) {
+               health_code_update();
+
+               health_poll_entry();
+               ret = lttng_consumer_poll_socket(consumer_sockpoll);
+               health_poll_exit();
+               if (ret) {
+                       if (ret > 0) {
+                               /* should exit */
+                               err = 0;
+                       }
+                       goto end;
+               }
+               DBG("Incoming command on sock");
+               ret = lttng_consumer_recv_cmd(ctx, sock, consumer_sockpoll);
+               if (ret <= 0) {
+                       /*
+                        * This could simply be a session daemon quitting. Don't output
+                        * ERR() here.
+                        */
+                       DBG("Communication interrupted on command socket");
+                       err = 0;
+                       goto end;
+               }
+               if (consumer_quit) {
+                       DBG("consumer_thread_receive_fds received quit from signal");
+                       err = 0;        /* All is OK */
+                       goto end;
+               }
+               DBG("received command on sock");
+       }
+       /* All is OK */
+       err = 0;
+
+end:
+       DBG("Consumer thread sessiond poll exiting");
+
+       /*
+        * Close metadata streams since the producer is the session daemon which
+        * just died.
+        *
+        * NOTE: for now, this only applies to the UST tracer.
+        */
+       lttng_consumer_close_all_metadata();
+
+       /*
+        * when all fds have hung up, the polling thread
+        * can exit cleanly
+        */
+       consumer_quit = 1;
+
+       /*
+        * Notify the data poll thread to poll back again and test the
+        * consumer_quit state that we just set so to quit gracefully.
+        */
+       notify_thread_lttng_pipe(ctx->consumer_data_pipe);
+
+       notify_channel_pipe(ctx, NULL, -1, CONSUMER_CHANNEL_QUIT);
+
+       notify_health_quit_pipe(health_quit_pipe);
+
+       /* Cleaning up possibly open sockets. */
+       if (sock >= 0) {
+               ret = close(sock);
+               if (ret < 0) {
+                       PERROR("close sock sessiond poll");
+               }
+       }
+       if (client_socket >= 0) {
+               ret = close(client_socket);
+               if (ret < 0) {
+                       PERROR("close client_socket sessiond poll");
+               }
+       }
+
+error_testpoint:
+       if (err) {
+               health_error();
+               ERR("Health error occurred in %s", __func__);
+       }
+       health_unregister(health_consumerd);
+
+       rcu_unregister_thread();
+       return NULL;
+}
+
+ssize_t lttng_consumer_read_subbuffer(struct lttng_consumer_stream *stream,
+               struct lttng_consumer_local_data *ctx)
+{
+       ssize_t ret;
+
+       pthread_mutex_lock(&stream->lock);
+       if (stream->metadata_flag) {
+               pthread_mutex_lock(&stream->metadata_rdv_lock);
+       }
+
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               ret = lttng_kconsumer_read_subbuffer(stream, ctx);
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               ret = lttng_ustconsumer_read_subbuffer(stream, ctx);
+               break;
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+               ret = -ENOSYS;
+               break;
+       }
+
+       if (stream->metadata_flag) {
+               pthread_cond_broadcast(&stream->metadata_rdv);
+               pthread_mutex_unlock(&stream->metadata_rdv_lock);
+       }
+       pthread_mutex_unlock(&stream->lock);
+       return ret;
+}
+
+int lttng_consumer_on_recv_stream(struct lttng_consumer_stream *stream)
+{
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               return lttng_kconsumer_on_recv_stream(stream);
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               return lttng_ustconsumer_on_recv_stream(stream);
+       default:
+               ERR("Unknown consumer_data type");
+               assert(0);
+               return -ENOSYS;
+       }
+}
+
+/*
+ * Allocate and set consumer data hash tables.
+ */
+int lttng_consumer_init(void)
+{
+       consumer_data.channel_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
+       if (!consumer_data.channel_ht) {
+               goto error;
+       }
+
+       consumer_data.relayd_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
+       if (!consumer_data.relayd_ht) {
+               goto error;
+       }
+
+       consumer_data.stream_list_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
+       if (!consumer_data.stream_list_ht) {
+               goto error;
+       }
+
+       consumer_data.stream_per_chan_id_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
+       if (!consumer_data.stream_per_chan_id_ht) {
+               goto error;
+       }
+
+       data_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
+       if (!data_ht) {
+               goto error;
+       }
+
+       metadata_ht = lttng_ht_new(0, LTTNG_HT_TYPE_U64);
+       if (!metadata_ht) {
+               goto error;
+       }
+
+       return 0;
+
+error:
+       return -1;
+}
+
+/*
+ * Process the ADD_RELAYD command receive by a consumer.
+ *
+ * This will create a relayd socket pair and add it to the relayd hash table.
+ * The caller MUST acquire a RCU read side lock before calling it.
+ */
+int consumer_add_relayd_socket(uint64_t net_seq_idx, int sock_type,
+               struct lttng_consumer_local_data *ctx, int sock,
+               struct pollfd *consumer_sockpoll,
+               struct lttcomm_relayd_sock *relayd_sock, uint64_t sessiond_id,
+               uint64_t relayd_session_id)
+{
+       int fd = -1, ret = -1, relayd_created = 0;
+       enum lttcomm_return_code ret_code = LTTCOMM_CONSUMERD_SUCCESS;
+       struct consumer_relayd_sock_pair *relayd = NULL;
+
+       assert(ctx);
+       assert(relayd_sock);
+
+       DBG("Consumer adding relayd socket (idx: %" PRIu64 ")", net_seq_idx);
+
+       /* Get relayd reference if exists. */
+       relayd = consumer_find_relayd(net_seq_idx);
+       if (relayd == NULL) {
+               assert(sock_type == LTTNG_STREAM_CONTROL);
+               /* Not found. Allocate one. */
+               relayd = consumer_allocate_relayd_sock_pair(net_seq_idx);
+               if (relayd == NULL) {
+                       ret = -ENOMEM;
+                       ret_code = LTTCOMM_CONSUMERD_ENOMEM;
+                       goto error;
+               } else {
+                       relayd->sessiond_session_id = sessiond_id;
+                       relayd_created = 1;
+               }
+
+               /*
+                * This code path MUST continue to the consumer send status message to
+                * we can notify the session daemon and continue our work without
+                * killing everything.
+                */
+       } else {
+               /*
+                * relayd key should never be found for control socket.
+                */
+               assert(sock_type != LTTNG_STREAM_CONTROL);
+       }
+
+       /* First send a status message before receiving the fds. */
+       ret = consumer_send_status_msg(sock, LTTCOMM_CONSUMERD_SUCCESS);
+       if (ret < 0) {
+               /* Somehow, the session daemon is not responding anymore. */
+               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_FATAL);
+               goto error_nosignal;
+       }
+
+       /* Poll on consumer socket. */
+       ret = lttng_consumer_poll_socket(consumer_sockpoll);
+       if (ret) {
+               /* Needing to exit in the middle of a command: error. */
+               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_POLL_ERROR);
+               ret = -EINTR;
+               goto error_nosignal;
+       }
+
+       /* Get relayd socket from session daemon */
+       ret = lttcomm_recv_fds_unix_sock(sock, &fd, 1);
+       if (ret != sizeof(fd)) {
+               ret = -1;
+               fd = -1;        /* Just in case it gets set with an invalid value. */
+
+               /*
+                * Failing to receive FDs might indicate a major problem such as
+                * reaching a fd limit during the receive where the kernel returns a
+                * MSG_CTRUNC and fails to cleanup the fd in the queue. Any case, we
+                * don't take any chances and stop everything.
+                *
+                * XXX: Feature request #558 will fix that and avoid this possible
+                * issue when reaching the fd limit.
+                */
+               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_ERROR_RECV_FD);
+               ret_code = LTTCOMM_CONSUMERD_ERROR_RECV_FD;
+               goto error;
+       }
+
+       /* Copy socket information and received FD */
+       switch (sock_type) {
+       case LTTNG_STREAM_CONTROL:
+               /* Copy received lttcomm socket */
+               lttcomm_copy_sock(&relayd->control_sock.sock, &relayd_sock->sock);
+               ret = lttcomm_create_sock(&relayd->control_sock.sock);
+               /* Handle create_sock error. */
+               if (ret < 0) {
+                       ret_code = LTTCOMM_CONSUMERD_ENOMEM;
+                       goto error;
+               }
+               /*
+                * Close the socket created internally by
+                * lttcomm_create_sock, so we can replace it by the one
+                * received from sessiond.
+                */
+               if (close(relayd->control_sock.sock.fd)) {
+                       PERROR("close");
+               }
+
+               /* Assign new file descriptor */
+               relayd->control_sock.sock.fd = fd;
+               fd = -1;        /* For error path */
+               /* Assign version values. */
+               relayd->control_sock.major = relayd_sock->major;
+               relayd->control_sock.minor = relayd_sock->minor;
+
+               relayd->relayd_session_id = relayd_session_id;
+
+               break;
+       case LTTNG_STREAM_DATA:
+               /* Copy received lttcomm socket */
+               lttcomm_copy_sock(&relayd->data_sock.sock, &relayd_sock->sock);
+               ret = lttcomm_create_sock(&relayd->data_sock.sock);
+               /* Handle create_sock error. */
+               if (ret < 0) {
+                       ret_code = LTTCOMM_CONSUMERD_ENOMEM;
+                       goto error;
+               }
+               /*
+                * Close the socket created internally by
+                * lttcomm_create_sock, so we can replace it by the one
+                * received from sessiond.
+                */
+               if (close(relayd->data_sock.sock.fd)) {
+                       PERROR("close");
+               }
+
+               /* Assign new file descriptor */
+               relayd->data_sock.sock.fd = fd;
+               fd = -1;        /* for eventual error paths */
+               /* Assign version values. */
+               relayd->data_sock.major = relayd_sock->major;
+               relayd->data_sock.minor = relayd_sock->minor;
+               break;
+       default:
+               ERR("Unknown relayd socket type (%d)", sock_type);
+               ret = -1;
+               ret_code = LTTCOMM_CONSUMERD_FATAL;
+               goto error;
+       }
+
+       DBG("Consumer %s socket created successfully with net idx %" PRIu64 " (fd: %d)",
+                       sock_type == LTTNG_STREAM_CONTROL ? "control" : "data",
+                       relayd->net_seq_idx, fd);
+
+       /* We successfully added the socket. Send status back. */
+       ret = consumer_send_status_msg(sock, ret_code);
+       if (ret < 0) {
+               /* Somehow, the session daemon is not responding anymore. */
+               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_FATAL);
+               goto error_nosignal;
+       }
+
+       /*
+        * Add relayd socket pair to consumer data hashtable. If object already
+        * exists or on error, the function gracefully returns.
+        */
+       add_relayd(relayd);
+
+       /* All good! */
+       return 0;
+
+error:
+       if (consumer_send_status_msg(sock, ret_code) < 0) {
+               lttng_consumer_send_error(ctx, LTTCOMM_CONSUMERD_FATAL);
+       }
+
+error_nosignal:
+       /* Close received socket if valid. */
+       if (fd >= 0) {
+               if (close(fd)) {
+                       PERROR("close received socket");
+               }
+       }
+
+       if (relayd_created) {
+               free(relayd);
+       }
+
+       return ret;
+}
+
+/*
+ * Try to lock the stream mutex.
+ *
+ * On success, 1 is returned else 0 indicating that the mutex is NOT lock.
+ */
+static int stream_try_lock(struct lttng_consumer_stream *stream)
+{
+       int ret;
+
+       assert(stream);
+
+       /*
+        * Try to lock the stream mutex. On failure, we know that the stream is
+        * being used else where hence there is data still being extracted.
+        */
+       ret = pthread_mutex_trylock(&stream->lock);
+       if (ret) {
+               /* For both EBUSY and EINVAL error, the mutex is NOT locked. */
+               ret = 0;
+               goto end;
+       }
+
+       ret = 1;
+
+end:
+       return ret;
+}
+
+/*
+ * Search for a relayd associated to the session id and return the reference.
+ *
+ * A rcu read side lock MUST be acquire before calling this function and locked
+ * until the relayd object is no longer necessary.
+ */
+static struct consumer_relayd_sock_pair *find_relayd_by_session_id(uint64_t id)
+{
+       struct lttng_ht_iter iter;
+       struct consumer_relayd_sock_pair *relayd = NULL;
+
+       /* Iterate over all relayd since they are indexed by net_seq_idx. */
+       cds_lfht_for_each_entry(consumer_data.relayd_ht->ht, &iter.iter, relayd,
+                       node.node) {
+               /*
+                * Check by sessiond id which is unique here where the relayd session
+                * id might not be when having multiple relayd.
+                */
+               if (relayd->sessiond_session_id == id) {
+                       /* Found the relayd. There can be only one per id. */
+                       goto found;
+               }
+       }
+
+       return NULL;
+
+found:
+       return relayd;
+}
+
+/*
+ * Check if for a given session id there is still data needed to be extract
+ * from the buffers.
+ *
+ * Return 1 if data is pending or else 0 meaning ready to be read.
+ */
+int consumer_data_pending(uint64_t id)
+{
+       int ret;
+       struct lttng_ht_iter iter;
+       struct lttng_ht *ht;
+       struct lttng_consumer_stream *stream;
+       struct consumer_relayd_sock_pair *relayd = NULL;
+       int (*data_pending)(struct lttng_consumer_stream *);
+
+       DBG("Consumer data pending command on session id %" PRIu64, id);
+
+       rcu_read_lock();
+       pthread_mutex_lock(&consumer_data.lock);
+
+       switch (consumer_data.type) {
+       case LTTNG_CONSUMER_KERNEL:
+               data_pending = lttng_kconsumer_data_pending;
+               break;
+       case LTTNG_CONSUMER32_UST:
+       case LTTNG_CONSUMER64_UST:
+               data_pending = lttng_ustconsumer_data_pending;
+               break;
+       default:
+               ERR("Unknown consumer data type");
+               assert(0);
+       }
+
+       /* Ease our life a bit */
+       ht = consumer_data.stream_list_ht;
+
+       relayd = find_relayd_by_session_id(id);
+       if (relayd) {
+               /* Send init command for data pending. */
+               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+               ret = relayd_begin_data_pending(&relayd->control_sock,
+                               relayd->relayd_session_id);
+               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+               if (ret < 0) {
+                       /* Communication error thus the relayd so no data pending. */
+                       goto data_not_pending;
+               }
+       }
+
+       cds_lfht_for_each_entry_duplicate(ht->ht,
+                       ht->hash_fct(&id, lttng_ht_seed),
+                       ht->match_fct, &id,
+                       &iter.iter, stream, node_session_id.node) {
+               /* If this call fails, the stream is being used hence data pending. */
+               ret = stream_try_lock(stream);
+               if (!ret) {
+                       goto data_pending;
+               }
+
+               /*
+                * A removed node from the hash table indicates that the stream has
+                * been deleted thus having a guarantee that the buffers are closed
+                * on the consumer side. However, data can still be transmitted
+                * over the network so don't skip the relayd check.
+                */
+               ret = cds_lfht_is_node_deleted(&stream->node.node);
+               if (!ret) {
+                       /* Check the stream if there is data in the buffers. */
+                       ret = data_pending(stream);
+                       if (ret == 1) {
+                               pthread_mutex_unlock(&stream->lock);
+                               goto data_pending;
+                       }
+               }
+
+               /* Relayd check */
+               if (relayd) {
+                       pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+                       if (stream->metadata_flag) {
+                               ret = relayd_quiescent_control(&relayd->control_sock,
+                                               stream->relayd_stream_id);
+                       } else {
+                               ret = relayd_data_pending(&relayd->control_sock,
+                                               stream->relayd_stream_id,
+                                               stream->next_net_seq_num - 1);
+                       }
+                       pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+                       if (ret == 1) {
+                               pthread_mutex_unlock(&stream->lock);
+                               goto data_pending;
+                       }
+               }
+               pthread_mutex_unlock(&stream->lock);
+       }
+
+       if (relayd) {
+               unsigned int is_data_inflight = 0;
+
+               /* Send init command for data pending. */
+               pthread_mutex_lock(&relayd->ctrl_sock_mutex);
+               ret = relayd_end_data_pending(&relayd->control_sock,
+                               relayd->relayd_session_id, &is_data_inflight);
+               pthread_mutex_unlock(&relayd->ctrl_sock_mutex);
+               if (ret < 0) {
+                       goto data_not_pending;
+               }
+               if (is_data_inflight) {
+                       goto data_pending;
+               }
+       }
+
+       /*
+        * Finding _no_ node in the hash table and no inflight data means that the
+        * stream(s) have been removed thus data is guaranteed to be available for
+        * analysis from the trace files.
+        */
+
+data_not_pending:
+       /* Data is available to be read by a viewer. */
+       pthread_mutex_unlock(&consumer_data.lock);
+       rcu_read_unlock();
+       return 0;
+
+data_pending:
+       /* Data is still being extracted from buffers. */
+       pthread_mutex_unlock(&consumer_data.lock);
+       rcu_read_unlock();
+       return 1;
+}
+
+/*
+ * Send a ret code status message to the sessiond daemon.
+ *
+ * Return the sendmsg() return value.
+ */
+int consumer_send_status_msg(int sock, int ret_code)
+{
+       struct lttcomm_consumer_status_msg msg;
+
+       memset(&msg, 0, sizeof(msg));
+       msg.ret_code = ret_code;
+
+       return lttcomm_send_unix_sock(sock, &msg, sizeof(msg));
+}
+
+/*
+ * Send a channel status message to the sessiond daemon.
+ *
+ * Return the sendmsg() return value.
+ */
+int consumer_send_status_channel(int sock,
+               struct lttng_consumer_channel *channel)
+{
+       struct lttcomm_consumer_status_channel msg;
+
+       assert(sock >= 0);
+
+       memset(&msg, 0, sizeof(msg));
+       if (!channel) {
+               msg.ret_code = LTTCOMM_CONSUMERD_CHANNEL_FAIL;
+       } else {
+               msg.ret_code = LTTCOMM_CONSUMERD_SUCCESS;
+               msg.key = channel->key;
+               msg.stream_count = channel->streams.count;
+       }
+
+       return lttcomm_send_unix_sock(sock, &msg, sizeof(msg));
+}
+
+unsigned long consumer_get_consume_start_pos(unsigned long consumed_pos,
+               unsigned long produced_pos, uint64_t nb_packets_per_stream,
+               uint64_t max_sb_size)
+{
+       unsigned long start_pos;
+
+       if (!nb_packets_per_stream) {
+               return consumed_pos;    /* Grab everything */
+       }
+       start_pos = produced_pos - offset_align_floor(produced_pos, max_sb_size);
+       start_pos -= max_sb_size * nb_packets_per_stream;
+       if ((long) (start_pos - consumed_pos) < 0) {
+               return consumed_pos;    /* Grab everything */
+       }
+       return start_pos;
+}
diff --git a/src/common/consumer/consumer.h b/src/common/consumer/consumer.h
new file mode 100644 (file)
index 0000000..c7ef3fb
--- /dev/null
@@ -0,0 +1,698 @@
+/*
+ * Copyright (C) 2011 - Julien Desfossez <julien.desfossez@polymtl.ca>
+ *                      Mathieu Desnoyers <mathieu.desnoyers@efficios.com>
+ *               2012 - David Goulet <dgoulet@efficios.com>
+ *
+ * This program is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU General Public License, version 2 only,
+ * as published by the Free Software Foundation.
+ *
+ * This program 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 General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along
+ * with this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+
+#ifndef LIB_CONSUMER_H
+#define LIB_CONSUMER_H
+
+#include <limits.h>
+#include <poll.h>
+#include <unistd.h>
+#include <urcu/list.h>
+
+#include <lttng/lttng.h>
+
+#include <common/hashtable/hashtable.h>
+#include <common/compat/fcntl.h>
+#include <common/compat/uuid.h>
+#include <common/sessiond-comm/sessiond-comm.h>
+#include <common/pipe.h>
+#include <common/index/ctf-index.h>
+
+/* Commands for consumer */
+enum lttng_consumer_command {
+       LTTNG_CONSUMER_ADD_CHANNEL,
+       LTTNG_CONSUMER_ADD_STREAM,
+       /* pause, delete, active depending on fd state */
+       LTTNG_CONSUMER_UPDATE_STREAM,
+       /* inform the consumer to quit when all fd has hang up */
+       LTTNG_CONSUMER_STOP,    /* deprecated */
+       LTTNG_CONSUMER_ADD_RELAYD_SOCKET,
+       /* Inform the consumer to kill a specific relayd connection */
+       LTTNG_CONSUMER_DESTROY_RELAYD,
+       /* Return to the sessiond if there is data pending for a session */
+       LTTNG_CONSUMER_DATA_PENDING,
+       /* Consumer creates a channel and returns it to sessiond. */
+       LTTNG_CONSUMER_ASK_CHANNEL_CREATION,
+       LTTNG_CONSUMER_GET_CHANNEL,
+       LTTNG_CONSUMER_DESTROY_CHANNEL,
+       LTTNG_CONSUMER_PUSH_METADATA,
+       LTTNG_CONSUMER_CLOSE_METADATA,
+       LTTNG_CONSUMER_SETUP_METADATA,
+       LTTNG_CONSUMER_FLUSH_CHANNEL,
+       LTTNG_CONSUMER_SNAPSHOT_CHANNEL,
+       LTTNG_CONSUMER_SNAPSHOT_METADATA,
+       LTTNG_CONSUMER_STREAMS_SENT,
+};
+
+/* State of each fd in consumer */
+enum lttng_consumer_stream_state {
+       LTTNG_CONSUMER_ACTIVE_STREAM,
+       LTTNG_CONSUMER_PAUSE_STREAM,
+       LTTNG_CONSUMER_DELETE_STREAM,
+};
+
+enum lttng_consumer_type {
+       LTTNG_CONSUMER_UNKNOWN = 0,
+       LTTNG_CONSUMER_KERNEL,
+       LTTNG_CONSUMER64_UST,
+       LTTNG_CONSUMER32_UST,
+};
+
+enum consumer_endpoint_status {
+       CONSUMER_ENDPOINT_ACTIVE,
+       CONSUMER_ENDPOINT_INACTIVE,
+};
+
+enum consumer_channel_output {
+       CONSUMER_CHANNEL_MMAP   = 0,
+       CONSUMER_CHANNEL_SPLICE = 1,
+};
+
+enum consumer_channel_type {
+       CONSUMER_CHANNEL_TYPE_METADATA  = 0,
+       CONSUMER_CHANNEL_TYPE_DATA      = 1,
+};
+
+extern struct lttng_consumer_global_data consumer_data;
+
+struct stream_list {
+       struct cds_list_head head;
+       unsigned int count;
+};
+
+/* Stub. */
+struct consumer_metadata_cache;
+
+struct lttng_consumer_channel {
+       /* HT node used for consumer_data.channel_ht */
+       struct lttng_ht_node_u64 node;
+       /* Indexed key. Incremented value in the consumer. */
+       uint64_t key;
+       /* Number of streams referencing this channel */
+       int refcount;
+       /* Tracing session id on the session daemon side. */
+       uint64_t session_id;
+       /*
+        * Session id when requesting metadata to the session daemon for
+        * a session with per-PID buffers.
+        */
+       uint64_t session_id_per_pid;
+       /* Channel trace file path name. */
+       char pathname[PATH_MAX];
+       /* Channel name. */
+       char name[LTTNG_SYMBOL_NAME_LEN];
+       /* UID and GID of the session owning this channel. */
+       uid_t uid;
+       gid_t gid;
+       /* Relayd id of the channel. -1ULL if it does not apply. */
+       uint64_t relayd_id;
+       /*
+        * Number of streams NOT initialized yet. This is used in order to not
+        * delete this channel if streams are getting initialized.
+        */
+       unsigned int nb_init_stream_left;
+       /* Output type (mmap or splice). */
+       enum consumer_channel_output output;
+       /* Channel type for stream */
+       enum consumer_channel_type type;
+
+       /* For UST */
+       uid_t ust_app_uid;      /* Application UID. */
+       struct ustctl_consumer_channel *uchan;
+       unsigned char uuid[UUID_STR_LEN];
+       /*
+        * Temporary stream list used to store the streams once created and waiting
+        * to be sent to the session daemon by receiving the
+        * LTTNG_CONSUMER_GET_CHANNEL.
+        */
+       struct stream_list streams;
+
+       /*
+        * Set if the channel is metadata. We keep a reference to the stream
+        * because we have to flush data once pushed by the session daemon. For a
+        * regular channel, this is always set to NULL.
+        */
+       struct lttng_consumer_stream *metadata_stream;
+
+       /* for UST */
+       int wait_fd;
+       /* Node within channel thread ht */
+       struct lttng_ht_node_u64 wait_fd_node;
+
+       /* Metadata cache is metadata channel */
+       struct consumer_metadata_cache *metadata_cache;
+       /* For UST metadata periodical flush */
+       int switch_timer_enabled;
+       timer_t switch_timer;
+       int switch_timer_error;
+
+       /* For the live mode */
+       int live_timer_enabled;
+       timer_t live_timer;
+       int live_timer_error;
+
+       /* On-disk circular buffer */
+       uint64_t tracefile_size;
+       uint64_t tracefile_count;
+       /*
+        * Monitor or not the streams of this channel meaning this indicates if the
+        * streams should be sent to the data/metadata thread or added to the no
+        * monitor list of the channel.
+        */
+       unsigned int monitor;
+
+       /*
+        * Channel lock.
+        *
+        * This lock protects against concurrent update of channel.
+        *
+        * This is nested INSIDE the consumer data lock.
+        * This is nested OUTSIDE the channel timer lock.
+        * This is nested OUTSIDE the metadata cache lock.
+        * This is nested OUTSIDE stream lock.
+        * This is nested OUTSIDE consumer_relayd_sock_pair lock.
+        */
+       pthread_mutex_t lock;
+
+       /*
+        * Channel teardown lock.
+        *
+        * This lock protect against teardown of channel. It is _never_
+        * taken by the timer handler.
+        *
+        * This is nested INSIDE the consumer data lock.
+        * This is nested INSIDE the channel lock.
+        * This is nested OUTSIDE the metadata cache lock.
+        * This is nested OUTSIDE stream lock.
+        * This is nested OUTSIDE consumer_relayd_sock_pair lock.
+        */
+       pthread_mutex_t timer_lock;
+
+       /* Timer value in usec for live streaming. */
+       unsigned int live_timer_interval;
+
+       int *stream_fds;
+       int nr_stream_fds;
+       char root_shm_path[PATH_MAX];
+       char shm_path[PATH_MAX];
+};
+
+/*
+ * Internal representation of the streams, sessiond_key is used to identify
+ * uniquely a stream.
+ */
+struct lttng_consumer_stream {
+       /* HT node used by the data_ht and metadata_ht */
+       struct lttng_ht_node_u64 node;
+       /* stream indexed per channel key node */
+       struct lttng_ht_node_u64 node_channel_id;
+       /* HT node used in consumer_data.stream_list_ht */
+       struct lttng_ht_node_u64 node_session_id;
+       /* Pointer to associated channel. */
+       struct lttng_consumer_channel *chan;
+
+       /* Key by which the stream is indexed for 'node'. */
+       uint64_t key;
+       /*
+        * File descriptor of the data output file. This can be either a file or a
+        * socket fd for relayd streaming.
+        */
+       int out_fd; /* output file to write the data */
+       /* Write position in the output file descriptor */
+       off_t out_fd_offset;
+       /* Amount of bytes written to the output */
+       uint64_t output_written;
+       enum lttng_consumer_stream_state state;
+       int shm_fd_is_copy;
+       int data_read;
+       int hangup_flush_done;
+
+       /*
+        * metadata_timer_lock protects flags waiting_on_metadata and
+        * missed_metadata_flush.
+        */
+       pthread_mutex_t metadata_timer_lock;
+       /*
+        * Flag set when awaiting metadata to be pushed. Used in the
+        * timer thread to skip waiting on the stream (and stream lock) to
+        * ensure we can proceed to flushing metadata in live mode.
+        */
+       bool waiting_on_metadata;
+       /* Raised when a timer misses a metadata flush. */
+       bool missed_metadata_flush;
+
+       enum lttng_event_output output;
+       /* Maximum subbuffer size. */
+       unsigned long max_sb_size;
+
+       /*
+        * Still used by the kernel for MMAP output. For UST, the ustctl getter is
+        * used for the mmap base and offset.
+        */
+       void *mmap_base;
+       unsigned long mmap_len;
+
+       /* For UST */
+
+       int wait_fd;
+       /* UID/GID of the user owning the session to which stream belongs */
+       uid_t uid;
+       gid_t gid;
+       /* Network sequence number. Indicating on which relayd socket it goes. */
+       uint64_t net_seq_idx;
+       /*
+        * Indicate if this stream was successfully sent to a relayd. This is set
+        * after the refcount of the relayd is incremented and is checked when the
+        * stream is closed before decrementing the refcount in order to avoid an
+        * unbalanced state.
+        */
+       unsigned int sent_to_relayd;
+
+       /* Identify if the stream is the metadata */
+       unsigned int metadata_flag;
+       /* Used when the stream is set for network streaming */
+       uint64_t relayd_stream_id;
+       /*
+        * When sending a stream packet to a relayd, this number is used to track
+        * the packet sent by the consumer and seen by the relayd. When sending the
+        * data header to the relayd, this number is sent and if the transmission
+        * was successful, it is incremented.
+        *
+        * Even if the full data is not fully transmitted it won't matter since
+        * only two possible error can happen after that where either the relayd
+        * died or a read error is detected on the stream making this value useless
+        * after that.
+        *
+        * This value SHOULD be read/updated atomically or with the lock acquired.
+        */
+       uint64_t next_net_seq_num;
+       /*
+        * Lock to use the stream FDs since they are used between threads.
+        *
+        * This is nested INSIDE the consumer_data lock.
+        * This is nested INSIDE the metadata cache lock.
+        * This is nested INSIDE the channel lock.
+        * This is nested INSIDE the channel timer lock.
+        * This is nested OUTSIDE consumer_relayd_sock_pair lock.
+        */
+       pthread_mutex_t lock;
+       /* Tracing session id */
+       uint64_t session_id;
+       /*
+        * Indicates if the stream end point is still active or not (network
+        * streaming or local file system). The thread "owning" the stream is
+        * handling this status and can be notified of a state change through the
+        * consumer data appropriate pipe.
+        */
+       enum consumer_endpoint_status endpoint_status;
+       /* Stream name. Format is: <channel_name>_<cpu_number> */
+       char name[LTTNG_SYMBOL_NAME_LEN];
+       /* Internal state of libustctl. */
+       struct ustctl_consumer_stream *ustream;
+       struct cds_list_head send_node;
+       /* On-disk circular buffer */
+       uint64_t tracefile_size_current;
+       uint64_t tracefile_count_current;
+       /*
+        * Monitor or not the streams of this channel meaning this indicates if the
+        * streams should be sent to the data/metadata thread or added to the no
+        * monitor list of the channel.
+        */
+       unsigned int monitor;
+       /*
+        * Indicate if the stream is globally visible meaning that it has been
+        * added to the multiple hash tables. If *not* set, NO lock should be
+        * acquired in the destroy path.
+        */
+       unsigned int globally_visible;
+       /*
+        * Pipe to wake up the metadata poll thread when the UST metadata
+        * cache is updated.
+        */
+       int ust_metadata_poll_pipe[2];
+       /*
+        * How much metadata was read from the metadata cache and sent
+        * to the channel.
+        */
+       uint64_t ust_metadata_pushed;
+       /*
+        * FD of the index file for this stream.
+        */
+       int index_fd;
+
+       /*
+        * Local pipe to extract data when using splice.
+        */
+       int splice_pipe[2];
+
+       /*
+        * Rendez-vous point between data and metadata stream in live mode.
+        */
+       pthread_cond_t metadata_rdv;
+       pthread_mutex_t metadata_rdv_lock;
+
+       /* Indicate if the stream still has some data to be read. */
+       unsigned int has_data:1;
+};
+
+/*
+ * Internal representation of a relayd socket pair.
+ */
+struct consumer_relayd_sock_pair {
+       /* Network sequence number. */
+       uint64_t net_seq_idx;
+       /* Number of stream associated with this relayd */
+       unsigned int refcount;
+
+       /*
+        * This flag indicates whether or not we should destroy this object. The
+        * destruction should ONLY occurs when this flag is set and the refcount is
+        * set to zero.
+        */
+       unsigned int destroy_flag;
+
+       /*
+        * Mutex protecting the control socket to avoid out of order packets
+        * between threads sending data to the relayd. Since metadata data is sent
+        * over that socket, at least two sendmsg() are needed (header + data)
+        * creating a race for packets to overlap between threads using it.
+        *
+        * This is nested INSIDE the consumer_data lock.
+        * This is nested INSIDE the stream lock.
+        */
+       pthread_mutex_t ctrl_sock_mutex;
+
+       /* Control socket. Command and metadata are passed over it */
+       struct lttcomm_relayd_sock control_sock;
+
+       /*
+        * We don't need a mutex at this point since we only splice or write single
+        * large chunk of data with a header appended at the begining. Moreover,
+        * this socket is for now only used in a single thread.
+        */
+       struct lttcomm_relayd_sock data_sock;
+       struct lttng_ht_node_u64 node;
+
+       /* Session id on both sides for the sockets. */
+       uint64_t relayd_session_id;
+       uint64_t sessiond_session_id;
+};
+
+/*
+ * UST consumer local data to the program. One or more instance per
+ * process.
+ */
+struct lttng_consumer_local_data {
+       /*
+        * Function to call when data is available on a buffer.
+        * Returns the number of bytes read, or negative error value.
+        */
+       ssize_t (*on_buffer_ready)(struct lttng_consumer_stream *stream,
+                       struct lttng_consumer_local_data *ctx);
+       /*
+        * function to call when we receive a new channel, it receives a
+        * newly allocated channel, depending on the return code of this
+        * function, the new channel will be handled by the application
+        * or the library.
+        *
+        * Returns:
+        *    > 0 (success, FD is kept by application)
+        *   == 0 (success, FD is left to library)
+        *    < 0 (error)
+        */
+       int (*on_recv_channel)(struct lttng_consumer_channel *channel);
+       /*
+        * function to call when we receive a new stream, it receives a
+        * newly allocated stream, depending on the return code of this
+        * function, the new stream will be handled by the application
+        * or the library.
+        *
+        * Returns:
+        *    > 0 (success, FD is kept by application)
+        *   == 0 (success, FD is left to library)
+        *    < 0 (error)
+        */
+       int (*on_recv_stream)(struct lttng_consumer_stream *stream);
+       /*
+        * function to call when a stream is getting updated by the session
+        * daemon, this function receives the sessiond key and the new
+        * state, depending on the return code of this function the
+        * update of state for the stream is handled by the application
+        * or the library.
+        *
+        * Returns:
+        *    > 0 (success, FD is kept by application)
+        *   == 0 (success, FD is left to library)
+        *    < 0 (error)
+        */
+       int (*on_update_stream)(uint64_t sessiond_key, uint32_t state);
+       enum lttng_consumer_type type;
+       /* socket to communicate errors with sessiond */
+       int consumer_error_socket;
+       /* socket to ask metadata to sessiond. */
+       int consumer_metadata_socket;
+       /*
+        * Protect consumer_metadata_socket.
+        *
+        * This is nested OUTSIDE the metadata cache lock.
+        */
+       pthread_mutex_t metadata_socket_lock;
+       /* socket to exchange commands with sessiond */
+       char *consumer_command_sock_path;
+       /* communication with splice */
+       int consumer_channel_pipe[2];
+       /* Data stream poll thread pipe. To transfer data stream to the thread */
+       struct lttng_pipe *consumer_data_pipe;
+
+       /*
+        * Data thread use that pipe to catch wakeup from read subbuffer that
+        * detects that there is still data to be read for the stream encountered.
+        * Before doing so, the stream is flagged to indicate that there is still
+        * data to be read.
+        *
+        * Both pipes (read/write) are owned and used inside the data thread.
+        */
+       struct lttng_pipe *consumer_wakeup_pipe;
+       /* Indicate if the wakeup thread has been notified. */
+       unsigned int has_wakeup:1;
+
+       /* to let the signal handler wake up the fd receiver thread */
+       int consumer_should_quit[2];
+       /* Metadata poll thread pipe. Transfer metadata stream to it */
+       struct lttng_pipe *consumer_metadata_pipe;
+};
+
+/*
+ * Library-level data. One instance per process.
+ */
+struct lttng_consumer_global_data {
+       /*
+        * At this time, this lock is used to ensure coherence between the count
+        * and number of element in the hash table. It's also a protection for
+        * concurrent read/write between threads.
+        *
+        * This is nested OUTSIDE the stream lock.
+        * This is nested OUTSIDE the consumer_relayd_sock_pair lock.
+        */
+       pthread_mutex_t lock;
+
+       /*
+        * Number of streams in the data stream hash table declared outside.
+        * Protected by consumer_data.lock.
+        */
+       int stream_count;
+
+       /* Channel hash table protected by consumer_data.lock. */
+       struct lttng_ht *channel_ht;
+       /*
+        * Flag specifying if the local array of FDs needs update in the
+        * poll function. Protected by consumer_data.lock.
+        */
+       unsigned int need_update;
+       enum lttng_consumer_type type;
+
+       /*
+        * Relayd socket(s) hashtable indexed by network sequence number. Each
+        * stream has an index which associate the right relayd socket to use.
+        */
+       struct lttng_ht *relayd_ht;
+
+       /*
+        * This hash table contains all streams (metadata and data) indexed by
+        * session id. In other words, the ht is indexed by session id and each
+        * bucket contains the list of associated streams.
+        *
+        * This HT uses the "node_session_id" of the consumer stream.
+        */
+       struct lttng_ht *stream_list_ht;
+
+       /*
+        * This HT uses the "node_channel_id" of the consumer stream.
+        */
+       struct lttng_ht *stream_per_chan_id_ht;
+};
+
+/*
+ * Init consumer data structures.
+ */
+int lttng_consumer_init(void);
+
+/*
+ * Set the error socket for communication with a session daemon.
+ */
+void lttng_consumer_set_error_sock(struct lttng_consumer_local_data *ctx,
+               int sock);
+
+/*
+ * Set the command socket path for communication with a session daemon.
+ */
+void lttng_consumer_set_command_sock_path(
+               struct lttng_consumer_local_data *ctx, char *sock);
+
+/*
+ * Send return code to session daemon.
+ *
+ * Returns the return code of sendmsg : the number of bytes transmitted or -1
+ * on error.
+ */
+int lttng_consumer_send_error(struct lttng_consumer_local_data *ctx, int cmd);
+
+/*
+ * Called from signal handler to ensure a clean exit.
+ */
+void lttng_consumer_should_exit(struct lttng_consumer_local_data *ctx);
+
+/*
+ * Cleanup the daemon's socket on exit.
+ */
+void lttng_consumer_cleanup(void);
+
+/*
+ * Flush pending writes to trace output disk file.
+ */
+void lttng_consumer_sync_trace_file(struct lttng_consumer_stream *stream,
+               off_t orig_offset);
+
+/*
+ * Poll on the should_quit pipe and the command socket return -1 on error and
+ * should exit, 0 if data is available on the command socket
+ */
+int lttng_consumer_poll_socket(struct pollfd *kconsumer_sockpoll);
+
+struct lttng_consumer_stream *consumer_allocate_stream(uint64_t channel_key,
+               uint64_t stream_key,
+               enum lttng_consumer_stream_state state,
+               const char *channel_name,
+               uid_t uid,
+               gid_t gid,
+               uint64_t relayd_id,
+               uint64_t session_id,
+               int cpu,
+               int *alloc_ret,
+               enum consumer_channel_type type,
+               unsigned int monitor);
+struct lttng_consumer_channel *consumer_allocate_channel(uint64_t key,
+               uint64_t session_id,
+               const char *pathname,
+               const char *name,
+               uid_t uid,
+               gid_t gid,
+               uint64_t relayd_id,
+               enum lttng_event_output output,
+               uint64_t tracefile_size,
+               uint64_t tracefile_count,
+               uint64_t session_id_per_pid,
+               unsigned int monitor,
+               unsigned int live_timer_interval,
+               const char *root_shm_path,
+               const char *shm_path);
+void consumer_del_stream(struct lttng_consumer_stream *stream,
+               struct lttng_ht *ht);
+void consumer_del_metadata_stream(struct lttng_consumer_stream *stream,
+               struct lttng_ht *ht);
+int consumer_add_channel(struct lttng_consumer_channel *channel,
+               struct lttng_consumer_local_data *ctx);
+void consumer_del_channel(struct lttng_consumer_channel *channel);
+
+/* lttng-relayd consumer command */
+struct consumer_relayd_sock_pair *consumer_find_relayd(uint64_t key);
+int consumer_send_relayd_stream(struct lttng_consumer_stream *stream, char *path);
+int consumer_send_relayd_streams_sent(uint64_t net_seq_idx);
+void close_relayd_stream(struct lttng_consumer_stream *stream);
+struct lttng_consumer_channel *consumer_find_channel(uint64_t key);
+int consumer_handle_stream_before_relayd(struct lttng_consumer_stream *stream,
+               size_t data_size);
+void consumer_steal_stream_key(int key, struct lttng_ht *ht);
+
+struct lttng_consumer_local_data *lttng_consumer_create(
+               enum lttng_consumer_type type,
+               ssize_t (*buffer_ready)(struct lttng_consumer_stream *stream,
+                       struct lttng_consumer_local_data *ctx),
+               int (*recv_channel)(struct lttng_consumer_channel *channel),
+               int (*recv_stream)(struct lttng_consumer_stream *stream),
+               int (*update_stream)(uint64_t sessiond_key, uint32_t state));
+void lttng_consumer_destroy(struct lttng_consumer_local_data *ctx);
+ssize_t lttng_consumer_on_read_subbuffer_mmap(
+               struct lttng_consumer_local_data *ctx,
+               struct lttng_consumer_stream *stream, unsigned long len,
+               unsigned long padding,
+               struct ctf_packet_index *index);
+ssize_t lttng_consumer_on_read_subbuffer_splice(
+               struct lttng_consumer_local_data *ctx,
+               struct lttng_consumer_stream *stream, unsigned long len,
+               unsigned long padding,
+               struct ctf_packet_index *index);
+int lttng_consumer_take_snapshot(struct lttng_consumer_stream *stream);
+int lttng_consumer_get_produced_snapshot(struct lttng_consumer_stream *stream,
+               unsigned long *pos);
+int lttng_ustconsumer_get_wakeup_fd(struct lttng_consumer_stream *stream);
+int lttng_ustconsumer_close_wakeup_fd(struct lttng_consumer_stream *stream);
+void *consumer_thread_metadata_poll(void *data);
+void *consumer_thread_data_poll(void *data);
+void *consumer_thread_sessiond_poll(void *data);
+void *consumer_thread_channel_poll(void *data);
+int lttng_consumer_recv_cmd(struct lttng_consumer_local_data *ctx,
+               int sock, struct pollfd *consumer_sockpoll);
+
+ssize_t lttng_consumer_read_subbuffer(struct lttng_consumer_stream *stream,
+               struct lttng_consumer_local_data *ctx);
+int lttng_consumer_on_recv_stream(struct lttng_consumer_stream *stream);
+int consumer_add_relayd_socket(uint64_t net_seq_idx, int sock_type,
+               struct lttng_consumer_local_data *ctx, int sock,
+               struct pollfd *consumer_sockpoll, struct lttcomm_relayd_sock *relayd_sock,
+               uint64_t sessiond_id, uint64_t relayd_session_id);
+void consumer_flag_relayd_for_destroy(
+               struct consumer_relayd_sock_pair *relayd);
+int consumer_data_pending(uint64_t id);
+int consumer_send_status_msg(int sock, int ret_code);
+int consumer_send_status_channel(int sock,
+               struct lttng_consumer_channel *channel);
+void notify_thread_del_channel(struct lttng_consumer_local_data *ctx,
+               uint64_t key);
+void consumer_destroy_relayd(struct consumer_relayd_sock_pair *relayd);
+unsigned long consumer_get_consume_start_pos(unsigned long consumed_pos,
+               unsigned long produced_pos, uint64_t nb_packets_per_stream,
+               uint64_t max_sb_size);
+int consumer_add_data_stream(struct lttng_consumer_stream *stream);
+void consumer_del_stream_for_data(struct lttng_consumer_stream *stream);
+int consumer_add_metadata_stream(struct lttng_consumer_stream *stream);
+void consumer_del_stream_for_metadata(struct lttng_consumer_stream *stream);
+int consumer_create_index_file(struct lttng_consumer_stream *stream);
+
+#endif /* LIB_CONSUMER_H */
index 7875e890b6668ddf99ed6ca79bb107d99e5d17e3..fac1f0b8a2d58b988d9a30cb6789b8c6bd0c3a5c 100644 (file)
@@ -39,9 +39,9 @@
 #include <common/pipe.h>
 #include <common/relayd/relayd.h>
 #include <common/utils.h>
-#include <common/consumer-stream.h>
+#include <common/consumer/consumer-stream.h>
 #include <common/index/index.h>
-#include <common/consumer-timer.h>
+#include <common/consumer/consumer-timer.h>
 
 #include "kernel-consumer.h"
 
index 1aad2733b28aa8def996b0a315e952eb29b43879..a07f5218828536f0f1cb6b7e444785c078d9bd35 100644 (file)
@@ -19,7 +19,7 @@
 #ifndef _LTTNG_KCONSUMER_H
 #define _LTTNG_KCONSUMER_H
 
-#include <common/consumer.h>
+#include <common/consumer/consumer.h>
 
 int lttng_kconsumer_take_snapshot(struct lttng_consumer_stream *stream);
 int lttng_kconsumer_get_produced_snapshot(struct lttng_consumer_stream *stream,
index 5bf3e7b5a4218b6925da7f0f64583c308b9ed664..44bcfdc374d9a0bd9b2ce869059fa912794a19c1 100644 (file)
@@ -38,9 +38,9 @@
 #include <common/relayd/relayd.h>
 #include <common/compat/fcntl.h>
 #include <common/compat/endian.h>
-#include <common/consumer-metadata-cache.h>
-#include <common/consumer-stream.h>
-#include <common/consumer-timer.h>
+#include <common/consumer/consumer-metadata-cache.h>
+#include <common/consumer/consumer-stream.h>
+#include <common/consumer/consumer-timer.h>
 #include <common/utils.h>
 #include <common/index/index.h>
 
index 339c52747428ef1ed6eeb54dfb4a40d244d39fce..01a50773c263ab562c9955105f7f5495c8fc06f7 100644 (file)
@@ -21,7 +21,7 @@
 
 #include <errno.h>
 
-#include <common/consumer.h>
+#include <common/consumer/consumer.h>
 
 #ifdef HAVE_LIBLTTNG_UST_CTL
 
This page took 0.196406 seconds and 4 git commands to generate.