diff --git a/src/box/relay.cc b/src/box/relay.cc
index 39d12126ff635cb44eb094b3bb42796eb13f2b79..1488b2ff75b92f06ff5cea616ff6355f6e36cb48 100644
--- a/src/box/relay.cc
+++ b/src/box/relay.cc
@@ -125,7 +125,7 @@ struct relay {
 		/** Current vclock sent by relay */
 		struct vclock vclock;
 		/** The condition is signaled at relay exit. */
-		struct ipc_cond exit_cond;
+		struct fiber_cond exit_cond;
 	} tx;
 };
 
@@ -286,7 +286,7 @@ static void
 tx_exit_cb(struct cmsg *msg)
 {
 	struct relay_exit_msg *m = (struct relay_exit_msg *)msg;
-	ipc_cond_signal(&m->relay->tx.exit_cond);
+	fiber_cond_signal(&m->relay->tx.exit_cond);
 }
 
 static void
@@ -467,15 +467,15 @@ relay_subscribe(int fd, uint64_t sync, struct replica *replica,
 	struct cord cord;
 	char name[FIBER_NAME_MAX];
 	snprintf(name, sizeof(name), "relay_%p", &relay);
-	ipc_cond_create(&relay.tx.exit_cond);
+	fiber_cond_create(&relay.tx.exit_cond);
 	cord_costart(&cord, name, relay_subscribe_f, &relay);
 	cpipe_create(&relay.relay_pipe, name);
 	/*
 	 * When relay exits, it sends a message which signals the
 	 * exit condition in tx thread.
 	 */
-	ipc_cond_wait(&relay.tx.exit_cond);
-	ipc_cond_destroy(&relay.tx.exit_cond);
+	fiber_cond_wait(&relay.tx.exit_cond);
+	fiber_cond_destroy(&relay.tx.exit_cond);
 	/*
 	 * Destroy the cpipe so that relay fiber can destroy
 	 * the corresponding endpoint and exit.
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index fc68f592dbff78422cf21be7d9e27eb6116ee41f..b698c7ef6608759d8461bfccc17fbbb5c6466312 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -602,9 +602,9 @@ struct vy_scheduler {
 	 * are used here instead of pthread_cond_t.
 	 */
 	struct ev_async scheduler_async;
-	struct ipc_cond scheduler_cond;
+	struct fiber_cond scheduler_cond;
 	/** Used for throttling tx when quota is full. */
-	struct ipc_cond quota_cond;
+	struct fiber_cond quota_cond;
 	/**
 	 * A queue with all vy_task objects created by the
 	 * scheduler and not yet taken by a worker.
@@ -664,7 +664,7 @@ struct vy_scheduler {
 	/** Time when the current dump round started. */
 	ev_tstamp dump_start;
 	/** Signaled on dump round completion. */
-	struct ipc_cond dump_cond;
+	struct fiber_cond dump_cond;
 };
 
 static void
@@ -1450,7 +1450,7 @@ static void
 vy_scheduler_quota_exceeded_cb(struct vy_quota *quota)
 {
 	struct vy_env *env = container_of(quota, struct vy_env, quota);
-	ipc_cond_signal(&env->scheduler->scheduler_cond);
+	fiber_cond_signal(&env->scheduler->scheduler_cond);
 }
 
 static ev_tstamp
@@ -1458,7 +1458,7 @@ vy_scheduler_quota_throttled_cb(struct vy_quota *quota, ev_tstamp timeout)
 {
 	struct vy_env *env = container_of(quota, struct vy_env, quota);
 	ev_tstamp wait_start = ev_now(loop());
-	if (ipc_cond_wait_timeout(&env->scheduler->quota_cond, timeout) != 0)
+	if (fiber_cond_wait_timeout(&env->scheduler->quota_cond, timeout) != 0)
 		return 0; /* timed out */
 	ev_tstamp wait_end = ev_now(loop());
 	return timeout - (wait_end - wait_start);
@@ -1468,7 +1468,7 @@ static void
 vy_scheduler_quota_released_cb(struct vy_quota *quota)
 {
 	struct vy_env *env = container_of(quota, struct vy_env, quota);
-	ipc_cond_broadcast(&env->scheduler->quota_cond);
+	fiber_cond_broadcast(&env->scheduler->quota_cond);
 }
 
 static void
@@ -1478,7 +1478,7 @@ vy_scheduler_async_cb(ev_loop *loop, struct ev_async *watcher, int events)
 	(void) events;
 	struct vy_scheduler *scheduler =
 		container_of(watcher, struct vy_scheduler, scheduler_async);
-	ipc_cond_signal(&scheduler->scheduler_cond);
+	fiber_cond_signal(&scheduler->scheduler_cond);
 }
 
 static struct vy_scheduler *
@@ -1492,7 +1492,7 @@ vy_scheduler_new(struct vy_env *env)
 	}
 	tt_pthread_mutex_init(&scheduler->mutex, NULL);
 	diag_create(&scheduler->diag);
-	ipc_cond_create(&scheduler->dump_cond);
+	fiber_cond_create(&scheduler->dump_cond);
 	vclock_create(&scheduler->last_checkpoint);
 	scheduler->env = env;
 	vy_compact_heap_create(&scheduler->compact_heap);
@@ -1500,8 +1500,8 @@ vy_scheduler_new(struct vy_env *env)
 	tt_pthread_cond_init(&scheduler->worker_cond, NULL);
 	scheduler->loop = loop();
 	ev_async_init(&scheduler->scheduler_async, vy_scheduler_async_cb);
-	ipc_cond_create(&scheduler->scheduler_cond);
-	ipc_cond_create(&scheduler->quota_cond);
+	fiber_cond_create(&scheduler->scheduler_cond);
+	fiber_cond_create(&scheduler->quota_cond);
 	mempool_create(&scheduler->task_pool, cord_slab_cache(),
 			sizeof(struct vy_task));
 	/* Start scheduler fiber. */
@@ -1518,7 +1518,7 @@ vy_scheduler_delete(struct vy_scheduler *scheduler)
 	/* Stop scheduler fiber. */
 	scheduler->scheduler = NULL;
 	/* Sic: fiber_cancel() can't be used here. */
-	ipc_cond_signal(&scheduler->scheduler_cond);
+	fiber_cond_signal(&scheduler->scheduler_cond);
 
 	if (scheduler->is_worker_pool_running)
 		vy_scheduler_stop_workers(scheduler);
@@ -1529,8 +1529,8 @@ vy_scheduler_delete(struct vy_scheduler *scheduler)
 	vy_dump_heap_destroy(&scheduler->dump_heap);
 	tt_pthread_cond_destroy(&scheduler->worker_cond);
 	TRASH(&scheduler->scheduler_async);
-	ipc_cond_destroy(&scheduler->scheduler_cond);
-	ipc_cond_destroy(&scheduler->quota_cond);
+	fiber_cond_destroy(&scheduler->scheduler_cond);
+	fiber_cond_destroy(&scheduler->quota_cond);
 	tt_pthread_mutex_destroy(&scheduler->mutex);
 	free(scheduler);
 }
@@ -1800,7 +1800,7 @@ vy_scheduler_f(va_list va)
 	 * are not started and the scheduler is idle until
 	 * shutdown or checkpoint.
 	 */
-	ipc_cond_wait(&scheduler->scheduler_cond);
+	fiber_cond_wait(&scheduler->scheduler_cond);
 	if (scheduler->scheduler == NULL)
 		return 0; /* destroyed */
 
@@ -1886,7 +1886,7 @@ vy_scheduler_f(va_list va)
 		continue;
 error:
 		/* Abort pending checkpoint. */
-		ipc_cond_signal(&scheduler->dump_cond);
+		fiber_cond_signal(&scheduler->dump_cond);
 		/*
 		 * A task can fail either due to lack of memory or IO
 		 * error. In either case it is pointless to schedule
@@ -1911,7 +1911,7 @@ vy_scheduler_f(va_list va)
 		continue;
 wait:
 		/* Wait for changes */
-		ipc_cond_wait(&scheduler->scheduler_cond);
+		fiber_cond_wait(&scheduler->scheduler_cond);
 	}
 
 	return 0;
@@ -2077,7 +2077,7 @@ vy_scheduler_complete_dump(struct vy_scheduler *scheduler)
 	vy_quota_release(quota, mem_dumped);
 
 	scheduler->dump_generation = min_generation;
-	ipc_cond_signal(&scheduler->dump_cond);
+	fiber_cond_signal(&scheduler->dump_cond);
 
 	/* Account dump bandwidth. */
 	struct vy_stat *stat = scheduler->env->stat;
@@ -2115,7 +2115,7 @@ vy_begin_checkpoint(struct vy_env *env)
 
 	vy_scheduler_trigger_dump(scheduler);
 	scheduler->checkpoint_in_progress = true;
-	ipc_cond_signal(&scheduler->scheduler_cond);
+	fiber_cond_signal(&scheduler->scheduler_cond);
 	return 0;
 }
 
@@ -2141,7 +2141,7 @@ vy_wait_checkpoint(struct vy_env *env, struct vclock *vclock)
 				       diag_last_error(&scheduler->diag));
 			goto error;
 		}
-		ipc_cond_wait(&scheduler->dump_cond);
+		fiber_cond_wait(&scheduler->dump_cond);
 	}
 
 	if (vy_log_rotate(vclock) != 0)
@@ -2171,7 +2171,7 @@ vy_end_checkpoint(struct vy_env *env)
 	 * done so that it can catch up.
 	 */
 	scheduler->checkpoint_in_progress = false;
-	ipc_cond_signal(&scheduler->scheduler_cond);
+	fiber_cond_signal(&scheduler->scheduler_cond);
 }
 
 /* Scheduler }}} */
@@ -6042,7 +6042,7 @@ struct vy_squash_queue {
 	/** Fiber doing background upsert squashing. */
 	struct fiber *fiber;
 	/** Used to wake up the fiber to process more requests. */
-	struct ipc_cond cond;
+	struct fiber_cond cond;
 	/** Queue of vy_squash objects to be processed. */
 	struct stailq queue;
 	/** Mempool for struct vy_squash. */
@@ -6261,7 +6261,7 @@ vy_squash_queue_new(void)
 		return NULL;
 	}
 	sq->fiber = NULL;
-	ipc_cond_create(&sq->cond);
+	fiber_cond_create(&sq->cond);
 	stailq_create(&sq->queue);
 	mempool_create(&sq->pool, cord_slab_cache(),
 		       sizeof(struct vy_squash));
@@ -6274,7 +6274,7 @@ vy_squash_queue_delete(struct vy_squash_queue *sq)
 	if (sq->fiber != NULL) {
 		sq->fiber = NULL;
 		/* Sic: fiber_cancel() can't be used here */
-		ipc_cond_signal(&sq->cond);
+		fiber_cond_signal(&sq->cond);
 	}
 	struct vy_squash *squash, *next;
 	stailq_foreach_entry_safe(squash, next, &sq->queue, next)
@@ -6288,7 +6288,7 @@ vy_squash_queue_f(va_list va)
 	struct vy_squash_queue *sq = va_arg(va, struct vy_squash_queue *);
 	while (sq->fiber != NULL) {
 		if (stailq_empty(&sq->queue)) {
-			ipc_cond_wait(&sq->cond);
+			fiber_cond_wait(&sq->cond);
 			continue;
 		}
 		struct vy_squash *squash;
@@ -6326,7 +6326,7 @@ vy_squash_schedule(struct vy_index *index, struct tuple *stmt, void *arg)
 		goto fail;
 
 	stailq_add_tail_entry(&sq->queue, squash, next);
-	ipc_cond_signal(&sq->cond);
+	fiber_cond_signal(&sq->cond);
 	return;
 fail:
 	error_log(diag_last_error(diag_get()));
diff --git a/src/box/vy_mem.c b/src/box/vy_mem.c
index 0816d546931e35deccfd2e90da4f575e02241197..1d97cdf41bfcf173ca7edde3f8c982f51ad61108 100644
--- a/src/box/vy_mem.c
+++ b/src/box/vy_mem.c
@@ -86,7 +86,7 @@ vy_mem_new(struct lsregion *allocator, int64_t generation,
 			   vy_mem_tree_extent_alloc,
 			   vy_mem_tree_extent_free, index);
 	rlist_create(&index->in_sealed);
-	ipc_cond_create(&index->pin_cond);
+	fiber_cond_create(&index->pin_cond);
 	return index;
 }
 
@@ -113,7 +113,7 @@ vy_mem_delete(struct vy_mem *index)
 	tuple_format_ref(index->format, -1);
 	tuple_format_ref(index->format_with_colmask, -1);
 	tuple_format_ref(index->upsert_format, -1);
-	ipc_cond_destroy(&index->pin_cond);
+	fiber_cond_destroy(&index->pin_cond);
 	TRASH(index);
 	free(index);
 }
diff --git a/src/box/vy_mem.h b/src/box/vy_mem.h
index 4de7df923ec56102867f1fafa50f63cc8bd54b0a..7adae3267dc7003eb21e3c2c46bc0a2be3bc8982 100644
--- a/src/box/vy_mem.h
+++ b/src/box/vy_mem.h
@@ -173,7 +173,7 @@ struct vy_mem {
 	 * Condition variable signaled by vy_mem_unpin()
 	 * if pin_count reaches 0.
 	 */
-	struct ipc_cond pin_cond;
+	struct fiber_cond pin_cond;
 };
 
 /**
@@ -198,7 +198,7 @@ vy_mem_unpin(struct vy_mem *mem)
 	assert(mem->pin_count > 0);
 	mem->pin_count--;
 	if (mem->pin_count == 0)
-		ipc_cond_broadcast(&mem->pin_cond);
+		fiber_cond_broadcast(&mem->pin_cond);
 }
 
 /**
@@ -208,7 +208,7 @@ static inline void
 vy_mem_wait_pinned(struct vy_mem *mem)
 {
 	while (mem->pin_count > 0)
-		ipc_cond_wait(&mem->pin_cond);
+		fiber_cond_wait(&mem->pin_cond);
 }
 
 /**
diff --git a/src/box/vy_run.c b/src/box/vy_run.c
index db3f6453b7ec7d14022965d0fa9904ced83ce7b6..a1e33e765192e46f0d8a3f35ad79d27aee54015b 100644
--- a/src/box/vy_run.c
+++ b/src/box/vy_run.c
@@ -370,7 +370,7 @@ vy_slice_new(int64_t id, struct vy_run *run,
 		tuple_ref(end);
 	slice->end = end;
 	rlist_create(&slice->in_range);
-	ipc_cond_create(&slice->pin_cond);
+	fiber_cond_create(&slice->pin_cond);
 	/** Lookup the first and the last pages spanned by the slice. */
 	bool unused;
 	if (slice->begin == NULL) {
@@ -416,7 +416,7 @@ vy_slice_delete(struct vy_slice *slice)
 		tuple_unref(slice->begin);
 	if (slice->end != NULL)
 		tuple_unref(slice->end);
-	ipc_cond_destroy(&slice->pin_cond);
+	fiber_cond_destroy(&slice->pin_cond);
 	TRASH(slice);
 	free(slice);
 }
diff --git a/src/box/vy_run.h b/src/box/vy_run.h
index 3e8e216bcb0be7303d3af1a9c15ee2ae8c615e41..a6fe45eb56730a8df52094b4663fbaee70f2f798 100644
--- a/src/box/vy_run.h
+++ b/src/box/vy_run.h
@@ -168,7 +168,7 @@ struct vy_slice {
 	 * Condition variable signaled by vy_slice_unpin()
 	 * if pin_count reaches 0.
 	 */
-	struct ipc_cond pin_cond;
+	struct fiber_cond pin_cond;
 	union {
 		/** Link in range->slices list. */
 		struct rlist in_range;
@@ -423,7 +423,7 @@ vy_slice_unpin(struct vy_slice *slice)
 {
 	assert(slice->pin_count > 0);
 	if (--slice->pin_count == 0)
-		ipc_cond_broadcast(&slice->pin_cond);
+		fiber_cond_broadcast(&slice->pin_cond);
 }
 
 /**
@@ -433,7 +433,7 @@ static inline void
 vy_slice_wait_pinned(struct vy_slice *slice)
 {
 	while (slice->pin_count > 0)
-		ipc_cond_wait(&slice->pin_cond);
+		fiber_cond_wait(&slice->pin_cond);
 }
 
 /**
diff --git a/src/cbus.c b/src/cbus.c
index 859c7f5d214b7d32ffcd7f29d439b5a35910812f..62f709af701ec86a8bf1b8cfc4f1fdbd17a8cc24 100644
--- a/src/cbus.c
+++ b/src/cbus.c
@@ -110,7 +110,7 @@ cbus_endpoint_poison_f(struct cmsg *msg)
 {
 	struct cbus_endpoint *endpoint = ((struct cmsg_poison *)msg)->endpoint;
 	--endpoint->n_pipes;
-	ipc_cond_signal(&endpoint->cond);
+	fiber_cond_signal(&endpoint->cond);
 	free(msg);
 }
 
@@ -193,7 +193,7 @@ cbus_endpoint_create(struct cbus_endpoint *endpoint, const char *name,
 	snprintf(endpoint->name, sizeof(endpoint->name), "%s", name);
 	endpoint->consumer = loop();
 	endpoint->n_pipes = 0;
-	ipc_cond_create(&endpoint->cond);
+	fiber_cond_create(&endpoint->cond);
 	tt_pthread_mutex_init(&endpoint->mutex, NULL);
 	stailq_create(&endpoint->output);
 	ev_async_init(&endpoint->async,
@@ -231,7 +231,7 @@ cbus_endpoint_destroy(struct cbus_endpoint *endpoint,
 			process_cb(endpoint);
 		if (endpoint->n_pipes == 0 && stailq_empty(&endpoint->output))
 			break;
-		 ipc_cond_wait(&endpoint->cond);
+		 fiber_cond_wait(&endpoint->cond);
 	}
 
 	/*
@@ -242,7 +242,7 @@ cbus_endpoint_destroy(struct cbus_endpoint *endpoint,
 	tt_pthread_mutex_unlock(&endpoint->mutex);
 	tt_pthread_mutex_destroy(&endpoint->mutex);
 	ev_async_stop(endpoint->consumer, &endpoint->async);
-	ipc_cond_destroy(&endpoint->cond);
+	fiber_cond_destroy(&endpoint->cond);
 	TRASH(endpoint);
 	return 0;
 }
@@ -404,7 +404,7 @@ cbus_call(struct cpipe *callee, struct cpipe *caller, struct cbus_call_msg *msg,
 struct cbus_flush_msg {
 	struct cmsg cmsg;
 	bool complete;
-	struct ipc_cond cond;
+	struct fiber_cond cond;
 };
 
 static void
@@ -419,7 +419,7 @@ cbus_flush_complete(struct cmsg *cmsg)
 	struct cbus_flush_msg *msg = container_of(cmsg,
 			struct cbus_flush_msg, cmsg);
 	msg->complete = true;
-	ipc_cond_signal(&msg->cond);
+	fiber_cond_signal(&msg->cond);
 }
 
 void
@@ -434,7 +434,7 @@ cbus_flush(struct cpipe *callee, struct cpipe *caller,
 
 	cmsg_init(&msg.cmsg, route);
 	msg.complete = false;
-	ipc_cond_create(&msg.cond);
+	fiber_cond_create(&msg.cond);
 
 	cpipe_push(callee, &msg.cmsg);
 
@@ -443,7 +443,7 @@ cbus_flush(struct cpipe *callee, struct cpipe *caller,
 			process_cb(caller->endpoint);
 		if (msg.complete)
 			break;
-		ipc_cond_wait(&msg.cond);
+		fiber_cond_wait(&msg.cond);
 	}
 }
 
diff --git a/src/cbus.h b/src/cbus.h
index df24e5e928198545a316a6ebdb1e4811304ae95c..d9f9cdcc4558208b1afaacd3b5ae33ae1cb74102 100644
--- a/src/cbus.h
+++ b/src/cbus.h
@@ -257,7 +257,7 @@ struct cbus_endpoint {
 	/** Count of connected pipes */
 	uint32_t n_pipes;
 	/** Condition for endpoint destroy */
-	struct ipc_cond cond;
+	struct fiber_cond cond;
 };
 
 /**
diff --git a/src/fiber_cond.c b/src/fiber_cond.c
index d81120d2db2e4af54831b1d6866b4a706e1664d1..783f2654b4afd217dc75b50481dd25e6a79feb79 100644
--- a/src/fiber_cond.c
+++ b/src/fiber_cond.c
@@ -36,20 +36,20 @@
 #include "fiber.h"
 
 void
-ipc_cond_create(struct ipc_cond *c)
+fiber_cond_create(struct fiber_cond *c)
 {
 	rlist_create(&c->waiters);
 }
 
 void
-ipc_cond_destroy(struct ipc_cond *c)
+fiber_cond_destroy(struct fiber_cond *c)
 {
 	(void)c;
 	assert(rlist_empty(&c->waiters));
 }
 
 void
-ipc_cond_signal(struct ipc_cond *e)
+fiber_cond_signal(struct fiber_cond *e)
 {
 	if (! rlist_empty(&e->waiters)) {
 		struct fiber *f;
@@ -59,7 +59,7 @@ ipc_cond_signal(struct ipc_cond *e)
 }
 
 void
-ipc_cond_broadcast(struct ipc_cond *e)
+fiber_cond_broadcast(struct fiber_cond *e)
 {
 	while (! rlist_empty(&e->waiters)) {
 		struct fiber *f;
@@ -69,7 +69,7 @@ ipc_cond_broadcast(struct ipc_cond *e)
 }
 
 int
-ipc_cond_wait_timeout(struct ipc_cond *c, double timeout)
+fiber_cond_wait_timeout(struct fiber_cond *c, double timeout)
 {
 	struct fiber *f = fiber();
 	rlist_add_tail_entry(&c->waiters, f, state);
@@ -81,7 +81,7 @@ ipc_cond_wait_timeout(struct ipc_cond *c, double timeout)
 }
 
 int
-ipc_cond_wait(struct ipc_cond *c)
+fiber_cond_wait(struct fiber_cond *c)
 {
-	return ipc_cond_wait_timeout(c, TIMEOUT_INFINITY);
+	return fiber_cond_wait_timeout(c, TIMEOUT_INFINITY);
 }
diff --git a/src/fiber_cond.h b/src/fiber_cond.h
index 8b1fe5436b30f9f87df6f1a31ee5b12c99f4750b..ed0c1f7280c2542cb196d11bf6aa9aae895cd376 100644
--- a/src/fiber_cond.h
+++ b/src/fiber_cond.h
@@ -37,7 +37,7 @@
 extern "C" {
 #endif /* defined(__cplusplus) */
 
-struct ipc_cond {
+struct fiber_cond {
 	struct rlist waiters;
 };
 
@@ -45,32 +45,32 @@ struct ipc_cond {
  * Initialize a cond - semantics as in POSIX condition variable.
  */
 void
-ipc_cond_create(struct ipc_cond *c);
+fiber_cond_create(struct fiber_cond *c);
 
 /**
  * Finalize a cond. UB if there are fibers waiting for a cond.
  */
 void
-ipc_cond_destroy(struct ipc_cond *c);
+fiber_cond_destroy(struct fiber_cond *c);
 
 /**
  * Wake one fiber waiting for the cond.
  * Does nothing if no one is waiting.
  */
 void
-ipc_cond_signal(struct ipc_cond *c);
+fiber_cond_signal(struct fiber_cond *c);
 
 /**
  * Wake all fibers waiting for the cond.
  */
 void
-ipc_cond_broadcast(struct ipc_cond *c);
+fiber_cond_broadcast(struct fiber_cond *c);
 
 int
-ipc_cond_wait_timeout(struct ipc_cond *c, double timeout);
+fiber_cond_wait_timeout(struct fiber_cond *c, double timeout);
 
 int
-ipc_cond_wait(struct ipc_cond *c);
+fiber_cond_wait(struct fiber_cond *c);
 
 #if defined(__cplusplus)
 } /* extern "C" */
diff --git a/src/fiber_pool.c b/src/fiber_pool.c
index c596a9df257b231fe69445dbaad36382bebd06e6..d3d6500e333b6121bd62f318c3df78e4e7ba6cbb 100644
--- a/src/fiber_pool.c
+++ b/src/fiber_pool.c
@@ -76,7 +76,7 @@ fiber_pool_f(va_list ap)
 		goto restart;
 	}
 	pool->size--;
-	ipc_cond_signal(&pool->worker_cond);
+	fiber_cond_signal(&pool->worker_cond);
 
 	return 0;
 }
@@ -147,7 +147,7 @@ fiber_pool_create(struct fiber_pool *pool, const char *name, int max_pool_size,
 	pool->size = 0;
 	pool->max_size = max_pool_size;
 	stailq_create(&pool->output);
-	ipc_cond_create(&pool->worker_cond);
+	fiber_cond_create(&pool->worker_cond);
 	/* Join fiber pool to cbus */
 	cbus_endpoint_create(&pool->endpoint, name, fiber_pool_cb, pool);
 }
@@ -172,7 +172,7 @@ fiber_pool_destroy(struct fiber_pool *pool)
 	 * Just wait on fiber exit condition until all fibers are done
 	 */
 	while (pool->size > 0)
-		ipc_cond_wait(&pool->worker_cond);
-	ipc_cond_destroy(&pool->worker_cond);
+		fiber_cond_wait(&pool->worker_cond);
+	fiber_cond_destroy(&pool->worker_cond);
 }
 
diff --git a/src/fiber_pool.h b/src/fiber_pool.h
index f79497f33b9de014fdae835c7b17de3e78f18fd0..d6a95105b00bae003d64176c0c535fa1a2ee5841 100644
--- a/src/fiber_pool.h
+++ b/src/fiber_pool.h
@@ -65,7 +65,7 @@ struct fiber_pool {
 		/** Timer for idle workers */
 		struct ev_timer idle_timer;
 		/** Condition for worker exit signaling */
-		struct ipc_cond worker_cond;
+		struct fiber_cond worker_cond;
 	};
 	struct {
 		/** The consumer thread loop. */
diff --git a/src/httpc.c b/src/httpc.c
index be35b045893ca3a08388d95b2db48700f72c275f..222456d467b953c8e7962d07049788d0df2a1c29 100644
--- a/src/httpc.c
+++ b/src/httpc.c
@@ -83,7 +83,7 @@ curl_multi_process(CURLM *multi, curl_socket_t sockfd, int events)
 		curl_easy_getinfo(easy, CURLINFO_PRIVATE, (void *) &resp);
 
 		resp->curl_code = (int) curl_code;
-		ipc_cond_signal(&resp->cond);
+		fiber_cond_signal(&resp->cond);
 	}
 }
 
@@ -488,7 +488,7 @@ httpc_response_new(struct httpc_env *ctx)
 	resp->curl_code = CURLE_OK;
 	region_create(&resp->headers, &cord()->slabc);
 	region_create(&resp->body, &cord()->slabc);
-	ipc_cond_create(&resp->cond);
+	fiber_cond_create(&resp->cond);
 	return resp;
 }
 
@@ -497,7 +497,7 @@ httpc_response_delete(struct httpc_response *resp)
 {
 	region_destroy(&resp->headers);
 	region_destroy(&resp->body);
-	ipc_cond_destroy(&resp->cond);
+	fiber_cond_destroy(&resp->cond);
 	mempool_free(&resp->ctx->resp_pool, resp);
 }
 
@@ -525,7 +525,7 @@ httpc_execute(struct httpc_request *req, double timeout)
 	/* Don't wait on a cond if request has already failed */
 	if (resp->curl_code == CURLE_OK) {
 		++env->stat.active_requests;
-		int rc = ipc_cond_wait_timeout(&resp->cond, timeout);
+		int rc = fiber_cond_wait_timeout(&resp->cond, timeout);
 		if (rc < 0 || fiber_is_cancelled())
 			resp->curl_code = CURLE_OPERATION_TIMEDOUT;
 		--env->stat.active_requests;
diff --git a/src/httpc.h b/src/httpc.h
index 02ba40467c64fb7a1f4f7cb8275ee384c7588199..21b6bf44f1e356606c4c722be0c40f4cc29d333f 100644
--- a/src/httpc.h
+++ b/src/httpc.h
@@ -262,7 +262,7 @@ struct httpc_response {
 	 * When request is given to curl-driver, client waits on this variable
 	 * until the handler (callback function) gives a signal within variable
 	 * */
-	struct ipc_cond cond;
+	struct fiber_cond cond;
 };
 
 /**
diff --git a/src/lua/fiber_cond.c b/src/lua/fiber_cond.c
index 12dba65d68b722de8f5e44e581d97f08d2b10682..61722696990b88f06002e8cb4ee5f5c59f84e6cc 100644
--- a/src/lua/fiber_cond.c
+++ b/src/lua/fiber_cond.c
@@ -41,52 +41,52 @@
 static const char cond_typename[] = "fiber.cond";
 
 static int
-lbox_ipc_cond(struct lua_State *L)
+luaT_fiber_cond_new(struct lua_State *L)
 {
-	struct ipc_cond *e = lua_newuserdata(L, sizeof(*e));
+	struct fiber_cond *e = lua_newuserdata(L, sizeof(*e));
 	if (e == NULL)
 		luaL_error(L, "fiber.cond: not enough memory");
-	ipc_cond_create(e);
+	fiber_cond_create(e);
 	luaL_getmetatable(L, cond_typename);
 	lua_setmetatable(L, -2);
 	return 1;
 }
 
-static inline struct ipc_cond *
-lbox_check_cond(struct lua_State *L, int index, const char *source)
+static inline struct fiber_cond *
+luaT_checkfibercond(struct lua_State *L, int index, const char *source)
 {
 	if (index > lua_gettop(L))
 		luaL_error(L, "usage: %s", source);
-	return (struct ipc_cond *)luaL_checkudata(L, index, cond_typename);
+	return (struct fiber_cond *)luaL_checkudata(L, index, cond_typename);
 }
 
 static int
-lbox_ipc_cond_gc(struct lua_State *L)
+luaT_fiber_cond_gc(struct lua_State *L)
 {
-	ipc_cond_destroy(lbox_check_cond(L, 1, "cond:destroy()"));
+	fiber_cond_destroy(luaT_checkfibercond(L, 1, "cond:destroy()"));
 	return 0;
 }
 
 static int
-lbox_ipc_cond_signal(struct lua_State *L)
+luaT_fiber_cond_signal(struct lua_State *L)
 {
-	ipc_cond_signal(lbox_check_cond(L, 1, "cond:signal()"));
+	fiber_cond_signal(luaT_checkfibercond(L, 1, "cond:signal()"));
 	return 0;
 }
 
 static int
-lbox_ipc_cond_broadcast(struct lua_State *L)
+luaT_fiber_cond_broadcast(struct lua_State *L)
 {
-	ipc_cond_broadcast(lbox_check_cond(L, 1, "cond:broadcast()"));
+	fiber_cond_broadcast(luaT_checkfibercond(L, 1, "cond:broadcast()"));
 	return 0;
 }
 
 static int
-lbox_ipc_cond_wait(struct lua_State *L)
+luaT_fiber_cond_wait(struct lua_State *L)
 {
 	static const char usage[] = "cond:wait([timeout])";
 	int rc;
-	struct ipc_cond *e = lbox_check_cond(L, 1, usage);
+	struct fiber_cond *e = luaT_checkfibercond(L, 1, usage);
 	ev_tstamp timeout = TIMEOUT_INFINITY;
 	if (!lua_isnoneornil(L, 2)) {
 		if (!lua_isnumber(L, 2) ||
@@ -94,7 +94,7 @@ lbox_ipc_cond_wait(struct lua_State *L)
 			luaL_error(L, "usage: %s", usage);
 		}
 	}
-	rc = ipc_cond_wait_timeout(e, timeout);
+	rc = fiber_cond_wait_timeout(e, timeout);
 	if (rc != 0)
 		luaL_testcancel(L);
 	lua_pushboolean(L, rc == 0);
@@ -102,9 +102,9 @@ lbox_ipc_cond_wait(struct lua_State *L)
 }
 
 static int
-lbox_ipc_cond_to_string(struct lua_State *L)
+luaT_fiber_cond_tostring(struct lua_State *L)
 {
-	struct ipc_cond *cond = lbox_check_cond(L, 1, "");
+	struct fiber_cond *cond = luaT_checkfibercond(L, 1, "");
 	(void)cond;
 	lua_pushstring(L, "cond");
 	return 1;
@@ -114,17 +114,17 @@ void
 tarantool_lua_fiber_cond_init(struct lua_State *L)
 {
 	static const struct luaL_Reg cond_meta[] = {
-		{"__gc",	lbox_ipc_cond_gc},
-		{"__tostring",	lbox_ipc_cond_to_string},
-		{"signal",	lbox_ipc_cond_signal},
-		{"broadcast",	lbox_ipc_cond_broadcast},
-		{"wait",	lbox_ipc_cond_wait},
+		{"__gc",	luaT_fiber_cond_gc},
+		{"__tostring",	luaT_fiber_cond_tostring},
+		{"signal",	luaT_fiber_cond_signal},
+		{"broadcast",	luaT_fiber_cond_broadcast},
+		{"wait",	luaT_fiber_cond_wait},
 		{NULL, NULL}
 	};
 	luaL_register_type(L, cond_typename, cond_meta);
 
 	static const struct luaL_Reg cond_lib[] = {
-		{"cond",	lbox_ipc_cond},
+		{"cond",	luaT_fiber_cond_new},
 		{NULL, NULL}
 	};