Fix: relayd: live: some listed sessions are not attacheable
[lttng-tools.git] / src / bin / lttng-relayd / live.c
index 5b9e4248c0e779deadc7b604ace4d928f7c9634f..8c4f501aa3446afd5b5cb5d7e72e42bf82dd2d11 100644 (file)
@@ -1,6 +1,7 @@
 /*
  * Copyright (C) 2013 - Julien Desfossez <jdesfossez@efficios.com>
  *                      David Goulet <dgoulet@efficios.com>
+ *               2015 - 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,
@@ -16,7 +17,7 @@
  * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
  */
 
-#define _GNU_SOURCE
+#define _LGPL_SOURCE
 #include <getopt.h>
 #include <grp.h>
 #include <limits.h>
 #include <inttypes.h>
 #include <urcu/futex.h>
 #include <urcu/uatomic.h>
+#include <urcu/rculist.h>
 #include <unistd.h>
 #include <fcntl.h>
-#include <config.h>
 
 #include <lttng/lttng.h>
 #include <common/common.h>
 #include <common/compat/poll.h>
 #include <common/compat/socket.h>
+#include <common/compat/endian.h>
 #include <common/defaults.h>
 #include <common/futex.h>
 #include <common/index/index.h>
@@ -63,6 +65,9 @@
 #include "session.h"
 #include "ctf-trace.h"
 #include "connection.h"
+#include "viewer-session.h"
+
+#define SESSION_BUF_DEFAULT_COUNT      16
 
 static struct lttng_uri *live_uri;
 
@@ -88,12 +93,14 @@ static pthread_t live_worker_thread;
 static struct relay_conn_queue viewer_conn_queue;
 
 static uint64_t last_relay_viewer_session_id;
+static pthread_mutex_t last_relay_viewer_session_id_lock =
+               PTHREAD_MUTEX_INITIALIZER;
 
 /*
  * Cleanup the daemon
  */
 static
-void cleanup(void)
+void cleanup_relayd_live(void)
 {
        DBG("Cleaning up");
 
@@ -112,9 +119,6 @@ ssize_t recv_request(struct lttcomm_sock *sock, void *buf, size_t size)
 {
        ssize_t ret;
 
-       assert(sock);
-       assert(buf);
-
        ret = sock->ops->recvmsg(sock, buf, size, 0);
        if (ret < 0 || ret != size) {
                if (ret == 0) {
@@ -141,9 +145,6 @@ ssize_t send_response(struct lttcomm_sock *sock, void *buf, size_t size)
 {
        ssize_t ret;
 
-       assert(sock);
-       assert(buf);
-
        ret = sock->ops->sendmsg(sock, buf, size, 0);
        if (ret < 0) {
                ERR("Relayd failed to send response.");
@@ -153,32 +154,38 @@ ssize_t send_response(struct lttcomm_sock *sock, void *buf, size_t size)
 }
 
 /*
- * Atomically check if new streams got added in the session since the last
- * check and reset the flag to 0.
+ * Atomically check if new streams got added in one of the sessions attached
+ * and reset the flag to 0.
  *
  * Returns 1 if new streams got added, 0 if nothing changed, a negative value
  * on error.
  */
 static
-int check_new_streams(uint64_t session_id, struct lttng_ht *sessions_ht)
+int check_new_streams(struct relay_connection *conn)
 {
-       int ret;
-       unsigned long current_val;
        struct relay_session *session;
+       unsigned long current_val;
+       int ret = 0;
 
-       assert(sessions_ht);
-
-       session = session_find_by_id(sessions_ht, session_id);
-       if (!session) {
-               DBG("Relay session %" PRIu64 " not found", session_id);
-               ret = -1;
-               goto error;
+       if (!conn->viewer_session) {
+               goto end;
        }
-
-       current_val = uatomic_cmpxchg(&session->new_streams, 1, 0);
-       ret = current_val;
-
-error:
+       rcu_read_lock();
+       cds_list_for_each_entry_rcu(session,
+                       &conn->viewer_session->session_list,
+                       viewer_session_node) {
+               if (!session_get(session)) {
+                       continue;
+               }
+               current_val = uatomic_cmpxchg(&session->new_streams, 1, 0);
+               ret = current_val;
+               session_put(session);
+               if (ret == 1) {
+                       goto end;
+               }
+       }
+end:
+       rcu_read_unlock();
        return ret;
 }
 
@@ -190,15 +197,13 @@ error:
  */
 static
 ssize_t send_viewer_streams(struct lttcomm_sock *sock,
-               struct relay_session *session, unsigned int ignore_sent_flag)
+               uint64_t session_id, unsigned int ignore_sent_flag)
 {
        ssize_t ret;
        struct lttng_viewer_stream send_stream;
        struct lttng_ht_iter iter;
        struct relay_viewer_stream *vstream;
 
-       assert(session);
-
        rcu_read_lock();
 
        cds_lfht_for_each_entry(viewer_streams_ht->ht, &iter.iter, vstream,
@@ -207,30 +212,50 @@ ssize_t send_viewer_streams(struct lttcomm_sock *sock,
 
                health_code_update();
 
+               if (!viewer_stream_get(vstream)) {
+                       continue;
+               }
+
+               pthread_mutex_lock(&vstream->stream->lock);
                /* Ignore if not the same session. */
-               if (vstream->session_id != session->id ||
+               if (vstream->stream->trace->session->id != session_id ||
                                (!ignore_sent_flag && vstream->sent_flag)) {
+                       pthread_mutex_unlock(&vstream->stream->lock);
+                       viewer_stream_put(vstream);
                        continue;
                }
 
-               ctf_trace = ctf_trace_find_by_path(session->ctf_traces_ht,
-                               vstream->path_name);
-               assert(ctf_trace);
-
-               send_stream.id = htobe64(vstream->stream_handle);
+               ctf_trace = vstream->stream->trace;
+               send_stream.id = htobe64(vstream->stream->stream_handle);
                send_stream.ctf_trace_id = htobe64(ctf_trace->id);
-               send_stream.metadata_flag = htobe32(vstream->metadata_flag);
-               strncpy(send_stream.path_name, vstream->path_name,
-                               sizeof(send_stream.path_name));
-               strncpy(send_stream.channel_name, vstream->channel_name,
-                               sizeof(send_stream.channel_name));
+               send_stream.metadata_flag = htobe32(
+                               vstream->stream->is_metadata);
+               if (lttng_strncpy(send_stream.path_name, vstream->path_name,
+                               sizeof(send_stream.path_name))) {
+                       pthread_mutex_unlock(&vstream->stream->lock);
+                       viewer_stream_put(vstream);
+                       ret = -1;       /* Error. */
+                       goto end_unlock;
+               }
+               if (lttng_strncpy(send_stream.channel_name,
+                               vstream->channel_name,
+                               sizeof(send_stream.channel_name))) {
+                       pthread_mutex_unlock(&vstream->stream->lock);
+                       viewer_stream_put(vstream);
+                       ret = -1;       /* Error. */
+                       goto end_unlock;
+               }
+
+               DBG("Sending stream %" PRIu64 " to viewer",
+                               vstream->stream->stream_handle);
+               vstream->sent_flag = 1;
+               pthread_mutex_unlock(&vstream->stream->lock);
 
-               DBG("Sending stream %" PRIu64 " to viewer", vstream->stream_handle);
                ret = send_response(sock, &send_stream, sizeof(send_stream));
+               viewer_stream_put(vstream);
                if (ret < 0) {
                        goto end_unlock;
                }
-               vstream->sent_flag = 1;
        }
 
        ret = 0;
@@ -246,31 +271,33 @@ end_unlock:
  * viewer stream of the session, the number of unsent stream and the number of
  * stream created. Those counters can be NULL and thus will be ignored.
  *
+ * session must be locked to ensure that we see either none or all initial
+ * streams for a session, but no intermediate state..
+ *
  * Return 0 on success or else a negative value.
  */
-static
-int make_viewer_streams(struct relay_session *session,
-               enum lttng_viewer_seek seek_t, uint32_t *nb_total, uint32_t *nb_unsent,
-               uint32_t *nb_created)
+static int make_viewer_streams(struct relay_session *session,
+               struct lttng_trace_chunk *viewer_trace_chunk,
+               enum lttng_viewer_seek seek_t,
+               uint32_t *nb_total,
+               uint32_t *nb_unsent,
+               uint32_t *nb_created,
+               bool *closed)
 {
        int ret;
        struct lttng_ht_iter iter;
        struct ctf_trace *ctf_trace;
 
        assert(session);
+       ASSERT_LOCKED(session->lock);
 
-       /*
-        * This is to make sure we create viewer streams for a full received
-        * channel. For instance, if we have 8 streams for a channel that are
-        * concurrently being flagged ready, we can end up creating just a subset
-        * of the 8 streams (the ones that are flagged). This lock avoids this
-        * limbo state.
-        */
-       pthread_mutex_lock(&session->viewer_ready_lock);
+       if (session->connection_closed) {
+               *closed = true;
+       }
 
        /*
-        * Create viewer streams for relay streams that are ready to be used for a
-        * the given session id only.
+        * Create viewer streams for relay streams that are ready to be
+        * used for a the given session id only.
         */
        rcu_read_lock();
        cds_lfht_for_each_entry(session->ctf_traces_ht->ht, &iter.iter, ctf_trace,
@@ -279,84 +306,87 @@ int make_viewer_streams(struct relay_session *session,
 
                health_code_update();
 
-               if (ctf_trace->invalid_flag) {
+               if (!ctf_trace_get(ctf_trace)) {
                        continue;
                }
 
-               cds_list_for_each_entry(stream, &ctf_trace->stream_list, trace_list) {
+               cds_list_for_each_entry_rcu(stream, &ctf_trace->stream_list, stream_node) {
                        struct relay_viewer_stream *vstream;
 
-                       if (!stream->viewer_ready) {
+                       if (!stream_get(stream)) {
                                continue;
                        }
-
-                       vstream = viewer_stream_find_by_id(stream->stream_handle);
+                       /*
+                        * stream published is protected by the session lock.
+                        */
+                       if (!stream->published) {
+                               goto next;
+                       }
+                       vstream = viewer_stream_get_by_id(stream->stream_handle);
                        if (!vstream) {
-                               vstream = viewer_stream_create(stream, seek_t, ctf_trace);
+                               vstream = viewer_stream_create(stream,
+                                               viewer_trace_chunk, seek_t);
                                if (!vstream) {
                                        ret = -1;
+                                       ctf_trace_put(ctf_trace);
+                                       stream_put(stream);
                                        goto error_unlock;
                                }
-                               /* Acquire reference to ctf_trace. */
-                               ctf_trace_get_ref(ctf_trace);
 
                                if (nb_created) {
                                        /* Update number of created stream counter. */
                                        (*nb_created)++;
                                }
-                       } else if (!vstream->sent_flag && nb_unsent) {
-                               /* Update number of unsent stream counter. */
-                               (*nb_unsent)++;
+                               /*
+                                * Ensure a self-reference is preserved even
+                                * after we have put our local reference.
+                                */
+                               if (!viewer_stream_get(vstream)) {
+                                       ERR("Unable to get self-reference on viewer stream, logic error.");
+                                       abort();
+                               }
+                       } else {
+                               if (!vstream->sent_flag && nb_unsent) {
+                                       /* Update number of unsent stream counter. */
+                                       (*nb_unsent)++;
+                               }
                        }
                        /* Update number of total stream counter. */
                        if (nb_total) {
-                               (*nb_total)++;
+                               if (stream->is_metadata) {
+                                       if (!stream->closed ||
+                                                       stream->metadata_received > vstream->metadata_sent) {
+                                               (*nb_total)++;
+                                       }
+                               } else {
+                                       if (!stream->closed ||
+                                               !(((int64_t) (stream->prev_data_seq - stream->last_net_seq_num)) >= 0)) {
+
+                                               (*nb_total)++;
+                                       }
+                               }
                        }
+                       /* Put local reference. */
+                       viewer_stream_put(vstream);
+               next:
+                       stream_put(stream);
                }
+               ctf_trace_put(ctf_trace);
        }
 
        ret = 0;
 
 error_unlock:
        rcu_read_unlock();
-       pthread_mutex_unlock(&session->viewer_ready_lock);
        return ret;
 }
 
-/*
- * Write to writable pipe used to notify a thread.
- */
-static
-int notify_thread_pipe(int wpipe)
-{
-       ssize_t ret;
-
-       ret = lttng_write(wpipe, "!", 1);
-       if (ret < 1) {
-               PERROR("write poll pipe");
-       }
-
-       return (int) ret;
-}
-
-/*
- * Stop all threads by closing the thread quit pipe.
- */
-static
-void stop_threads(void)
+int relayd_live_stop(void)
 {
-       int ret;
-
-       /* Stopping all threads */
-       DBG("Terminating all live threads");
-       ret = notify_thread_pipe(live_conn_pipe[1]);
-       if (ret < 0) {
-               ERR("write error on thread quit pipe");
-       }
-
-       /* Dispatch thread */
+       /* Stop dispatch thread */
        CMM_STORE_SHARED(live_dispatch_thread_exit, 1);
        futex_nto1_wake(&viewer_conn_queue.futex);
+       return 0;
 }
 
 /*
@@ -378,7 +408,7 @@ int create_thread_poll_set(struct lttng_poll_event *events, int size)
        }
 
        /* Add quit pipe */
-       ret = lttng_poll_add(events, live_conn_pipe[0], LPOLLIN | LPOLLERR);
+       ret = lttng_poll_add(events, thread_quit_pipe[0], LPOLLIN | LPOLLERR);
        if (ret < 0) {
                goto error;
        }
@@ -395,9 +425,9 @@ error:
  * Return 1 if it was triggered else 0;
  */
 static
-int check_live_conn_pipe(int fd, uint32_t events)
+int check_thread_quit_pipe(int fd, uint32_t events)
 {
-       if (fd == live_conn_pipe[0] && (events & LPOLLIN)) {
+       if (fd == thread_quit_pipe[0] && (events & LPOLLIN)) {
                return 1;
        }
 
@@ -423,10 +453,11 @@ struct lttcomm_sock *init_socket(struct lttng_uri *uri)
        if (ret < 0) {
                goto error;
        }
-       DBG("Listening on sock %d for live", sock->fd);
+       DBG("Listening on sock %d for lttng-live", sock->fd);
 
        ret = sock->ops->bind(sock);
        if (ret < 0) {
+               PERROR("Failed to bind lttng-live socket");
                goto error;
        }
 
@@ -514,33 +545,26 @@ restart:
                        pollfd = LTTNG_POLL_GETFD(&events, i);
 
                        /* Thread quit pipe has been closed. Killing thread. */
-                       ret = check_live_conn_pipe(pollfd, revents);
+                       ret = check_thread_quit_pipe(pollfd, revents);
                        if (ret) {
                                err = 0;
                                goto exit;
                        }
 
-                       if (revents & (LPOLLERR | LPOLLHUP | LPOLLRDHUP)) {
-                               ERR("socket poll error");
-                               goto error;
-                       } else if (revents & LPOLLIN) {
+                       if (revents & LPOLLIN) {
                                /*
-                                * Get allocated in this thread, enqueued to a global queue,
-                                * dequeued and freed in the worker thread.
+                                * A new connection is requested, therefore a
+                                * viewer connection is allocated in this
+                                * thread, enqueued to a global queue and
+                                * dequeued (and freed) in the worker thread.
                                 */
                                int val = 1;
                                struct relay_connection *new_conn;
                                struct lttcomm_sock *newsock;
 
-                               new_conn = connection_create();
-                               if (!new_conn) {
-                                       goto error;
-                               }
-
                                newsock = live_control_sock->ops->accept(live_control_sock);
                                if (!newsock) {
                                        PERROR("accepting control sock");
-                                       connection_free(new_conn);
                                        goto error;
                                }
                                DBG("Relay viewer connection accepted socket %d", newsock->fd);
@@ -550,19 +574,32 @@ restart:
                                if (ret < 0) {
                                        PERROR("setsockopt inet");
                                        lttcomm_destroy_sock(newsock);
-                                       connection_free(new_conn);
                                        goto error;
                                }
-                               new_conn->sock = newsock;
+                               new_conn = connection_create(newsock, RELAY_CONNECTION_UNKNOWN);
+                               if (!new_conn) {
+                                       lttcomm_destroy_sock(newsock);
+                                       goto error;
+                               }
+                               /* Ownership assumed by the connection. */
+                               newsock = NULL;
 
                                /* Enqueue request for the dispatcher thread. */
-                               cds_wfq_enqueue(&viewer_conn_queue.queue, &new_conn->qnode);
+                               cds_wfcq_enqueue(&viewer_conn_queue.head, &viewer_conn_queue.tail,
+                                                &new_conn->qnode);
 
                                /*
-                                * Wake the dispatch queue futex. Implicit memory barrier with
-                                * the exchange in cds_wfq_enqueue.
+                                * Wake the dispatch queue futex.
+                                * Implicit memory barrier with the
+                                * exchange in cds_wfcq_enqueue.
                                 */
                                futex_nto1_wake(&viewer_conn_queue.futex);
+                       } else if (revents & (LPOLLERR | LPOLLHUP | LPOLLRDHUP)) {
+                               ERR("socket poll error");
+                               goto error;
+                       } else {
+                               ERR("Unexpected poll events %u for sock %d", revents, pollfd);
+                               goto error;
                        }
                }
        }
@@ -587,7 +624,9 @@ error_sock_control:
        }
        health_unregister(health_relayd);
        DBG("Live viewer listener thread cleanup complete");
-       stop_threads();
+       if (lttng_relay_stop_threads()) {
+               ERR("Error stopping threads");
+       }
        return NULL;
 }
 
@@ -599,7 +638,7 @@ void *thread_dispatcher(void *data)
 {
        int err = -1;
        ssize_t ret;
-       struct cds_wfq_node *node;
+       struct cds_wfcq_node *node;
        struct relay_connection *conn = NULL;
 
        DBG("[thread] Live viewer relay dispatcher started");
@@ -612,17 +651,22 @@ void *thread_dispatcher(void *data)
 
        health_code_update();
 
-       while (!CMM_LOAD_SHARED(live_dispatch_thread_exit)) {
+       for (;;) {
                health_code_update();
 
                /* Atomically prepare the queue futex */
                futex_nto1_prepare(&viewer_conn_queue.futex);
 
+               if (CMM_LOAD_SHARED(live_dispatch_thread_exit)) {
+                       break;
+               }
+
                do {
                        health_code_update();
 
                        /* Dequeue commands */
-                       node = cds_wfq_dequeue_blocking(&viewer_conn_queue.queue);
+                       node = cds_wfcq_dequeue_blocking(&viewer_conn_queue.head,
+                                                        &viewer_conn_queue.tail);
                        if (node == NULL) {
                                DBG("Woken up but nothing in the live-viewer "
                                                "relay command queue");
@@ -634,14 +678,15 @@ void *thread_dispatcher(void *data)
                                        conn->sock->fd);
 
                        /*
-                        * Inform worker thread of the new request. This call is blocking
-                        * so we can be assured that the data will be read at some point in
-                        * time or wait to the end of the world :)
+                        * Inform worker thread of the new request. This
+                        * call is blocking so we can be assured that
+                        * the data will be read at some point in time
+                        * or wait to the end of the world :)
                         */
                        ret = lttng_write(live_conn_pipe[1], &conn, sizeof(conn));
                        if (ret < 0) {
                                PERROR("write conn pipe");
-                               connection_destroy(conn);
+                               connection_put(conn);
                                goto error;
                        }
                } while (node != NULL);
@@ -663,7 +708,9 @@ error_testpoint:
        }
        health_unregister(health_relayd);
        DBG("Live viewer dispatch thread dying");
-       stop_threads();
+       if (lttng_relay_stop_threads()) {
+               ERR("Error stopping threads");
+       }
        return NULL;
 }
 
@@ -678,8 +725,6 @@ int viewer_connect(struct relay_connection *conn)
        int ret;
        struct lttng_viewer_connect reply, msg;
 
-       assert(conn);
-
        conn->version_check_done = 1;
 
        health_code_update();
@@ -693,6 +738,7 @@ int viewer_connect(struct relay_connection *conn)
 
        health_code_update();
 
+       memset(&reply, 0, sizeof(reply));
        reply.major = RELAYD_VERSION_COMM_MAJOR;
        reply.minor = RELAYD_VERSION_COMM_MINOR;
 
@@ -725,7 +771,15 @@ int viewer_connect(struct relay_connection *conn)
        reply.major = htobe32(reply.major);
        reply.minor = htobe32(reply.minor);
        if (conn->type == RELAY_VIEWER_COMMAND) {
-               reply.viewer_session_id = htobe64(++last_relay_viewer_session_id);
+               /*
+                * Increment outside of htobe64 macro, because the argument can
+                * be used more than once within the macro, and thus the
+                * operation may be undefined.
+                */
+               pthread_mutex_lock(&last_relay_viewer_session_id_lock);
+               last_relay_viewer_session_id++;
+               pthread_mutex_unlock(&last_relay_viewer_session_id_lock);
+               reply.viewer_session_id = htobe64(last_relay_viewer_session_id);
        }
 
        health_code_update();
@@ -746,107 +800,134 @@ end:
 
 /*
  * Send the viewer the list of current sessions.
+ * We need to create a copy of the hash table content because otherwise
+ * we cannot assume the number of entries stays the same between getting
+ * the number of HT elements and iteration over the HT.
  *
  * Return 0 on success or else a negative value.
  */
 static
 int viewer_list_sessions(struct relay_connection *conn)
 {
-       int ret;
+       int ret = 0;
        struct lttng_viewer_list_sessions session_list;
-       unsigned long count;
-       long approx_before, approx_after;
        struct lttng_ht_iter iter;
-       struct lttng_viewer_session send_session;
        struct relay_session *session;
+       struct lttng_viewer_session *send_session_buf = NULL;
+       uint32_t buf_count = SESSION_BUF_DEFAULT_COUNT;
+       uint32_t count = 0;
 
        DBG("List sessions received");
 
-       rcu_read_lock();
-       cds_lfht_count_nodes(conn->sessions_ht->ht, &approx_before, &count,
-                       &approx_after);
-       session_list.sessions_count = htobe32(count);
-
-       health_code_update();
-
-       ret = send_response(conn->sock, &session_list, sizeof(session_list));
-       if (ret < 0) {
-               goto end_unlock;
+       send_session_buf = zmalloc(SESSION_BUF_DEFAULT_COUNT * sizeof(*send_session_buf));
+       if (!send_session_buf) {
+               return -1;
        }
 
-       health_code_update();
-
-       cds_lfht_for_each_entry(conn->sessions_ht->ht, &iter.iter, session,
+       rcu_read_lock();
+       cds_lfht_for_each_entry(sessions_ht->ht, &iter.iter, session,
                        session_n.node) {
-               health_code_update();
-
-               strncpy(send_session.session_name, session->session_name,
-                               sizeof(send_session.session_name));
-               strncpy(send_session.hostname, session->hostname,
-                               sizeof(send_session.hostname));
-               send_session.id = htobe64(session->id);
-               send_session.live_timer = htobe32(session->live_timer);
-               send_session.clients = htobe32(session->viewer_refcount);
-               send_session.streams = htobe32(session->stream_count);
+               struct lttng_viewer_session *send_session;
 
                health_code_update();
 
-               ret = send_response(conn->sock, &send_session, sizeof(send_session));
-               if (ret < 0) {
-                       goto end_unlock;
+               pthread_mutex_lock(&session->lock);
+               if (session->connection_closed) {
+                       /* Skip closed session */
+                       goto next_session;
+               }
+               if (!session->current_trace_chunk) {
+                       /*
+                        * Skip un-attachable session. It is either
+                        * being destroyed or has not had a trace
+                        * chunk created against it yet.
+                        */
+                       goto next_session;
                }
-       }
-       health_code_update();
 
-       rcu_read_unlock();
-       ret = 0;
-       goto end;
+               if (count >= buf_count) {
+                       struct lttng_viewer_session *newbuf;
+                       uint32_t new_buf_count = buf_count << 1;
 
-end_unlock:
+                       newbuf = realloc(send_session_buf,
+                               new_buf_count * sizeof(*send_session_buf));
+                       if (!newbuf) {
+                               ret = -1;
+                               goto break_loop;
+                       }
+                       send_session_buf = newbuf;
+                       buf_count = new_buf_count;
+               }
+               send_session = &send_session_buf[count];
+               if (lttng_strncpy(send_session->session_name,
+                               session->session_name,
+                               sizeof(send_session->session_name))) {
+                       ret = -1;
+                       goto break_loop;
+               }
+               if (lttng_strncpy(send_session->hostname, session->hostname,
+                               sizeof(send_session->hostname))) {
+                       ret = -1;
+                       goto break_loop;
+               }
+               send_session->id = htobe64(session->id);
+               send_session->live_timer = htobe32(session->live_timer);
+               if (session->viewer_attached) {
+                       send_session->clients = htobe32(1);
+               } else {
+                       send_session->clients = htobe32(0);
+               }
+               send_session->streams = htobe32(session->stream_count);
+               count++;
+       next_session:
+               pthread_mutex_unlock(&session->lock);
+               continue;
+       break_loop:
+               pthread_mutex_unlock(&session->lock);
+               break;
+       }
        rcu_read_unlock();
+       if (ret < 0) {
+               goto end_free;
+       }
 
-end:
-       return ret;
-}
+       session_list.sessions_count = htobe32(count);
 
-/*
- * Check if a connection is attached to a session.
- * Return 1 if attached, 0 if not attached, a negative value on error.
- */
-static
-int session_attached(struct relay_connection *conn, uint64_t session_id)
-{
-       struct relay_session *session;
-       int found = 0;
+       health_code_update();
 
-       if (!conn->viewer_session) {
-               goto end;
+       ret = send_response(conn->sock, &session_list, sizeof(session_list));
+       if (ret < 0) {
+               goto end_free;
        }
-       cds_list_for_each_entry(session,
-                       &conn->viewer_session->sessions_head,
-                       viewer_session_list) {
-               if (session->id == session_id) {
-                       found = 1;
-                       goto end;
-               }
+
+       health_code_update();
+
+       ret = send_response(conn->sock, send_session_buf,
+                       count * sizeof(*send_session_buf));
+       if (ret < 0) {
+               goto end_free;
        }
+       health_code_update();
 
-end:
-       return found;
+       ret = 0;
+end_free:
+       free(send_session_buf);
+       return ret;
 }
 
 /*
- * Send the viewer the list of current sessions.
+ * Send the viewer the list of current streams.
  */
 static
 int viewer_get_new_streams(struct relay_connection *conn)
 {
        int ret, send_streams = 0;
-       uint32_t nb_created = 0, nb_unsent = 0, nb_streams = 0;
+       uint32_t nb_created = 0, nb_unsent = 0, nb_streams = 0, nb_total = 0;
        struct lttng_viewer_new_streams_request request;
        struct lttng_viewer_new_streams_response response;
-       struct relay_session *session;
+       struct relay_session *session = NULL;
        uint64_t session_id;
+       bool closed = false;
 
        assert(conn);
 
@@ -863,15 +944,16 @@ int viewer_get_new_streams(struct relay_connection *conn)
 
        health_code_update();
 
-       rcu_read_lock();
-       session = session_find_by_id(conn->sessions_ht, session_id);
+       memset(&response, 0, sizeof(response));
+
+       session = session_get_by_id(session_id);
        if (!session) {
                DBG("Relay session %" PRIu64 " not found", session_id);
                response.status = htobe32(LTTNG_VIEWER_NEW_STREAMS_ERR);
                goto send_reply;
        }
 
-       if (!session_attached(conn, session_id)) {
+       if (!viewer_session_is_attached(conn->viewer_session, session)) {
                send_streams = 0;
                response.status = htobe32(LTTNG_VIEWER_NEW_STREAMS_ERR);
                goto send_reply;
@@ -880,33 +962,36 @@ int viewer_get_new_streams(struct relay_connection *conn)
        send_streams = 1;
        response.status = htobe32(LTTNG_VIEWER_NEW_STREAMS_OK);
 
-       if (!send_streams) {
-               goto send_reply;
+       pthread_mutex_lock(&session->lock);
+       if (!conn->viewer_session->current_trace_chunk &&
+                       session->current_trace_chunk) {
+               ret = viewer_session_set_trace_chunk(conn->viewer_session,
+                               session->current_trace_chunk);
+               if (ret) {
+                       goto error_unlock_session;
+               }
        }
-
-       ret = make_viewer_streams(session, LTTNG_VIEWER_SEEK_LAST, NULL, &nb_unsent,
-                       &nb_created);
+       ret = make_viewer_streams(session,
+                       conn->viewer_session->current_trace_chunk,
+                       LTTNG_VIEWER_SEEK_LAST, &nb_total, &nb_unsent,
+                       &nb_created, &closed);
        if (ret < 0) {
-               goto end_unlock;
+               goto error_unlock_session;
        }
+       pthread_mutex_unlock(&session->lock);
+
        /* Only send back the newly created streams with the unsent ones. */
        nb_streams = nb_created + nb_unsent;
        response.streams_count = htobe32(nb_streams);
 
        /*
-        * If the session is closed and we have no new streams to send,
-        * it means that the viewer has already received the whole trace
-        * for this session and should now close it.
+        * If the session is closed, HUP when there are no more streams
+        * with data.
         */
-       if (nb_streams == 0 && session->close_flag) {
+       if (closed && nb_total == 0) {
                send_streams = 0;
+               response.streams_count = 0;
                response.status = htobe32(LTTNG_VIEWER_NEW_STREAMS_HUP);
-               /*
-                * Remove the session from the attached list of the connection
-                * and try to destroy it.
-                */
-               cds_list_del(&session->viewer_session_list);
-               session_viewer_try_destroy(conn->sessions_ht, session);
                goto send_reply;
        }
 
@@ -914,32 +999,39 @@ send_reply:
        health_code_update();
        ret = send_response(conn->sock, &response, sizeof(response));
        if (ret < 0) {
-               goto end_unlock;
+               goto end_put_session;
        }
        health_code_update();
 
        /*
-        * Unknown or empty session, just return gracefully, the viewer knows what
-        * is happening.
+        * Unknown or empty session, just return gracefully, the viewer
+        * knows what is happening.
         */
        if (!send_streams || !nb_streams) {
                ret = 0;
-               goto end_unlock;
+               goto end_put_session;
        }
 
        /*
-        * Send stream and *DON'T* ignore the sent flag so every viewer streams
-        * that were not sent from that point will be sent to the viewer.
+        * Send stream and *DON'T* ignore the sent flag so every viewer
+        * streams that were not sent from that point will be sent to
+        * the viewer.
         */
-       ret = send_viewer_streams(conn->sock, session, 0);
+       ret = send_viewer_streams(conn->sock, session_id, 0);
        if (ret < 0) {
-               goto end_unlock;
+               goto end_put_session;
        }
 
-end_unlock:
-       rcu_read_unlock();
+end_put_session:
+       if (session) {
+               session_put(session);
+       }
 error:
        return ret;
+error_unlock_session:
+       pthread_mutex_unlock(&session->lock);
+       session_put(session);
+       return ret;
 }
 
 /*
@@ -954,7 +1046,9 @@ int viewer_attach_session(struct relay_connection *conn)
        enum lttng_viewer_seek seek_type;
        struct lttng_viewer_attach_session_request request;
        struct lttng_viewer_attach_session_response response;
-       struct relay_session *session;
+       struct relay_session *session = NULL;
+       bool closed = false;
+       uint64_t session_id;
 
        assert(conn);
 
@@ -966,45 +1060,44 @@ int viewer_attach_session(struct relay_connection *conn)
                goto error;
        }
 
+       session_id = be64toh(request.session_id);
        health_code_update();
 
+       memset(&response, 0, sizeof(response));
+
        if (!conn->viewer_session) {
                DBG("Client trying to attach before creating a live viewer session");
                response.status = htobe32(LTTNG_VIEWER_ATTACH_NO_SESSION);
                goto send_reply;
        }
 
-       rcu_read_lock();
-       session = session_find_by_id(conn->sessions_ht,
-                       be64toh(request.session_id));
+       session = session_get_by_id(session_id);
        if (!session) {
-               DBG("Relay session %" PRIu64 " not found",
-                               be64toh(request.session_id));
+               DBG("Relay session %" PRIu64 " not found", session_id);
                response.status = htobe32(LTTNG_VIEWER_ATTACH_UNK);
                goto send_reply;
        }
-       session_viewer_attach(session);
-       DBG("Attach session ID %" PRIu64 " received", be64toh(request.session_id));
+       DBG("Attach session ID %" PRIu64 " received", session_id);
 
-       if (uatomic_read(&session->viewer_refcount) > 1) {
-               DBG("Already a viewer attached");
-               response.status = htobe32(LTTNG_VIEWER_ATTACH_ALREADY);
-               session_viewer_detach(session);
-               goto send_reply;
-       } else if (session->live_timer == 0) {
+       pthread_mutex_lock(&session->lock);
+       if (session->live_timer == 0) {
                DBG("Not live session");
                response.status = htobe32(LTTNG_VIEWER_ATTACH_NOT_LIVE);
                goto send_reply;
-       } else {
-               send_streams = 1;
-               response.status = htobe32(LTTNG_VIEWER_ATTACH_OK);
-               cds_list_add(&session->viewer_session_list,
-                               &conn->viewer_session->sessions_head);
+       }
+
+       send_streams = 1;
+       ret = viewer_session_attach(conn->viewer_session, session);
+       if (ret) {
+               DBG("Already a viewer attached");
+               response.status = htobe32(LTTNG_VIEWER_ATTACH_ALREADY);
+               goto send_reply;
        }
 
        switch (be32toh(request.seek)) {
        case LTTNG_VIEWER_SEEK_BEGINNING:
        case LTTNG_VIEWER_SEEK_LAST:
+               response.status = htobe32(LTTNG_VIEWER_ATTACH_OK);
                seek_type = be32toh(request.seek);
                break;
        default:
@@ -1014,233 +1107,365 @@ int viewer_attach_session(struct relay_connection *conn)
                goto send_reply;
        }
 
-       if (!send_streams) {
-               goto send_reply;
+       if (!conn->viewer_session->current_trace_chunk &&
+                       session->current_trace_chunk) {
+               ret = viewer_session_set_trace_chunk(conn->viewer_session,
+                               session->current_trace_chunk);
+               if (ret) {
+                       goto end_put_session;
+               }
        }
-
-       ret = make_viewer_streams(session, seek_type, &nb_streams, NULL, NULL);
+       ret = make_viewer_streams(session,
+                       conn->viewer_session->current_trace_chunk, seek_type,
+                       &nb_streams, NULL, NULL, &closed);
        if (ret < 0) {
-               goto end_unlock;
+               goto end_put_session;
        }
+       pthread_mutex_unlock(&session->lock);
+       session_put(session);
+       session = NULL;
+
        response.streams_count = htobe32(nb_streams);
+       /*
+        * If the session is closed when the viewer is attaching, it
+        * means some of the streams may have been concurrently removed,
+        * so we don't allow the viewer to attach, even if there are
+        * streams available.
+        */
+       if (closed) {
+               send_streams = 0;
+               response.streams_count = 0;
+               response.status = htobe32(LTTNG_VIEWER_ATTACH_UNK);
+               goto send_reply;
+       }
 
 send_reply:
        health_code_update();
        ret = send_response(conn->sock, &response, sizeof(response));
        if (ret < 0) {
-               goto end_unlock;
+               goto end_put_session;
        }
        health_code_update();
 
        /*
-        * Unknown or empty session, just return gracefully, the viewer knows what
-        * is happening.
+        * Unknown or empty session, just return gracefully, the viewer
+        * knows what is happening.
         */
        if (!send_streams || !nb_streams) {
                ret = 0;
-               goto end_unlock;
+               goto end_put_session;
        }
 
        /* Send stream and ignore the sent flag. */
-       ret = send_viewer_streams(conn->sock, session, 1);
+       ret = send_viewer_streams(conn->sock, session_id, 1);
        if (ret < 0) {
-               goto end_unlock;
+               goto end_put_session;
        }
 
-end_unlock:
-       rcu_read_unlock();
+end_put_session:
+       if (session) {
+               pthread_mutex_unlock(&session->lock);
+               session_put(session);
+       }
 error:
        return ret;
 }
 
 /*
- * Send the next index for a stream.
+ * Open the index file if needed for the given vstream.
  *
- * Return 0 on success or else a negative value.
+ * If an index file is successfully opened, the vstream will set it as its
+ * current index file.
+ *
+ * Return 0 on success, a negative value on error (-ENOENT if not ready yet).
+ *
+ * Called with rstream lock held.
  */
-static
-int viewer_get_next_index(struct relay_connection *conn)
+static int try_open_index(struct relay_viewer_stream *vstream,
+               struct relay_stream *rstream)
 {
-       int ret;
-       struct lttng_viewer_get_next_index request_index;
-       struct lttng_viewer_index viewer_index;
-       struct ctf_packet_index packet_index;
-       struct relay_viewer_stream *vstream;
-       struct relay_stream *rstream;
-       struct ctf_trace *ctf_trace;
-       struct relay_session *session;
-
-       assert(conn);
-
-       DBG("Viewer get next index");
-
-       health_code_update();
+       int ret = 0;
+       const uint32_t connection_major = rstream->trace->session->major;
+       const uint32_t connection_minor = rstream->trace->session->minor;
 
-       ret = recv_request(conn->sock, &request_index, sizeof(request_index));
-       if (ret < 0) {
+       if (vstream->index_file) {
                goto end;
        }
-       health_code_update();
-
-       rcu_read_lock();
-       vstream = viewer_stream_find_by_id(be64toh(request_index.stream_id));
-       if (!vstream) {
-               ret = -1;
-               goto end_unlock;
-       }
-
-       session = session_find_by_id(conn->sessions_ht, vstream->session_id);
-       if (!session) {
-               ret = -1;
-               goto end_unlock;
-       }
-
-       ctf_trace = ctf_trace_find_by_path(session->ctf_traces_ht, vstream->path_name);
-       assert(ctf_trace);
-
-       memset(&viewer_index, 0, sizeof(viewer_index));
 
        /*
-        * The viewer should not ask for index on metadata stream.
+        * First time, we open the index file and at least one index is ready.
         */
-       if (vstream->metadata_flag) {
-               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_HUP);
-               goto send_reply;
+       if (rstream->index_received_seqcount == 0) {
+               ret = -ENOENT;
+               goto end;
        }
-
-       /* First time, we open the index file */
-       if (vstream->index_read_fd < 0) {
-               ret = index_open(vstream->path_name, vstream->channel_name,
-                               vstream->tracefile_count, vstream->tracefile_count_current);
-               if (ret == -ENOENT) {
-                       /*
-                        * The index is created only when the first data packet arrives, it
-                        * might not be ready at the beginning of the session
-                        */
-                       viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_RETRY);
-                       goto send_reply;
-               } else if (ret < 0) {
-                       viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_ERR);
-                       goto send_reply;
-               }
-               vstream->index_read_fd = ret;
+       vstream->index_file = lttng_index_file_create_from_trace_chunk_read_only(
+                       vstream->stream_file.trace_chunk, rstream->path_name,
+                       rstream->channel_name, rstream->tracefile_size,
+                       vstream->current_tracefile_id,
+                       lttng_to_index_major(connection_major, connection_minor),
+                       lttng_to_index_minor(connection_major, connection_minor));
+       if (!vstream->index_file) {
+               ret = -1;
        }
 
-       rstream = stream_find_by_id(relay_streams_ht, vstream->stream_handle);
-       assert(rstream);
+end:
+       return ret;
+}
 
-       if (!rstream->close_flag) {
-               if (vstream->abort_flag) {
-                       /* Rotate on abort (overwrite). */
-                       DBG("Viewer rotate because of overwrite");
-                       ret = viewer_stream_rotate(vstream, rstream);
-                       if (ret < 0) {
-                               goto end_unlock;
-                       } else if (ret == 1) {
-                               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_HUP);
-                               viewer_stream_delete(vstream);
-                               viewer_stream_destroy(ctf_trace, vstream);
-                               goto send_reply;
-                       }
-                       /* ret == 0 means successful so we continue. */
-               }
+/*
+ * Check the status of the index for the given stream. This function
+ * updates the index structure if needed and can put (close) the vstream
+ * in the HUP situation.
+ *
+ * Return 0 means that we can proceed with the index. A value of 1 means
+ * that the index has been updated and is ready to be sent to the
+ * client. A negative value indicates an error that can't be handled.
+ *
+ * Called with rstream lock held.
+ */
+static int check_index_status(struct relay_viewer_stream *vstream,
+               struct relay_stream *rstream, struct ctf_trace *trace,
+               struct lttng_viewer_index *index)
+{
+       int ret;
 
-               pthread_mutex_lock(&rstream->viewer_stream_rotation_lock);
-               if (rstream->tracefile_count_current == vstream->tracefile_count_current) {
-                       if (rstream->beacon_ts_end != -1ULL &&
-                               vstream->last_sent_index == rstream->total_index_received) {
-                               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_INACTIVE);
-                               viewer_index.timestamp_end = htobe64(rstream->beacon_ts_end);
-                               pthread_mutex_unlock(&rstream->viewer_stream_rotation_lock);
-                               goto send_reply;
-                       } else if (rstream->total_index_received <= vstream->last_sent_index
-                                       && !vstream->close_write_flag) {
-                               /*
-                                * Reader and writer are working in the same tracefile, so we care
-                                * about the number of index received and sent. Otherwise, we read
-                                * up to EOF.
-                                */
-                               pthread_mutex_unlock(&rstream->viewer_stream_rotation_lock);
-                               /* No new index to send, retry later. */
-                               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_RETRY);
-                               goto send_reply;
-                       }
+       if ((trace->session->connection_closed || rstream->closed)
+                       && rstream->index_received_seqcount
+                               == vstream->index_sent_seqcount) {
+               /*
+                * Last index sent and session connection or relay
+                * stream are closed.
+                */
+               index->status = htobe32(LTTNG_VIEWER_INDEX_HUP);
+               goto hup;
+       } else if (rstream->beacon_ts_end != -1ULL &&
+                       rstream->index_received_seqcount
+                               == vstream->index_sent_seqcount) {
+               /*
+                * We've received a synchronization beacon and the last index
+                * available has been sent, the index for now is inactive.
+                *
+                * In this case, we have received a beacon which allows us to
+                * inform the client of a time interval during which we can
+                * guarantee that there are no events to read (and never will
+                * be).
+                */
+               index->status = htobe32(LTTNG_VIEWER_INDEX_INACTIVE);
+               index->timestamp_end = htobe64(rstream->beacon_ts_end);
+               index->stream_id = htobe64(rstream->ctf_stream_id);
+               goto index_ready;
+       } else if (rstream->index_received_seqcount
+                       == vstream->index_sent_seqcount) {
+               /*
+                * This checks whether received == sent seqcount. In
+                * this case, we have not received a beacon. Therefore,
+                * we can only ask the client to retry later.
+                */
+               index->status = htobe32(LTTNG_VIEWER_INDEX_RETRY);
+               goto index_ready;
+       } else if (!tracefile_array_seq_in_file(rstream->tfa,
+                       vstream->current_tracefile_id,
+                       vstream->index_sent_seqcount)) {
+               /*
+                * The next index we want to send cannot be read either
+                * because we need to perform a rotation, or due to
+                * the producer having overwritten its trace file.
+                */
+               DBG("Viewer stream %" PRIu64 " rotation",
+                               vstream->stream->stream_handle);
+               ret = viewer_stream_rotate(vstream);
+               if (ret < 0) {
+                       goto end;
+               } else if (ret == 1) {
+                       /* EOF across entire stream. */
+                       index->status = htobe32(LTTNG_VIEWER_INDEX_HUP);
+                       goto hup;
                }
-               pthread_mutex_unlock(&rstream->viewer_stream_rotation_lock);
-       } else if (rstream->close_flag && vstream->close_write_flag &&
-                       vstream->total_index_received == vstream->last_sent_index) {
-               /* Last index sent and current tracefile closed in write */
-               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_HUP);
-               viewer_stream_delete(vstream);
-               viewer_stream_destroy(ctf_trace, vstream);
+               /*
+                * If we have been pushed due to overwrite, it
+                * necessarily means there is data that can be read in
+                * the stream. If we rotated because we reached the end
+                * of a tracefile, it means the following tracefile
+                * needs to contain at least one index, else we would
+                * have already returned LTTNG_VIEWER_INDEX_RETRY to the
+                * viewer. The updated index_sent_seqcount needs to
+                * point to a readable index entry now.
+                *
+                * In the case where we "rotate" on a single file, we
+                * can end up in a case where the requested index is
+                * still unavailable.
+                */
+               if (rstream->tracefile_count == 1 &&
+                               !tracefile_array_seq_in_file(
+                                       rstream->tfa,
+                                       vstream->current_tracefile_id,
+                                       vstream->index_sent_seqcount)) {
+                       index->status = htobe32(LTTNG_VIEWER_INDEX_RETRY);
+                       goto index_ready;
+               }
+               assert(tracefile_array_seq_in_file(rstream->tfa,
+                               vstream->current_tracefile_id,
+                               vstream->index_sent_seqcount));
+       }
+       /* ret == 0 means successful so we continue. */
+       ret = 0;
+end:
+       return ret;
+
+hup:
+       viewer_stream_put(vstream);
+index_ready:
+       return 1;
+}
+
+/*
+ * Send the next index for a stream.
+ *
+ * Return 0 on success or else a negative value.
+ */
+static
+int viewer_get_next_index(struct relay_connection *conn)
+{
+       int ret;
+       struct lttng_viewer_get_next_index request_index;
+       struct lttng_viewer_index viewer_index;
+       struct ctf_packet_index packet_index;
+       struct relay_viewer_stream *vstream = NULL;
+       struct relay_stream *rstream = NULL;
+       struct ctf_trace *ctf_trace = NULL;
+       struct relay_viewer_stream *metadata_viewer_stream = NULL;
+
+       assert(conn);
+
+       DBG("Viewer get next index");
+
+       memset(&viewer_index, 0, sizeof(viewer_index));
+       health_code_update();
+
+       ret = recv_request(conn->sock, &request_index, sizeof(request_index));
+       if (ret < 0) {
+               goto end;
+       }
+       health_code_update();
+
+       vstream = viewer_stream_get_by_id(be64toh(request_index.stream_id));
+       if (!vstream) {
+               DBG("Client requested index of unknown stream id %" PRIu64,
+                               (uint64_t) be64toh(request_index.stream_id));
+               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_ERR);
                goto send_reply;
-       } else {
-               vstream->close_write_flag = 1;
        }
 
-       if (!ctf_trace->metadata_received ||
-                       ctf_trace->metadata_received > ctf_trace->metadata_sent) {
-               viewer_index.flags |= LTTNG_VIEWER_FLAG_NEW_METADATA;
+       /* Use back. ref. Protected by refcounts. */
+       rstream = vstream->stream;
+       ctf_trace = rstream->trace;
+
+       /* metadata_viewer_stream may be NULL. */
+       metadata_viewer_stream =
+                       ctf_trace_get_viewer_metadata_stream(ctf_trace);
+
+       pthread_mutex_lock(&rstream->lock);
+
+       /*
+        * The viewer should not ask for index on metadata stream.
+        */
+       if (rstream->is_metadata) {
+               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_HUP);
+               goto send_reply;
        }
 
-       ret = check_new_streams(vstream->session_id, conn->sessions_ht);
+       /* Try to open an index if one is needed for that stream. */
+       ret = try_open_index(vstream, rstream);
        if (ret < 0) {
-               goto end_unlock;
-       } else if (ret == 1) {
-               viewer_index.flags |= LTTNG_VIEWER_FLAG_NEW_STREAM;
+               if (ret == -ENOENT) {
+                       /*
+                        * The index is created only when the first data
+                        * packet arrives, it might not be ready at the
+                        * beginning of the session
+                        */
+                       viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_RETRY);
+               } else {
+                       /* Unhandled error. */
+                       viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_ERR);
+               }
+               goto send_reply;
        }
 
-       pthread_mutex_lock(&vstream->overwrite_lock);
-       if (vstream->abort_flag) {
+       ret = check_index_status(vstream, rstream, ctf_trace, &viewer_index);
+       if (ret < 0) {
+               goto error_put;
+       } else if (ret == 1) {
                /*
-                * The file is being overwritten by the writer, we cannot * use it.
+                * We have no index to send and check_index_status has populated
+                * viewer_index's status.
                 */
-               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_RETRY);
-               pthread_mutex_unlock(&vstream->overwrite_lock);
-               ret = viewer_stream_rotate(vstream, rstream);
-               if (ret < 0) {
-                       goto end_unlock;
-               } else if (ret == 1) {
-                       viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_HUP);
-                       viewer_stream_delete(vstream);
-                       viewer_stream_destroy(ctf_trace, vstream);
-                       goto send_reply;
-               }
                goto send_reply;
        }
+       /* At this point, ret is 0 thus we will be able to read the index. */
+       assert(!ret);
 
-       ret = lttng_read(vstream->index_read_fd, &packet_index,
-                       sizeof(packet_index));
-       pthread_mutex_unlock(&vstream->overwrite_lock);
-       if (ret < sizeof(packet_index)) {
-               /*
-                * The tracefile is closed in write, so we read up to EOF.
-                */
-               if (vstream->close_write_flag == 1) {
-                       viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_RETRY);
-                       /* Rotate on normal EOF */
-                       ret = viewer_stream_rotate(vstream, rstream);
-                       if (ret < 0) {
-                               goto end_unlock;
-                       } else if (ret == 1) {
-                               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_HUP);
-                               viewer_stream_delete(vstream);
-                               viewer_stream_destroy(ctf_trace, vstream);
-                               goto send_reply;
+       /*
+        * vstream->stream_fd may be NULL if it has been closed by
+        * tracefile rotation, or if we are at the beginning of the
+        * stream. We open the data stream file here to protect against
+        * overwrite caused by tracefile rotation (in association with
+        * unlink performed before overwrite).
+        */
+       if (!vstream->stream_file.fd) {
+               int fd;
+               char file_path[LTTNG_PATH_MAX];
+               enum lttng_trace_chunk_status status;
+
+               ret = utils_stream_file_path(rstream->path_name,
+                               rstream->channel_name, rstream->tracefile_size,
+                               vstream->current_tracefile_id, NULL, file_path,
+                               sizeof(file_path));
+               if (ret < 0) {
+                       goto error_put;
+               }
+
+               status = lttng_trace_chunk_open_file(
+                               vstream->stream_file.trace_chunk,
+                               file_path, O_RDONLY, 0, &fd);
+               if (status != LTTNG_TRACE_CHUNK_STATUS_OK) {
+                       PERROR("Failed to open trace file for viewer stream");
+                       goto error_put;
+               }
+               vstream->stream_file.fd = stream_fd_create(fd);
+               if (!vstream->stream_file.fd) {
+                       if (close(fd)) {
+                               PERROR("Failed to close viewer stream file");
                        }
-               } else {
-                       PERROR("Relay reading index file %d", vstream->index_read_fd);
-                       viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_ERR);
+                       goto error_put;
                }
+       }
+
+       ret = check_new_streams(conn);
+       if (ret < 0) {
+               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_ERR);
+               goto send_reply;
+       } else if (ret == 1) {
+               viewer_index.flags |= LTTNG_VIEWER_FLAG_NEW_STREAM;
+       }
+
+       ret = lttng_index_file_read(vstream->index_file, &packet_index);
+       if (ret) {
+               ERR("Relay error reading index file %d",
+                               vstream->index_file->fd);
+               viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_ERR);
                goto send_reply;
        } else {
                viewer_index.status = htobe32(LTTNG_VIEWER_INDEX_OK);
-               vstream->last_sent_index++;
+               vstream->index_sent_seqcount++;
        }
 
        /*
         * Indexes are stored in big endian, no need to switch before sending.
         */
+       DBG("Sending viewer index for stream %" PRIu64 " offset %" PRIu64,
+               rstream->stream_handle,
+               (uint64_t) be64toh(packet_index.offset));
        viewer_index.offset = packet_index.offset;
        viewer_index.packet_size = packet_index.packet_size;
        viewer_index.content_size = packet_index.content_size;
@@ -1250,22 +1475,53 @@ int viewer_get_next_index(struct relay_connection *conn)
        viewer_index.stream_id = packet_index.stream_id;
 
 send_reply:
+       if (rstream) {
+               pthread_mutex_unlock(&rstream->lock);
+       }
+
+       if (metadata_viewer_stream) {
+               pthread_mutex_lock(&metadata_viewer_stream->stream->lock);
+               DBG("get next index metadata check: recv %" PRIu64
+                               " sent %" PRIu64,
+                       metadata_viewer_stream->stream->metadata_received,
+                       metadata_viewer_stream->metadata_sent);
+               if (!metadata_viewer_stream->stream->metadata_received ||
+                               metadata_viewer_stream->stream->metadata_received >
+                                       metadata_viewer_stream->metadata_sent) {
+                       viewer_index.flags |= LTTNG_VIEWER_FLAG_NEW_METADATA;
+               }
+               pthread_mutex_unlock(&metadata_viewer_stream->stream->lock);
+       }
+
        viewer_index.flags = htobe32(viewer_index.flags);
        health_code_update();
 
        ret = send_response(conn->sock, &viewer_index, sizeof(viewer_index));
        if (ret < 0) {
-               goto end_unlock;
+               goto end;
        }
        health_code_update();
 
-       DBG("Index %" PRIu64 " for stream %" PRIu64 " sent",
-                       vstream->last_sent_index, vstream->stream_handle);
-
-end_unlock:
-       rcu_read_unlock();
-
+       if (vstream) {
+               DBG("Index %" PRIu64 " for stream %" PRIu64 " sent",
+                               vstream->index_sent_seqcount,
+                               vstream->stream->stream_handle);
+       }
 end:
+       if (metadata_viewer_stream) {
+               viewer_stream_put(metadata_viewer_stream);
+       }
+       if (vstream) {
+               viewer_stream_put(vstream);
+       }
+       return ret;
+
+error_put:
+       pthread_mutex_unlock(&rstream->lock);
+       if (metadata_viewer_stream) {
+               viewer_stream_put(metadata_viewer_stream);
+       }
+       viewer_stream_put(vstream);
        return ret;
 }
 
@@ -1277,162 +1533,104 @@ end:
 static
 int viewer_get_packet(struct relay_connection *conn)
 {
-       int ret, send_data = 0;
-       char *data = NULL;
-       uint32_t len = 0;
-       ssize_t read_len;
+       int ret;
+       off_t lseek_ret;
+       char *reply = NULL;
        struct lttng_viewer_get_packet get_packet_info;
-       struct lttng_viewer_trace_packet reply;
-       struct relay_viewer_stream *stream;
-       struct relay_session *session;
-       struct ctf_trace *ctf_trace;
-
-       assert(conn);
+       struct lttng_viewer_trace_packet reply_header;
+       struct relay_viewer_stream *vstream = NULL;
+       uint32_t reply_size = sizeof(reply_header);
+       uint32_t packet_data_len = 0;
+       ssize_t read_len;
 
        DBG2("Relay get data packet");
 
        health_code_update();
 
-       ret = recv_request(conn->sock, &get_packet_info, sizeof(get_packet_info));
+       ret = recv_request(conn->sock, &get_packet_info,
+                       sizeof(get_packet_info));
        if (ret < 0) {
                goto end;
        }
        health_code_update();
 
        /* From this point on, the error label can be reached. */
-       memset(&reply, 0, sizeof(reply));
+       memset(&reply_header, 0, sizeof(reply_header));
 
-       rcu_read_lock();
-       stream = viewer_stream_find_by_id(be64toh(get_packet_info.stream_id));
-       if (!stream) {
-               goto error;
+       vstream = viewer_stream_get_by_id(be64toh(get_packet_info.stream_id));
+       if (!vstream) {
+               DBG("Client requested packet of unknown stream id %" PRIu64,
+                               (uint64_t) be64toh(get_packet_info.stream_id));
+               reply_header.status = htobe32(LTTNG_VIEWER_GET_PACKET_ERR);
+               goto send_reply_nolock;
+       } else {
+               packet_data_len = be32toh(get_packet_info.len);
+               reply_size += packet_data_len;
        }
 
-       session = session_find_by_id(conn->sessions_ht, stream->session_id);
-       if (!session) {
-               ret = -1;
+       reply = zmalloc(reply_size);
+       if (!reply) {
+               PERROR("packet reply zmalloc");
+               reply_size = sizeof(reply_header);
                goto error;
        }
 
-       ctf_trace = ctf_trace_find_by_path(session->ctf_traces_ht,
-                       stream->path_name);
-       assert(ctf_trace);
-
-       /*
-        * First time we read this stream, we need open the tracefile, we should
-        * only arrive here if an index has already been sent to the viewer, so the
-        * tracefile must exist, if it does not it is a fatal error.
-        */
-       if (stream->read_fd < 0) {
-               char fullpath[PATH_MAX];
-
-               if (stream->tracefile_count > 0) {
-                       ret = snprintf(fullpath, PATH_MAX, "%s/%s_%" PRIu64, stream->path_name,
-                                       stream->channel_name,
-                                       stream->tracefile_count_current);
-               } else {
-                       ret = snprintf(fullpath, PATH_MAX, "%s/%s", stream->path_name,
-                                       stream->channel_name);
-               }
-               if (ret < 0) {
-                       goto error;
-               }
-               ret = open(fullpath, O_RDONLY);
-               if (ret < 0) {
-                       PERROR("Relay opening trace file");
-                       goto error;
-               }
-               stream->read_fd = ret;
-       }
-
-       if (!ctf_trace->metadata_received ||
-                       ctf_trace->metadata_received > ctf_trace->metadata_sent) {
-               reply.status = htobe32(LTTNG_VIEWER_GET_PACKET_ERR);
-               reply.flags |= LTTNG_VIEWER_FLAG_NEW_METADATA;
-               goto send_reply;
-       }
-
-       ret = check_new_streams(stream->session_id, conn->sessions_ht);
-       if (ret < 0) {
-               goto end_unlock;
-       } else if (ret == 1) {
-               reply.status = htobe32(LTTNG_VIEWER_GET_PACKET_ERR);
-               reply.flags |= LTTNG_VIEWER_FLAG_NEW_STREAM;
-               goto send_reply;
-       }
-
-       len = be32toh(get_packet_info.len);
-       data = zmalloc(len);
-       if (!data) {
-               PERROR("relay data zmalloc");
+       pthread_mutex_lock(&vstream->stream->lock);
+       lseek_ret = lseek(vstream->stream_file.fd->fd,
+                       be64toh(get_packet_info.offset), SEEK_SET);
+       if (lseek_ret < 0) {
+               PERROR("lseek fd %d to offset %" PRIu64,
+                               vstream->stream_file.fd->fd,
+                               (uint64_t) be64toh(get_packet_info.offset));
                goto error;
        }
-
-       ret = lseek(stream->read_fd, be64toh(get_packet_info.offset), SEEK_SET);
-       if (ret < 0) {
-               /*
-                * If the read fd was closed by the streaming side, the
-                * abort_flag will be set to 1, otherwise it is an error.
-                */
-               if (stream->abort_flag == 0) {
-                       PERROR("lseek");
-                       goto error;
-               }
-               reply.status = htobe32(LTTNG_VIEWER_GET_PACKET_EOF);
-               goto send_reply;
-       }
-       read_len = lttng_read(stream->read_fd, data, len);
-       if (read_len < len) {
-               /*
-                * If the read fd was closed by the streaming side, the
-                * abort_flag will be set to 1, otherwise it is an error.
-                */
-               if (stream->abort_flag == 0) {
-                       PERROR("Relay reading trace file, fd: %d, offset: %" PRIu64,
-                                       stream->read_fd,
-                                       be64toh(get_packet_info.offset));
-                       goto error;
-               } else {
-                       reply.status = htobe32(LTTNG_VIEWER_GET_PACKET_EOF);
-                       goto send_reply;
-               }
+       read_len = lttng_read(vstream->stream_file.fd->fd,
+                       reply + sizeof(reply_header), packet_data_len);
+       if (read_len < packet_data_len) {
+               PERROR("Relay reading trace file, fd: %d, offset: %" PRIu64,
+                               vstream->stream_file.fd->fd,
+                               (uint64_t) be64toh(get_packet_info.offset));
+               goto error;
        }
-       reply.status = htobe32(LTTNG_VIEWER_GET_PACKET_OK);
-       reply.len = htobe32(len);
-       send_data = 1;
+       reply_header.status = htobe32(LTTNG_VIEWER_GET_PACKET_OK);
+       reply_header.len = htobe32(packet_data_len);
        goto send_reply;
 
 error:
-       reply.status = htobe32(LTTNG_VIEWER_GET_PACKET_ERR);
+       reply_header.status = htobe32(LTTNG_VIEWER_GET_PACKET_ERR);
 
 send_reply:
-       reply.flags = htobe32(reply.flags);
+       if (vstream) {
+               pthread_mutex_unlock(&vstream->stream->lock);
+       }
+send_reply_nolock:
 
        health_code_update();
 
-       ret = send_response(conn->sock, &reply, sizeof(reply));
-       if (ret < 0) {
-               goto end_unlock;
+       if (reply) {
+               memcpy(reply, &reply_header, sizeof(reply_header));
+               ret = send_response(conn->sock, reply, reply_size);
+       } else {
+               /* No reply to send. */
+               ret = send_response(conn->sock, &reply_header,
+                               reply_size);
        }
-       health_code_update();
 
-       if (send_data) {
-               health_code_update();
-               ret = send_response(conn->sock, data, len);
-               if (ret < 0) {
-                       goto end_unlock;
-               }
-               health_code_update();
+       health_code_update();
+       if (ret < 0) {
+               PERROR("sendmsg of packet data failed");
+               goto end_free;
        }
 
-       DBG("Sent %u bytes for stream %" PRIu64, len,
-                       be64toh(get_packet_info.stream_id));
-
-end_unlock:
-       free(data);
-       rcu_read_unlock();
+       DBG("Sent %u bytes for stream %" PRIu64, reply_size,
+                       (uint64_t) be64toh(get_packet_info.stream_id));
 
+end_free:
+       free(reply);
 end:
+       if (vstream) {
+               viewer_stream_put(vstream);
+       }
        return ret;
 }
 
@@ -1450,9 +1648,7 @@ int viewer_get_metadata(struct relay_connection *conn)
        char *data = NULL;
        struct lttng_viewer_get_metadata request;
        struct lttng_viewer_metadata_packet reply;
-       struct relay_viewer_stream *stream;
-       struct ctf_trace *ctf_trace;
-       struct relay_session *session;
+       struct relay_viewer_stream *vstream = NULL;
 
        assert(conn);
 
@@ -1466,45 +1662,66 @@ int viewer_get_metadata(struct relay_connection *conn)
        }
        health_code_update();
 
-       rcu_read_lock();
-       stream = viewer_stream_find_by_id(be64toh(request.stream_id));
-       if (!stream || !stream->metadata_flag) {
-               ERR("Invalid metadata stream");
-               goto error;
-       }
+       memset(&reply, 0, sizeof(reply));
 
-       session = session_find_by_id(conn->sessions_ht, stream->session_id);
-       if (!session) {
-               ret = -1;
+       vstream = viewer_stream_get_by_id(be64toh(request.stream_id));
+       if (!vstream) {
+               /*
+                * The metadata stream can be closed by a CLOSE command
+                * just before we attach. It can also be closed by
+                * per-pid tracing during tracing. Therefore, it is
+                * possible that we cannot find this viewer stream.
+                * Reply back to the client with an error if we cannot
+                * find it.
+                */
+               DBG("Client requested metadata of unknown stream id %" PRIu64,
+                               (uint64_t) be64toh(request.stream_id));
+               reply.status = htobe32(LTTNG_VIEWER_METADATA_ERR);
+               goto send_reply;
+       }
+       pthread_mutex_lock(&vstream->stream->lock);
+       if (!vstream->stream->is_metadata) {
+               ERR("Invalid metadata stream");
                goto error;
        }
 
-       ctf_trace = ctf_trace_find_by_path(session->ctf_traces_ht,
-                       stream->path_name);
-       assert(ctf_trace);
-       assert(ctf_trace->metadata_sent <= ctf_trace->metadata_received);
+       assert(vstream->metadata_sent <= vstream->stream->metadata_received);
 
-       len = ctf_trace->metadata_received - ctf_trace->metadata_sent;
+       len = vstream->stream->metadata_received - vstream->metadata_sent;
        if (len == 0) {
                reply.status = htobe32(LTTNG_VIEWER_NO_NEW_METADATA);
                goto send_reply;
        }
 
        /* first time, we open the metadata file */
-       if (stream->read_fd < 0) {
-               char fullpath[PATH_MAX];
-
-               ret = snprintf(fullpath, PATH_MAX, "%s/%s", stream->path_name,
-                               stream->channel_name);
+       if (!vstream->stream_file.fd) {
+               int fd;
+               char file_path[LTTNG_PATH_MAX];
+               enum lttng_trace_chunk_status status;
+               struct relay_stream *rstream = vstream->stream;
+
+               ret = utils_stream_file_path(rstream->path_name,
+                               rstream->channel_name, rstream->tracefile_size,
+                               vstream->current_tracefile_id, NULL, file_path,
+                               sizeof(file_path));
                if (ret < 0) {
                        goto error;
                }
-               ret = open(fullpath, O_RDONLY);
-               if (ret < 0) {
-                       PERROR("Relay opening metadata file");
+
+               status = lttng_trace_chunk_open_file(
+                               vstream->stream_file.trace_chunk,
+                               file_path, O_RDONLY, 0, &fd);
+               if (status != LTTNG_TRACE_CHUNK_STATUS_OK) {
+                       PERROR("Failed to open metadata file for viewer stream");
+                       goto error;
+               }
+               vstream->stream_file.fd = stream_fd_create(fd);
+               if (!vstream->stream_file.fd) {
+                       if (close(fd)) {
+                               PERROR("Failed to close viewer metadata file");
+                       }
                        goto error;
                }
-               stream->read_fd = ret;
        }
 
        reply.len = htobe64(len);
@@ -1514,13 +1731,20 @@ int viewer_get_metadata(struct relay_connection *conn)
                goto error;
        }
 
-       read_len = lttng_read(stream->read_fd, data, len);
+       read_len = lttng_read(vstream->stream_file.fd->fd, data, len);
        if (read_len < len) {
                PERROR("Relay reading metadata file");
                goto error;
        }
-       ctf_trace->metadata_sent += read_len;
+       vstream->metadata_sent += read_len;
+       if (vstream->metadata_sent == vstream->stream->metadata_received
+                       && vstream->stream->closed) {
+               /* Release ownership for the viewer metadata stream. */
+               viewer_stream_put(vstream);
+       }
+
        reply.status = htobe32(LTTNG_VIEWER_METADATA_OK);
+
        goto send_reply;
 
 error:
@@ -1528,28 +1752,33 @@ error:
 
 send_reply:
        health_code_update();
+       if (vstream) {
+               pthread_mutex_unlock(&vstream->stream->lock);
+       }
        ret = send_response(conn->sock, &reply, sizeof(reply));
        if (ret < 0) {
-               goto end_unlock;
+               goto end_free;
        }
        health_code_update();
 
        if (len > 0) {
                ret = send_response(conn->sock, data, len);
                if (ret < 0) {
-                       goto end_unlock;
+                       goto end_free;
                }
        }
 
        DBG("Sent %" PRIu64 " bytes of metadata for stream %" PRIu64, len,
-                       be64toh(request.stream_id));
+                       (uint64_t) be64toh(request.stream_id));
 
        DBG("Metadata sent");
 
-end_unlock:
+end_free:
        free(data);
-       rcu_read_unlock();
 end:
+       if (vstream) {
+               viewer_stream_put(vstream);
+       }
        return ret;
 }
 
@@ -1566,14 +1795,14 @@ int viewer_create_session(struct relay_connection *conn)
 
        DBG("Viewer create session received");
 
+       memset(&resp, 0, sizeof(resp));
        resp.status = htobe32(LTTNG_VIEWER_CREATE_SESSION_OK);
-       conn->viewer_session = zmalloc(sizeof(conn->viewer_session));
+       conn->viewer_session = viewer_session_create();
        if (!conn->viewer_session) {
                ERR("Allocation viewer session");
                resp.status = htobe32(LTTNG_VIEWER_CREATE_SESSION_ERR);
                goto send_reply;
        }
-       CDS_INIT_LIST_HEAD(&conn->viewer_session->sessions_head);
 
 send_reply:
        health_code_update();
@@ -1588,6 +1817,78 @@ end:
        return ret;
 }
 
+/*
+ * Detach a viewer session.
+ *
+ * Return 0 on success or else a negative value.
+ */
+static
+int viewer_detach_session(struct relay_connection *conn)
+{
+       int ret;
+       struct lttng_viewer_detach_session_response response;
+       struct lttng_viewer_detach_session_request request;
+       struct relay_session *session = NULL;
+       uint64_t viewer_session_to_close;
+
+       DBG("Viewer detach session received");
+
+       assert(conn);
+
+       health_code_update();
+
+       /* Receive the request from the connected client. */
+       ret = recv_request(conn->sock, &request, sizeof(request));
+       if (ret < 0) {
+               goto end;
+       }
+       viewer_session_to_close = be64toh(request.session_id);
+
+       if (!conn->viewer_session) {
+               DBG("Client trying to detach before creating a live viewer session");
+               response.status = htobe32(LTTNG_VIEWER_DETACH_SESSION_ERR);
+               goto send_reply;
+       }
+
+       health_code_update();
+
+       memset(&response, 0, sizeof(response));
+       DBG("Detaching from session ID %" PRIu64, viewer_session_to_close);
+
+       session = session_get_by_id(be64toh(request.session_id));
+       if (!session) {
+               DBG("Relay session %" PRIu64 " not found",
+                               (uint64_t) be64toh(request.session_id));
+               response.status = htobe32(LTTNG_VIEWER_DETACH_SESSION_UNK);
+               goto send_reply;
+       }
+
+       ret = viewer_session_is_attached(conn->viewer_session, session);
+       if (ret != 1) {
+               DBG("Not attached to this session");
+               response.status = htobe32(LTTNG_VIEWER_DETACH_SESSION_ERR);
+               goto send_reply_put;
+       }
+
+       viewer_session_close_one_session(conn->viewer_session, session);
+       response.status = htobe32(LTTNG_VIEWER_DETACH_SESSION_OK);
+       DBG("Session %" PRIu64 " detached.", viewer_session_to_close);
+
+send_reply_put:
+       session_put(session);
+
+send_reply:
+       health_code_update();
+       ret = send_response(conn->sock, &response, sizeof(response));
+       if (ret < 0) {
+               goto end;
+       }
+       health_code_update();
+       ret = 0;
+
+end:
+       return ret;
+}
 
 /*
  * live_relay_unknown_command: send -1 if received unknown command
@@ -1597,6 +1898,7 @@ void live_relay_unknown_command(struct relay_connection *conn)
 {
        struct lttcomm_relayd_generic_reply reply;
 
+       memset(&reply, 0, sizeof(reply));
        reply.ret_code = htobe32(LTTNG_ERR_UNK);
        (void) send_response(conn->sock, &reply, sizeof(reply));
 }
@@ -1611,9 +1913,6 @@ int process_control(struct lttng_viewer_cmd *recv_hdr,
        int ret = 0;
        uint32_t msg_value;
 
-       assert(recv_hdr);
-       assert(conn);
-
        msg_value = be32toh(recv_hdr->cmd);
 
        /*
@@ -1651,8 +1950,12 @@ int process_control(struct lttng_viewer_cmd *recv_hdr,
        case LTTNG_VIEWER_CREATE_SESSION:
                ret = viewer_create_session(conn);
                break;
+       case LTTNG_VIEWER_DETACH_SESSION:
+               ret = viewer_detach_session(conn);
+               break;
        default:
-               ERR("Received unknown viewer command (%u)", be32toh(recv_hdr->cmd));
+               ERR("Received unknown viewer command (%u)",
+                               be32toh(recv_hdr->cmd));
                live_relay_unknown_command(conn);
                ret = -1;
                goto end;
@@ -1667,8 +1970,6 @@ void cleanup_connection_pollfd(struct lttng_poll_event *events, int pollfd)
 {
        int ret;
 
-       assert(events);
-
        (void) lttng_poll_del(events, pollfd);
 
        ret = close(pollfd);
@@ -1677,95 +1978,6 @@ void cleanup_connection_pollfd(struct lttng_poll_event *events, int pollfd)
        }
 }
 
-/*
- * Delete all streams for a specific session ID.
- */
-static void destroy_viewer_streams_by_session(struct relay_session *session)
-{
-       struct relay_viewer_stream *stream;
-       struct lttng_ht_iter iter;
-
-       assert(session);
-
-       rcu_read_lock();
-       cds_lfht_for_each_entry(viewer_streams_ht->ht, &iter.iter, stream,
-                       stream_n.node) {
-               struct ctf_trace *ctf_trace;
-
-               health_code_update();
-               if (stream->session_id != session->id) {
-                       continue;
-               }
-
-               ctf_trace = ctf_trace_find_by_path(session->ctf_traces_ht,
-                               stream->path_name);
-               assert(ctf_trace);
-
-               viewer_stream_delete(stream);
-
-               if (stream->metadata_flag) {
-                       ctf_trace->metadata_sent = 0;
-                       ctf_trace->viewer_metadata_stream = NULL;
-               }
-
-               viewer_stream_destroy(ctf_trace, stream);
-       }
-       rcu_read_unlock();
-}
-
-static void try_destroy_streams(struct relay_session *session)
-{
-       struct ctf_trace *ctf_trace;
-       struct lttng_ht_iter iter;
-
-       assert(session);
-
-       cds_lfht_for_each_entry(session->ctf_traces_ht->ht, &iter.iter, ctf_trace,
-                       node.node) {
-               /* Attempt to destroy the ctf trace of that session. */
-               ctf_trace_try_destroy(session, ctf_trace);
-       }
-}
-
-/*
- * Delete and destroy a connection.
- *
- * RCU read side lock MUST be acquired.
- */
-static void destroy_connection(struct lttng_ht *relay_connections_ht,
-               struct relay_connection *conn)
-{
-       struct relay_session *session, *tmp_session;
-
-       assert(relay_connections_ht);
-       assert(conn);
-
-       connection_delete(relay_connections_ht, conn);
-
-       if (!conn->viewer_session) {
-               goto end;
-       }
-
-       rcu_read_lock();
-       cds_list_for_each_entry_safe(session, tmp_session,
-                       &conn->viewer_session->sessions_head,
-                       viewer_session_list) {
-               DBG("Cleaning connection of session ID %" PRIu64, session->id);
-               /*
-                * Very important that this is done before destroying the session so we
-                * can put back every viewer stream reference from the ctf_trace.
-                */
-               destroy_viewer_streams_by_session(session);
-               try_destroy_streams(session);
-               cds_list_del(&session->viewer_session_list);
-               session_viewer_try_destroy(conn->sessions_ht, session);
-       }
-       rcu_read_unlock();
-
-end:
-       connection_destroy(conn);
-}
-
 /*
  * This thread does the actual work
  */
@@ -1774,13 +1986,11 @@ void *thread_worker(void *data)
 {
        int ret, err = -1;
        uint32_t nb_fd;
-       struct relay_connection *conn;
        struct lttng_poll_event events;
-       struct lttng_ht *relay_connections_ht;
+       struct lttng_ht *viewer_connections_ht;
        struct lttng_ht_iter iter;
        struct lttng_viewer_cmd recv_hdr;
-       struct relay_local_data *relay_ctx = (struct relay_local_data *) data;
-       struct lttng_ht *sessions_ht = relay_ctx->sessions_ht;
+       struct relay_connection *destroy_conn;
 
        DBG("[thread] Live viewer relay worker started");
 
@@ -1793,9 +2003,9 @@ void *thread_worker(void *data)
        }
 
        /* table of connections indexed on socket */
-       relay_connections_ht = lttng_ht_new(0, LTTNG_HT_TYPE_ULONG);
-       if (!relay_connections_ht) {
-               goto relay_connections_ht_error;
+       viewer_connections_ht = lttng_ht_new(0, LTTNG_HT_TYPE_ULONG);
+       if (!viewer_connections_ht) {
+               goto viewer_connections_ht_error;
        }
 
        ret = create_thread_poll_set(&events, 2);
@@ -1844,60 +2054,77 @@ restart:
                        health_code_update();
 
                        /* Thread quit pipe has been closed. Killing thread. */
-                       ret = check_live_conn_pipe(pollfd, revents);
+                       ret = check_thread_quit_pipe(pollfd, revents);
                        if (ret) {
                                err = 0;
                                goto exit;
                        }
 
-                       /* Inspect the relay conn pipe for new connection */
+                       /* Inspect the relay conn pipe for new connection. */
                        if (pollfd == live_conn_pipe[0]) {
-                               if (revents & (LPOLLERR | LPOLLHUP | LPOLLRDHUP)) {
-                                       ERR("Relay live pipe error");
-                                       goto error;
-                               } else if (revents & LPOLLIN) {
-                                       ret = lttng_read(live_conn_pipe[0], &conn, sizeof(conn));
+                               if (revents & LPOLLIN) {
+                                       struct relay_connection *conn;
+
+                                       ret = lttng_read(live_conn_pipe[0],
+                                                       &conn, sizeof(conn));
                                        if (ret < 0) {
                                                goto error;
                                        }
-                                       conn->sessions_ht = sessions_ht;
-                                       connection_init(conn);
-                                       lttng_poll_add(&events, conn->sock->fd,
+                                       ret = lttng_poll_add(&events,
+                                                       conn->sock->fd,
                                                        LPOLLIN | LPOLLRDHUP);
-                                       rcu_read_lock();
-                                       lttng_ht_add_unique_ulong(relay_connections_ht,
-                                                       &conn->sock_n);
-                                       rcu_read_unlock();
-                                       DBG("Connection socket %d added", conn->sock->fd);
+                                       if (ret) {
+                                               ERR("Failed to add new live connection file descriptor to poll set");
+                                               goto error;
+                                       }
+                                       connection_ht_add(viewer_connections_ht, conn);
+                                       DBG("Connection socket %d added to poll", conn->sock->fd);
+                               } else if (revents & (LPOLLERR | LPOLLHUP | LPOLLRDHUP)) {
+                                       ERR("Relay live pipe error");
+                                       goto error;
+                               } else {
+                                       ERR("Unexpected poll events %u for sock %d", revents, pollfd);
+                                       goto error;
                                }
                        } else {
-                               rcu_read_lock();
-                               conn = connection_find_by_sock(relay_connections_ht, pollfd);
-                               /* If not found, there is a synchronization issue. */
-                               assert(conn);
+                               /* Connection activity. */
+                               struct relay_connection *conn;
 
-                               if (revents & (LPOLLERR | LPOLLHUP | LPOLLRDHUP)) {
-                                       cleanup_connection_pollfd(&events, pollfd);
-                                       destroy_connection(relay_connections_ht, conn);
-                               } else if (revents & LPOLLIN) {
+                               conn = connection_get_by_sock(viewer_connections_ht, pollfd);
+                               if (!conn) {
+                                       continue;
+                               }
+
+                               if (revents & LPOLLIN) {
                                        ret = conn->sock->ops->recvmsg(conn->sock, &recv_hdr,
                                                        sizeof(recv_hdr), 0);
                                        if (ret <= 0) {
-                                               /* Connection closed */
+                                               /* Connection closed. */
                                                cleanup_connection_pollfd(&events, pollfd);
-                                               destroy_connection(relay_connections_ht, conn);
+                                               /* Put "create" ownership reference. */
+                                               connection_put(conn);
                                                DBG("Viewer control conn closed with %d", pollfd);
                                        } else {
                                                ret = process_control(&recv_hdr, conn);
                                                if (ret < 0) {
                                                        /* Clear the session on error. */
                                                        cleanup_connection_pollfd(&events, pollfd);
-                                                       destroy_connection(relay_connections_ht, conn);
+                                                       /* Put "create" ownership reference. */
+                                                       connection_put(conn);
                                                        DBG("Viewer connection closed with %d", pollfd);
                                                }
                                        }
+                               } else if (revents & (LPOLLERR | LPOLLHUP | LPOLLRDHUP)) {
+                                       cleanup_connection_pollfd(&events, pollfd);
+                                       /* Put "create" ownership reference. */
+                                       connection_put(conn);
+                               } else {
+                                       ERR("Unexpected poll events %u for sock %d", revents, pollfd);
+                                       connection_put(conn);
+                                       goto error;
                                }
-                               rcu_read_unlock();
+                               /* Put local "get_by_sock" reference. */
+                               connection_put(conn);
                        }
                }
        }
@@ -1906,17 +2133,18 @@ exit:
 error:
        lttng_poll_clean(&events);
 
-       /* Cleanup reamaining connection object. */
+       /* Cleanup remaining connection object. */
        rcu_read_lock();
-       cds_lfht_for_each_entry(relay_connections_ht->ht, &iter.iter, conn,
+       cds_lfht_for_each_entry(viewer_connections_ht->ht, &iter.iter,
+                       destroy_conn,
                        sock_n.node) {
                health_code_update();
-               destroy_connection(relay_connections_ht, conn);
+               connection_put(destroy_conn);
        }
        rcu_read_unlock();
 error_poll_create:
-       lttng_ht_destroy(relay_connections_ht);
-relay_connections_ht_error:
+       lttng_ht_destroy(viewer_connections_ht);
+viewer_connections_ht_error:
        /* Close relay conn pipes */
        utils_close_pipe(live_conn_pipe);
        if (err) {
@@ -1929,7 +2157,9 @@ error_testpoint:
                ERR("Health error occurred in %s", __func__);
        }
        health_unregister(health_relayd);
-       stop_threads();
+       if (lttng_relay_stop_threads()) {
+               ERR("Error stopping threads");
+       }
        rcu_unregister_thread();
        return NULL;
 }
@@ -1940,55 +2170,53 @@ error_testpoint:
  */
 static int create_conn_pipe(void)
 {
-       int ret;
-
-       ret = utils_create_pipe_cloexec(live_conn_pipe);
-
-       return ret;
+       return utils_create_pipe_cloexec(live_conn_pipe);
 }
 
-void live_stop_threads(void)
+int relayd_live_join(void)
 {
-       int ret;
+       int ret, retval = 0;
        void *status;
 
-       stop_threads();
-
        ret = pthread_join(live_listener_thread, &status);
-       if (ret != 0) {
+       if (ret) {
+               errno = ret;
                PERROR("pthread_join live listener");
-               goto error;     /* join error, exit without cleanup */
+               retval = -1;
        }
 
        ret = pthread_join(live_worker_thread, &status);
-       if (ret != 0) {
+       if (ret) {
+               errno = ret;
                PERROR("pthread_join live worker");
-               goto error;     /* join error, exit without cleanup */
+               retval = -1;
        }
 
        ret = pthread_join(live_dispatcher_thread, &status);
-       if (ret != 0) {
+       if (ret) {
+               errno = ret;
                PERROR("pthread_join live dispatcher");
-               goto error;     /* join error, exit without cleanup */
+               retval = -1;
        }
 
-       cleanup();
+       cleanup_relayd_live();
 
-error:
-       return;
+       return retval;
 }
 
 /*
  * main
  */
-int live_start_threads(struct lttng_uri *uri,
-               struct relay_local_data *relay_ctx)
+int relayd_live_create(struct lttng_uri *uri)
 {
-       int ret = 0;
+       int ret = 0, retval = 0;
        void *status;
        int is_root;
 
-       assert(uri);
+       if (!uri) {
+               retval = -1;
+               goto exit_init_data;
+       }
        live_uri = uri;
 
        /* Check if daemon is UID = 0 */
@@ -1997,74 +2225,86 @@ int live_start_threads(struct lttng_uri *uri,
        if (!is_root) {
                if (live_uri->port < 1024) {
                        ERR("Need to be root to use ports < 1024");
-                       ret = -1;
-                       goto exit;
+                       retval = -1;
+                       goto exit_init_data;
                }
        }
 
        /* Setup the thread apps communication pipe. */
-       if ((ret = create_conn_pipe()) < 0) {
-               goto exit;
+       if (create_conn_pipe()) {
+               retval = -1;
+               goto exit_init_data;
        }
 
        /* Init relay command queue. */
-       cds_wfq_init(&viewer_conn_queue.queue);
+       cds_wfcq_init(&viewer_conn_queue.head, &viewer_conn_queue.tail);
 
        /* Set up max poll set size */
-       lttng_poll_set_max_size();
+       if (lttng_poll_set_max_size()) {
+               retval = -1;
+               goto exit_init_data;
+       }
 
        /* Setup the dispatcher thread */
-       ret = pthread_create(&live_dispatcher_thread, NULL,
+       ret = pthread_create(&live_dispatcher_thread, default_pthread_attr(),
                        thread_dispatcher, (void *) NULL);
-       if (ret != 0) {
+       if (ret) {
+               errno = ret;
                PERROR("pthread_create viewer dispatcher");
-               goto exit_dispatcher;
+               retval = -1;
+               goto exit_dispatcher_thread;
        }
 
        /* Setup the worker thread */
-       ret = pthread_create(&live_worker_thread, NULL,
-                       thread_worker, relay_ctx);
-       if (ret != 0) {
+       ret = pthread_create(&live_worker_thread, default_pthread_attr(),
+                       thread_worker, NULL);
+       if (ret) {
+               errno = ret;
                PERROR("pthread_create viewer worker");
-               goto exit_worker;
+               retval = -1;
+               goto exit_worker_thread;
        }
 
        /* Setup the listener thread */
-       ret = pthread_create(&live_listener_thread, NULL,
+       ret = pthread_create(&live_listener_thread, default_pthread_attr(),
                        thread_listener, (void *) NULL);
-       if (ret != 0) {
+       if (ret) {
+               errno = ret;
                PERROR("pthread_create viewer listener");
-               goto exit_listener;
+               retval = -1;
+               goto exit_listener_thread;
        }
 
-       ret = 0;
-       goto end;
+       /*
+        * All OK, started all threads.
+        */
+       return retval;
 
-exit_listener:
-       ret = pthread_join(live_listener_thread, &status);
-       if (ret != 0) {
-               PERROR("pthread_join live listener");
-               goto error;     /* join error, exit without cleanup */
-       }
+       /*
+        * Join on the live_listener_thread should anything be added after
+        * the live_listener thread's creation.
+        */
+
+exit_listener_thread:
 
-exit_worker:
        ret = pthread_join(live_worker_thread, &status);
-       if (ret != 0) {
+       if (ret) {
+               errno = ret;
                PERROR("pthread_join live worker");
-               goto error;     /* join error, exit without cleanup */
+               retval = -1;
        }
+exit_worker_thread:
 
-exit_dispatcher:
        ret = pthread_join(live_dispatcher_thread, &status);
-       if (ret != 0) {
+       if (ret) {
+               errno = ret;
                PERROR("pthread_join live dispatcher");
-               goto error;     /* join error, exit without cleanup */
+               retval = -1;
        }
+exit_dispatcher_thread:
 
-exit:
-       cleanup();
+exit_init_data:
+       cleanup_relayd_live();
 
-end:
-error:
-       return ret;
+       return retval;
 }
This page took 0.100971 seconds and 4 git commands to generate.