diff --git a/core/log_io.m b/core/log_io.m
index 4339f4145713b06d752aa5e698fd169ceb06c40b..345901a9cbafa84da31f2b1825a93b393f8533d9 100644
--- a/core/log_io.m
+++ b/core/log_io.m
@@ -43,6 +43,7 @@
 #include <say.h>
 #include <pickle.h>
 #include <crc32.h>
+#include <tarantool_pthread.h>
 
 const u16 snap_tag = -1;
 const u16 wal_tag = -2;
@@ -58,8 +59,25 @@ const char snap_mark[] = "SNAP\n";
 const char xlog_mark[] = "XLOG\n";
 static const int HEADER_SIZE_MAX = sizeof(v11) + sizeof(snap_mark) + 2;
 
+struct recovery_state *recovery_state;
+
 #define ROW_EOF (void *)1
 
+/* Context of the WAL writer thread. */
+
+struct wal_writer
+{
+	STAILQ_HEAD(wal_fifo, wal_write_request) input;
+	pthread_t thread;
+	pthread_mutex_t mutex;
+	pthread_cond_t cond;
+	bool is_shutdown;
+};
+
+static pthread_once_t wal_writer_once = PTHREAD_ONCE_INIT;
+
+static struct wal_writer wal_writer;
+
 static struct tbuf *row_reader_v11(FILE *f, struct palloc_pool *pool);
 
 struct log_io_iter {
@@ -211,6 +229,13 @@ close_iter(struct log_io_iter *i)
 	tarantool_coro_destroy(&i->coro);
 }
 
+/**
+ * Read logfile contents using designated format, panic if
+ * the log is corrupted/unreadable.
+ *
+ * @param i	iterator object, encapsulating log specifics.
+ *
+ */
 static void
 read_rows(struct log_io_iter *i)
 {
@@ -273,8 +298,8 @@ read_rows(struct log_io_iter *i)
 
 		if (++row_count % 100000 == 0)
 			say_info("%.1fM rows processed", row_count / 1000000.);
-	}
-      eof:
+	} /* for loop */
+eof:
 	/*
 	 * then only two cases of fully read file:
 	 * 1. eof_marker_size > 0 and it is the last record in file
@@ -295,7 +320,7 @@ read_rows(struct log_io_iter *i)
 		goto out;
 	}
 
-      out:
+out:
 	l->rows += row_count;
 
 	fseeko(l->f, good_offset, SEEK_SET);	/* seek back to last known good offset */
@@ -653,10 +678,9 @@ static struct log_io *
 log_io_open_for_read(struct recovery_state *recover, struct log_io_class *class, i64 lsn, int suffix,
 		     const char *filename)
 {
-	struct log_io *l = NULL;
 	const char *errmsg;
 
-	l = calloc(1, sizeof(*l));
+	struct log_io *l = calloc(1, sizeof(*l));
 	if (l == NULL) {
 		say_syserror("calloc");
 		return NULL;
@@ -674,7 +698,7 @@ log_io_open_for_read(struct recovery_state *recover, struct log_io_class *class,
 		strncpy(l->filename, filename, PATH_MAX);
 	}
 
-	say_debug("find_log for reading `%s'", l->filename);
+	say_debug("%s: opening %s'", __func__, l->filename);
 
 	l->f = fopen(l->filename, "r");
 	if (l->f == NULL) {
@@ -688,8 +712,8 @@ log_io_open_for_read(struct recovery_state *recover, struct log_io_class *class,
 
 	return l;
 error:
-	say_error("log_io_open_for_read: failed to open `%s': %s", l->filename,
-		  errmsg);
+	say_error("%s: failed to open %s: %s", __func__,
+		  l->filename, errmsg);
 	if (l->f != NULL)
 		fclose(l->f);
 	free(l);
@@ -709,8 +733,8 @@ log_io_open_for_write(struct recovery_state *recover, struct log_io_class *class
 	l = calloc(1, sizeof(*l));
 	if (l == NULL) {
 		*save_errno = errno;
-		errmsg = strerror(errno);
-		goto error;
+		say_syserror("calloc");
+		return NULL;
 	}
 	l->mode = LOG_WRITE;
 	l->class = class;
@@ -759,14 +783,13 @@ log_io_open_for_write(struct recovery_state *recover, struct log_io_class *class
 	say_info("creating `%s'", l->filename);
 	write_header(l);
 	return l;
-      error:
-	say_error("find_log: failed to open `%s': %s", l->filename,
+
+error:
+	say_error("%s: failed to open `%s': %s", __func__, l->filename,
 		  errmsg);
-	if (l != NULL) {
-		if (l->f != NULL)
-			fclose(l->f);
-		free(l);
-	}
+	if (l->f != NULL)
+		fclose(l->f);
+	free(l);
 	return NULL;
 }
 
@@ -1176,38 +1199,151 @@ recover_finalize(struct recovery_state *r)
 	}
 }
 
-static struct wal_write_request *
-wal_write_request(const struct tbuf *t)
+static void
+wal_writer_child()
 {
-	return t->data;
+	recovery_state->writer = NULL;
 }
 
-static struct tbuf *
-write_to_disk(void *_state, struct tbuf *t)
+static void
+wal_writer_init_once()
+{
+	pthread_atfork(NULL, NULL, wal_writer_child);
+}
+
+static void
+wal_writer_init(struct wal_writer *writer)
+{
+	/* I. Initialize the state. */
+	pthread_mutexattr_t errorcheck;
+
+	tt_pthread_mutexattr_init(&errorcheck);
+
+#ifndef NDEBUG
+	tt_pthread_mutexattr_settype(&errorcheck, PTHREAD_MUTEX_ERRORCHECK);
+#endif
+	/* Initialize queue lock mutex. */
+	tt_pthread_mutex_init(&writer->mutex, &errorcheck);
+	tt_pthread_mutexattr_destroy(&errorcheck);
+
+	pthread_condattr_t clock_monotonic;
+	tt_pthread_condattr_init(&clock_monotonic);
+
+	/* CLOCK_REALTIME is default for pthread timeouts, yet
+	 * we'd want to use a faster CLOCK_MONOTONIC.
+	 */
+	tt_pthread_condattr_setclock(&clock_monotonic, CLOCK_MONOTONIC);
+
+	tt_pthread_cond_init(&writer->cond, &clock_monotonic);
+	tt_pthread_condattr_destroy(&clock_monotonic);
+
+	STAILQ_INIT(&writer->input);
+
+	tt_pthread_once(&wal_writer_once, wal_writer_init_once);
+}
+
+static void
+wal_writer_destroy(struct wal_writer *writer)
+{
+	tt_pthread_mutex_destroy(&writer->mutex);
+	tt_pthread_cond_destroy(&writer->cond);
+}
+
+/** WAL writer thread routine. */
+static void *wal_writer_thread(void *worker_args);
+
+/**
+ * Initialize WAL writer, start the thread.
+ *
+ * @param state			WAL writer meta-data.
+ *
+ * @return 0 success, -1 on error. On success, recovery->writer
+ *         points to a newly created WAL writer.
+ */
+static int
+wal_writer_start(struct recovery_state *state)
+{
+	assert(state->writer == NULL);
+	assert(wal_writer.is_shutdown == false);
+
+	/* I. Initialize the state. */
+	wal_writer_init(&wal_writer);
+	state->writer = &wal_writer;
+
+	/* II. Start the thread. */
+
+	if (pthread_create(&wal_writer.thread, NULL, wal_writer_thread,
+			   state)) {
+		wal_writer_destroy(&wal_writer);
+		state->writer = NULL;
+		return -1;
+	}
+	return 0;
+}
+
+static int
+wal_writer_stop(struct recovery_state *state)
+{
+	struct wal_writer *writer = state->writer;
+
+	state->writer = NULL;
+
+	/* Stop the worker thread. */
+
+	tt_pthread_mutex_lock(&writer->mutex);
+	writer->is_shutdown= true;
+	tt_pthread_cond_signal(&writer->cond);
+	tt_pthread_mutex_unlock(&writer->mutex);
+
+	if (pthread_join(writer->thread, NULL) == 0) {
+		wal_writer_destroy(writer);
+		return 0;
+	}
+	say_syserror("WAL writer: thread join failed");
+	return -1;
+}
+
+struct wal_fifo
+wal_writer_pop(struct wal_writer *writer, bool wait)
+{
+	struct wal_fifo input;
+	do {
+		input = writer->input;
+		STAILQ_INIT(&writer->input);
+		if (STAILQ_EMPTY(&input) == false || wait == false)
+			break;
+		tt_pthread_cond_wait(&writer->cond, &writer->mutex);
+	} while (writer->is_shutdown == false);
+	return input;
+}
+
+static int
+write_to_disk(struct recovery_state *r, struct wal_write_request *req)
 {
 	static struct log_io *wal = NULL, *wal_to_close = NULL;
 	static ev_tstamp last_flush = 0;
-	struct tbuf *reply, *header;
-	struct recovery_state *r = _state;
-	u32 result = 0;
 
+#if 0
 	/* we're not running inside ev_loop, so update ev_now manually */
 	ev_now_update();
+#endif
 
 	/* caller requested termination */
-	if (t == NULL) {
+	if (req == NULL) {
 		if (wal != NULL)
 			log_io_close(&wal);
+#if 0
+		if (wal_to_close != NULL)
+			log_io_close(&wal_to_close);
 		recover_free((struct recovery_state*)_state);
-		return NULL;
+#endif
+		return 0;
 	}
 
-	reply = tbuf_alloc(t->pool);
-
 	if (wal == NULL) {
 		int unused;
 		/* Open WAL with '.inprogress' suffix. */
-		wal = log_io_open_for_write(r, r->wal_class, wal_write_request(t)->lsn, -1,
+		wal = log_io_open_for_write(r, r->wal_class, req->lsn, -1,
 					    &unused);
 	}
 	else if (wal->rows == 1) {
@@ -1227,41 +1363,29 @@ write_to_disk(void *_state, struct tbuf *t)
 		say_syserror("can't open wal");
 		goto fail;
 	}
-	if (fwrite(&wal->class->marker, wal->class->marker_size, 1, wal->f) != 1) {
-		say_syserror("can't write marker to wal");
-		goto fail;
-	}
-
-	header = tbuf_alloc(t->pool);
-	tbuf_ensure(header, sizeof(struct row_v11));
-	header->size = sizeof(struct row_v11);
-
-	row_v11(header)->lsn = wal_write_request(t)->lsn;
-	row_v11(header)->tm = ev_now();
-	row_v11(header)->len = wal_write_request(t)->len;
-	row_v11(header)->data_crc32c =
-		crc32_calc(0, wal_write_request(t)->data, wal_write_request(t)->len);
-	row_v11(header)->header_crc32c =
-		crc32_calc(0, header->data + field_sizeof(struct row_v11, header_crc32c),
-		       sizeof(struct row_v11) - field_sizeof(struct row_v11, header_crc32c));
-
-	if (fwrite(header->data, header->size, 1, wal->f) != 1) {
+	req->marker = marker_v11;
+	req->tm = ev_now();
+	req->data_crc32c = crc32_calc(0, (u8 *) &req->tag, req->len);
+	/* Header size. */
+	size_t sz = (sizeof(req->lsn) + sizeof(req->tm) + sizeof(req->len) +
+		     sizeof(req->data_crc32c));
+	req->header_crc32c = crc32_calc(0, (u8 *) &req->lsn, sz);
+	/* Total size. */
+	sz += sizeof(req->marker) + sizeof(req->header_crc32c) + req->len;
+	/* Write the request. */
+	if (fwrite(&req->marker, sz, 1, wal->f) != 1) {
 		say_syserror("can't write row header to wal");
 		goto fail;
 	}
 
-	if (fwrite(wal_write_request(t)->data, wal_write_request(t)->len, 1, wal->f) != 1) {
-		say_syserror("can't write row data to wal");
-		goto fail;
-	}
-
 	/* flush stdio buffer to keep replication in sync */
 	if (fflush(wal->f) < 0) {
 		say_syserror("can't flush wal");
 		goto fail;
 	}
 
-	if (wal->class->fsync_delay > 0 && ev_now() - last_flush >= wal->class->fsync_delay) {
+	if (wal->class->fsync_delay > 0 &&
+	    ev_now() - last_flush >= wal->class->fsync_delay) {
 		if (log_io_flush(wal) < 0) {
 			say_syserror("can't flush wal");
 			goto fail;
@@ -1271,58 +1395,100 @@ write_to_disk(void *_state, struct tbuf *t)
 
 	wal->rows++;
 	if (wal->class->rows_per_file <= wal->rows ||
-	    (wal_write_request(t)->lsn + 1) % wal->class->rows_per_file == 0) {
+	    (req->lsn + 1) % wal->class->rows_per_file == 0) {
 		wal_to_close = wal;
 		wal = NULL;
 	}
 
-	tbuf_append(reply, &result, sizeof(result));
-	return reply;
+	req->out_lsn = req->lsn;
+	return 0;
 
-      fail:
-	result = 1;
-	tbuf_append(reply, &result, sizeof(result));
-	return reply;
+fail:
+	req->out_lsn = 0;
+	return -1;
+}
+
+static void *
+wal_writer_thread(void *worker_args)
+{
+	struct recovery_state *r = worker_args;
+	struct wal_writer *writer = r->writer;
+	struct wal_fifo output = STAILQ_HEAD_INITIALIZER(output);
+	struct wal_write_request *req;
+
+	tt_pthread_mutex_lock(&writer->mutex);
+	while (writer->is_shutdown == false) {
+		struct wal_fifo input =
+			wal_writer_pop(writer, STAILQ_EMPTY(&output));
+		pthread_mutex_unlock(&writer->mutex);
+		/*
+		 * Check the old list of fibers to wakeup *here*
+		 * since we needed a membar for its out_lsn's to
+		 * sync up.
+		 */
+		STAILQ_FOREACH(req, &output, wal_fifo_entry) {
+			/*
+			 * @todo:
+			 * Even though wal_write() is not
+			 * a cancellation point, check the fiber
+			 * wasn't cancelled and recycled.
+			 * */
+			fiber_wakeup(req->fiber);
+		}
+		STAILQ_FOREACH(req, &input, wal_fifo_entry) {
+			(void) write_to_disk(r, req);
+		}
+		output = input;
+		tt_pthread_mutex_lock(&writer->mutex);
+	}
+	tt_pthread_mutex_unlock(&writer->mutex);
+	write_to_disk(r, NULL);
+	return NULL;
 }
 
 int
 wal_write(struct recovery_state *r, u16 tag, u16 op, u64 cookie,
 	  i64 lsn, struct tbuf *row)
 {
-	struct tbuf *m = tbuf_alloc(fiber->gc_pool);
-	struct msg *a;
-
 	say_debug("wal_write lsn=%" PRIi64, lsn);
-	tbuf_reserve(m, sizeof(struct wal_write_request) +
-		     sizeof(tag) + sizeof(cookie) + sizeof(op) + row->size);
-	m->size = sizeof(struct wal_write_request);
-	wal_write_request(m)->lsn = lsn;
-	wal_write_request(m)->len = sizeof(tag) + sizeof(cookie) + sizeof(op) + row->size;
-	tbuf_append(m, &tag, sizeof(tag));
-	tbuf_append(m, &cookie, sizeof(cookie));
-	tbuf_append(m, &op, sizeof(op));
-	tbuf_append(m, row->data, row->size);
-
-	if (write_inbox(r->wal_writer->out, m) == false) {
-		say_warn("wal writer inbox is full");
-		return -1;
-	}
-	a = read_inbox();
+	struct wal_writer *writer = r->writer;
+
+	struct wal_write_request *req =
+		palloc(fiber->gc_pool, sizeof(struct wal_write_request)
+		       + row->size);
+
+	req->fiber = fiber;
+	req->lsn = lsn;
+	req->tag = tag;
+	req->cookie = cookie;
+	req->op = op;
+	req->len = sizeof(tag) + sizeof(cookie) + sizeof(op) + row->size;
+	memcpy(&req->data, row->data, row->size);
+
+	tt_pthread_mutex_lock(&writer->mutex);
+
+	bool was_empty = STAILQ_EMPTY(&writer->input);
+
+	STAILQ_INSERT_TAIL(&writer->input, req, wal_fifo_entry);
+
+	if (was_empty)
+		tt_pthread_cond_signal(&writer->cond);
 
-	u32 reply = read_u32(a->msg);
-	say_debug("wal_write reply=%" PRIu32, reply);
-	if (reply != 0)
-		say_warn("wal writer returned error status");
-	return reply ? -1 : 0;
+	tt_pthread_mutex_unlock(&writer->mutex);
+
+	fiber_yield();
+
+	return req->out_lsn == 0 ? -1 : 0;
 }
 
-struct recovery_state *
-recover_init(const char *snap_dirname, const char *wal_dirname,
+void
+recovery_init(const char *snap_dirname, const char *wal_dirname,
 	     row_handler row_handler, int rows_per_file,
-	     const char *wal_mode, double fsync_delay,
-	     int inbox_size, int flags, void *data)
+	     const char *wal_mode, double fsync_delay, int flags, void *data)
 {
-	struct recovery_state *r = p0alloc(eter_pool, sizeof(*r));
+	assert(recovery_state == NULL);
+	recovery_state = p0alloc(eter_pool, sizeof(struct recovery_state));
+	struct recovery_state *r = recovery_state;
 
 	if (rows_per_file <= 1)
 		panic("unacceptable value of 'rows_per_file'");
@@ -1341,32 +1507,32 @@ recover_init(const char *snap_dirname, const char *wal_dirname,
 	wait_lsn_clear(&r->wait_lsn);
 
 	if ((flags & RECOVER_READONLY) == 0)
-		r->wal_writer = spawn_child("wal_writer", inbox_size, write_to_disk, r);
-
-	return r;
+		wal_writer_start(r);
 }
 
 void
-recovery_update_mode(struct recovery_state *r, const char *mode,
-		     double fsync_delay)
+recovery_update_mode(const char *mode, double fsync_delay)
 {
+	struct recovery_state *r = recovery_state;
 	(void) mode;
 	r->wal_class->fsync_delay = fsync_delay;
 }
 
 void
-recover_free(struct recovery_state *recovery)
+recovery_free()
 {
-	struct child *writer = recovery->wal_writer;
-	if (writer && writer->out && writer->out->fd > 0) {
-		close(writer->out->fd);
-		usleep(1000);
-	}
+	struct recovery_state *recovery = recovery_state;
+	if (recovery == NULL)
+		return;
+	if (recovery->writer)
+		wal_writer_stop(recovery);
 
 	v11_class_free(recovery->snap_class);
 	v11_class_free(recovery->wal_class);
 	if (recovery->current_wal)
 		log_io_close(&recovery->current_wal);
+
+	recovery_state = NULL;
 }
 
 void
@@ -1396,6 +1562,7 @@ write_rows(struct log_io_iter *i)
 			panic("fwrite");
 
 		row_v11(row)->lsn = 0;	/* unused */
+		/* @todo: check if we can safely use ev_now() here. */
 		row_v11(row)->tm = ev_now();
 		row_v11(row)->len = data->size;
 		row_v11(row)->data_crc32c = crc32_calc(0, data->data, data->size);
diff --git a/core/replication.m b/core/replication.m
index 85298fde11126069f4421538a541f0f73c222535..0c58898e9a3352a10c42d54dfb9a8fd7d383eff2 100644
--- a/core/replication.m
+++ b/core/replication.m
@@ -607,11 +607,12 @@ replication_relay_loop(int client_sock)
 	ev_io_init(&sock_read_ev, replication_relay_recv, sock_read_fd, EV_READ);
 	ev_io_start(&sock_read_ev);
 
-	/* init reovery porcess */
-	log_io = recover_init(NULL, cfg.wal_dir,
-			      replication_relay_send_row,
-			      INT32_MAX, "fsync_delay", 0, 64,
-			      RECOVER_READONLY, false);
+	/* Initialize the recovery process */
+	recovery_init(NULL, cfg.wal_dir, replication_relay_send_row,
+		      INT32_MAX, "fsync_delay", 0,
+		      RECOVER_READONLY, false);
+
+	log_io = recovery_state;
 
 	recover(log_io, lsn);
 	recover_follow(log_io, 0.1);
diff --git a/core/tarantool.m b/core/tarantool.m
index b778cc901d20c9fd47b0f6539da61325a6e939fc..ee745d90bd217a90c407a622d8ecbc2942e46d5f 100644
--- a/core/tarantool.m
+++ b/core/tarantool.m
@@ -70,7 +70,6 @@ char **main_argv;
 int main_argc;
 static void *main_opt = NULL;
 struct tarantool_cfg cfg;
-struct recovery_state *recovery_state;
 static ev_signal *sigs = NULL;
 
 bool init_storage, booting = true;
@@ -155,7 +154,7 @@ core_reload_config(const struct tarantool_cfg *old_conf,
 		say_debug("%s: wal_fsync_delay [%f] -> [%f]",
 			__func__, old_conf->wal_fsync_delay, new_delay);
 
-	recovery_update_mode(recovery_state, new_conf->wal_mode, new_delay);
+	recovery_update_mode(new_conf->wal_mode, new_delay);
 
 	return 0;
 }
@@ -405,8 +404,7 @@ error:
 void
 tarantool_free(void)
 {
-	if (recovery_state != NULL)
-		recover_free(recovery_state);
+	recovery_free();
 	stat_free();
 
 	if (cfg_filename_fullpath)
diff --git a/include/log_io.h b/include/log_io.h
index 9dba46403cf91019e7aa96d8cd1abd5069dc10f9..0450d48b32ced148cd9f10569e5ca7da51cea3f0 100644
--- a/include/log_io.h
+++ b/include/log_io.h
@@ -34,6 +34,7 @@
 #include <util.h>
 #include <palloc.h>
 #include <netinet/in.h> /* struct sockaddr_in */
+#include <third_party/queue.h>
 
 struct tbuf;
 
@@ -102,13 +103,15 @@ struct log_io {
 	bool is_inprogress;
 };
 
+struct wal_writer;
+
 struct recovery_state {
 	i64 lsn, confirmed_lsn;
 
 	struct log_io *current_wal;	/* the WAL we'r currently reading/writing from/to */
 	struct log_io_class *snap_class;
 	struct log_io_class *wal_class;
-	struct child *wal_writer;
+	struct wal_writer *writer;
 
 	/* row_handler will be presented by most recent format of data
 	   log_io_class->reader is responsible of converting data from old format */
@@ -129,12 +132,28 @@ struct recovery_state {
 	void *data;
 };
 
+struct recovery_state *recovery_state;
+
 struct wal_write_request {
+	STAILQ_ENTRY(wal_write_request) wal_fifo_entry;
+	/* Auxiliary. */
+	u64 out_lsn;
+	struct fiber *fiber;
+	/** Header. */
+	u32 marker;
+	u32 header_crc32c;
 	i64 lsn;
+	double tm;
 	u32 len;
+	u32 data_crc32c;
+	/* Data. */
+	u16 tag;
+	u64 cookie;
+	u16 op;
 	u8 data[];
 } __attribute__((packed));
 
+/* @todo: merge with wal_write_request. */
 struct row_v11 {
 	u32 header_crc32c;
 	i64 lsn;
@@ -144,23 +163,21 @@ struct row_v11 {
 	u8 data[];
 } __attribute__((packed));
 
+
 static inline struct row_v11 *row_v11(const struct tbuf *t)
 {
 	return (struct row_v11 *)t->data;
 }
 
-void mach_setup_crc32 ();
-
 struct tbuf *convert_to_v11(struct tbuf *orig, u16 tag, u64 cookie, i64 lsn);
 
-struct recovery_state *recover_init(const char *snap_dirname, const char *xlog_dirname,
-				    row_handler row_handler,
-				    int rows_per_file, const char *wal_mode,
-				    double fsync_delay, int inbox_size,
-				    int flags, void *data);
-void recovery_update_mode(struct recovery_state *r,
-			  const char *wal_mode, double fsync_delay);
-void recover_free(struct recovery_state *recovery);
+void recovery_init(const char *snap_dirname, const char *xlog_dirname,
+		   row_handler row_handler,
+		   int rows_per_file, const char *wal_mode,
+		   double fsync_delay,
+		   int flags, void *data);
+void recovery_update_mode(const char *wal_mode, double fsync_delay);
+void recovery_free();
 int recover(struct recovery_state *, i64 lsn);
 void recover_follow(struct recovery_state *r, ev_tstamp wal_dir_rescan_delay);
 void recover_finalize(struct recovery_state *r);
diff --git a/include/tarantool.h b/include/tarantool.h
index 9e63e889fd8ad7fe8f2ec1373d87c7964d4a8c91..886f15575dc99c0f51867cc0ea41cb5f67e756d5 100644
--- a/include/tarantool.h
+++ b/include/tarantool.h
@@ -33,7 +33,6 @@
 struct lua_State;
 struct luaL_Reg;
 
-extern struct recovery_state *recovery_state;
 void mod_init(void);
 void mod_free(void);
 struct tarantool_cfg;
diff --git a/include/tarantool_pthread.h b/include/tarantool_pthread.h
index 784350f9180b34b967b2ab6cff6002c3aaeec6d8..7c102660f5183ba33a8def3d6fa413db80b90310 100644
--- a/include/tarantool_pthread.h
+++ b/include/tarantool_pthread.h
@@ -154,4 +154,10 @@
 		tt_pthread_error(e);		\
 })
 
+#define tt_pthread_once(control, function)	\
+({	int e = pthread_once(control, function);\
+	tt_pthread_error(e);			\
+})
+
+
 #endif /* TARANTOOL_PTHREAD_H_INCLUDED */
diff --git a/mod/box/box.m b/mod/box/box.m
index 043984512e39467b630495a252684b4012c6951c..115c83af0d454ea3db91c3af43d6557ef452f8f3 100644
--- a/mod/box/box.m
+++ b/mod/box/box.m
@@ -2148,10 +2148,10 @@ mod_init(void)
 	space_init();
 
 	/* recovery initialization */
-	recovery_state = recover_init(cfg.snap_dir, cfg.wal_dir,
-				      recover_row, cfg.rows_per_wal, cfg.wal_mode,
-				      cfg.wal_fsync_delay, cfg.wal_writer_inbox_size,
-				      init_storage ? RECOVER_READONLY : 0, NULL);
+	recovery_init(cfg.snap_dir, cfg.wal_dir,
+		      recover_row, cfg.rows_per_wal, cfg.wal_mode,
+		      cfg.wal_fsync_delay,
+		      init_storage ? RECOVER_READONLY : 0, NULL);
 
 	recovery_state->snap_io_rate_limit = cfg.snap_io_rate_limit * 1024 * 1024;
 	recovery_setup_panic(recovery_state, cfg.panic_on_snap_error, cfg.panic_on_wal_error);
@@ -2254,8 +2254,6 @@ mod_info(struct tbuf *out)
 	tbuf_printf(out, "  version: \"%s\"" CRLF, tarantool_version());
 	tbuf_printf(out, "  uptime: %i" CRLF, (int)tarantool_uptime());
 	tbuf_printf(out, "  pid: %i" CRLF, getpid());
-	tbuf_printf(out, "  wal_writer_pid: %" PRIi64 CRLF,
-		    (i64) recovery_state->wal_writer->pid);
 	tbuf_printf(out, "  lsn: %" PRIi64 CRLF, recovery_state->confirmed_lsn);
 	tbuf_printf(out, "  recovery_lag: %.3f" CRLF, recovery_state->recovery_lag);
 	tbuf_printf(out, "  recovery_last_update: %.3f" CRLF,
diff --git a/test/box/admin.result b/test/box/admin.result
index 2b376cb59e490f6ba0f60f8a6b499d6fceab2e24..078f65688c395807c0750840e0b45b50b5987776 100644
--- a/test/box/admin.result
+++ b/test/box/admin.result
@@ -98,7 +98,6 @@ info:
   version: "1.minor.patch-<rev>-<commit>"
   uptime: <uptime>
   pid: <pid>
-  wal_writer_pid: <pid>
   lsn: 3
   recovery_lag: 0.000
   recovery_last_update: 0.000
diff --git a/test/box/xlog.result b/test/box/xlog.result
index ba345f63be7ea6ca88bcef3857b726ffe294ab32..8faf70ef7bf22a5ae04a09d3289844e222f3d3c3 100644
--- a/test/box/xlog.result
+++ b/test/box/xlog.result
@@ -16,7 +16,7 @@ Insert OK, 1 row affected
 Stopping the server...
 00000000000000000004.xlog.inprogress has been successfully renamed
 
-# An inprogress xlog fle with one record must be renamed during recovery.
+# An inprogress xlog file with one record must be renamed during recovery.
 
 00000000000000000005.xlog.inprogress hash been successfully renamed
 
diff --git a/test/box/xlog.test b/test/box/xlog.test
index 748634e95b85b79ba873d47ffe962ac90420dfa2..170bbec31e1a76806a5833accc0be4b576d474db 100644
--- a/test/box/xlog.test
+++ b/test/box/xlog.test
@@ -44,7 +44,7 @@ if os.access(wal, os.F_OK) and not os.access(wal_inprogress, os.F_OK):
   print "00000000000000000004.xlog.inprogress has been successfully renamed"
 
 print """
-# An inprogress xlog fle with one record must be renamed during recovery.
+# An inprogress xlog file with one record must be renamed during recovery.
 """
 
 wal_inprogress = os.path.join(vardir, "00000000000000000005.xlog.inprogress")