Make app socket directories per-user v2
[ust.git] / libust / tracectl.c
index 3d09cc2a136ea5e49e48ba19d6d1c40b443a79cb..ae92b7e17577f279d30456622842037d371455ca 100644 (file)
@@ -39,6 +39,7 @@
 #include <ust/marker.h>
 #include <ust/tracepoint.h>
 #include <ust/tracectl.h>
+#include <ust/clock.h>
 #include "tracer.h"
 #include "usterr.h"
 #include "ustcomm.h"
  */
 s64 pidunique = -1LL;
 
+/* The process pid is used to detect a non-traceable fork
+ * and allow the non-traceable fork to be ignored
+ * by destructor sequences in libust
+ */
+static pid_t processpid = 0;
+
 static struct ustcomm_header _receive_header;
 static struct ustcomm_header *receive_header = &_receive_header;
 static char receive_buffer[USTCOMM_BUFFER_SIZE];
 static char send_buffer[USTCOMM_BUFFER_SIZE];
 
 static int epoll_fd;
+
+/*
+ * Listener thread data vs fork() protection mechanism. Ensures that no listener
+ * thread mutexes and data structures are being concurrently modified or held by
+ * other threads when fork() is executed.
+ */
+static pthread_mutex_t listener_thread_data_mutex = PTHREAD_MUTEX_INITIALIZER;
+
+/* Mutex protecting listen_sock. Nests inside listener_thread_data_mutex. */
+static pthread_mutex_t listen_sock_mutex = PTHREAD_MUTEX_INITIALIZER;
 static struct ustcomm_sock *listen_sock;
 
 extern struct chan_info_struct chan_infos[];
 
-static struct list_head open_buffers_list = LIST_HEAD_INIT(open_buffers_list);
-
-static struct list_head ust_socks = LIST_HEAD_INIT(ust_socks);
+static struct cds_list_head ust_socks = CDS_LIST_HEAD_INIT(ust_socks);
 
 /* volatile because shared between the listener and the main thread */
 int buffers_to_export = 0;
 
+int ust_clock_source;
+
 static long long make_pidunique(void)
 {
        s64 retval;
@@ -91,11 +108,11 @@ static void print_markers(FILE *fp)
 
        while (iter.marker) {
                fprintf(fp, "marker: %s/%s %d \"%s\" %p\n",
-                       iter.marker->channel,
-                       iter.marker->name,
-                       (int)imv_read(iter.marker->state),
-                       iter.marker->format,
-                       iter.marker->location);
+                       (*iter.marker)->channel,
+                       (*iter.marker)->name,
+                       (int)imv_read((*iter.marker)->state),
+                       (*iter.marker)->format,
+                       (*iter.marker)->location);
                marker_iter_next(&iter);
        }
        unlock_markers();
@@ -110,16 +127,16 @@ static void print_trace_events(FILE *fp)
        trace_event_iter_start(&iter);
 
        while (iter.trace_event) {
-               fprintf(fp, "trace_event: %s\n", iter.trace_event->name);
+               fprintf(fp, "trace_event: %s\n", (*iter.trace_event)->name);
                trace_event_iter_next(&iter);
        }
        unlock_trace_events();
 }
 
-static int connect_ustd(void)
+static int connect_ustconsumer(void)
 {
        int result, fd;
-       char default_daemon_path[] = SOCK_DIR "/ustd";
+       char default_daemon_path[] = SOCK_DIR "/ustconsumer";
        char *explicit_daemon_path, *daemon_path;
 
        explicit_daemon_path = getenv("UST_DAEMON_SOCKET");
@@ -133,7 +150,7 @@ static int connect_ustd(void)
 
        result = ustcomm_connect_path(daemon_path, &fd);
        if (result < 0) {
-               WARN("connect_ustd failed, daemon_path: %s",
+               WARN("connect_ustconsumer failed, daemon_path: %s",
                     daemon_path);
                return result;
        }
@@ -143,8 +160,9 @@ static int connect_ustd(void)
 
 
 static void request_buffer_consumer(int sock,
-                                  const char *channel,
-                                  int cpu)
+                                   const char *trace,
+                                   const char *channel,
+                                   int cpu)
 {
        struct ustcomm_header send_header, recv_header;
        struct ustcomm_buffer_info buf_inf;
@@ -152,6 +170,7 @@ static void request_buffer_consumer(int sock,
 
        result = ustcomm_pack_buffer_info(&send_header,
                                          &buf_inf,
+                                         trace,
                                          channel,
                                          cpu);
 
@@ -186,12 +205,12 @@ static void inform_consumer_daemon(const char *trace_name)
        struct ust_trace *trace;
        const char *ch_name;
 
-       sock = connect_ustd();
+       sock = connect_ustconsumer();
        if (sock < 0) {
                return;
        }
 
-       DBG("Connected to ustd");
+       DBG("Connected to ustconsumer");
 
        ltt_lock_traces();
 
@@ -206,9 +225,10 @@ static void inform_consumer_daemon(const char *trace_name)
                        /* iterate on all cpus */
                        for (j=0; j<trace->channels[i].n_cpus; j++) {
                                ch_name = trace->channels[i].channel_name;
-                               request_buffer_consumer(sock, ch_name, j);
-                               STORE_SHARED(buffers_to_export,
-                                            LOAD_SHARED(buffers_to_export)+1);
+                               request_buffer_consumer(sock, trace_name,
+                                                       ch_name, j);
+                               CMM_STORE_SHARED(buffers_to_export,
+                                            CMM_LOAD_SHARED(buffers_to_export)+1);
                        }
                }
        }
@@ -343,7 +363,7 @@ static int set_subbuf_size(const char *trace_name, const char *ch_name,
        }
 
        channel->subbuf_size = power;
-       DBG("the set_subbuf_size for the requested channel is %u", channel->subbuf_size);
+       DBG("the set_subbuf_size for the requested channel is %zu", channel->subbuf_size);
 
 unlock_traces:
        ltt_unlock_traces();
@@ -381,7 +401,7 @@ static int set_subbuf_num(const char *trace_name, const char *ch_name,
        }
 
        channel->subbuf_cnt = num;
-       DBG("the set_subbuf_cnt for the requested channel is %zd", channel->subbuf_cnt);
+       DBG("the set_subbuf_cnt for the requested channel is %u", channel->subbuf_cnt);
 
 unlock_traces:
        ltt_unlock_traces();
@@ -464,14 +484,9 @@ static int notify_buffer_mapped(const char *trace_name,
         */
        if (uatomic_read(&buf->consumed) == 0) {
                DBG("decrementing buffers_to_export");
-               STORE_SHARED(buffers_to_export, LOAD_SHARED(buffers_to_export)-1);
+               CMM_STORE_SHARED(buffers_to_export, CMM_LOAD_SHARED(buffers_to_export)-1);
        }
 
-       /* The buffer has been exported, ergo, we can add it to the
-        * list of open buffers
-        */
-       list_add(&buf->open_buffers_list, &open_buffers_list);
-
 unlock_traces:
        ltt_unlock_traces();
 
@@ -521,43 +536,53 @@ unlock_traces:
        return retval;
 }
 
+static void release_listener_mutex(void *ptr)
+{
+       pthread_mutex_unlock(&listener_thread_data_mutex);
+}
+
 static void listener_cleanup(void *ptr)
 {
-       ustcomm_del_named_sock(listen_sock, 0);
+       pthread_mutex_lock(&listen_sock_mutex);
+       if (listen_sock) {
+               ustcomm_del_named_sock(listen_sock, 0);
+               listen_sock = NULL;
+       }
+       pthread_mutex_unlock(&listen_sock_mutex);
 }
 
-static void force_subbuf_switch()
+static int force_subbuf_switch(const char *trace_name)
 {
-       struct ust_buffer *buf;
+       struct ust_trace *trace;
+       int i, j, retval = 0;
+
+       ltt_lock_traces();
+       trace = _ltt_trace_find(trace_name);
+       if (!trace) {
+                retval = -ENODATA;
+                DBG("Cannot find trace. It was likely destroyed by the user.");
+                goto unlock_traces;
+        }
 
-       list_for_each_entry(buf, &open_buffers_list,
-                           open_buffers_list) {
-               ltt_force_switch(buf, FORCE_FLUSH);
+       for (i = 0; i < trace->nr_channels; i++) {
+               for (j = 0; j < trace->channels[i].n_cpus; j++) {
+                       ltt_force_switch(trace->channels[i].buf[j],
+                                        FORCE_FLUSH);
+               }
        }
+
+unlock_traces:
+       ltt_unlock_traces();
+
+       return retval;
 }
 
-/* Simple commands are those which need only respond with a return value. */
-static int process_simple_client_cmd(int command, char *recv_buf)
+static int process_trace_cmd(int command, char *trace_name)
 {
        int result;
        char trace_type[] = "ustrelay";
-       char trace_name[] = "auto";
 
        switch(command) {
-       case SET_SOCK_PATH:
-       {
-               struct ustcomm_sock_path *sock_msg;
-               sock_msg = (struct ustcomm_sock_path *)recv_buf;
-               sock_msg->sock_path =
-                       ustcomm_restore_ptr(sock_msg->sock_path,
-                                           sock_msg->data,
-                                           sizeof(sock_msg->data));
-               if (!sock_msg->sock_path) {
-
-                       return -EINVAL;
-               }
-               return setenv("UST_DAEMON_SOCKET", sock_msg->sock_path, 1);
-       }
        case START:
                /* start is an operation that setups the trace, allocates it and starts it */
                result = ltt_trace_setup(trace_name);
@@ -670,18 +695,20 @@ static int process_simple_client_cmd(int command, char *recv_buf)
                }
                return 0;
        case FORCE_SUBBUF_SWITCH:
-               /* FIXME: return codes? */
-               force_subbuf_switch();
-
-               break;
+               DBG("force switch");
 
-       default:
-               return -EINVAL;
+               result = force_subbuf_switch(trace_name);
+               if (result < 0) {
+                       ERR("force_subbuf_switch failed");
+                       return result;
+               }
+               return 0;
        }
 
        return 0;
 }
 
+
 static void process_channel_cmd(int sock, int command,
                                struct ustcomm_channel_info *ch_inf)
 {
@@ -689,14 +716,13 @@ static void process_channel_cmd(int sock, int command,
        struct ustcomm_header *reply_header = &_reply_header;
        struct ustcomm_channel_info *reply_msg =
                (struct ustcomm_channel_info *)send_buffer;
-       char trace_name[] = "auto";
        int result, offset = 0, num, size;
 
        memset(reply_header, 0, sizeof(*reply_header));
 
        switch (command) {
        case GET_SUBBUF_NUM_SIZE:
-               result = get_subbuf_num_size(trace_name,
+               result = get_subbuf_num_size(ch_inf->trace,
                                             ch_inf->channel,
                                             &num, &size);
                if (result < 0) {
@@ -713,13 +739,13 @@ static void process_channel_cmd(int sock, int command,
 
                break;
        case SET_SUBBUF_NUM:
-               reply_header->result = set_subbuf_num(trace_name,
+               reply_header->result = set_subbuf_num(ch_inf->trace,
                                                      ch_inf->channel,
                                                      ch_inf->subbuf_num);
 
                break;
        case SET_SUBBUF_SIZE:
-               reply_header->result = set_subbuf_size(trace_name,
+               reply_header->result = set_subbuf_size(ch_inf->trace,
                                                       ch_inf->channel,
                                                       ch_inf->subbuf_size);
 
@@ -738,7 +764,6 @@ static void process_buffer_cmd(int sock, int command,
        struct ustcomm_header *reply_header = &_reply_header;
        struct ustcomm_buffer_info *reply_msg =
                (struct ustcomm_buffer_info *)send_buffer;
-       char trace_name[] = "auto";
        int result, offset = 0, buf_shmid, buf_struct_shmid, buf_pipe_fd;
        long consumed_old;
 
@@ -746,7 +771,8 @@ static void process_buffer_cmd(int sock, int command,
 
        switch (command) {
        case GET_BUF_SHMID_PIPE_FD:
-               result = get_buffer_shmid_pipe_fd(trace_name, buf_inf->channel,
+               result = get_buffer_shmid_pipe_fd(buf_inf->trace,
+                                                 buf_inf->channel,
                                                  buf_inf->ch_cpu,
                                                  &buf_shmid,
                                                  &buf_struct_shmid,
@@ -771,12 +797,12 @@ static void process_buffer_cmd(int sock, int command,
 
        case NOTIFY_BUF_MAPPED:
                reply_header->result =
-                       notify_buffer_mapped(trace_name,
+                       notify_buffer_mapped(buf_inf->trace,
                                             buf_inf->channel,
                                             buf_inf->ch_cpu);
                break;
        case GET_SUBBUFFER:
-               result = get_subbuffer(trace_name, buf_inf->channel,
+               result = get_subbuffer(buf_inf->trace, buf_inf->channel,
                                       buf_inf->ch_cpu, &consumed_old);
                if (result < 0) {
                        reply_header->result = result;
@@ -790,7 +816,7 @@ static void process_buffer_cmd(int sock, int command,
 
                break;
        case PUT_SUBBUFFER:
-               result = put_subbuffer(trace_name, buf_inf->channel,
+               result = put_subbuffer(buf_inf->trace, buf_inf->channel,
                                       buf_inf->ch_cpu,
                                       buf_inf->consumed_old);
                reply_header->result = result;
@@ -809,7 +835,7 @@ static void process_marker_cmd(int sock, int command,
 {
        struct ustcomm_header _reply_header;
        struct ustcomm_header *reply_header = &_reply_header;
-       int result;
+       int result = 0;
 
        memset(reply_header, 0, sizeof(*reply_header));
 
@@ -918,7 +944,7 @@ static void process_client_cmd(struct ustcomm_header *recv_header,
                print_markers(fp);
                fclose(fp);
 
-               reply_header->size = size;
+               reply_header->size = size + 1;  /* Include final \0 */
 
                result = ustcomm_send(sock, reply_header, ptr);
 
@@ -944,7 +970,7 @@ static void process_client_cmd(struct ustcomm_header *recv_header,
                print_trace_events(fp);
                fclose(fp);
 
-               reply_header->size = size;
+               reply_header->size = size + 1;  /* Include final \0 */
 
                result = ustcomm_send(sock, reply_header, ptr);
 
@@ -981,34 +1007,72 @@ static void process_client_cmd(struct ustcomm_header *recv_header,
        }
        case GET_SOCK_PATH:
        {
-               struct ustcomm_sock_path *sock_msg;
+               struct ustcomm_single_field *sock_msg;
                char *sock_path_env;
 
-               sock_msg = (struct ustcomm_sock_path *)send_buf;
+               sock_msg = (struct ustcomm_single_field *)send_buf;
 
                sock_path_env = getenv("UST_DAEMON_SOCKET");
 
                if (!sock_path_env) {
-                       result = ustcomm_pack_sock_path(reply_header,
-                                                       sock_msg,
-                                                       SOCK_DIR "/ustd");
+                       result = ustcomm_pack_single_field(reply_header,
+                                                          sock_msg,
+                                                          SOCK_DIR "/ustconsumer");
 
                } else {
-                       result = ustcomm_pack_sock_path(reply_header,
-                                                       sock_msg,
-                                                       sock_path_env);
+                       result = ustcomm_pack_single_field(reply_header,
+                                                          sock_msg,
+                                                          sock_path_env);
                }
                reply_header->result = result;
 
                goto send_response;
        }
-       default:
+       case SET_SOCK_PATH:
+       {
+               struct ustcomm_single_field *sock_msg;
+               sock_msg = (struct ustcomm_single_field *)recv_buf;
+               result = ustcomm_unpack_single_field(sock_msg);
+               if (result < 0) {
+                       reply_header->result = -EINVAL;
+                       goto send_response;
+               }
+
+               reply_header->result = setenv("UST_DAEMON_SOCKET",
+                                             sock_msg->field, 1);
+
+               goto send_response;
+       }
+       case START:
+       case SETUP_TRACE:
+       case ALLOC_TRACE:
+       case CREATE_TRACE:
+       case START_TRACE:
+       case STOP_TRACE:
+       case DESTROY_TRACE:
+       case FORCE_SUBBUF_SWITCH:
+       {
+               struct ustcomm_single_field *trace_inf =
+                       (struct ustcomm_single_field *)recv_buf;
+
+               result = ustcomm_unpack_single_field(trace_inf);
+               if (result < 0) {
+                       ERR("couldn't unpack trace info");
+                       reply_header->result = -EINVAL;
+                       goto send_response;
+               }
+
                reply_header->result =
-                       process_simple_client_cmd(recv_header->command,
-                                                 recv_buf);
+                       process_trace_cmd(recv_header->command,
+                                         trace_inf->field);
                goto send_response;
 
        }
+       default:
+               reply_header->result = -EINVAL;
+
+               goto send_response;
+       }
 
        return;
 
@@ -1037,6 +1101,8 @@ void *listener_main(void *p)
                }
 
                for (i = 0; i < nfds; i++) {
+                       pthread_mutex_lock(&listener_thread_data_mutex);
+                       pthread_cleanup_push(release_listener_mutex, NULL);
                        epoll_sock = (struct ustcomm_sock *)events[i].data.ptr;
                        if (epoll_sock == listen_sock) {
                                addr_size = sizeof(struct sockaddr);
@@ -1065,6 +1131,7 @@ void *listener_main(void *p)
                                                           epoll_sock->fd);
                                }
                        }
+                       pthread_cleanup_pop(1); /* release listener mutex */
                }
        }
 
@@ -1154,41 +1221,46 @@ static void auto_probe_connect(struct marker *m)
 
 static struct ustcomm_sock * init_app_socket(int epoll_fd)
 {
-       char *name;
+       char *dir_name, *sock_name;
        int result;
-       struct ustcomm_sock *sock;
+       struct ustcomm_sock *sock = NULL;
 
-       result = asprintf(&name, "%s/%d", SOCK_DIR, (int)getpid());
+       dir_name = ustcomm_user_sock_dir();
+       if (!dir_name)
+               return NULL;
+
+       result = asprintf(&sock_name, "%s/%d", dir_name, (int)getpid());
        if (result < 0) {
                ERR("string overflow allocating socket name, "
                    "UST thread bailing");
-               return NULL;
+               goto free_dir_name;
        }
 
-       result = ensure_dir_exists(SOCK_DIR);
+       result = ensure_dir_exists(dir_name);
        if (result == -1) {
                ERR("Unable to create socket directory %s, UST thread bailing",
-                   SOCK_DIR);
-               goto free_name;
+                   dir_name);
+               goto free_sock_name;
        }
 
-       sock = ustcomm_init_named_socket(name, epoll_fd);
+       sock = ustcomm_init_named_socket(sock_name, epoll_fd);
        if (!sock) {
                ERR("Error initializing named socket (%s). Check that directory"
-                   "exists and that it is writable. UST thread bailing", name);
-               goto free_name;
+                   "exists and that it is writable. UST thread bailing", sock_name);
+               goto free_sock_name;
        }
 
-       free(name);
-       return sock;
+free_sock_name:
+       free(sock_name);
+free_dir_name:
+       free(dir_name);
 
-free_name:
-       free(name);
-       return NULL;
+       return sock;
 }
 
 static void __attribute__((constructor)) init()
 {
+       struct timespec ts;
        int result;
        char* autoprobe_val = NULL;
        char* subbuffer_size_val = NULL;
@@ -1201,6 +1273,7 @@ static void __attribute__((constructor)) init()
         * pid, (before and after an exec).
         */
        pidunique = make_pidunique();
+       processpid = getpid();
 
        DBG("Tracectl constructor");
 
@@ -1221,6 +1294,15 @@ static void __attribute__((constructor)) init()
 
        create_listener();
 
+       /* Get clock the clock source type */
+
+       /* Default clock source */
+       ust_clock_source = CLOCK_TRACE;
+       if (clock_gettime(ust_clock_source, &ts) != 0) {
+               ust_clock_source = CLOCK_MONOTONIC;
+               DBG("UST traces will not be synchronized with LTTng traces");
+       }
+
        autoprobe_val = getenv("UST_AUTOPROBE");
        if (autoprobe_val) {
                struct marker_iter iter;
@@ -1260,8 +1342,8 @@ static void __attribute__((constructor)) init()
 
                DBG("now iterating on markers already registered");
                while (iter.marker) {
-                       DBG("now iterating on marker %s", iter.marker->name);
-                       auto_probe_connect(iter.marker);
+                       DBG("now iterating on marker %s", (*iter.marker)->name);
+                       auto_probe_connect(*iter.marker);
                        marker_iter_next(&iter);
                }
        }
@@ -1269,7 +1351,7 @@ static void __attribute__((constructor)) init()
        if (getenv("UST_OVERWRITE")) {
                int val = atoi(getenv("UST_OVERWRITE"));
                if (val == 0 || val == 1) {
-                       STORE_SHARED(ust_channels_overwrite_by_default, val);
+                       CMM_STORE_SHARED(ust_channels_overwrite_by_default, val);
                } else {
                        WARN("invalid value for UST_OVERWRITE");
                }
@@ -1278,7 +1360,7 @@ static void __attribute__((constructor)) init()
        if (getenv("UST_AUTOCOLLECT")) {
                int val = atoi(getenv("UST_AUTOCOLLECT"));
                if (val == 0 || val == 1) {
-                       STORE_SHARED(ust_channels_request_collection_by_default, val);
+                       CMM_STORE_SHARED(ust_channels_request_collection_by_default, val);
                } else {
                        WARN("invalid value for UST_AUTOCOLLECT");
                }
@@ -1410,7 +1492,7 @@ static int trace_recording(void)
 
        ltt_lock_traces();
 
-       list_for_each_entry(trace, &ltt_traces.head, list) {
+       cds_list_for_each_entry(trace, &ltt_traces.head, list) {
                if (trace->active) {
                        retval = 1;
                        break;
@@ -1455,7 +1537,7 @@ static void stop_listener(void)
 }
 
 /* This destructor keeps the process alive for a few seconds in order
- * to leave time to ustd to connect to its buffers. This is necessary
+ * to leave time for ustconsumer to connect to its buffers. This is necessary
  * for programs whose execution is very short. It is also useful in all
  * programs when tracing is started close to the end of the program
  * execution.
@@ -1466,10 +1548,14 @@ static void stop_listener(void)
 
 static void __attribute__((destructor)) keepalive()
 {
-       if (trace_recording() && LOAD_SHARED(buffers_to_export)) {
+       if (processpid != getpid()) {
+               return;
+       }
+
+       if (trace_recording() && CMM_LOAD_SHARED(buffers_to_export)) {
                int total = 0;
                DBG("Keeping process alive for consumer daemon...");
-               while (LOAD_SHARED(buffers_to_export)) {
+               while (CMM_LOAD_SHARED(buffers_to_export)) {
                        const int interv = 200000;
                        restarting_usleep(interv);
                        total += interv;
@@ -1509,43 +1595,46 @@ void ust_potential_exec(void)
 
 static void ust_fork(void)
 {
-       struct ust_buffer *buf, *buf_tmp;
        struct ustcomm_sock *sock, *sock_tmp;
+       struct ust_trace *trace, *trace_tmp;
        int result;
 
        /* FIXME: technically, the locks could have been taken before the fork */
        DBG("ust: forking");
 
-       /* break lock if necessary */
-       ltt_unlock_traces();
+       /* Get the pid of the new process */
+       processpid = getpid();
+
+       /*
+        * FIXME: This could be prettier, we loop over the list twice and
+        * following good locking practice should lock around the loop
+        */
+       cds_list_for_each_entry_safe(trace, trace_tmp, &ltt_traces.head, list) {
+               ltt_trace_stop(trace->trace_name);
+       }
 
-       ltt_trace_stop("auto");
-       ltt_trace_destroy("auto", 1);
        /* Delete all active connections, but leave them in the epoll set */
-       list_for_each_entry_safe(sock, sock_tmp, &ust_socks, list) {
+       cds_list_for_each_entry_safe(sock, sock_tmp, &ust_socks, list) {
                ustcomm_del_sock(sock, 1);
        }
 
-       /* Delete all blocked consumers */
-       list_for_each_entry_safe(buf, buf_tmp, &open_buffers_list,
-                                open_buffers_list) {
-               result = close(buf->data_ready_fd_read);
-               if (result == -1) {
-                       PERROR("close");
-               }
-               result = close(buf->data_ready_fd_write);
-               if (result == -1) {
-                       PERROR("close");
-               }
-               list_del(&buf->open_buffers_list);
+       /*
+        * FIXME: This could be prettier, we loop over the list twice and
+        * following good locking practice should lock around the loop
+        */
+       cds_list_for_each_entry_safe(trace, trace_tmp, &ltt_traces.head, list) {
+               ltt_trace_destroy(trace->trace_name, 1);
        }
 
-       /* Clean up the listener socket and epoll, keeping the scoket file */
-       ustcomm_del_named_sock(listen_sock, 1);
+       /* Clean up the listener socket and epoll, keeping the socket file */
+       if (listen_sock) {
+               ustcomm_del_named_sock(listen_sock, 1);
+               listen_sock = NULL;
+       }
        close(epoll_fd);
 
        /* Re-start the launch sequence */
-       STORE_SHARED(buffers_to_export, 0);
+       CMM_STORE_SHARED(buffers_to_export, 0);
        have_listener = 0;
 
        /* Set up epoll */
@@ -1596,6 +1685,17 @@ void ust_before_fork(ust_fork_info_t *fork_info)
                 PERROR("sigprocmask");
                 return;
         }
+
+       /*
+        * Take the fork lock to make sure we are not in the middle of
+        * something in the listener thread.
+        */
+       pthread_mutex_lock(&listener_thread_data_mutex);
+       /*
+        * Hold listen_sock_mutex to protect from listen_sock teardown.
+        */
+       pthread_mutex_lock(&listen_sock_mutex);
+       rcu_bp_before_fork();
 }
 
 /* Don't call this function directly in a traced program */
@@ -1603,6 +1703,9 @@ static void ust_after_fork_common(ust_fork_info_t *fork_info)
 {
        int result;
 
+       pthread_mutex_unlock(&listen_sock_mutex);
+       pthread_mutex_unlock(&listener_thread_data_mutex);
+
         /* Restore signals */
         result = sigprocmask(SIG_SETMASK, &fork_info->orig_sigs, NULL);
         if (result == -1) {
@@ -1613,16 +1716,20 @@ static void ust_after_fork_common(ust_fork_info_t *fork_info)
 
 void ust_after_fork_parent(ust_fork_info_t *fork_info)
 {
-       /* Reenable signals */
+       rcu_bp_after_fork_parent();
+       /* Release mutexes and reenable signals */
        ust_after_fork_common(fork_info);
 }
 
 void ust_after_fork_child(ust_fork_info_t *fork_info)
 {
-       /* First sanitize the child */
+       /* Release urcu mutexes */
+       rcu_bp_after_fork_child();
+
+       /* Sanitize the child */
        ust_fork();
 
-       /* Then reenable interrupts */
+       /* Then release mutexes and reenable signals */
        ust_after_fork_common(fork_info);
 }
 
This page took 0.032915 seconds and 4 git commands to generate.