diff --git a/src/box/box.cc b/src/box/box.cc
index 7dfe29c56ed392456bda9140e94f3f8ee9f55958..23004fff1a245dd424de37ea181f832d97c0218b 100644
--- a/src/box/box.cc
+++ b/src/box/box.cc
@@ -51,6 +51,7 @@
 #include "user.h"
 #include "cfg.h"
 #include "iobuf.h"
+#include "coeio.h"
 
 static void process_ro(struct request *request, struct port *port);
 box_process_func box_process = process_ro;
@@ -58,6 +59,7 @@ box_process_func box_process = process_ro;
 struct recovery_state *recovery;
 
 int snapshot_pid = 0; /* snapshot processes pid */
+int64_t snapshot_last_lsn = 0;
 
 static void
 box_snapshot_cb(struct xlog *l);
@@ -237,7 +239,7 @@ box_leave_local_standby_mode(void *data __attribute__((unused)))
 	/*
 	 * notify engines about end of recovery.
 	*/
-	space_end_recover();
+	engine_end_recover();
 
 	stat_cleanup(stat_base, IPROTO_TYPE_STAT_MAX);
 	box_set_wal_mode(cfg_gets("wal_mode"));
@@ -432,21 +434,24 @@ box_init()
 				     cfg_geti("panic_on_wal_error"));
 
 		if (recovery_has_data(recovery)) {
+			/* Tell Sophia engine LSN it must recover to. */
+			snapshot_last_lsn = recovery_snap_lsn(recovery);
+			engine_begin_recover_snapshot(snapshot_last_lsn);
 			/* Process existing snapshot */
 			recover_snap(recovery);
-			space_end_recover_snapshot();
+			engine_end_recover_snapshot();
 		} else if (recovery_has_remote(recovery)) {
 			/* Initialize a new replica */
 			replica_bootstrap(recovery);
-			space_end_recover_snapshot();
-			snapshot_save(recovery, box_snapshot_cb);
+			engine_end_recover_snapshot();
+			box_deploy(recovery);
 		} else {
 			/* Initialize the first server of a new cluster */
 			recovery_bootstrap(recovery);
 			box_set_cluster_uuid();
 			box_set_server_uuid();
-			space_end_recover_snapshot();
-			snapshot_save(recovery, box_snapshot_cb);
+			engine_end_recover_snapshot();
+			box_deploy(recovery);
 		}
 		fiber_gc();
 	} catch (Exception *e) {
@@ -532,45 +537,134 @@ box_snapshot_cb(struct xlog *l)
 	space_foreach(snapshot_space, l);
 }
 
+static inline void
+engine_start_snapshot(EngineFactory *f, void *udate)
+{
+	int64_t lsn = *(int64_t*)udate;
+	f->snapshot(SNAPSHOT_START, lsn);
+}
+
+static inline void
+engine_delete_snapshot(EngineFactory *f, void *udate)
+{
+	int64_t lsn = *(int64_t*)udate;
+	f->snapshot(SNAPSHOT_DELETE, lsn);
+}
+
+static inline void
+engine_wait_snapshot(EngineFactory *f, void *udate)
+{
+	int64_t lsn = *(int64_t*)udate;
+	f->snapshot(SNAPSHOT_WAIT, lsn);
+}
+
+static ssize_t
+box_snapshot_rename_cb(va_list ap)
+{
+	uint64_t id = *va_arg(ap, uint64_t*);
+	int *status = va_arg(ap, int*);
+	*status = snapshot_rename(&recovery->snap_dir, id);
+	return 0;
+}
+
 int
 box_snapshot(void)
 {
 	if (snapshot_pid)
 		return EINPROGRESS;
 
-	/* flush buffers to avoid multiple output */
-	/* https://github.com/tarantool/tarantool/issues/366 */
+	/* flush buffers to avoid multiple output
+	 *
+	 * https://github.com/tarantool/tarantool/issues/366
+	*/
 	fflush(stdout);
 	fflush(stderr);
-	pid_t p = fork();
-	if (p < 0) {
-		say_syserror("fork");
-		return -1;
-	}
-	if (p > 0) {
-		snapshot_pid = p;
-		/* increment snapshot version */
-		tuple_begin_snapshot();
-		int status = wait_for_child(p);
-		tuple_end_snapshot();
-		snapshot_pid = 0;
-		return (WIFSIGNALED(status) ? EINTR : WEXITSTATUS(status));
-	}
 
-	slab_arena_mprotect(&memtx_arena);
+	/* create snapshot file */
+	int64_t snap_lsn = vclock_signature(&recovery->vclock);
+
+	/* create engine snapshot */
+	engine_foreach(engine_start_snapshot, &snap_lsn);
 
-	cord_set_name("snap");
-	title("dumper", "%" PRIu32, getppid());
-	fiber_set_name(fiber(), "dumper");
 	/*
-	 * Safety: make sure we don't double-write
-	 * parent stdio buffers at exit().
+	 * Due to fork nature, no threads are recreated.
+	 * This is the only consistency guarantee here for a
+	 * multi-threaded engine.
 	 */
-	close_all_xcpt(1, log_fd);
-	snapshot_save(recovery, box_snapshot_cb);
+	int rc;
+	int status = 0;
+	pid_t pid = fork();
+	switch (pid) {
+	case -1:
+		say_syserror("fork");
+		status = errno;
+		goto error;
+	case  0: /* dumper */
+		slab_arena_mprotect(&memtx_arena);
+		cord_set_name("snap");
+		title("dumper", "%" PRIu32, getppid());
+		fiber_set_name(fiber(), "dumper");
+		/* make sure we don't double-write parent stdio
+		 * buffers at exit() during panic */
+		close_all_xcpt(1, log_fd);
+		/* do not rename snapshot */
+		snapshot_save(recovery, box_snapshot_cb, false);
+		exit(EXIT_SUCCESS);
+		return 0;
+	default: /* waiter */
+		snapshot_pid = pid;
+	}
+
+	/* increment snapshot version */
+	tuple_begin_snapshot();
+
+	/* wait for memtx-part snapshot completion */
+	status = wait_for_child(pid);
+	if (WIFSIGNALED(status))
+		status = EINTR;
+	else
+		status = WEXITSTATUS(status);
+	if (status != 0)
+		goto error;
+
+	/* complete snapshot */
+	tuple_end_snapshot();
+
+	/* wait for engine snapshot completion */
+	engine_foreach(engine_wait_snapshot, &snap_lsn);
 
-	exit(EXIT_SUCCESS);
+	/* rename snapshot on completion */
+	rc = coeio_custom(box_snapshot_rename_cb,
+	                  TIMEOUT_INFINITY, &snap_lsn, &status);
+	if (rc == -1 || status == -1)
+		goto error;
+
+	/* remove previous snapshot reference */
+	engine_foreach(engine_delete_snapshot, &snapshot_last_lsn);
+
+	snapshot_last_lsn = snap_lsn;
+	snapshot_pid = 0;
 	return 0;
+
+error:
+	/* rollback snapshot creation */
+	if (snap_lsn != snapshot_last_lsn)
+		engine_foreach(engine_delete_snapshot, &snap_lsn);
+	tuple_end_snapshot();
+	snapshot_pid = 0;
+	return status;
+}
+
+void
+box_deploy(struct recovery_state *r)
+{
+	/* create memtx snapshot */
+	snapshot_save(r, box_snapshot_cb, true);
+
+	/* create engine snapshot and wait for completion */
+	uint64_t snap_lsn = vclock_signature(&r->vclock);
+	engine_foreach(engine_start_snapshot, &snap_lsn);
+	engine_foreach(engine_wait_snapshot, &snap_lsn);
 }
 
 const char *
diff --git a/src/box/box.h b/src/box/box.h
index 4729c99d10ed36f7f5d13affaf51c53b5431b102..94c79e663fcb20273e9bf182d195a2b92ba9afcf 100644
--- a/src/box/box.h
+++ b/src/box/box.h
@@ -83,6 +83,7 @@ extern uint32_t snapshot_version;
  * snapshot file.
  */
 int box_snapshot(void);
+void box_deploy(struct recovery_state*);
 
 /** Basic initialization of the storage dir. */
 void
diff --git a/src/box/engine.cc b/src/box/engine.cc
index 13cb1492246f00d8567cd085e034cd6e066edaa2..6d3f896c0ea0792c282a3a427f5291b4dff0aaf1 100644
--- a/src/box/engine.cc
+++ b/src/box/engine.cc
@@ -28,6 +28,8 @@
  */
 #include "engine.h"
 #include "space.h"
+#include "exception.h"
+#include "schema.h"
 #include "salad/rlist.h"
 #include <stdlib.h>
 #include <string.h>
@@ -107,3 +109,68 @@ void engine_shutdown()
 		delete e;
 	}
 }
+
+static inline void
+engine_begin_recover_cb(EngineFactory *f, void *udate)
+{
+	int64_t lsn = *(int64_t*)udate;
+	f->snapshot(SNAPSHOT_RECOVER, lsn);
+}
+
+void
+engine_begin_recover_snapshot(int64_t snapshot_lsn)
+{
+	/* recover engine snapshot */
+	engine_foreach(engine_begin_recover_cb, &snapshot_lsn);
+}
+
+static void
+do_one_recover_step(struct space *space, void * /* param */)
+{
+	if (space_index(space, 0)) {
+		space->engine->recover(space);
+	} else {
+		/* in case of space has no primary index,
+		 * derive it's engine handler recovery state from
+		 * the global one. */
+		space->engine->initRecovery();
+	}
+}
+
+static inline void
+engine_end_recover_snapshot_cb(EngineFactory *f, void *udate)
+{
+	(void)udate;
+	f->recoveryEvent(END_RECOVERY_SNAPSHOT);
+}
+
+void
+engine_end_recover_snapshot()
+{
+	/*
+	 * For all new spaces created from now on, when the
+	 * PRIMARY key is added, enable it right away.
+	 */
+	engine_foreach(engine_end_recover_snapshot_cb, NULL);
+	space_foreach(do_one_recover_step, NULL);
+}
+
+static inline void
+engine_end_recover_cb(EngineFactory *f, void *udate)
+{
+	(void)udate;
+	f->recoveryEvent(END_RECOVERY);
+}
+
+void
+engine_end_recover()
+{
+	/*
+	 * For all new spaces created after recovery is complete,
+	 * when the primary key is added, enable all keys.
+	 */
+	engine_foreach(engine_end_recover_cb, NULL);
+
+	space_foreach(do_one_recover_step, NULL);
+}
+
diff --git a/src/box/engine.h b/src/box/engine.h
index c811cef6d5074f6e8c461464b11e1ddaa0ca440b..276ef3c269bae54ebb385ee017dda818e30528f1 100644
--- a/src/box/engine.h
+++ b/src/box/engine.h
@@ -74,6 +74,16 @@ enum engine_recovery_event {
 	END_RECOVERY
 };
 
+/**
+ * Engine specific snapshot event.
+ */
+enum engine_snapshot_event {
+	SNAPSHOT_START,
+	SNAPSHOT_RECOVER,
+	SNAPSHOT_DELETE,
+	SNAPSHOT_WAIT
+};
+
 typedef void (*engine_recover_f)(struct space*);
 
 typedef struct tuple *
@@ -127,6 +137,10 @@ class EngineFactory: public Object {
 	virtual void begin(struct txn*, struct space*);
 	virtual void commit(struct txn*);
 	virtual void rollback(struct txn*);
+	/**
+	 * Engine snapshotting support.
+	 */
+	virtual void snapshot(enum engine_snapshot_event, int64_t) = 0;
 public:
 	/** Name of the engine. */
 	const char *name;
@@ -203,4 +217,25 @@ engine_id(Engine *engine)
 	return engine->factory->id;
 }
 
+/**
+ * Tell the engine what the last LSN to recover from is
+ * (during server start.
+ */
+void
+engine_begin_recover_snapshot(int64_t snapshot_lsn);
+
+/**
+ * Called at the end of recovery from snapshot.
+ * Build primary keys in all spaces.
+ * */
+void
+engine_end_recover_snapshot();
+
+/**
+ * Called at the end of recovery.
+ * Build secondary keys in all spaces.
+ */
+void
+engine_end_recover();
+
 #endif /* TARANTOOL_BOX_ENGINE_H_INCLUDED */
diff --git a/src/box/engine_memtx.cc b/src/box/engine_memtx.cc
index 2011f03742b94176b6912aba562f878bde5b2452..c021d8431dce616edd2bec72d6903e496558cbc6 100644
--- a/src/box/engine_memtx.cc
+++ b/src/box/engine_memtx.cc
@@ -211,3 +211,11 @@ void MemtxFactory::rollback(struct txn *txn)
 		}
 	}
 }
+
+void MemtxFactory::snapshot(enum engine_snapshot_event, int64_t /* lsn */)
+{
+	/*
+	 * memtx snapshotting supported directly by box.
+	 * do nothing here.
+	 */
+}
diff --git a/src/box/engine_memtx.h b/src/box/engine_memtx.h
index d9fe35bfb32152422e8c9103d799f2e8bd2ad757..693b6362a7a18c4d1572e614767fd6b4d6537aa7 100644
--- a/src/box/engine_memtx.h
+++ b/src/box/engine_memtx.h
@@ -37,6 +37,7 @@ struct MemtxFactory: public EngineFactory {
 	virtual void keydefCheck(struct key_def *key_def);
 	virtual void recoveryEvent(enum engine_recovery_event event);
 	virtual void rollback(struct txn*);
+	virtual void snapshot(enum engine_snapshot_event, int64_t);
 };
 
 #endif /* TARANTOOL_BOX_ENGINE_MEMTX_H_INCLUDED */
diff --git a/src/box/engine_sophia.cc b/src/box/engine_sophia.cc
index 5489202748522a555627548f13878a070c31da5b..4da8630361d5a4a2b7e9ec272f47f971f8a5e837 100644
--- a/src/box/engine_sophia.cc
+++ b/src/box/engine_sophia.cc
@@ -87,7 +87,9 @@ sophia_recovery_end(struct space *space)
 	r->state   = READY_ALL_KEYS;
 	r->replace = sophia_replace;
 	r->recover = space_noop;
+	/*
 	sophia_complete_recovery(space);
+	*/
 }
 
 static void
@@ -109,8 +111,8 @@ SophiaFactory::SophiaFactory()
 	:EngineFactory("sophia")
 {
 	flags = ENGINE_TRANSACTIONAL;
-	env   = NULL;
-	tx    = NULL;
+	env = NULL;
+	tx  = NULL;
 	recovery.state   = READY_NO_KEYS;
 	recovery.recover = sophia_recovery_begin_snapshot;
 	recovery.replace = sophia_replace_recover;
@@ -151,6 +153,10 @@ SophiaFactory::recoveryEvent(enum engine_recovery_event event)
 		recovery.recover = sophia_recovery_end_snapshot;
 		break;
 	case END_RECOVERY:
+		/* complete two-phase recovery */
+		int rc = sp_open(env);
+		if (rc == -1)
+			sophia_raise(env);
 		recovery.state   = READY_NO_KEYS;
 		recovery.replace = sophia_replace;
 		recovery.recover = space_noop;
@@ -271,39 +277,19 @@ SophiaFactory::commit(struct txn *txn)
 		tx = NULL;
 	});
 
-	/* a. prepare transaction for commit */
-	int rc = sp_prepare(tx);
-	if (rc == -1)
-		sophia_raise(env);
-	assert(rc == 0);
-
-	/* b. create transaction log cursor and
-	 *    forge each statement's LSN number.
-	*/
-	void *lc = sp_ctl(tx, "log_cursor");
-	if (lc == NULL) {
-		sp_rollback(tx);
-		sophia_raise(env);
-	}
+	/* a. get max lsn for commit */
+	int64_t lsn = 0;
 	struct txn_stmt *stmt;
 	rlist_foreach_entry(stmt, &txn->stmts, next) {
-		if (stmt->new_tuple == NULL && stmt->old_tuple == NULL)
-			continue;
-		void *v = sp_get(lc);
-		assert(v != NULL);
-		sp_set(v, "lsn", stmt->row->lsn);
-		/* remove tuple reference */
-		if (stmt->new_tuple) {
-			/* 2 refs: iproto case */
-			/* 3 refs: lua case */
-			assert(stmt->new_tuple->refs >= 2);
-			tuple_unref(stmt->new_tuple);
-		}
+		if (stmt->row->lsn > lsn)
+			lsn = stmt->row->lsn;
 	}
-	assert(sp_get(lc) == NULL);
-	sp_destroy(lc);
 
-	/* c. commit transaction */
+	/* b. commit transaction */
+	int rc = sp_prepare(tx, lsn);
+	assert(rc == 0);
+	if (rc == -1)
+		sophia_raise(env);
 	rc = sp_commit(tx);
 	if (rc == -1)
 		sophia_raise(env);
@@ -320,3 +306,106 @@ SophiaFactory::rollback(struct txn *)
 	});
 	sp_rollback(tx);
 }
+
+static inline void
+sophia_snapshot(void *env, int64_t lsn)
+{
+	/* start asynchronous checkpoint */
+	void *c = sp_ctl(env);
+	int rc = sp_set(c, "scheduler.checkpoint");
+	if (rc == -1)
+		sophia_raise(env);
+	char snapshot[32];
+	snprintf(snapshot, sizeof(snapshot), "snapshot.%" PRIu64, lsn);
+	/* ensure snapshot is not already exists */
+	void *o = sp_get(c, snapshot);
+	if (o) {
+		return;
+	}
+	snprintf(snapshot, sizeof(snapshot), "%" PRIu64, lsn);
+	rc = sp_set(c, "snapshot", snapshot);
+	if (rc == -1)
+		sophia_raise(env);
+}
+
+static inline void
+sophia_snapshot_recover(void *env, int64_t lsn)
+{
+	/* recovered snapshot lsn is >= then last
+	 * engine lsn */
+	char snapshot_lsn[32];
+	snprintf(snapshot_lsn, sizeof(snapshot_lsn), "%" PRIu64, lsn);
+	void *c = sp_ctl(env);
+	int rc = sp_set(c, "snapshot", snapshot_lsn);
+	if (rc == -1)
+		sophia_raise(env);
+	char snapshot[32];
+	snprintf(snapshot, sizeof(snapshot), "snapshot.%" PRIu64 ".lsn", lsn);
+	rc = sp_set(c, snapshot, snapshot_lsn);
+	if (rc == -1)
+		sophia_raise(env);
+}
+
+static inline int
+sophia_snapshot_ready(void *env, int64_t lsn)
+{
+	/* get sophia lsn associated with snapshot */
+	char snapshot[32];
+	snprintf(snapshot, sizeof(snapshot), "snapshot.%" PRIu64 ".lsn", lsn);
+	void *c = sp_ctl(env);
+	void *o = sp_get(c, snapshot);
+	if (o == NULL) {
+		if (sp_error(env))
+			sophia_raise(env);
+		panic("sophia snapshot %" PRIu64 " does not exist", lsn);
+	}
+	char *pe;
+	char *p = (char *)sp_get(o, "value", NULL);
+	int64_t snapshot_start_lsn = strtoull(p, &pe, 10);
+	sp_destroy(o);
+
+	/* compare with a latest completed checkpoint lsn */
+	o = sp_get(c, "scheduler.checkpoint_lsn_last");
+	if (o == NULL)
+		sophia_raise(env);
+	p = (char *)sp_get(o, "value", NULL);
+	int64_t last_lsn = strtoull(p, &pe, 10);
+	sp_destroy(o);
+	return last_lsn >= snapshot_start_lsn;
+}
+
+static inline void
+sophia_snapshot_delete(void *env, int64_t lsn)
+{
+	char snapshot[32];
+	snprintf(snapshot, sizeof(snapshot), "snapshot.%" PRIu64, lsn);
+	void *c = sp_ctl(env);
+	void *s = sp_get(c, snapshot);
+	if (s == NULL) {
+		if (sp_error(env))
+			sophia_raise(env);
+		panic("sophia snapshot %" PRIu64 " does not exist", lsn);
+	}
+	int rc = sp_destroy(s);
+	if (rc == -1)
+		sophia_raise(env);
+}
+
+void SophiaFactory::snapshot(enum engine_snapshot_event e, int64_t lsn)
+{
+	switch (e) {
+	case SNAPSHOT_START:
+		sophia_snapshot(env, lsn);
+		break;
+	case SNAPSHOT_RECOVER:
+		sophia_snapshot_recover(env, lsn);
+		break;
+	case SNAPSHOT_DELETE:
+		sophia_snapshot_delete(env, lsn);
+		break;
+	case SNAPSHOT_WAIT:
+		while (! sophia_snapshot_ready(env, lsn))
+			fiber_yield_timeout(.020);
+		break;
+	}
+}
diff --git a/src/box/engine_sophia.h b/src/box/engine_sophia.h
index e37ba9f74efa2ed65aeb5c2d5d8716fef7112d2e..5c1176a83eb158243e3ddf12c48085b4510a2c68 100644
--- a/src/box/engine_sophia.h
+++ b/src/box/engine_sophia.h
@@ -40,6 +40,7 @@ struct SophiaFactory: public EngineFactory {
 	virtual void commit(struct txn*);
 	virtual void rollback(struct txn*);
 	virtual void recoveryEvent(enum engine_recovery_event);
+	virtual void snapshot(enum engine_snapshot_event, int64_t);
 	void *env;
 	void *tx;
 };
diff --git a/src/box/recovery.cc b/src/box/recovery.cc
index c4c0cdd5876104c4659126dcb1043a83fbb1084f..f6e3c3fe32a36fd96d7b53e863b7a662cbf2b7eb 100644
--- a/src/box/recovery.cc
+++ b/src/box/recovery.cc
@@ -1138,6 +1138,14 @@ wal_write(struct recovery_state *r, struct xrow_header *row)
 
 /* {{{ box.snapshot() */
 
+int64_t
+recovery_snap_lsn(struct recovery_state *r)
+{
+	/* recover last snapshot lsn */
+	struct vclock *vclock = vclockset_last(&r->snap_dir.index);
+	return vclock ? vclock_signature(vclock) : -1;
+}
+
 void
 snapshot_write_row(struct recovery_state *r, struct xlog *l,
 		   struct xrow_header *row)
@@ -1216,7 +1224,8 @@ snapshot_write_row(struct recovery_state *r, struct xlog *l,
 }
 
 void
-snapshot_save(struct recovery_state *r, snapshot_f snapshot_handler)
+snapshot_save(struct recovery_state *r, snapshot_f snapshot_handler,
+	      bool rename)
 {
 	struct xlog *snap = xlog_create(&r->snap_dir, &r->vclock);
 	if (snap == NULL)
@@ -1230,10 +1239,23 @@ snapshot_save(struct recovery_state *r, snapshot_f snapshot_handler)
 
 	snapshot_handler(snap);
 
+	snap->is_inprogress = rename;
 	xlog_close(snap);
 
 	say_info("done");
 }
 
+int snapshot_rename(struct xdir *dir, int64_t lsn)
+{
+	char dest[PATH_MAX + 1];
+	snprintf(dest, sizeof(dest), "%s",
+	         format_filename(dir, lsn, NONE));
+	const char *from = format_filename(dir, lsn, INPROGRESS);
+	int rc = rename(from, dest);
+	if (rc == -1)
+		say_syserror("can't rename %s to %s", from, dest);
+	return rc;
+}
+
 /* }}} */
 
diff --git a/src/box/recovery.h b/src/box/recovery.h
index 9b3187195fab3e11d83aa824e199012f5f7a0b3c..16e9f7578250c1ce7fb74c1ba13bc2a7e95715f3 100644
--- a/src/box/recovery.h
+++ b/src/box/recovery.h
@@ -172,6 +172,13 @@ void recovery_apply_row(struct recovery_state *r, struct xrow_header *packet);
 
 struct fio_batch;
 
+/**
+ * Return LSN of the most recent snapshot or -1 if there is
+ * no snapshot.
+ */
+int64_t
+recovery_snap_lsn(struct recovery_state *r);
+
 void
 snapshot_write_row(struct recovery_state *r, struct xlog *l,
 		   struct xrow_header *packet);
@@ -179,7 +186,11 @@ snapshot_write_row(struct recovery_state *r, struct xlog *l,
 typedef void (snapshot_f)(struct xlog *);
 
 void
-snapshot_save(struct recovery_state *r, snapshot_f snapshot_handler);
+snapshot_save(struct recovery_state *r, snapshot_f snapshot_handler,
+	      bool rename);
+
+int
+snapshot_rename(struct xdir *dir, int64_t lsn);
 
 #if defined(__cplusplus)
 } /* extern "C" */
diff --git a/src/box/schema.cc b/src/box/schema.cc
index 6ce440d4290dc1162a92448a0ed9dc300dd0c623..be69a719736149386c75e3c2f2c2225e17da02d3 100644
--- a/src/box/schema.cc
+++ b/src/box/schema.cc
@@ -163,19 +163,6 @@ space_cache_replace(struct space *space)
 	return p_old ? (struct space *) p_old->val : NULL;
 }
 
-static void
-do_one_recover_step(struct space *space, void * /* param */)
-{
-	if (space_index(space, 0)) {
-		space->engine->recover(space);
-	} else {
-		/* in case of space has no primary index,
-		 * derive it's engine handler recovery state from
-		 * the global one. */
-		space->engine->initRecovery();
-	}
-}
-
 /** A wrapper around space_new() for data dictionary spaces. */
 struct space *
 sc_space_new(struct space_def *space_def,
@@ -307,44 +294,6 @@ schema_init()
 	key_def_delete(key_def);
 }
 
-static inline void
-space_end_recover_snapshot_cb(EngineFactory *f, void *udate)
-{
-	(void)udate;
-	f->recoveryEvent(END_RECOVERY_SNAPSHOT);
-}
-
-void
-space_end_recover_snapshot()
-{
-	/*
-	 * For all new spaces created from now on, when the
-	 * PRIMARY key is added, enable it right away.
-	 */
-	engine_foreach(space_end_recover_snapshot_cb, NULL);
-
-	space_foreach(do_one_recover_step, NULL);
-}
-
-static inline void
-space_end_recover_cb(EngineFactory *f, void *udate)
-{
-	(void)udate;
-	f->recoveryEvent(END_RECOVERY);
-}
-
-void
-space_end_recover()
-{
-	/*
-	 * For all new spaces created after recovery is complete,
-	 * when the primary key is added, enable all keys.
-	 */
-	engine_foreach(space_end_recover_cb, NULL);
-
-	space_foreach(do_one_recover_step, NULL);
-}
-
 void
 schema_free(void)
 {
diff --git a/src/box/schema.h b/src/box/schema.h
index 201293ec78339c88905a0d220ec32e37669d31b3..a63fa229dd3df72f67c90502c2283936f1544cda 100644
--- a/src/box/schema.h
+++ b/src/box/schema.h
@@ -105,20 +105,6 @@ schema_init();
 void
 schema_free();
 
-/**
- * Called at the end of recovery from snapshot.
- * Build primary keys in all spaces.
- * */
-void
-space_end_recover_snapshot();
-
-/**
- * Called at the end of recovery.
- * Build secondary keys in all spaces.
- */
-void
-space_end_recover();
-
 struct space *schema_space(uint32_t id);
 
 /*
diff --git a/src/box/sophia_index.cc b/src/box/sophia_index.cc
index 8191090c1fac3af21a31a502fb5ac1c2747d5903..135bebc0320077d6845053f71760d101fef1c7ed 100644
--- a/src/box/sophia_index.cc
+++ b/src/box/sophia_index.cc
@@ -131,15 +131,18 @@ sophia_configure(struct space *space, struct key_def *key_def)
 		(SophiaFactory*)space->engine->factory;
 	void *env = factory->env;
 	void *c = sp_ctl(env);
+	char pointer[128];
 	char name[128];
 	snprintf(name, sizeof(name), "%" PRIu32, key_def->space_id);
 	sp_set(c, "db", name);
 	snprintf(name, sizeof(name), "db.%" PRIu32 ".index.cmp",
 	         key_def->space_id);
-	sp_set(c, name, sophia_index_compare);
+	snprintf(pointer, sizeof(pointer), "pointer: %p", (void*)sophia_index_compare);
+	sp_set(c, name, pointer);
 	snprintf(name, sizeof(name), "db.%" PRIu32 ".index.cmp_arg",
 	         key_def->space_id);
-	sp_set(c, name, key_def);
+	snprintf(pointer, sizeof(pointer), "pointer: %p", (void*)key_def);
+	sp_set(c, name, pointer);
 	snprintf(name, sizeof(name), "db.%" PRIu32, key_def->space_id);
 	void *db = sp_get(c, name);
 	if (db == NULL)
@@ -164,14 +167,6 @@ SophiaIndex::SophiaIndex(struct key_def *key_def_arg __attribute__((unused)))
 	int rc = sp_open(db);
 	if (rc == -1)
 		sophia_raise(env);
-	/* auto-complete any space created
-	 * after recovery */
-	engine_recovery *r = &factory->recovery;
-	if (r->recover == space_noop) {
-		rc = sp_open(db);
-		if (rc == -1)
-			sophia_raise(env);
-	}
 	tuple_format_ref(space->format, 1);
 }
 
@@ -211,7 +206,8 @@ SophiaIndex::random(uint32_t rnd) const
 	if (o == NULL)
 		sophia_raise(env);
 	sp_set(o, "key", &rnd, sizeof(rnd));
-	void *c = sp_cursor(db, "random", o);
+	sp_set(o, "order", "random");
+	void *c = sp_cursor(db, o);
 	if (c == NULL)
 		sophia_raise(env);
 	auto scoped_guard =
@@ -313,6 +309,8 @@ SophiaIndex::replace(struct tuple *old_tuple, struct tuple *new_tuple,
 	SophiaFactory *factory =
 		(SophiaFactory *)space->engine->factory;
 	assert(factory->tx != NULL);
+
+	/* insert, replace or update */
 	if (new_tuple) {
 		const char *key = tuple_field(new_tuple, key_def->parts[0].fieldno);
 		const char *keyptr = key;
@@ -337,7 +335,9 @@ SophiaIndex::replace(struct tuple *old_tuple, struct tuple *new_tuple,
 		}
 		if (dup_tuple)
 			return dup_tuple;
+		return old_tuple;
 	}
+	/* delete */
 	if (old_tuple)
 		sophia_index_stmt(factory->tx, db, 1, key_def, old_tuple);
 	return old_tuple;
@@ -477,9 +477,10 @@ SophiaIndex::initIterator(struct iterator *ptr,
 	void *o = sp_object(db);
 	if (o == NULL)
 		sophia_raise(env);
+	sp_set(o, "order", compare);
 	if (key)
 		sp_set(o, "key", key, keysize);
-	it->cursor = sp_cursor(db, compare, o);
+	it->cursor = sp_cursor(db, o);
 	if (it->cursor == NULL)
 		sophia_raise(env);
 }
diff --git a/src/box/xlog.cc b/src/box/xlog.cc
index b8e4960426c0c5cae4ee8b8011fb2d2efd1ce8b2..4879d62d9be1ce6370d8fc2cf78d150f8f8597d0 100644
--- a/src/box/xlog.cc
+++ b/src/box/xlog.cc
@@ -616,7 +616,8 @@ xlog_rename(struct xlog *l)
 	new_filename[suffix - filename] = '\0';
 
 	if (rename(filename, new_filename) != 0) {
-		say_syserror("%s: rename to %s failed", filename, new_filename);
+		say_syserror("%s: rename to %s failed", filename,
+			     new_filename);
 
 		return -1;
 	}
diff --git a/test/sophia/box.lua b/test/sophia/box.lua
index e11166512e1535d3f11c58294912487d18174492..de85f237bdc1b686dbebc761e6de0b4d8e585c05 100644
--- a/test/sophia/box.lua
+++ b/test/sophia/box.lua
@@ -12,7 +12,19 @@ function sophia_rmdir(dir)
 	os.execute("rm -rf sophia_test")
 end
 
-sophia_rmdir()
+function file_exists(name)
+	local f = io.open(name,"r")
+	if f ~= nil then
+		io.close(f)
+		return true
+	else
+		return false
+	end
+end
+
+if not file_exists("lock") then
+	sophia_rmdir()
+end
 
 local sophia = {
 	memory_limit = 0,
diff --git a/test/sophia/crud.result b/test/sophia/crud.result
index 548a312a5ee65233f69274ddbd5d1b9644038d79..f96bcff12fb39f99356a4c5edba7b3a8518d6481 100644
--- a/test/sophia/crud.result
+++ b/test/sophia/crud.result
@@ -446,6 +446,10 @@ for key = 1, 132 do space:delete({key}) end
 for key = 1, 132 do assert(space:get({key}) == nil) end
 ---
 ...
+-- delete nonexistent
+space:delete({1234})
+---
+...
 -- select
 for key = 1, 96 do space:insert({key}) end
 ---
diff --git a/test/sophia/crud.test.lua b/test/sophia/crud.test.lua
index fea140c1aecf7857ff59579b42c6858066a1c4ed..1bc09b18c2b72d8b4f223ca53eb4388fabe20d16 100644
--- a/test/sophia/crud.test.lua
+++ b/test/sophia/crud.test.lua
@@ -27,6 +27,9 @@ t
 for key = 1, 132 do space:delete({key}) end
 for key = 1, 132 do assert(space:get({key}) == nil) end
 
+-- delete nonexistent
+space:delete({1234})
+
 -- select
 
 for key = 1, 96 do space:insert({key}) end
diff --git a/test/sophia/dml.result b/test/sophia/dml.result
index 923f9560857fd857b36c678eb0dbea438a03305d..10ce0b15df4f81cad172509f06bbb3d889346707 100644
--- a/test/sophia/dml.result
+++ b/test/sophia/dml.result
@@ -4,18 +4,14 @@ space = box.schema.create_space('test', { id = 100, engine = 'sophia' })
 ...
 sophia_printdir()
 ---
-- 'snapshot
-
-'
+- 
 ...
 space:drop()
 ---
 ...
 sophia_printdir()
 ---
-- 'snapshot
-
-'
+- 
 ...
 -- index create/drop
 space = box.schema.create_space('test', { id = 101, engine = 'sophia' })
@@ -28,8 +24,6 @@ sophia_printdir()
 ---
 - '101
 
-  snapshot
-
 '
 ...
 space:drop()
@@ -37,9 +31,7 @@ space:drop()
 ...
 sophia_printdir()
 ---
-- 'snapshot
-
-'
+- 
 ...
 -- index create/drop alter
 space = box.schema.create_space('test', { id = 102, engine = 'sophia' })
@@ -52,8 +44,6 @@ sophia_printdir()
 ---
 - '102
 
-  snapshot
-
 '
 ...
 _index = box.space[box.schema.INDEX_ID]
@@ -65,9 +55,7 @@ _index:delete{102, 0}
 ...
 sophia_printdir()
 ---
-- 'snapshot
-
-'
+- 
 ...
 space:drop()
 ---
@@ -87,8 +75,6 @@ sophia_printdir()
 ---
 - '103
 
-  snapshot
-
 '
 ...
 space:drop()
@@ -109,8 +95,6 @@ sophia_printdir()
 ---
 - '104
 
-  snapshot
-
 '
 ...
 space:drop()
@@ -144,9 +128,7 @@ space:drop()
 ...
 sophia_printdir()
 ---
-- 'snapshot
-
-'
+- 
 ...
 -- index size
 space = box.schema.create_space('test', { id = 107, engine = 'sophia' })
diff --git a/test/sophia/gh.result b/test/sophia/gh.result
index 45d9befc9f2e0487f9cdc1d24aeb611a652cf1dd..9a60d57f5fba8e5f7bf951f9dd90d443df857cc2 100644
--- a/test/sophia/gh.result
+++ b/test/sophia/gh.result
@@ -147,3 +147,21 @@ box.space['name_of_space']:select{'a'}
 s:drop()
 ---
 ...
+-- gh-680: Sophia: assertion on update
+s = box.schema.space.create('tester', {engine='sophia'})
+---
+...
+i = s:create_index('primary',{type = 'tree', parts = {2, 'STR'}})
+---
+...
+s:insert{1,'X'}
+---
+- [1, 'X']
+...
+s:update({'X'}, {{'=', 2, 'Y'}})
+---
+- [1, 'Y']
+...
+s:drop()
+---
+...
diff --git a/test/sophia/gh.test.lua b/test/sophia/gh.test.lua
index 01763075f1905afb6ec9a0602580f7cf667b932d..35fe9512e680f8cbaa998086bf78d403d6e78369 100644
--- a/test/sophia/gh.test.lua
+++ b/test/sophia/gh.test.lua
@@ -59,3 +59,10 @@ box.space['name_of_space']:select{'a'}
 box.space['name_of_space']:truncate()
 box.space['name_of_space']:select{'a'}
 s:drop()
+
+-- gh-680: Sophia: assertion on update
+s = box.schema.space.create('tester', {engine='sophia'})
+i = s:create_index('primary',{type = 'tree', parts = {2, 'STR'}})
+s:insert{1,'X'}
+s:update({'X'}, {{'=', 2, 'Y'}})
+s:drop()
diff --git a/test/sophia/recover.result b/test/sophia/recover.result
deleted file mode 100644
index d4e97c29e34df0d393483092e26576e3cb97a572..0000000000000000000000000000000000000000
--- a/test/sophia/recover.result
+++ /dev/null
@@ -1,26 +0,0 @@
--- snapshot
-space = box.schema.create_space('test', { id = 100, engine = 'sophia' })
----
-...
-index = space:create_index('primary')
----
-...
-sophia_printdir()
----
-- '100
-
-  snapshot
-
-'
-...
-box.snapshot()
----
-- ok
-...
-space:drop()
----
-...
-box.snapshot()
----
-- ok
-...
diff --git a/test/sophia/recover.test.lua b/test/sophia/recover.test.lua
deleted file mode 100644
index c8814c2f45798c411e392bf75e54bd64149a11fa..0000000000000000000000000000000000000000
--- a/test/sophia/recover.test.lua
+++ /dev/null
@@ -1,9 +0,0 @@
-
--- snapshot
-
-space = box.schema.create_space('test', { id = 100, engine = 'sophia' })
-index = space:create_index('primary')
-sophia_printdir()
-box.snapshot()
-space:drop()
-box.snapshot()
diff --git a/test/sophia/snapshot.result b/test/sophia/snapshot.result
new file mode 100644
index 0000000000000000000000000000000000000000..6452499061ef5f40faa85eb16aa78e053f1e1c2f
--- /dev/null
+++ b/test/sophia/snapshot.result
@@ -0,0 +1,396 @@
+-- snapshot
+space = box.schema.create_space('test', { id = 100, engine = 'sophia' })
+---
+...
+index = space:create_index('primary')
+---
+...
+sophia_printdir()
+---
+- '100
+
+'
+...
+for key = 1, 351 do space:insert({key}) end
+---
+...
+box.snapshot()
+---
+- ok
+...
+os.execute("touch lock")
+---
+- 0
+...
+--# stop server default
+--# start server default
+space = box.space['test']
+---
+...
+t = {}
+---
+...
+for key = 1, 351 do table.insert(t, space:get({key})) end
+---
+...
+t
+---
+- - [1]
+  - [2]
+  - [3]
+  - [4]
+  - [5]
+  - [6]
+  - [7]
+  - [8]
+  - [9]
+  - [10]
+  - [11]
+  - [12]
+  - [13]
+  - [14]
+  - [15]
+  - [16]
+  - [17]
+  - [18]
+  - [19]
+  - [20]
+  - [21]
+  - [22]
+  - [23]
+  - [24]
+  - [25]
+  - [26]
+  - [27]
+  - [28]
+  - [29]
+  - [30]
+  - [31]
+  - [32]
+  - [33]
+  - [34]
+  - [35]
+  - [36]
+  - [37]
+  - [38]
+  - [39]
+  - [40]
+  - [41]
+  - [42]
+  - [43]
+  - [44]
+  - [45]
+  - [46]
+  - [47]
+  - [48]
+  - [49]
+  - [50]
+  - [51]
+  - [52]
+  - [53]
+  - [54]
+  - [55]
+  - [56]
+  - [57]
+  - [58]
+  - [59]
+  - [60]
+  - [61]
+  - [62]
+  - [63]
+  - [64]
+  - [65]
+  - [66]
+  - [67]
+  - [68]
+  - [69]
+  - [70]
+  - [71]
+  - [72]
+  - [73]
+  - [74]
+  - [75]
+  - [76]
+  - [77]
+  - [78]
+  - [79]
+  - [80]
+  - [81]
+  - [82]
+  - [83]
+  - [84]
+  - [85]
+  - [86]
+  - [87]
+  - [88]
+  - [89]
+  - [90]
+  - [91]
+  - [92]
+  - [93]
+  - [94]
+  - [95]
+  - [96]
+  - [97]
+  - [98]
+  - [99]
+  - [100]
+  - [101]
+  - [102]
+  - [103]
+  - [104]
+  - [105]
+  - [106]
+  - [107]
+  - [108]
+  - [109]
+  - [110]
+  - [111]
+  - [112]
+  - [113]
+  - [114]
+  - [115]
+  - [116]
+  - [117]
+  - [118]
+  - [119]
+  - [120]
+  - [121]
+  - [122]
+  - [123]
+  - [124]
+  - [125]
+  - [126]
+  - [127]
+  - [128]
+  - [129]
+  - [130]
+  - [131]
+  - [132]
+  - [133]
+  - [134]
+  - [135]
+  - [136]
+  - [137]
+  - [138]
+  - [139]
+  - [140]
+  - [141]
+  - [142]
+  - [143]
+  - [144]
+  - [145]
+  - [146]
+  - [147]
+  - [148]
+  - [149]
+  - [150]
+  - [151]
+  - [152]
+  - [153]
+  - [154]
+  - [155]
+  - [156]
+  - [157]
+  - [158]
+  - [159]
+  - [160]
+  - [161]
+  - [162]
+  - [163]
+  - [164]
+  - [165]
+  - [166]
+  - [167]
+  - [168]
+  - [169]
+  - [170]
+  - [171]
+  - [172]
+  - [173]
+  - [174]
+  - [175]
+  - [176]
+  - [177]
+  - [178]
+  - [179]
+  - [180]
+  - [181]
+  - [182]
+  - [183]
+  - [184]
+  - [185]
+  - [186]
+  - [187]
+  - [188]
+  - [189]
+  - [190]
+  - [191]
+  - [192]
+  - [193]
+  - [194]
+  - [195]
+  - [196]
+  - [197]
+  - [198]
+  - [199]
+  - [200]
+  - [201]
+  - [202]
+  - [203]
+  - [204]
+  - [205]
+  - [206]
+  - [207]
+  - [208]
+  - [209]
+  - [210]
+  - [211]
+  - [212]
+  - [213]
+  - [214]
+  - [215]
+  - [216]
+  - [217]
+  - [218]
+  - [219]
+  - [220]
+  - [221]
+  - [222]
+  - [223]
+  - [224]
+  - [225]
+  - [226]
+  - [227]
+  - [228]
+  - [229]
+  - [230]
+  - [231]
+  - [232]
+  - [233]
+  - [234]
+  - [235]
+  - [236]
+  - [237]
+  - [238]
+  - [239]
+  - [240]
+  - [241]
+  - [242]
+  - [243]
+  - [244]
+  - [245]
+  - [246]
+  - [247]
+  - [248]
+  - [249]
+  - [250]
+  - [251]
+  - [252]
+  - [253]
+  - [254]
+  - [255]
+  - [256]
+  - [257]
+  - [258]
+  - [259]
+  - [260]
+  - [261]
+  - [262]
+  - [263]
+  - [264]
+  - [265]
+  - [266]
+  - [267]
+  - [268]
+  - [269]
+  - [270]
+  - [271]
+  - [272]
+  - [273]
+  - [274]
+  - [275]
+  - [276]
+  - [277]
+  - [278]
+  - [279]
+  - [280]
+  - [281]
+  - [282]
+  - [283]
+  - [284]
+  - [285]
+  - [286]
+  - [287]
+  - [288]
+  - [289]
+  - [290]
+  - [291]
+  - [292]
+  - [293]
+  - [294]
+  - [295]
+  - [296]
+  - [297]
+  - [298]
+  - [299]
+  - [300]
+  - [301]
+  - [302]
+  - [303]
+  - [304]
+  - [305]
+  - [306]
+  - [307]
+  - [308]
+  - [309]
+  - [310]
+  - [311]
+  - [312]
+  - [313]
+  - [314]
+  - [315]
+  - [316]
+  - [317]
+  - [318]
+  - [319]
+  - [320]
+  - [321]
+  - [322]
+  - [323]
+  - [324]
+  - [325]
+  - [326]
+  - [327]
+  - [328]
+  - [329]
+  - [330]
+  - [331]
+  - [332]
+  - [333]
+  - [334]
+  - [335]
+  - [336]
+  - [337]
+  - [338]
+  - [339]
+  - [340]
+  - [341]
+  - [342]
+  - [343]
+  - [344]
+  - [345]
+  - [346]
+  - [347]
+  - [348]
+  - [349]
+  - [350]
+  - [351]
+...
+space:drop()
+---
+...
+os.execute("rm -f lock")
+---
+- 0
+...
diff --git a/test/sophia/snapshot.test.lua b/test/sophia/snapshot.test.lua
new file mode 100644
index 0000000000000000000000000000000000000000..81d47b4d030ab9807cc52b3d5addd95f8e16e8ee
--- /dev/null
+++ b/test/sophia/snapshot.test.lua
@@ -0,0 +1,22 @@
+
+-- snapshot
+
+space = box.schema.create_space('test', { id = 100, engine = 'sophia' })
+index = space:create_index('primary')
+sophia_printdir()
+
+for key = 1, 351 do space:insert({key}) end
+box.snapshot()
+
+os.execute("touch lock")
+
+--# stop server default
+--# start server default
+
+space = box.space['test']
+t = {}
+for key = 1, 351 do table.insert(t, space:get({key})) end
+t
+space:drop()
+
+os.execute("rm -f lock")
diff --git a/third_party/sophia b/third_party/sophia
index f3fe222f99d5c6734e6171a2df635aa3d041146c..13d0195e8c2694e01219d87592a5dd8e6a2e3b41 160000
--- a/third_party/sophia
+++ b/third_party/sophia
@@ -1 +1 @@
-Subproject commit f3fe222f99d5c6734e6171a2df635aa3d041146c
+Subproject commit 13d0195e8c2694e01219d87592a5dd8e6a2e3b41