unix: make loops and watchers usable after fork()

Added the uv_loop_fork() API that must be called in a child process to
continue using an existing loop. Internally this calls a uv__io_fork
function for each supported platform, similar to the way
uv__platform_loop_init works.

After this call, existing and new IO, async and signal watchers will
contiue working as before on all platforms, as will the
threadpool (although any threads it was using are of course gone).

On Linux and BSDs that use kqueue, existing and new fsevent watchers
will also continue to work as expected. On OS X, though, directory
fsevents will not be able to use the optimized CoreFoundation path if
they had already been used in the parent process, instead falling back
to the kqueue path used on other BSDs.

Existing fsevent watchers will not function on AIX or SunOS. This
could be relatively easily fixed by someone with AIX knowledge in the
future, but SunOS will require some additional work to keep track if
the watchers.

A new test file, test/test-fork.c, was added to contain fork-related
tests to verify functionality in the child process.

PR-URL: https://github.com/libuv/libuv/pull/846
Reviewed-By: Ben Noordhuis <info@bnoordhuis.nl>
Reviewed-By: Saúl Ibarra Corretgé <saghul@gmail.com>
This commit is contained in:
Jason Madden 2016-04-22 14:16:31 -05:00 committed by Ben Noordhuis
parent 7b9f379923
commit fd7ce57f2b
17 changed files with 999 additions and 2 deletions

View File

@ -172,6 +172,7 @@ test_run_tests_SOURCES = test/blackhole-server.c \
test/test-fs-event.c \
test/test-fs-poll.c \
test/test-fs.c \
test/test-fork.c \
test/test-get-currentexe.c \
test/test-get-loadavg.c \
test/test-get-memory.c \

View File

@ -165,3 +165,52 @@ API
.. c:function:: void uv_walk(uv_loop_t* loop, uv_walk_cb walk_cb, void* arg)
Walk the list of handles: `walk_cb` will be executed with the given `arg`.
.. c:function:: int uv_loop_fork(uv_loop_t* loop)
.. versionadded:: 1.12.0
Reinitialize any kernel state necessary in the child process after
a :man:`fork(2)` system call.
Previously started watchers will continue to be started in the
child process.
It is necessary to explicitly call this function on every event
loop created in the parent process that you plan to continue to
use in the child, including the default loop (even if you don't
continue to use it in the parent). This function must be called
before calling :c:func:`uv_run` or any other API function using
the loop in the child. Failure to do so will result in undefined
behaviour, possibly including duplicate events delivered to both
parent and child or aborting the child process.
When possible, it is preferred to create a new loop in the child
process instead of reusing a loop created in the parent. New loops
created in the child process after the fork should not use this
function.
This function is not implemented on Windows, where it returns ``UV_ENOSYS``.
.. note::
On Mac OS X, if directory FS event handles were in use in the
parent process *for any event loop*, the child process will no
longer be able to use the most efficient FSEvent
implementation. Instead, uses of directory FS event handles in
the child will fall back to the same implementation used for
files and on other kqueue-based systems.
.. caution::
On AIX and SunOS, FS event handles that were already started in
the parent process at the time of forking will *not* deliver
events in the child process; they must be closed and restarted.
On all other platforms, they will continue to work normally
without any further intervention.
.. caution::
Any previous value returned from :c:func`uv_backend_fd` is now
invalid. That function must be called again to determine the
correct backend file descriptor.

View File

@ -274,6 +274,7 @@ UV_EXTERN void uv_loop_delete(uv_loop_t*);
UV_EXTERN size_t uv_loop_size(void);
UV_EXTERN int uv_loop_alive(const uv_loop_t* loop);
UV_EXTERN int uv_loop_configure(uv_loop_t* loop, uv_loop_option option, ...);
UV_EXTERN int uv_loop_fork(uv_loop_t* loop);
UV_EXTERN int uv_run(uv_loop_t*, uv_run_mode mode);
UV_EXTERN void uv_stop(uv_loop_t*);

View File

@ -139,7 +139,7 @@ UV_DESTRUCTOR(static void cleanup(void)) {
#endif
static void init_once(void) {
static void init_threads(void) {
unsigned int i;
const char* val;
@ -177,6 +177,27 @@ static void init_once(void) {
}
#ifndef _WIN32
static void reset_once(void) {
uv_once_t child_once = UV_ONCE_INIT;
memcpy(&once, &child_once, sizeof(child_once));
}
#endif
static void init_once(void) {
#ifndef _WIN32
/* Re-initialize the threadpool after fork.
* Note that this discards the global mutex and condition as well
* as the work queue.
*/
if (pthread_atfork(NULL, NULL, &reset_once))
abort();
#endif
init_threads();
}
void uv__work_submit(uv_loop_t* loop,
struct uv__work* w,
void (*work)(struct uv__work* w),

View File

@ -96,6 +96,13 @@ void uv__platform_loop_delete(uv_loop_t* loop) {
}
int uv__io_fork(uv_loop_t* loop) {
uv__platform_loop_delete(loop);
return uv__platform_loop_init(loop);
}
int uv__io_check_fd(uv_loop_t* loop, int fd) {
struct poll_ctl pc;

View File

@ -200,6 +200,16 @@ static int uv__async_start(uv_loop_t* loop) {
}
int uv__async_fork(uv_loop_t* loop) {
if (loop->async_io_watcher.fd == -1) /* never started */
return 0;
uv__async_stop(loop);
return uv__async_start(loop);
}
void uv__async_stop(uv_loop_t* loop) {
if (loop->async_io_watcher.fd == -1)
return;

View File

@ -192,9 +192,12 @@ void uv__io_feed(uv_loop_t* loop, uv__io_t* w);
int uv__io_active(const uv__io_t* w, unsigned int events);
int uv__io_check_fd(uv_loop_t* loop, int fd);
void uv__io_poll(uv_loop_t* loop, int timeout); /* in milliseconds or -1 */
int uv__io_fork(uv_loop_t* loop);
/* async */
void uv__async_stop(uv_loop_t* loop);
int uv__async_fork(uv_loop_t* loop);
/* loop */
void uv__run_idle(uv_loop_t* loop);
@ -230,6 +233,7 @@ int uv__next_timeout(const uv_loop_t* loop);
void uv__signal_close(uv_signal_t* handle);
void uv__signal_global_once_init(void);
void uv__signal_loop_cleanup(uv_loop_t* loop);
int uv__signal_loop_fork(uv_loop_t* loop);
/* platform specific */
uint64_t uv__hrtime(uv_clocktype_t type);
@ -324,4 +328,8 @@ UV_UNUSED(static char* uv__basename_r(const char* path)) {
return s + 1;
}
#if defined(__linux__)
int uv__inotify_fork(uv_loop_t* loop, void* old_watchers);
#endif
#endif /* UV_UNIX_INTERNAL_H_ */

View File

@ -48,6 +48,37 @@ int uv__kqueue_init(uv_loop_t* loop) {
}
static int uv__has_forked_with_cfrunloop;
int uv__io_fork(uv_loop_t* loop) {
int err;
uv__close(loop->backend_fd);
loop->backend_fd = -1;
err = uv__kqueue_init(loop);
if (err)
return err;
#if defined(__APPLE__)
if (loop->cf_state != NULL) {
/* We cannot start another CFRunloop and/or thread in the child
process; CF aborts if you try or if you try to touch the thread
at all to kill it. So the best we can do is ignore it from now
on. This means we can't watch directories in the same way
anymore (like other BSDs). It also means we cannot properly
clean up the allocated resources; calling
uv__fsevents_loop_delete from uv_loop_close will crash the
process. So we sidestep the issue by pretending like we never
started it in the first place.
*/
uv__has_forked_with_cfrunloop = 1;
uv__free(loop->cf_state);
loop->cf_state = NULL;
}
#endif
return err;
}
int uv__io_check_fd(uv_loop_t* loop, int fd) {
struct kevent ev;
int rc;
@ -404,6 +435,9 @@ int uv_fs_event_start(uv_fs_event_t* handle,
handle->cb = cb;
#if defined(__APPLE__)
if (uv__has_forked_with_cfrunloop)
goto fallback;
/* Nullify field to perform checks later */
handle->cf_cb = NULL;
handle->realpath = NULL;
@ -438,7 +472,7 @@ int uv_fs_event_stop(uv_fs_event_t* handle) {
uv__handle_stop(handle);
#if defined(__APPLE__)
if (uv__fsevents_close(handle))
if (uv__has_forked_with_cfrunloop || uv__fsevents_close(handle))
#endif /* defined(__APPLE__) */
{
uv__io_close(handle->loop, &handle->event_watcher);

View File

@ -107,6 +107,24 @@ int uv__platform_loop_init(uv_loop_t* loop) {
}
int uv__io_fork(uv_loop_t* loop) {
int err;
void* old_watchers;
old_watchers = loop->inotify_watchers;
uv__close(loop->backend_fd);
loop->backend_fd = -1;
uv__platform_loop_delete(loop);
err = uv__platform_loop_init(loop);
if (err)
return err;
return uv__inotify_fork(loop, old_watchers);
}
void uv__platform_loop_delete(uv_loop_t* loop) {
if (loop->inotify_fd == -1) return;
uv__io_stop(loop, &loop->inotify_read_watcher, POLLIN);

View File

@ -61,6 +61,8 @@ static void uv__inotify_read(uv_loop_t* loop,
uv__io_t* w,
unsigned int revents);
static void maybe_free_watcher_list(struct watcher_list* w,
uv_loop_t* loop);
static int new_inotify_fd(void) {
int err;
@ -108,6 +110,71 @@ static int init_inotify(uv_loop_t* loop) {
}
int uv__inotify_fork(uv_loop_t* loop, void* old_watchers) {
/* Open the inotify_fd, and re-arm all the inotify watchers. */
int err;
struct watcher_list* tmp_watcher_list_iter;
struct watcher_list* watcher_list;
struct watcher_list tmp_watcher_list;
QUEUE queue;
QUEUE* q;
uv_fs_event_t* handle;
char* tmp_path;
if (old_watchers != NULL) {
/* We must restore the old watcher list to be able to close items
* out of it.
*/
loop->inotify_watchers = old_watchers;
QUEUE_INIT(&tmp_watcher_list.watchers);
/* Note that the queue we use is shared with the start and stop()
* functions, making QUEUE_FOREACH unsafe to use. So we use the
* QUEUE_MOVE trick to safely iterate. Also don't free the watcher
* list until we're done iterating. c.f. uv__inotify_read.
*/
RB_FOREACH_SAFE(watcher_list, watcher_root,
CAST(&old_watchers), tmp_watcher_list_iter) {
watcher_list->iterating = 1;
QUEUE_MOVE(&watcher_list->watchers, &queue);
while (!QUEUE_EMPTY(&queue)) {
q = QUEUE_HEAD(&queue);
handle = QUEUE_DATA(q, uv_fs_event_t, watchers);
/* It's critical to keep a copy of path here, because it
* will be set to NULL by stop() and then deallocated by
* maybe_free_watcher_list
*/
tmp_path = uv__strdup(handle->path);
assert(tmp_path != NULL);
QUEUE_REMOVE(q);
QUEUE_INSERT_TAIL(&watcher_list->watchers, q);
uv_fs_event_stop(handle);
QUEUE_INSERT_TAIL(&tmp_watcher_list.watchers, &handle->watchers);
handle->path = tmp_path;
}
watcher_list->iterating = 0;
maybe_free_watcher_list(watcher_list, loop);
}
QUEUE_MOVE(&tmp_watcher_list.watchers, &queue);
while (!QUEUE_EMPTY(&queue)) {
q = QUEUE_HEAD(&queue);
QUEUE_REMOVE(q);
handle = QUEUE_DATA(q, uv_fs_event_t, watchers);
tmp_path = handle->path;
handle->path = NULL;
err = uv_fs_event_start(handle, handle->cb, tmp_path, 0);
uv__free(tmp_path);
if (err)
return err;
}
}
return 0;
}
static struct watcher_list* find_watcher(uv_loop_t* loop, int wd) {
struct watcher_list w;
w.wd = wd;

View File

@ -109,6 +109,39 @@ fail_signal_init:
}
int uv_loop_fork(uv_loop_t* loop) {
int err;
unsigned int i;
uv__io_t* w;
err = uv__io_fork(loop);
if (err)
return err;
err = uv__async_fork(loop);
if (err)
return err;
err = uv__signal_loop_fork(loop);
if (err)
return err;
/* Rearm all the watchers that aren't re-queued by the above. */
for (i = 0; i < loop->nwatchers; i++) {
w = loop->watchers[i];
if (w == NULL)
continue;
if (w->pevents != 0 && QUEUE_EMPTY(&w->watcher_queue)) {
w->events = 0; /* Force re-registration in uv__io_poll. */
QUEUE_INSERT_TAIL(&loop->watcher_queue, &w->watcher_queue);
}
}
return 0;
}
void uv__loop_close(uv_loop_t* loop) {
uv__signal_loop_cleanup(loop);
uv__platform_loop_delete(loop);

View File

@ -58,8 +58,19 @@ RB_GENERATE_STATIC(uv__signal_tree_s,
uv_signal_s, tree_entry,
uv__signal_compare)
static void uv__signal_global_reinit(void);
static void uv__signal_global_init(void) {
if (!uv__signal_lock_pipefd[0])
/* pthread_atfork can register before and after handlers, one
* for each child. This only registers one for the child. That
* state is both persistent and cumulative, so if we keep doing
* it the handler functions will be called multiple times. Thus
* we only want to do it once.
*/
if (pthread_atfork(NULL, NULL, &uv__signal_global_reinit))
abort();
if (uv__make_pipe(uv__signal_lock_pipefd, 0))
abort();
@ -68,6 +79,22 @@ static void uv__signal_global_init(void) {
}
static void uv__signal_global_reinit(void) {
/* We can only use signal-safe functions here.
* That includes read/write and close, fortunately.
* We do all of this directly here instead of resetting
* uv__signal_global_init_guard because
* uv__signal_global_once_init is only called from uv_loop_init
* and this needs to function in existing loops.
*/
uv__close(uv__signal_lock_pipefd[0]);
uv__signal_lock_pipefd[0] = -1;
uv__close(uv__signal_lock_pipefd[1]);
uv__signal_lock_pipefd[1] = -1;
uv__signal_global_init();
}
void uv__signal_global_once_init(void) {
uv_once(&uv__signal_global_init_guard, uv__signal_global_init);
}
@ -235,6 +262,16 @@ static int uv__signal_loop_once_init(uv_loop_t* loop) {
}
int uv__signal_loop_fork(uv_loop_t* loop) {
uv__io_stop(loop, &loop->signal_io_watcher, POLLIN);
uv__close(loop->signal_pipefd[0]);
uv__close(loop->signal_pipefd[1]);
loop->signal_pipefd[0] = -1;
loop->signal_pipefd[1] = -1;
return uv__signal_loop_once_init(loop);
}
void uv__signal_loop_cleanup(uv_loop_t* loop) {
QUEUE* q;

View File

@ -99,6 +99,18 @@ void uv__platform_loop_delete(uv_loop_t* loop) {
}
int uv__io_fork(uv_loop_t* loop) {
#if defined(PORT_SOURCE_FILE)
if (loop->fs_fd != -1) {
/* stop the watcher before we blow away its fileno */
uv__io_stop(loop, &loop->fs_event_watcher, POLLIN);
}
#endif
uv__platform_loop_delete(loop);
return uv__platform_loop_init(loop);
}
void uv__platform_invalidate_fd(uv_loop_t* loop, int fd) {
struct port_event* events;
uintptr_t i;

View File

@ -332,6 +332,11 @@ int uv_backend_fd(const uv_loop_t* loop) {
}
int uv_loop_fork(uv_loop_t* loop) {
return UV_ENOSYS;
}
int uv_backend_timeout(const uv_loop_t* loop) {
if (loop->stop_flag != 0)
return 0;

669
test/test-fork.c Normal file
View File

@ -0,0 +1,669 @@
/* Copyright libuv project contributors. All rights reserved.
*
* Permission is hereby granted, free of charge, to any person obtaining a copy
* of this software and associated documentation files (the "Software"), to
* deal in the Software without restriction, including without limitation the
* rights to use, copy, modify, merge, publish, distribute, sublicense, and/or
* sell copies of the Software, and to permit persons to whom the Software is
* furnished to do so, subject to the following conditions:
*
* The above copyright notice and this permission notice shall be included in
* all copies or substantial portions of the Software.
*
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
* FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
* AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
* LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
* FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
* IN THE SOFTWARE.
*/
/* These tests are Unix only. */
#ifndef _WIN32
#include <unistd.h>
#include <sys/wait.h>
#include <sys/socket.h>
#include <string.h>
#include "uv.h"
#include "task.h"
static int timer_cb_called;
static int socket_cb_called;
static void timer_cb(uv_timer_t* timer) {
timer_cb_called++;
uv_close((uv_handle_t*) timer, NULL);
}
static int socket_cb_read_fd;
static int socket_cb_read_size;
static char socket_cb_read_buf[1024];
static void socket_cb(uv_poll_t* poll, int status, int events) {
ssize_t cnt;
cnt = 0;
socket_cb_called++;
ASSERT(0 == status);
printf("Socket cb got events %d\n", events);
ASSERT(UV_READABLE == (events & UV_READABLE));
if (socket_cb_read_fd) {
cnt = read(socket_cb_read_fd, socket_cb_read_buf, socket_cb_read_size);
ASSERT(cnt == socket_cb_read_size);
}
uv_close((uv_handle_t*) poll, NULL);
}
static void run_timer_loop_once(void) {
uv_loop_t* loop;
uv_timer_t timer_handle;
loop = uv_default_loop();
timer_cb_called = 0; /* Reset for the child. */
ASSERT(0 == uv_timer_init(loop, &timer_handle));
ASSERT(0 == uv_timer_start(&timer_handle, timer_cb, 1, 0));
ASSERT(0 == uv_run(loop, UV_RUN_DEFAULT));
ASSERT(1 == timer_cb_called);
}
static void assert_wait_child(pid_t child_pid) {
pid_t waited_pid;
int child_stat;
waited_pid = waitpid(child_pid, &child_stat, 0);
printf("Waited pid is %d with status %d\n", waited_pid, child_stat);
if (waited_pid == -1) {
perror("Failed to wait");
}
ASSERT(child_pid == waited_pid);
ASSERT(WIFEXITED(child_stat)); /* Clean exit, not a signal. */
ASSERT(!WIFSIGNALED(child_stat));
ASSERT(0 == WEXITSTATUS(child_stat));
}
TEST_IMPL(fork_timer) {
/* Timers continue to work after we fork. */
/*
* Establish the loop before we fork to make sure that it
* has state to get reset after the fork.
*/
pid_t child_pid;
run_timer_loop_once();
child_pid = fork();
ASSERT(child_pid != -1);
if (child_pid != 0) {
/* parent */
assert_wait_child(child_pid);
} else {
/* child */
ASSERT(0 == uv_loop_fork(uv_default_loop()));
run_timer_loop_once();
}
MAKE_VALGRIND_HAPPY();
return 0;
}
TEST_IMPL(fork_socketpair) {
/* A socket opened in the parent and accept'd in the
child works after a fork. */
pid_t child_pid;
int socket_fds[2];
uv_poll_t poll_handle;
/* Prime the loop. */
run_timer_loop_once();
ASSERT(0 == socketpair(AF_UNIX, SOCK_STREAM, 0, socket_fds));
/* Create the server watcher in the parent, use it in the child. */
ASSERT(0 == uv_poll_init(uv_default_loop(), &poll_handle, socket_fds[0]));
child_pid = fork();
ASSERT(child_pid != -1);
if (child_pid != 0) {
/* parent */
ASSERT(3 == send(socket_fds[1], "hi\n", 3, 0));
assert_wait_child(child_pid);
} else {
/* child */
ASSERT(0 == uv_loop_fork(uv_default_loop()));
ASSERT(0 == socket_cb_called);
ASSERT(0 == uv_poll_start(&poll_handle, UV_READABLE, socket_cb));
printf("Going to run the loop in the child\n");
ASSERT(0 == uv_run(uv_default_loop(), UV_RUN_DEFAULT));
ASSERT(1 == socket_cb_called);
}
MAKE_VALGRIND_HAPPY();
return 0;
}
TEST_IMPL(fork_socketpair_started) {
/* A socket opened in the parent and accept'd in the
child works after a fork, even if the watcher was already
started, and then stopped in the parent. */
pid_t child_pid;
int socket_fds[2];
int sync_pipe[2];
char sync_buf[1];
uv_poll_t poll_handle;
ASSERT(0 == pipe(sync_pipe));
/* Prime the loop. */
run_timer_loop_once();
ASSERT(0 == socketpair(AF_UNIX, SOCK_STREAM, 0, socket_fds));
/* Create and start the server watcher in the parent, use it in the child. */
ASSERT(0 == uv_poll_init(uv_default_loop(), &poll_handle, socket_fds[0]));
ASSERT(0 == uv_poll_start(&poll_handle, UV_READABLE, socket_cb));
/* Run the loop AFTER the poll watcher is registered to make sure it
gets passed to the kernel. Use NOWAIT and expect a non-zero
return to prove the poll watcher is active.
*/
ASSERT(1 == uv_run(uv_default_loop(), UV_RUN_NOWAIT));
child_pid = fork();
ASSERT(child_pid != -1);
if (child_pid != 0) {
/* parent */
ASSERT(0 == uv_poll_stop(&poll_handle));
uv_close((uv_handle_t*)&poll_handle, NULL);
ASSERT(0 == uv_run(uv_default_loop(), UV_RUN_DEFAULT));
ASSERT(0 == socket_cb_called);
ASSERT(1 == write(sync_pipe[1], "1", 1)); /* alert child */
ASSERT(3 == send(socket_fds[1], "hi\n", 3, 0));
ASSERT(0 == uv_run(uv_default_loop(), UV_RUN_DEFAULT));
ASSERT(0 == socket_cb_called);
assert_wait_child(child_pid);
} else {
/* child */
printf("Child is %d\n", getpid());
ASSERT(1 == read(sync_pipe[0], sync_buf, 1)); /* wait for parent */
ASSERT(0 == uv_loop_fork(uv_default_loop()));
ASSERT(0 == socket_cb_called);
printf("Going to run the loop in the child\n");
socket_cb_read_fd = socket_fds[0];
socket_cb_read_size = 3;
ASSERT(0 == uv_run(uv_default_loop(), UV_RUN_DEFAULT));
ASSERT(1 == socket_cb_called);
printf("Buf %s\n", socket_cb_read_buf);
ASSERT(0 == strcmp("hi\n", socket_cb_read_buf));
}
MAKE_VALGRIND_HAPPY();
return 0;
}
static int fork_signal_cb_called;
void fork_signal_to_child_cb(uv_signal_t* handle, int signum)
{
fork_signal_cb_called = signum;
uv_close((uv_handle_t*)handle, NULL);
}
TEST_IMPL(fork_signal_to_child) {
/* A signal handler installed before forking
is run only in the child when the child is signalled. */
uv_signal_t signal_handle;
pid_t child_pid;
int sync_pipe[2];
char sync_buf[1];
fork_signal_cb_called = 0; /* reset */
ASSERT(0 == pipe(sync_pipe));
/* Prime the loop. */
run_timer_loop_once();
ASSERT(0 == uv_signal_init(uv_default_loop(), &signal_handle));
ASSERT(0 == uv_signal_start(&signal_handle, fork_signal_to_child_cb, SIGUSR1));
child_pid = fork();
ASSERT(child_pid != -1);
if (child_pid != 0) {
/* parent */
ASSERT(1 == read(sync_pipe[0], sync_buf, 1)); /* wait for child */
ASSERT(0 == kill(child_pid, SIGUSR1));
/* Run the loop, make sure we don't get the signal. */
printf("Running loop in parent\n");
uv_unref((uv_handle_t*)&signal_handle);
ASSERT(0 == uv_run(uv_default_loop(), UV_RUN_NOWAIT));
ASSERT(0 == fork_signal_cb_called);
printf("Waiting for child in parent\n");
assert_wait_child(child_pid);
} else {
/* child */
ASSERT(0 == uv_loop_fork(uv_default_loop()));
ASSERT(1 == write(sync_pipe[1], "1", 1)); /* alert parent */
/* Get the signal. */
ASSERT(0 != uv_loop_alive(uv_default_loop()));
printf("Running loop in child\n");
ASSERT(0 == uv_run(uv_default_loop(), UV_RUN_ONCE));
ASSERT(SIGUSR1 == fork_signal_cb_called);
}
MAKE_VALGRIND_HAPPY();
return 0;
}
TEST_IMPL(fork_signal_to_child_closed) {
/* A signal handler installed before forking
doesn't get received anywhere when the child is signalled,
but isnt running the loop. */
uv_signal_t signal_handle;
pid_t child_pid;
int sync_pipe[2];
int sync_pipe2[2];
char sync_buf[1];
fork_signal_cb_called = 0; /* reset */
ASSERT(0 == pipe(sync_pipe));
ASSERT(0 == pipe(sync_pipe2));
/* Prime the loop. */
run_timer_loop_once();
ASSERT(0 == uv_signal_init(uv_default_loop(), &signal_handle));
ASSERT(0 == uv_signal_start(&signal_handle, fork_signal_to_child_cb, SIGUSR1));
child_pid = fork();
ASSERT(child_pid != -1);
if (child_pid != 0) {
/* parent */
printf("Wating on child in parent\n");
ASSERT(1 == read(sync_pipe[0], sync_buf, 1)); /* wait for child */
printf("Parent killing child\n");
ASSERT(0 == kill(child_pid, SIGUSR1));
/* Run the loop, make sure we don't get the signal. */
printf("Running loop in parent\n");
uv_unref((uv_handle_t*)&signal_handle); /* so the loop can exit;
we *shouldn't* get any signals */
run_timer_loop_once(); /* but while we share a pipe, we do, so
have something active. */
ASSERT(0 == uv_run(uv_default_loop(), UV_RUN_ONCE));
printf("Signal in parent %d\n", fork_signal_cb_called);
ASSERT(0 == fork_signal_cb_called);
ASSERT(1 == write(sync_pipe2[1], "1", 1)); /* alert child */
printf("Waiting for child in parent\n");
assert_wait_child(child_pid);
} else {
/* child */
/* Our signal handler should still be installed. */
ASSERT(0 == uv_loop_fork(uv_default_loop()));
printf("Checking loop in child\n");
ASSERT(0 != uv_loop_alive(uv_default_loop()));
printf("Alerting parent in child\n");
ASSERT(1 == write(sync_pipe[1], "1", 1)); /* alert parent */
/* Don't run the loop. Wait for the parent to call us */
printf("Waiting on parent in child\n");
/* Wait for parent. read may fail if the parent tripped an ASSERT
and exited, so this isn't in an ASSERT.
*/
read(sync_pipe2[0], sync_buf, 1);
ASSERT(0 == fork_signal_cb_called);
printf("Exiting child \n");
/* Note that we're deliberately not running the loop
* in the child, and also not closing the loop's handles,
* so the child default loop can't be cleanly closed.
* We need te explicitly exit to avoid an automatic failure
* in that case.
*/
exit(0);
}
MAKE_VALGRIND_HAPPY();
return 0;
}
static void create_file(const char* name) {
int r;
uv_file file;
uv_fs_t req;
r = uv_fs_open(NULL, &req, name, O_WRONLY | O_CREAT, S_IWUSR | S_IRUSR, NULL);
ASSERT(r >= 0);
file = r;
uv_fs_req_cleanup(&req);
r = uv_fs_close(NULL, &req, file, NULL);
ASSERT(r == 0);
uv_fs_req_cleanup(&req);
}
static void touch_file(const char* name) {
int r;
uv_file file;
uv_fs_t req;
uv_buf_t buf;
r = uv_fs_open(NULL, &req, name, O_RDWR, 0, NULL);
ASSERT(r >= 0);
file = r;
uv_fs_req_cleanup(&req);
buf = uv_buf_init("foo", 4);
r = uv_fs_write(NULL, &req, file, &buf, 1, -1, NULL);
ASSERT(r >= 0);
uv_fs_req_cleanup(&req);
r = uv_fs_close(NULL, &req, file, NULL);
ASSERT(r == 0);
uv_fs_req_cleanup(&req);
}
static int timer_cb_touch_called;
static void timer_cb_touch(uv_timer_t* timer) {
uv_close((uv_handle_t*)timer, NULL);
touch_file("watch_file");
timer_cb_touch_called++;
}
static int fs_event_cb_called;
static void fs_event_cb_file_current_dir(uv_fs_event_t* handle,
const char* filename,
int events,
int status) {
ASSERT(fs_event_cb_called == 0);
++fs_event_cb_called;
ASSERT(status == 0);
#if defined(__APPLE__) || defined(__linux__)
ASSERT(strcmp(filename, "watch_file") == 0);
#else
ASSERT(filename == NULL || strcmp(filename, "watch_file") == 0);
#endif
uv_close((uv_handle_t*)handle, NULL);
}
static void assert_watch_file_current_dir(uv_loop_t* const loop, int file_or_dir) {
uv_timer_t timer;
uv_fs_event_t fs_event;
int r;
/* Setup */
remove("watch_file");
create_file("watch_file");
r = uv_fs_event_init(loop, &fs_event);
ASSERT(r == 0);
/* watching a dir is the only way to get fsevents involved on apple
platforms */
r = uv_fs_event_start(&fs_event,
fs_event_cb_file_current_dir,
file_or_dir == 1 ? "." : "watch_file",
0);
ASSERT(r == 0);
r = uv_timer_init(loop, &timer);
ASSERT(r == 0);
r = uv_timer_start(&timer, timer_cb_touch, 100, 0);
ASSERT(r == 0);
ASSERT(timer_cb_touch_called == 0);
ASSERT(fs_event_cb_called == 0);
uv_run(loop, UV_RUN_DEFAULT);
ASSERT(timer_cb_touch_called == 1);
ASSERT(fs_event_cb_called == 1);
/* Cleanup */
remove("watch_file");
fs_event_cb_called = 0;
timer_cb_touch_called = 0;
uv_run(loop, UV_RUN_DEFAULT); /* flush pending closes */
}
#define FS_TEST_FILE 0
#define FS_TEST_DIR 1
static int _do_fork_fs_events_child(int file_or_dir) {
/* basic fsevents work in the child after a fork */
pid_t child_pid;
uv_loop_t loop;
/* Watch in the parent, prime the loop and/or threads. */
assert_watch_file_current_dir(uv_default_loop(), file_or_dir);
child_pid = fork();
ASSERT(child_pid != -1);
if (child_pid != 0) {
/* parent */
assert_wait_child(child_pid);
} else {
/* child */
/* Ee can watch in a new loop, but dirs only work
if we're on linux. */
#if defined(__APPLE__)
file_or_dir = FS_TEST_FILE;
#endif
printf("Running child\n");
uv_loop_init(&loop);
printf("Child first watch\n");
assert_watch_file_current_dir(&loop, file_or_dir);
ASSERT(0 == uv_loop_close(&loop));
printf("Child second watch default loop\n");
/* Ee can watch in the default loop. */
ASSERT(0 == uv_loop_fork(uv_default_loop()));
/* On some platforms (OS X), if we don't update the time now,
* the timer cb fires before the event loop enters uv__io_poll,
* instead of after, meaning we don't see the change! This may be
* a general race.
*/
uv_update_time(uv_default_loop());
assert_watch_file_current_dir(uv_default_loop(), file_or_dir);
/* We can close the parent loop successfully too. This is
especially important on Apple platforms where if we're not
careful trying to touch the CFRunLoop, even just to shut it
down, that we allocated in the FS_TEST_DIR case would crash. */
ASSERT(0 == uv_loop_close(uv_default_loop()));
printf("Exiting child \n");
}
MAKE_VALGRIND_HAPPY();
return 0;
}
TEST_IMPL(fork_fs_events_child) {
return _do_fork_fs_events_child(FS_TEST_FILE);
}
TEST_IMPL(fork_fs_events_child_dir) {
#if defined(__APPLE__) || defined (__linux__)
return _do_fork_fs_events_child(FS_TEST_DIR);
#else
/* You can't spin up a cfrunloop thread on an apple platform
and then fork. See
http://objectivistc.tumblr.com/post/16187948939/you-must-exec-a-core-foundation-fork-safety-tale
*/
return 0;
#endif
}
TEST_IMPL(fork_fs_events_file_parent_child) {
#if defined(__sun) || defined(_AIX)
/* It's not possible to implement this without additional
* bookkeeping on SunOS. For AIX it is possible, but has to be
* written. See https://github.com/libuv/libuv/pull/846#issuecomment-287170420
*/
return 0;
#else
/* Establishing a started fs events watcher in the parent should
still work in the child. */
uv_timer_t timer;
uv_fs_event_t fs_event;
int r;
pid_t child_pid;
uv_loop_t* loop;
loop = uv_default_loop();
/* Setup */
remove("watch_file");
create_file("watch_file");
r = uv_fs_event_init(loop, &fs_event);
ASSERT(r == 0);
r = uv_fs_event_start(&fs_event,
fs_event_cb_file_current_dir,
"watch_file",
0);
ASSERT(r == 0);
r = uv_timer_init(loop, &timer);
ASSERT(r == 0);
child_pid = fork();
ASSERT(child_pid != -1);
if (child_pid != 0) {
/* parent */
assert_wait_child(child_pid);
} else {
/* child */
printf("Running child\n");
ASSERT(0 == uv_loop_fork(loop));
r = uv_timer_start(&timer, timer_cb_touch, 100, 0);
ASSERT(r == 0);
ASSERT(timer_cb_touch_called == 0);
ASSERT(fs_event_cb_called == 0);
printf("Running loop in child \n");
uv_run(loop, UV_RUN_DEFAULT);
ASSERT(timer_cb_touch_called == 1);
ASSERT(fs_event_cb_called == 1);
/* Cleanup */
remove("watch_file");
fs_event_cb_called = 0;
timer_cb_touch_called = 0;
uv_run(loop, UV_RUN_DEFAULT); /* Flush pending closes. */
}
MAKE_VALGRIND_HAPPY();
return 0;
#endif
}
static int work_cb_count;
static int after_work_cb_count;
static void work_cb(uv_work_t* req) {
work_cb_count++;
}
static void after_work_cb(uv_work_t* req, int status) {
ASSERT(status == 0);
after_work_cb_count++;
}
static void assert_run_work(uv_loop_t* const loop) {
uv_work_t work_req;
int r;
ASSERT(work_cb_count == 0);
ASSERT(after_work_cb_count == 0);
printf("Queue in %d\n", getpid());
r = uv_queue_work(loop, &work_req, work_cb, after_work_cb);
ASSERT(r == 0);
printf("Running in %d\n", getpid());
uv_run(loop, UV_RUN_DEFAULT);
ASSERT(work_cb_count == 1);
ASSERT(after_work_cb_count == 1);
/* cleanup */
work_cb_count = 0;
after_work_cb_count = 0;
}
TEST_IMPL(fork_threadpool_queue_work_simple) {
/* The threadpool works in a child process. */
pid_t child_pid;
uv_loop_t loop;
/* Prime the pool and default loop. */
assert_run_work(uv_default_loop());
child_pid = fork();
ASSERT(child_pid != -1);
if (child_pid != 0) {
/* parent */
/* We can still run work. */
assert_run_work(uv_default_loop());
assert_wait_child(child_pid);
} else {
/* child */
/* We can work in a new loop. */
printf("Running child in %d\n", getpid());
uv_loop_init(&loop);
printf("Child first watch\n");
assert_run_work(&loop);
uv_loop_close(&loop);
printf("Child second watch default loop\n");
/* We can work in the default loop. */
ASSERT(0 == uv_loop_fork(uv_default_loop()));
assert_run_work(uv_default_loop());
printf("Exiting child \n");
}
MAKE_VALGRIND_HAPPY();
return 0;
}
#endif /* !_WIN32 */

View File

@ -371,6 +371,18 @@ HELPER_DECLARE (pipe_echo_server)
TEST_DECLARE (queue_foreach_delete)
#ifndef _WIN32
TEST_DECLARE (fork_timer)
TEST_DECLARE (fork_socketpair)
TEST_DECLARE (fork_socketpair_started)
TEST_DECLARE (fork_signal_to_child)
TEST_DECLARE (fork_signal_to_child_closed)
TEST_DECLARE (fork_fs_events_child)
TEST_DECLARE (fork_fs_events_child_dir)
TEST_DECLARE (fork_fs_events_file_parent_child)
TEST_DECLARE (fork_threadpool_queue_work_simple)
#endif
TASK_LIST_START
TEST_ENTRY_CUSTOM (platform_output, 0, 1, 5000)
@ -797,6 +809,18 @@ TASK_LIST_START
TEST_ENTRY (queue_foreach_delete)
#ifndef _WIN32
TEST_ENTRY (fork_timer)
TEST_ENTRY (fork_socketpair)
TEST_ENTRY (fork_socketpair_started)
TEST_ENTRY (fork_signal_to_child)
TEST_ENTRY (fork_signal_to_child_closed)
TEST_ENTRY (fork_fs_events_child)
TEST_ENTRY (fork_fs_events_child_dir)
TEST_ENTRY (fork_fs_events_file_parent_child)
TEST_ENTRY (fork_threadpool_queue_work_simple)
#endif
#if 0
/* These are for testing the test runner. */
TEST_ENTRY (fail_always)

1
uv.gyp
View File

@ -349,6 +349,7 @@
'test/test-emfile.c',
'test/test-env-vars.c',
'test/test-fail-always.c',
'test/test-fork.c',
'test/test-fs.c',
'test/test-fs-event.c',
'test/test-get-currentexe.c',