diff --git a/src/box/alter.cc b/src/box/alter.cc
index 0e2480de8bc772977734885291ccc3d750a68b16..b158efa04a51dbd48f49e6b78e5adc3c9e766a86 100644
--- a/src/box/alter.cc
+++ b/src/box/alter.cc
@@ -66,6 +66,10 @@
 #define INDEX_165_PART_COUNT 5
 #define INDEX_165_PARTS 6
 
+/** _truncate columnts */
+#define TRUNCATE_ID      0
+#define TRUNCATE_COUNT   1
+
 /** _user columns */
 #define USER_TYPE        3
 #define AUTH_MECH_LIST   4
@@ -412,127 +416,6 @@ index_def_new_from_tuple(struct tuple *tuple, struct space *old_space)
 	return index_def;
 }
 
-static char *
-opt_encode(char *data, char *data_end, const void *opts,
-	   const void *default_opts, const struct opt_def *def)
-{
-	int64_t ival;
-	double dval;
-	const char *opt = ((const char *) opts) + def->offset;
-	const char *default_opt = ((const char *) default_opts) + def->offset;
-	if (memcmp(opt, default_opt, def->len) == 0)
-		return data;
-	uint32_t optlen = strlen(def->name);
-	if (data + mp_sizeof_str(optlen) > data_end)
-		return data_end;
-	data = mp_encode_str(data, def->name, optlen);
-	switch (def->type) {
-	case OPT_BOOL:
-		if (data + mp_sizeof_bool(true) > data_end)
-			return data_end;
-		data = mp_encode_bool(data, load_bool(opt));
-		break;
-	case OPT_INT:
-		ival = load_u64(opt);
-		if ((ival < 0 && data + mp_sizeof_int(ival) > data_end) ||
-		    (ival >= 0 && data + mp_sizeof_uint(ival) > data_end))
-			return data_end;
-		if (ival < 0)
-			data = mp_encode_int(data, ival);
-		else
-			data = mp_encode_uint(data, ival);
-		break;
-	case OPT_FLOAT:
-		dval = load_double(opt);
-		if (data + mp_sizeof_double(dval) > data_end)
-			return data_end;
-		data = mp_encode_double(data, dval);
-		break;
-	case OPT_STR:
-		optlen = strlen(opt);
-		if (data + mp_sizeof_str(optlen) > data_end)
-			return data_end;
-		data = mp_encode_str(data, opt, optlen);
-		break;
-	default:
-		unreachable();
-	}
-	return data;
-}
-
-static char *
-opts_encode(char *data, char *data_end, const void *opts,
-	    const void *default_opts, const struct opt_def *reg)
-{
-	char *p = data;
-	uint32_t n_opts = 0;
-	for (const struct opt_def *def = reg; def->name != NULL; def++) {
-		char *end = opt_encode(p, data_end, opts, default_opts, def);
-		if (end == data_end)
-			return end;
-		if (p != end) {
-			p = end;
-			n_opts++;
-		}
-	}
-	ptrdiff_t len = p - data;
-	if (data + mp_sizeof_map(n_opts) > data_end)
-		return data_end;
-	memmove(data + mp_sizeof_map(n_opts), data, len);
-	data = mp_encode_map(data, n_opts);
-	data += len;
-	return data;
-}
-
-
-/**
- * Encode options index_opts into msgpack stream.
- * @pre output buffer is reserved to contain enough space for the
- * output.
- *
- * @return a pointer to the end of the stream.
- */
-static char *
-index_opts_encode(char *data, char *data_end, const struct index_opts *opts)
-{
-	return opts_encode(data, data_end, opts, &index_opts_default,
-			   index_opts_reg);
-}
-
-struct tuple *
-index_def_tuple_update_lsn(struct tuple *tuple, int64_t lsn)
-{
-	bool is_166plus;
-	index_def_check_tuple(tuple, &is_166plus);
-	if (!is_166plus)
-		return tuple;
-	struct index_opts opts;
-	const char *opts_field = tuple_field(tuple, INDEX_OPTS);
-	const char *opts_field_end = index_opts_create(&opts, opts_field);
-	opts.lsn = lsn;
-	size_t size = (opts_field_end - opts_field) + 64;
-	char *buf = (char *)malloc(size);
-	if (buf == NULL)
-		tnt_raise(OutOfMemory, size, "malloc", "buf");
-	char *buf_end = buf;
-	buf_end = mp_encode_array(buf_end, 2);
-	buf_end = mp_encode_array(buf_end, 3);
-	buf_end = mp_encode_str(buf_end, "#", 1);
-	buf_end = mp_encode_uint(buf_end, INDEX_OPTS + 1);
-	buf_end = mp_encode_uint(buf_end, 1);
-	buf_end = mp_encode_array(buf_end, 3);
-	buf_end = mp_encode_str(buf_end, "!", 1);
-	buf_end = mp_encode_uint(buf_end, INDEX_OPTS + 1);
-	buf_end = index_opts_encode(buf_end, buf + size, &opts);
-	/* No check of return value: buf is checked by box_tuple_update */
-	assert(buf_end < buf + size);
-	tuple = box_tuple_update(tuple, buf, buf_end);
-	free(buf);
-	if (tuple == NULL)
-		diag_raise();
-	return tuple;
-}
-
 /**
  * Fill space opts from the msgpack stream (MP_MAP field in the
  * tuple).
@@ -1444,6 +1327,120 @@ on_replace_dd_index(struct trigger * /* trigger */, void *event)
 	scoped_guard.is_active = false;
 }
 
+/* {{{ space truncate */
+
+struct truncate_space {
+	/** Space being truncated. */
+	struct space *old_space;
+	/** Space created as a result of truncation. */
+	struct space *new_space;
+	/** Trigger executed to commit truncation. */
+	struct trigger on_commit;
+	/** Trigger executed to rollback truncation. */
+	struct trigger on_rollback;
+};
+
+/**
+ * Call the engine specific method to commit truncation
+ * and delete the old space.
+ */
+static void
+truncate_space_commit(struct trigger *trigger, void * /* event */)
+{
+	struct truncate_space *truncate =
+		(struct truncate_space *) trigger->data;
+	truncate->new_space->handler->commitTruncateSpace(truncate->old_space,
+							  truncate->new_space);
+	space_delete(truncate->old_space);
+}
+
+/**
+ * Move the old space back to the cache and delete
+ * the new space.
+ */
+static void
+truncate_space_rollback(struct trigger *trigger, void * /* event */)
+{
+	struct truncate_space *truncate =
+		(struct truncate_space *) trigger->data;
+	if (space_cache_replace(truncate->old_space) != truncate->new_space)
+		unreachable();
+	space_delete(truncate->new_space);
+}
+
+/**
+ * A trigger invoked on replace in space _truncate.
+ *
+ * In a nutshell, we truncate a space by replacing it with
+ * a new empty space with the same definition and indexes.
+ * Note, although we instantiate the new space before WAL
+ * write, we don't propagate changes to the old space in
+ * case a WAL write error happens and we have to rollback.
+ * This is OK, because a WAL write error implies cascading
+ * rollback of all transactions following this one.
+ */
+static void
+on_replace_dd_truncate(struct trigger * /* trigger */, void *event)
+{
+	struct txn *txn = (struct txn *) event;
+	struct txn_stmt *stmt = txn_current_stmt(txn);
+	struct tuple *old_tuple = stmt->old_tuple;
+	struct tuple *new_tuple = stmt->new_tuple;
+
+	if (old_tuple == NULL || new_tuple == NULL) {
+		/* Space create or drop. */
+		return;
+	}
+
+	/*
+	 * Truncate counter is updated - truncate the space.
+	 */
+	uint32_t space_id = tuple_field_u32_xc(new_tuple, TRUNCATE_ID);
+	uint64_t truncate_count = tuple_field_u64_xc(new_tuple, TRUNCATE_COUNT);
+	struct space *old_space = space_cache_find(space_id);
+
+	struct truncate_space *truncate =
+		region_calloc_object_xc(&fiber()->gc, struct truncate_space);
+
+	/* Create an empty copy of the old space. */
+	struct rlist key_list;
+	space_dump_def(old_space, &key_list);
+	struct space *new_space = space_new(&old_space->def, &key_list);
+	new_space->truncate_count = truncate_count;
+	auto guard = make_scoped_guard([=] { space_delete(new_space); });
+
+	/* Notify the engine about upcoming space truncation. */
+	new_space->handler->prepareTruncateSpace(old_space, new_space);
+
+	guard.is_active = false;
+
+	/*
+	 * Replace the old space with the new one in the space
+	 * cache. Requests processed after this point will see
+	 * the space as truncated.
+	 */
+	if (space_cache_replace(new_space) != old_space)
+		unreachable();
+
+	/*
+	 * Register the trigger that will commit or rollback
+	 * truncation depending on whether WAL write succeeds
+	 * or fails.
+	 */
+	truncate->old_space = old_space;
+	truncate->new_space = new_space;
+
+	trigger_create(&truncate->on_commit,
+		       truncate_space_commit, truncate, NULL);
+	txn_on_commit(txn, &truncate->on_commit);
+
+	trigger_create(&truncate->on_rollback,
+		       truncate_space_rollback, truncate, NULL);
+	txn_on_rollback(txn, &truncate->on_rollback);
+}
+
+/* }}} */
+
 /* {{{ access control */
 
 /** True if the space has records identified by key 'uid'
@@ -2126,6 +2123,10 @@ struct trigger alter_space_on_replace_index = {
 	RLIST_LINK_INITIALIZER, on_replace_dd_index, NULL, NULL
 };
 
+struct trigger on_replace_truncate = {
+	RLIST_LINK_INITIALIZER, on_replace_dd_truncate, NULL, NULL
+};
+
 struct trigger on_replace_schema = {
 	RLIST_LINK_INITIALIZER, on_replace_dd_schema, NULL, NULL
 };
diff --git a/src/box/alter.h b/src/box/alter.h
index df204f476f7f53e3aedb7723de3493704d5eb2d1..87bab00b052f43f71ec51e01ec997fd980bc3925 100644
--- a/src/box/alter.h
+++ b/src/box/alter.h
@@ -34,6 +34,7 @@
 
 extern struct trigger alter_space_on_replace_space;
 extern struct trigger alter_space_on_replace_index;
+extern struct trigger on_replace_truncate;
 extern struct trigger on_replace_schema;
 extern struct trigger on_replace_user;
 extern struct trigger on_replace_func;
diff --git a/src/box/bootstrap.snap b/src/box/bootstrap.snap
index 80a99d3f64da45b7834551a40c661358ad848eb7..35cef226606cf2527fd41ac34677a0cb3e4adffd 100644
Binary files a/src/box/bootstrap.snap and b/src/box/bootstrap.snap differ
diff --git a/src/box/box.cc b/src/box/box.cc
index a8f1dbea5dec7af4d48a065a934ee683ad8f8f0c..7558dd916fff623f4cdbfabf447117ce7090446e 100644
--- a/src/box/box.cc
+++ b/src/box/box.cc
@@ -824,71 +824,31 @@ box_upsert(uint32_t space_id, uint32_t index_id, const char *tuple,
 	return box_process1(request, result);
 }
 
+/**
+ * Trigger space truncation by bumping a counter
+ * in _truncate space.
+ */
 static void
 space_truncate(struct space *space)
 {
-	if (!space_index(space, 0)) {
-		/* empty space without indexes, nothing to truncate */
-		return;
-	}
-
-	char key_buf[20];
-	char *key_buf_end;
-	key_buf_end = mp_encode_uint(key_buf, space_id(space));
-	assert(key_buf_end <= key_buf + sizeof(key_buf));
-
-	/* BOX_INDEX_ID is id of _index space, we need 0 index of that space */
-	struct space *space_index = space_cache_find(BOX_INDEX_ID);
-	Index *index = index_find_xc(space_index, 0);
-	struct iterator *it = index->allocIterator();
-	auto guard_it_free = make_scoped_guard([=]{
-		it->free(it);
-	});
-	index->initIterator(it, ITER_EQ, key_buf, 1);
-	int index_count = 0;
-	struct tuple *indexes[BOX_INDEX_MAX] = { NULL };
-	struct tuple *tuple;
-
-	/* select all indexes of given space */
-	auto guard_indexes_unref = make_scoped_guard([=]{
-		for (int i = 0; i < index_count; i++)
-			tuple_unref(indexes[i]);
-	});
-	while ((tuple = it->next(it)) != NULL) {
-		tuple_ref_xc(tuple);
-		indexes[index_count++] = tuple;
-	}
-	assert(index_count <= BOX_INDEX_MAX);
-
-	/* box_delete() invalidates space pointer */
-	uint32_t x_space_id = space_id(space);
-	space = NULL;
-
-	/* drop all selected indexes */
-	for (int i = index_count - 1; i >= 0; --i) {
-		uint32_t index_id = tuple_field_u32_xc(indexes[i], 1);
-		key_buf_end = mp_encode_array(key_buf, 2);
-		key_buf_end = mp_encode_uint(key_buf_end, x_space_id);
-		key_buf_end = mp_encode_uint(key_buf_end, index_id);
-		assert(key_buf_end <= key_buf + sizeof(key_buf));
-		if (box_delete(BOX_INDEX_ID, 0, key_buf, key_buf_end, NULL))
-			diag_raise();
-	}
-
-	/* create all indexes again, now they are empty */
-	for (int i = 0; i < index_count; i++) {
-		int64_t lsn = vclock_sum(&replicaset_vclock);
-		/*
-		 * The returned tuple is blessed and will be
-		 * collected automatically.
-		 */
-		tuple = index_def_tuple_update_lsn(indexes[i], lsn);
-		TupleRefNil ref(tuple);
-		uint32_t bsize;
-		const char *data = tuple_data_range(tuple, &bsize);
-		if (box_insert(BOX_INDEX_ID, data, data + bsize, NULL))
-			diag_raise();
-	}
+	char key_buf[16];
+	char *key_buf_end = key_buf;
+	key_buf_end = mp_encode_array(key_buf_end, 1);
+	key_buf_end = mp_encode_uint(key_buf_end, space_id(space));
+	assert(key_buf_end < key_buf + sizeof(key_buf));
+
+	char ops_buf[128];
+	char *ops_buf_end = ops_buf;
+	ops_buf_end = mp_encode_array(ops_buf_end, 1);
+	ops_buf_end = mp_encode_array(ops_buf_end, 3);
+	ops_buf_end = mp_encode_str(ops_buf_end, "+", 1);
+	ops_buf_end = mp_encode_uint(ops_buf_end, 1);
+	ops_buf_end = mp_encode_uint(ops_buf_end, 1);
+	assert(ops_buf_end < ops_buf + sizeof(ops_buf));
+
+	if (box_update(BOX_TRUNCATE_ID, 0, key_buf, key_buf_end,
+		       ops_buf, ops_buf_end, 0, NULL) != 0)
+		diag_raise();
 }
 
 int
diff --git a/src/box/engine.cc b/src/box/engine.cc
index cde26c99e77db877e0e2391d8b8a0c02f3dba401..0705a9b72cb4fa212b715625fdbcc8da79db10df 100644
--- a/src/box/engine.cc
+++ b/src/box/engine.cc
@@ -177,7 +177,6 @@ Handler::executeUpsert(struct txn *, struct space *, struct request *)
 	tnt_raise(ClientError, ER_UNSUPPORTED, engine->name, "upsert");
 }
 
-
 void
 Handler::executeSelect(struct txn *, struct space *space,
 		       uint32_t index_id, uint32_t iterator,
@@ -249,6 +248,16 @@ Handler::buildSecondaryKey(struct space *, struct space *, Index *)
 	tnt_raise(ClientError, ER_UNSUPPORTED, engine->name, "buildSecondaryKey");
 }
 
+void
+Handler::prepareTruncateSpace(struct space *, struct space *)
+{
+}
+
+void
+Handler::commitTruncateSpace(struct space *, struct space *)
+{
+}
+
 void
 Handler::prepareAlterSpace(struct space *, struct space *)
 {
diff --git a/src/box/engine.h b/src/box/engine.h
index 2fb4ea34d5912d3e11b56064bbc0ff0e2d92bf91..a710515edcff1cade09a7d87c23999be750b2aab 100644
--- a/src/box/engine.h
+++ b/src/box/engine.h
@@ -237,6 +237,23 @@ struct Handler {
 	virtual void buildSecondaryKey(struct space *old_space,
 				       struct space *new_space,
 				       Index *new_index);
+	/**
+	 * Notify the enigne about upcoming space truncation
+	 * so that it can prepare new_space object.
+	 */
+	virtual void prepareTruncateSpace(struct space *old_space,
+					  struct space *new_space);
+	/**
+	 * Commit space truncation. Called after space truncate
+	 * record was written to WAL hence must not fail.
+	 *
+	 * The old_space is the space that was replaced with the
+	 * new_space as a result of truncation. The callback is
+	 * supposed to release resources associated with the
+	 * old_space and commit the new_space.
+	 */
+	virtual void commitTruncateSpace(struct space *old_space,
+					 struct space *new_space);
 	/**
 	 * Notify the engine about the changed space,
 	 * before it's done, to prepare 'new_space'
diff --git a/src/box/key_def.h b/src/box/key_def.h
index 997f1182bdb7d614f76235e8beeed7e3b0248312..54815d2d7491c994c71c11c13905fd841f342955 100644
--- a/src/box/key_def.h
+++ b/src/box/key_def.h
@@ -694,20 +694,6 @@ space_def_check(struct space_def *def, uint32_t namelen,
                 uint32_t engine_namelen,
                 int32_t errcode);
 
-/**
- * Given a tuple with an index definition, update the LSN stored
- * in the index options.
- *
- * @return a tuple with updated lsn in key def. The returned tuple
- *         is blessed (referenced by box_tuple_bless()).
- *
- * Throws an exception if error.
- *
- * @note Implemented in alter.cc
- */
-extern struct tuple *
-index_def_tuple_update_lsn(struct tuple *tuple, int64_t lsn);
-
 /**
  * Check object identifier for invalid symbols.
  * The function checks \a str for matching [a-zA-Z_][a-zA-Z0-9_]* expression.
diff --git a/src/box/lua/schema.lua b/src/box/lua/schema.lua
index e7432842afc05921c12c850676b6675e798d357a..0041286b63932a3d8fb0bd50940ace2fdb528b59 100644
--- a/src/box/lua/schema.lua
+++ b/src/box/lua/schema.lua
@@ -286,6 +286,10 @@ box.schema.space.create = function(name, options)
     }, { __serialize = 'map' })
     _space:insert{id, uid, name, options.engine, options.field_count,
         space_options, format}
+
+    local _truncate = box.space[box.schema.TRUNCATE_ID]
+    _truncate:insert{id, 0}
+
     return box.space[id], "created"
 end
 
@@ -311,6 +315,7 @@ box.schema.space.drop = function(space_id, space_name, opts)
     local _space = box.space[box.schema.SPACE_ID]
     local _index = box.space[box.schema.INDEX_ID]
     local _priv = box.space[box.schema.PRIV_ID]
+    local _truncate = box.space[box.schema.TRUNCATE_ID]
     local keys = _index:select(space_id)
     for i = #keys, 1, -1 do
         local v = keys[i]
@@ -328,6 +333,7 @@ box.schema.space.drop = function(space_id, space_name, opts)
             box.error(box.error.NO_SUCH_SPACE, space_name)
         end
     end
+    _truncate:delete{space_id}
 end
 
 box.schema.space.rename = function(space_id, space_name)
diff --git a/src/box/lua/space.cc b/src/box/lua/space.cc
index 8bd7041c9f51ce68b664a602a055e125f389f76e..3c35e889847cd1c387dfa8004defae3e1230b69a 100644
--- a/src/box/lua/space.cc
+++ b/src/box/lua/space.cc
@@ -310,6 +310,8 @@ box_lua_space_init(struct lua_State *L)
 	lua_setfield(L, -2, "VPRIV_ID");
 	lua_pushnumber(L, BOX_CLUSTER_ID);
 	lua_setfield(L, -2, "CLUSTER_ID");
+	lua_pushnumber(L, BOX_TRUNCATE_ID);
+	lua_setfield(L, -2, "TRUNCATE_ID");
 	lua_pushnumber(L, BOX_SYSTEM_ID_MIN);
 	lua_setfield(L, -2, "SYSTEM_ID_MIN");
 	lua_pushnumber(L, BOX_SYSTEM_ID_MAX);
diff --git a/src/box/lua/upgrade.lua b/src/box/lua/upgrade.lua
index be1a7851fe3c9016b4531e2e1fde8d149f574ef1..e72d481d38e5757b79c9aaa5a276227d316d8e0c 100644
--- a/src/box/lua/upgrade.lua
+++ b/src/box/lua/upgrade.lua
@@ -65,6 +65,7 @@ local function erase()
     truncate(box.space._user)
     truncate(box.space._func)
     truncate(box.space._priv)
+    truncate(box.space._truncate)
     --truncate(box.space._schema)
     box.space._schema:delete('version')
     box.space._schema:delete('max_id')
@@ -533,6 +534,36 @@ local function upgrade_to_1_7_2()
     box.space._schema:replace({'version', 1, 7, 2})
 end
 
+--------------------------------------------------------------------------------
+-- Tarantool 1.7.4
+--------------------------------------------------------------------------------
+
+local function create_truncate_space()
+    local _truncate = box.space[box.schema.TRUNCATE_ID]
+
+    log.info("create space _truncate")
+    box.space._space:insert{_truncate.id, ADMIN, '_truncate', 'memtx', 0, setmap({}),
+                            {{name = 'id', type = 'num'}, {name = 'count', type = 'num'}}}
+
+    log.info("create index primary on _truncate")
+    box.space._index:insert{_truncate.id, 0, 'primary', 'tree', {unique = true}, {{0, 'unsigned'}}}
+
+    for _, def in box.space._space:pairs() do
+        _truncate:insert{def[1], 0}
+    end
+end
+
+local function upgrade_to_1_7_4()
+    if VERSION_ID >= version_id(1, 7, 4) then
+        return
+    end
+
+    create_truncate_space()
+
+    log.info("set schema version to 1.7.4")
+    box.space._schema:replace({'version', 1, 7, 4})
+end
+
 --------------------------------------------------------------------------------
 
 local function upgrade()
@@ -549,6 +580,7 @@ local function upgrade()
     upgrade_to_1_6_8()
     upgrade_to_1_7_1()
     upgrade_to_1_7_2()
+    upgrade_to_1_7_4()
 end
 
 local function bootstrap()
diff --git a/src/box/memtx_index.cc b/src/box/memtx_index.cc
index 7600ef4890641391680db090efe5612cf57489fe..144869a2b2e1b4d59e35f0a07de1542d3331fc93 100644
--- a/src/box/memtx_index.cc
+++ b/src/box/memtx_index.cc
@@ -38,14 +38,11 @@
 #include "space.h"
 
 void
-MemtxIndex::commitDrop()
+MemtxIndex::truncate()
 {
 	if (index_def->iid != 0)
 		return; /* nothing to do for secondary keys */
-	/*
-	 * Delete all tuples in the old space if dropping
-	 * the primary key.
-	 */
+
 	struct iterator *it = position();
 	initIterator(it, ITER_ALL, NULL, 0);
 	struct tuple *tuple;
diff --git a/src/box/memtx_index.h b/src/box/memtx_index.h
index 0e59f87c089adcdbb43c17f3089e75e6544b5948..af08d802eed1e45bbcbfdc8ba7ba8b35a1a9d108 100644
--- a/src/box/memtx_index.h
+++ b/src/box/memtx_index.h
@@ -42,7 +42,13 @@ class MemtxIndex: public Index {
 			m_position->free(m_position);
 	}
 
-	virtual void commitDrop() override;
+	/** Delete all tuples in the index. */
+	void truncate();
+
+	virtual void commitDrop() override
+	{
+		truncate();
+	}
 
 	virtual struct tuple *min(const char *key,
 				  uint32_t part_count) const override;
diff --git a/src/box/memtx_space.cc b/src/box/memtx_space.cc
index e71c89e42346c73d6bafe89da835c115b59615cf..0f5afd63c89727486e02d323d86626e1140fa0fd 100644
--- a/src/box/memtx_space.cc
+++ b/src/box/memtx_space.cc
@@ -705,6 +705,24 @@ MemtxSpace::buildSecondaryKey(struct space *old_space,
 	}
 }
 
+void
+MemtxSpace::prepareTruncateSpace(struct space *old_space,
+				 struct space *new_space)
+{
+	(void)new_space;
+	MemtxSpace *handler = (MemtxSpace *) old_space->handler;
+	replace = handler->replace;
+}
+
+void
+MemtxSpace::commitTruncateSpace(struct space *old_space,
+				struct space *new_space)
+{
+	(void)new_space;
+	struct MemtxIndex *index = (MemtxIndex *) space_index(old_space, 0);
+	if (index != NULL)
+		index->truncate();
+}
 void
 MemtxSpace::prepareAlterSpace(struct space *old_space, struct space *new_space)
 {
diff --git a/src/box/memtx_space.h b/src/box/memtx_space.h
index 25b45e7015f535a54dca597a94d83fefec1a646a..68d1b781b64852b1bfaa4bb3662b5bdaf1e79af3 100644
--- a/src/box/memtx_space.h
+++ b/src/box/memtx_space.h
@@ -86,6 +86,10 @@ struct MemtxSpace: public Handler {
 	virtual void buildSecondaryKey(struct space *old_space,
 				       struct space *new_space,
 				       Index *new_index) override;
+	virtual void prepareTruncateSpace(struct space *old_space,
+					  struct space *new_space) override;
+	virtual void commitTruncateSpace(struct space *old_space,
+					 struct space *new_space) override;
 	virtual void prepareAlterSpace(struct space *old_space,
 				       struct space *new_space) override;
 	virtual void initSystemSpace(struct space *space) override;
diff --git a/src/box/schema.cc b/src/box/schema.cc
index 0c6965b52596cfe73e43a74efe048ac553ae2023..11489cbc695cbe9529b7ef606efc3960f69d642c 100644
--- a/src/box/schema.cc
+++ b/src/box/schema.cc
@@ -254,15 +254,14 @@ schema_init()
 	 */
 	/* _schema - key/value space with schema description */
 	struct space_def def = {
-		BOX_SCHEMA_ID, ADMIN, 0, "_schema", "memtx", {false}
+		BOX_SCHEMA_ID, ADMIN, 0, "_schema", "memtx", space_opts_default
 	};
-	struct index_opts opts = index_opts_default;
 	struct index_def *index_def = index_def_new(def.id,
 						    "_schema",
 						    0 /* index id */,
 						    "primary", /* name */
 						    TREE /* index type */,
-						    &opts,
+						    &index_opts_default,
 						    1); /* part count */
 	if (index_def == NULL)
 		diag_raise();
@@ -280,6 +279,11 @@ schema_init()
 	(void) sc_space_new(&def, index_def, &alter_space_on_replace_space,
 			    &on_stmt_begin_space);
 
+	/* _truncate - auxiliary space for triggering space truncation. */
+	index_def->space_id = def.id = BOX_TRUNCATE_ID;
+	snprintf(def.name, sizeof(def.name), "_truncate");
+	(void) sc_space_new(&def, index_def, &on_replace_truncate);
+
 	/* _user - all existing users */
 	index_def->space_id = def.id = BOX_USER_ID;
 	snprintf(def.name, sizeof(def.name), "_user");
@@ -313,7 +317,7 @@ schema_init()
 				  0 /* index id */,
 				  "primary",
 				  TREE /* index type */,
-				  &opts,
+				  &index_opts_default,
 				  2); /* part count */
 	if (index_def == NULL)
 		diag_raise();
diff --git a/src/box/schema.h b/src/box/schema.h
index cf8b6f15ff34afe55e5b7b29f13678e37020ee9d..a2fc8670083316a6a99b7cb1f12eff14281b7dba 100644
--- a/src/box/schema.h
+++ b/src/box/schema.h
@@ -59,6 +59,8 @@ enum {
 	BOX_VPRIV_ID = 313,
 	/** Space id of _cluster. */
 	BOX_CLUSTER_ID = 320,
+	/** Space if of _truncate. */
+	BOX_TRUNCATE_ID = 330,
 	/** End of the reserved range of system spaces. */
 	BOX_SYSTEM_ID_MAX = 511,
 	BOX_ID_NIL = 2147483647
diff --git a/src/box/space.h b/src/box/space.h
index 9cbd8f0ae9e7a057e71c7872a569bc43c391bfc7..b436230712088811d0ad15d5ee36c30fd1d741e8 100644
--- a/src/box/space.h
+++ b/src/box/space.h
@@ -72,6 +72,12 @@ struct space {
 	uint32_t index_id_max;
 	/** Space meta. */
 	struct space_def def;
+	/**
+	 * Number of times the space has been truncated.
+	 * Updating this counter via _truncate space triggers
+	 * space truncation.
+	 */
+	uint64_t truncate_count;
 	/** Enable/disable triggers. */
 	bool run_triggers;
 
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index 4bc86b767f4ebdf939d95a47d988c8e8f6a00d07..5ef529087c1b7686308583cc5406e4e9357e34fa 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -597,6 +597,15 @@ struct vy_index {
 	 * Used to make sure that the primary index is dumped last.
 	 */
 	int pin_count;
+	/**
+	 * The number of times the index was truncated.
+	 *
+	 * After recovery is complete, it equals space->truncate_count.
+	 * On local recovery, it is loaded from the metadata log and may
+	 * be greater than space->truncate_count, which indicates that
+	 * the space is truncated in WAL.
+	 */
+	uint64_t truncate_count;
 };
 
 /** Return index name. Used for logging. */
@@ -2396,6 +2405,10 @@ vy_index_recovery_cb(const struct vy_log_record *record, void *cb_arg)
 		assert(record->index_lsn == index->opts.lsn);
 		index->dump_lsn = record->dump_lsn;
 		break;
+	case VY_LOG_TRUNCATE_INDEX:
+		assert(record->index_lsn == index->opts.lsn);
+		index->truncate_count = record->truncate_count;
+		break;
 	case VY_LOG_DROP_INDEX:
 		assert(record->index_lsn == index->opts.lsn);
 		index->is_dropped = true;
@@ -4526,6 +4539,32 @@ vy_index_commit_create(struct vy_index *index)
 	vy_scheduler_add_index(env->scheduler, index);
 }
 
+/*
+ * Delete all runs, ranges, and slices of a given index
+ * from the metadata log.
+ */
+static void
+vy_log_index_prune(struct vy_index *index)
+{
+	int loops = 0;
+	for (struct vy_range *range = vy_range_tree_first(index->tree);
+	     range != NULL; range = vy_range_tree_next(index->tree, range)) {
+		struct vy_slice *slice;
+		rlist_foreach_entry(slice, &range->slices, in_range)
+			vy_log_delete_slice(slice->id);
+		vy_log_delete_range(range->id);
+		if (++loops % VY_YIELD_LOOPS == 0)
+			fiber_sleep(0);
+	}
+	struct vy_run *run;
+	int64_t gc_lsn = vclock_sum(&index->env->scheduler->last_checkpoint);
+	rlist_foreach_entry(run, &index->runs, in_index) {
+		vy_log_drop_run(run->id, gc_lsn);
+		if (++loops % VY_YIELD_LOOPS == 0)
+			fiber_sleep(0);
+	}
+}
+
 void
 vy_index_commit_drop(struct vy_index *index)
 {
@@ -4545,29 +4584,161 @@ vy_index_commit_drop(struct vy_index *index)
 	index->is_dropped = true;
 
 	vy_log_tx_begin();
-	int loops = 0;
-	for (struct vy_range *range = vy_range_tree_first(index->tree);
-	     range != NULL; range = vy_range_tree_next(index->tree, range)) {
-		struct vy_slice *slice;
-		rlist_foreach_entry(slice, &range->slices, in_range)
-			vy_log_delete_slice(slice->id);
-		vy_log_delete_range(range->id);
-		if (++loops % VY_YIELD_LOOPS == 0)
-			fiber_sleep(0);
-	}
-	struct vy_run *run;
-	int64_t gc_lsn = vclock_sum(&env->scheduler->last_checkpoint);
-	rlist_foreach_entry(run, &index->runs, in_index) {
-		vy_log_drop_run(run->id, gc_lsn);
-		if (++loops % VY_YIELD_LOOPS == 0)
-			fiber_sleep(0);
-	}
+	vy_log_index_prune(index);
 	vy_log_drop_index(index->opts.lsn);
 	if (vy_log_tx_try_commit() < 0)
 		say_warn("failed to log drop index: %s",
 			 diag_last_error(diag_get())->errmsg);
 }
 
+/**
+ * Swap disk contents (ranges, runs, and corresponding stats)
+ * between two indexes. Used only on recovery, to skip reloading
+ * indexes of a truncated space. The in-memory tree of the index
+ * can't be populated - see vy_is_committed_one().
+ */
+static void
+vy_index_swap(struct vy_index *old_index, struct vy_index *new_index)
+{
+	assert(old_index->mem_used == 0);
+	assert(new_index->mem_used == 0);
+
+	SWAP(old_index->dump_lsn, new_index->dump_lsn);
+	SWAP(old_index->range_count, new_index->range_count);
+	SWAP(old_index->run_count, new_index->run_count);
+	SWAP(old_index->page_count, new_index->page_count);
+	SWAP(old_index->stmt_count, new_index->stmt_count);
+	SWAP(old_index->size, new_index->size);
+	SWAP(old_index->run_hist, new_index->run_hist);
+	SWAP(old_index->tree, new_index->tree);
+	rlist_swap(&old_index->runs, &new_index->runs);
+}
+
+int
+vy_prepare_truncate_space(struct space *old_space, struct space *new_space)
+{
+	assert(old_space->index_count == new_space->index_count);
+	uint32_t index_count = new_space->index_count;
+	if (index_count == 0)
+		return 0;
+
+	struct vy_index *pk = vy_index(old_space->index[0]);
+	struct vy_env *env = pk->env;
+
+	/*
+	 * On local recovery, we need to handle the following
+	 * scenarios:
+	 *
+	 * - Space truncation was successfully logged before restart.
+	 *   In this case indexes of the old space contain data added
+	 *   after truncation (recovered by vy_index_recover()) and
+	 *   hence we just need to swap contents between old and new
+	 *   spaces.
+	 *
+	 * - We failed to log space truncation before restart.
+	 *   In this case we have to replay space truncation the
+	 *   same way we handle it during normal operation.
+	 *
+	 * See also vy_commit_truncate_space().
+	 */
+	bool truncate_done = (env->status == VINYL_FINAL_RECOVERY_LOCAL &&
+			      pk->truncate_count > old_space->truncate_count);
+
+	for (uint32_t i = 0; i < index_count; i++) {
+		struct vy_index *old_index = vy_index(old_space->index[i]);
+		struct vy_index *new_index = vy_index(new_space->index[i]);
+
+		if (truncate_done) {
+			/*
+			 * We are replaying truncate from WAL and the
+			 * old space already contains data added after
+			 * truncate (recovered from vylog). Avoid
+			 * reloading the space content from vylog,
+			 * simply swap the contents of old and new
+			 * spaces instead.
+			 */
+			vy_index_swap(old_index, new_index);
+			new_index->is_dropped = old_index->is_dropped;
+			new_index->truncate_count = old_index->truncate_count;
+			vy_scheduler_add_index(env->scheduler, new_index);
+			continue;
+		}
+
+		if (vy_index_create(new_index) != 0)
+			return -1;
+
+		new_index->truncate_count = new_space->truncate_count;
+	}
+	return 0;
+}
+
+void
+vy_commit_truncate_space(struct space *old_space, struct space *new_space)
+{
+	assert(old_space->index_count == new_space->index_count);
+	uint32_t index_count = new_space->index_count;
+	if (index_count == 0)
+		return;
+
+	struct vy_index *pk = vy_index(old_space->index[0]);
+	struct vy_env *env = pk->env;
+
+	/*
+	 * See the comment in vy_prepare_truncate_space().
+	 */
+	if (env->status == VINYL_FINAL_RECOVERY_LOCAL &&
+	    pk->truncate_count > old_space->truncate_count)
+		return;
+
+	/*
+	 * Mark old indexes as dropped. After this point no task can
+	 * be scheduled or completed for any of them (only aborted).
+	 */
+	for (uint32_t i = 0; i < index_count; i++) {
+		struct vy_index *index = vy_index(old_space->index[i]);
+		index->is_dropped = true;
+	}
+
+	/*
+	 * Log change in metadata.
+	 *
+	 * Since we can't fail here, in case of vylog write failure
+	 * we leave records we failed to write in vylog buffer so
+	 * that they get flushed along with the next write. If they
+	 * don't, we will replay them during WAL recovery.
+	 */
+	vy_log_tx_begin();
+	for (uint32_t i = 0; i < index_count; i++) {
+		struct vy_index *old_index = vy_index(old_space->index[i]);
+		struct vy_index *new_index = vy_index(new_space->index[i]);
+		struct vy_range *range = vy_range_tree_first(new_index->tree);
+
+		assert(!new_index->is_dropped);
+		assert(new_index->truncate_count == new_space->truncate_count);
+		assert(new_index->range_count == 1);
+
+		vy_log_index_prune(old_index);
+		vy_log_insert_range(new_index->opts.lsn, range->id, NULL, NULL);
+		vy_log_truncate_index(new_index->opts.lsn,
+				      new_index->truncate_count);
+	}
+	if (vy_log_tx_try_commit() < 0)
+		say_warn("failed to log index truncation: %s",
+			 diag_last_error(diag_get())->errmsg);
+
+	/*
+	 * After we committed space truncation in the metadata log,
+	 * we can make new indexes eligible for dump and compaction.
+	 */
+	for (uint32_t i = 0; i < index_count; i++) {
+		struct vy_index *index = vy_index(new_space->index[i]);
+		struct vy_range *range = vy_range_tree_first(index->tree);
+
+		vy_scheduler_add_range(env->scheduler, range);
+		vy_scheduler_add_index(env->scheduler, index);
+	}
+}
+
 extern struct tuple_format_vtab vy_tuple_format_vtab;
 
 struct vy_index *
@@ -4984,17 +5155,21 @@ vy_tx_set(struct vy_tx *tx, struct vy_index *index, struct tuple *stmt)
  * replaying records already present in the database. In this
  * case avoid overwriting a newer version with an older one.
  *
- * If the index is going to be dropped on WAL recovery,
- * there's no point in replaying statements for it either.
+ * If the index is going to be dropped or truncated on WAL
+ * recovery, there's no point in replaying statements for it,
+ * either.
  */
 static inline bool
-vy_is_committed_one(struct vy_tx *tx, struct vy_index *index)
+vy_is_committed_one(struct vy_tx *tx, struct space *space,
+		    struct vy_index *index)
 {
 	struct vy_env *env = tx->xm->env;
 	if (likely(env->status != VINYL_FINAL_RECOVERY_LOCAL))
 		return false;
 	if (index->is_dropped)
 		return true;
+	if (index->truncate_count > space->truncate_count)
+		return true;
 	if (vclock_sum(env->recovery_vclock) <= index->dump_lsn)
 		return true;
 	return false;
@@ -5012,7 +5187,7 @@ vy_is_committed(struct vy_tx *tx, struct space *space)
 		return false;
 	for (uint32_t iid = 0; iid < space->index_count; iid++) {
 		struct vy_index *index = vy_index(space->index[iid]);
-		if (!vy_is_committed_one(tx, index))
+		if (!vy_is_committed_one(tx, space, index))
 			return false;
 	}
 	return true;
@@ -5259,7 +5434,7 @@ vy_replace_impl(struct vy_tx *tx, struct space *space, struct request *request,
 	if (pk == NULL) /* space has no primary key */
 		return -1;
 	/* Primary key is dumped last. */
-	assert(!vy_is_committed_one(tx, pk));
+	assert(!vy_is_committed_one(tx, space, pk));
 	assert(pk->id == 0);
 	new_stmt = vy_stmt_new_replace(space->format, request->tuple,
 				       request->tuple_end);
@@ -5291,7 +5466,7 @@ vy_replace_impl(struct vy_tx *tx, struct space *space, struct request *request,
 	for (uint32_t iid = 1; iid < space->index_count; ++iid) {
 		struct vy_index *index;
 		index = vy_index(space->index[iid]);
-		if (vy_is_committed_one(tx, index))
+		if (vy_is_committed_one(tx, space, index))
 			continue;
 		/*
 		 * Delete goes first, so if old and new keys
@@ -5444,7 +5619,7 @@ vy_delete_impl(struct vy_tx *tx, struct space *space,
 	if (pk == NULL)
 		return -1;
 	/* Primary key is dumped last. */
-	assert(!vy_is_committed_one(tx, pk));
+	assert(!vy_is_committed_one(tx, space, pk));
 	struct tuple *delete =
 		vy_stmt_new_surrogate_delete(space->format, tuple);
 	if (delete == NULL)
@@ -5456,7 +5631,7 @@ vy_delete_impl(struct vy_tx *tx, struct space *space,
 	struct vy_index *index;
 	for (uint32_t i = 1; i < space->index_count; ++i) {
 		index = vy_index(space->index[i]);
-		if (vy_is_committed_one(tx, index))
+		if (vy_is_committed_one(tx, space, index))
 			continue;
 		if (vy_tx_set(tx, index, delete) != 0)
 			goto error;
@@ -5596,7 +5771,7 @@ vy_update(struct vy_tx *tx, struct txn_stmt *stmt, struct space *space,
 	assert(pk != NULL);
 	assert(pk->id == 0);
 	/* Primary key is dumped last. */
-	assert(!vy_is_committed_one(tx, pk));
+	assert(!vy_is_committed_one(tx, space, pk));
 	uint64_t column_mask = 0;
 	const char *new_tuple, *new_tuple_end;
 	uint32_t new_size, old_size;
@@ -5660,7 +5835,7 @@ vy_update(struct vy_tx *tx, struct txn_stmt *stmt, struct space *space,
 	assert(delete != NULL);
 	for (uint32_t i = 1; i < space->index_count; ++i) {
 		index = vy_index(space->index[i]);
-		if (vy_is_committed_one(tx, index))
+		if (vy_is_committed_one(tx, space, index))
 			continue;
 		if (vy_tx_set(tx, index, delete) != 0)
 			goto error;
@@ -5762,7 +5937,7 @@ vy_upsert(struct vy_tx *tx, struct txn_stmt *stmt, struct space *space,
 	if (pk == NULL)
 		return -1;
 	/* Primary key is dumped last. */
-	assert(!vy_is_committed_one(tx, pk));
+	assert(!vy_is_committed_one(tx, space, pk));
 	if (tuple_validate_raw(space->format, tuple))
 		return -1;
 
@@ -5867,7 +6042,7 @@ vy_upsert(struct vy_tx *tx, struct txn_stmt *stmt, struct space *space,
 	assert(delete != NULL);
 	for (uint32_t i = 1; i < space->index_count; ++i) {
 		index = vy_index(space->index[i]);
-		if (vy_is_committed_one(tx, index))
+		if (vy_is_committed_one(tx, space, index))
 			continue;
 		if (vy_tx_set(tx, index, delete) != 0)
 			goto error;
diff --git a/src/box/vinyl.h b/src/box/vinyl.h
index 9ef5c8663017a721805ff289e53f25d06816817d..5bc20067d528ea9c91316e6d0ee6cd4935ff6a40 100644
--- a/src/box/vinyl.h
+++ b/src/box/vinyl.h
@@ -256,6 +256,21 @@ vy_index_ref(struct vy_index *index);
 void
 vy_index_unref(struct vy_index *index);
 
+/**
+ * Handle vinyl space truncation.
+ *
+ * This function initializes indexes of the new space
+ * so that it can replace the old space on truncation.
+ */
+int
+vy_prepare_truncate_space(struct space *old_space, struct space *new_space);
+
+/**
+ * Commit space truncation in the metadata log.
+ */
+void
+vy_commit_truncate_space(struct space *old_space, struct space *new_space);
+
 /**
  * Hook on an preparation of space alter event.
  * @param old_space Old space.
diff --git a/src/box/vinyl_space.cc b/src/box/vinyl_space.cc
index 37f3ba722d4542ce14e4776e039121688f493116..62350f6dc10518f82e85bfa11e2509ab4ffb4c46 100644
--- a/src/box/vinyl_space.cc
+++ b/src/box/vinyl_space.cc
@@ -220,6 +220,21 @@ VinylSpace::buildSecondaryKey(struct space *old_space,
 	 */
 }
 
+void
+VinylSpace::prepareTruncateSpace(struct space *old_space,
+				 struct space *new_space)
+{
+	if (vy_prepare_truncate_space(old_space, new_space) != 0)
+		diag_raise();
+}
+
+void
+VinylSpace::commitTruncateSpace(struct space *old_space,
+				struct space *new_space)
+{
+	vy_commit_truncate_space(old_space, new_space);
+}
+
 void
 VinylSpace::prepareAlterSpace(struct space *old_space, struct space *new_space)
 {
diff --git a/src/box/vinyl_space.h b/src/box/vinyl_space.h
index e5780f50b56062b39f8a710bae85c2bccf78e189..42a1199f3676e2fb200a1424b2b3bbd88ce24ef9 100644
--- a/src/box/vinyl_space.h
+++ b/src/box/vinyl_space.h
@@ -57,15 +57,18 @@ struct VinylSpace: public Handler {
 	virtual void buildSecondaryKey(struct space *old_space,
 				       struct space *new_space,
 				       Index *new_index) override;
+	virtual void prepareTruncateSpace(struct space *old_space,
+					  struct space *new_space) override;
+	virtual void commitTruncateSpace(struct space *old_space,
+					 struct space *new_space) override;
 	virtual void prepareAlterSpace(struct space *old_space,
 				       struct space *new_space) override;
 	/**
 	 * If space was altered then this method updates
 	 * pointers to the primary index in all secondary ones.
 	 */
-	virtual void
-	commitAlterSpace(struct space *old_space, struct space *new_space)
-		override;
+	virtual void commitAlterSpace(struct space *old_space,
+				      struct space *new_space) override;
 };
 
 #endif /* TARANTOOL_BOX_VINYL_SPACE_H_INCLUDED */
diff --git a/src/box/vy_log.c b/src/box/vy_log.c
index e1d604aabc411c84905644ffd6ada12577d0a892..3ff9e048d3a090ef0eded050f655a984e10c0127 100644
--- a/src/box/vy_log.c
+++ b/src/box/vy_log.c
@@ -78,6 +78,7 @@ enum vy_log_key {
 	VY_LOG_KEY_SLICE_ID		= 8,
 	VY_LOG_KEY_DUMP_LSN		= 9,
 	VY_LOG_KEY_GC_LSN		= 10,
+	VY_LOG_KEY_TRUNCATE_COUNT	= 11,
 };
 
 /** vy_log_key -> human readable name. */
@@ -93,6 +94,7 @@ static const char *vy_log_key_name[] = {
 	[VY_LOG_KEY_SLICE_ID]		= "slice_id",
 	[VY_LOG_KEY_DUMP_LSN]		= "dump_lsn",
 	[VY_LOG_KEY_GC_LSN]		= "gc_lsn",
+	[VY_LOG_KEY_TRUNCATE_COUNT]	= "truncate_count",
 };
 
 /** vy_log_type -> human readable name. */
@@ -109,6 +111,7 @@ static const char *vy_log_type_name[] = {
 	[VY_LOG_DELETE_SLICE]		= "delete_slice",
 	[VY_LOG_DUMP_INDEX]		= "dump_index",
 	[VY_LOG_SNAPSHOT]		= "snapshot",
+	[VY_LOG_TRUNCATE_INDEX]		= "truncate_index",
 };
 
 struct vy_recovery;
@@ -192,6 +195,8 @@ struct vy_index_recovery_info {
 	bool is_dropped;
 	/** LSN of the last index dump. */
 	int64_t dump_lsn;
+	/** Truncate count. */
+	int64_t truncate_count;
 	/**
 	 * List of all ranges in the index, linked by
 	 * vy_range_recovery_info::in_index.
@@ -336,6 +341,10 @@ vy_log_record_snprint(char *buf, int size, const struct vy_log_record *record)
 		SNPRINT(total, snprintf, buf, size, "%s=%"PRIi64", ",
 			vy_log_key_name[VY_LOG_KEY_GC_LSN],
 			record->gc_lsn);
+	if (record->truncate_count > 0)
+		SNPRINT(total, snprintf, buf, size, "%s=%"PRIi64", ",
+			vy_log_key_name[VY_LOG_KEY_TRUNCATE_COUNT],
+			record->truncate_count);
 	SNPRINT(total, snprintf, buf, size, "}");
 	return total;
 }
@@ -441,6 +450,11 @@ vy_log_record_encode(const struct vy_log_record *record,
 		size += mp_sizeof_uint(record->gc_lsn);
 		n_keys++;
 	}
+	if (record->truncate_count > 0) {
+		size += mp_sizeof_uint(VY_LOG_KEY_TRUNCATE_COUNT);
+		size += mp_sizeof_uint(record->truncate_count);
+		n_keys++;
+	}
 	size += mp_sizeof_map(n_keys);
 
 	/*
@@ -506,6 +520,10 @@ vy_log_record_encode(const struct vy_log_record *record,
 		pos = mp_encode_uint(pos, VY_LOG_KEY_GC_LSN);
 		pos = mp_encode_uint(pos, record->gc_lsn);
 	}
+	if (record->truncate_count > 0) {
+		pos = mp_encode_uint(pos, VY_LOG_KEY_TRUNCATE_COUNT);
+		pos = mp_encode_uint(pos, record->truncate_count);
+	}
 	assert(pos == tuple + size);
 
 	/*
@@ -622,6 +640,9 @@ vy_log_record_decode(struct vy_log_record *record,
 		case VY_LOG_KEY_GC_LSN:
 			record->gc_lsn = mp_decode_uint(&pos);
 			break;
+		case VY_LOG_KEY_TRUNCATE_COUNT:
+			record->truncate_count = mp_decode_uint(&pos);
+			break;
 		default:
 			diag_set(ClientError, ER_INVALID_VYLOG_FILE,
 				 tt_sprintf("Bad record: unknown key %u",
@@ -1234,6 +1255,7 @@ vy_recovery_create_index(struct vy_recovery *recovery, int64_t index_lsn,
 	memcpy(index->key_def, key_def, key_def_sizeof(key_def->part_count));
 	index->is_dropped = false;
 	index->dump_lsn = -1;
+	index->truncate_count = 0;
 	rlist_create(&index->ranges);
 	rlist_create(&index->runs);
 	return 0;
@@ -1309,6 +1331,33 @@ vy_recovery_dump_index(struct vy_recovery *recovery,
 	return 0;
 }
 
+/**
+ * Handle a VY_LOG_TRUNCATE_INDEX log record.
+ * This function updates truncate_count of the index with ID @index_lsn.
+ * Returns 0 on success, -1 if ID not found or index is dropped.
+ */
+static int
+vy_recovery_truncate_index(struct vy_recovery *recovery,
+			   int64_t index_lsn, int64_t truncate_count)
+{
+	struct vy_index_recovery_info *index;
+	index = vy_recovery_lookup_index(recovery, index_lsn);
+	if (index == NULL) {
+		diag_set(ClientError, ER_INVALID_VYLOG_FILE,
+			 tt_sprintf("Truncation of unregistered index %lld",
+				    (long long)index_lsn));
+		return -1;
+	}
+	if (index->is_dropped) {
+		diag_set(ClientError, ER_INVALID_VYLOG_FILE,
+			 tt_sprintf("Truncation of deleted index %lld",
+				    (long long)index_lsn));
+		return -1;
+	}
+	index->truncate_count = truncate_count;
+	return 0;
+}
+
 /**
  * Allocate a vinyl run with ID @run_id and insert it to the hash.
  * Return the new run on success, NULL on OOM.
@@ -1750,6 +1799,10 @@ vy_recovery_process_record(struct vy_recovery *recovery,
 		rc = vy_recovery_dump_index(recovery, record->index_lsn,
 					    record->dump_lsn);
 		break;
+	case VY_LOG_TRUNCATE_INDEX:
+		rc = vy_recovery_truncate_index(recovery, record->index_lsn,
+						record->truncate_count);
+		break;
 	default:
 		unreachable();
 	}
@@ -1912,6 +1965,15 @@ vy_recovery_iterate_index(struct vy_index_recovery_info *index,
 	if (vy_recovery_cb_call(cb, cb_arg, &record) != 0)
 		return -1;
 
+	if (index->truncate_count > 0) {
+		vy_log_record_init(&record);
+		record.type = VY_LOG_TRUNCATE_INDEX;
+		record.index_lsn = index->index_lsn;
+		record.truncate_count = index->truncate_count;
+		if (vy_recovery_cb_call(cb, cb_arg, &record) != 0)
+			return -1;
+	}
+
 	if (!include_deleted && index->is_dropped) {
 		/*
 		 * Do not load the index as it is going to be
diff --git a/src/box/vy_log.h b/src/box/vy_log.h
index 8e653053d445c3b9facc1bc1e8064f121097d722..1f529af1b4c8f8a67dcd4d0c3de6aa72db3ae68a 100644
--- a/src/box/vy_log.h
+++ b/src/box/vy_log.h
@@ -150,6 +150,11 @@ enum vy_log_record_type {
 	 * See also: @only_snapshot argument of vy_recovery_new().
 	 */
 	VY_LOG_SNAPSHOT			= 11,
+	/**
+	 * Update truncate count of a vinyl index.
+	 * Requires vy_log_record::index_lsn, truncate_count.
+	 */
+	VY_LOG_TRUNCATE_INDEX		= 12,
 
 	vy_log_record_type_MAX
 };
@@ -195,6 +200,8 @@ struct vy_log_record {
 	 * that uses this run.
 	 */
 	int64_t gc_lsn;
+	/** Index truncate count. */
+	int64_t truncate_count;
 	/** Link in vy_log::tx. */
 	struct stailq_entry in_tx;
 };
@@ -534,6 +541,7 @@ vy_log_delete_slice(int64_t slice_id)
 	vy_log_write(&record);
 }
 
+/** Helper to log index dump. */
 static inline void
 vy_log_dump_index(int64_t index_lsn, int64_t dump_lsn)
 {
@@ -545,6 +553,18 @@ vy_log_dump_index(int64_t index_lsn, int64_t dump_lsn)
 	vy_log_write(&record);
 }
 
+/** Helper to log index truncation. */
+static inline void
+vy_log_truncate_index(int64_t index_lsn, int64_t truncate_count)
+{
+	struct vy_log_record record;
+	vy_log_record_init(&record);
+	record.type = VY_LOG_TRUNCATE_INDEX;
+	record.index_lsn = index_lsn;
+	record.truncate_count = truncate_count;
+	vy_log_write(&record);
+}
+
 #if defined(__cplusplus)
 } /* extern "C" */
 #endif /* defined(__cplusplus) */
diff --git a/src/trivia/util.h b/src/trivia/util.h
index 3548028390bd10f0c75f354828d7d082e57e3fdc..4164c2b9430756f8289e2c7b60cd7b477818e91d 100644
--- a/src/trivia/util.h
+++ b/src/trivia/util.h
@@ -56,6 +56,12 @@ extern "C" {
 # define MIN(a, b) ((a) < (b) ? (a) : (b))
 #endif
 
+#define SWAP(a, b) do {							\
+	typeof(a) tmp = (a);						\
+	(a) = (b);							\
+	(b) = tmp;							\
+} while (0)
+
 #define DIV_ROUND_UP(n, d) (((n) + (d) - 1) / (d))
 
 /* Macros to define enum and corresponding strings. */
diff --git a/test/app-tap/tarantoolctl.test.lua b/test/app-tap/tarantoolctl.test.lua
index 50ee33b540bc47e5884ecbe5a258f3c097c1cb60..55a1e1203bfc3c14421ccdc5280a9618ff753b52 100755
--- a/test/app-tap/tarantoolctl.test.lua
+++ b/test/app-tap/tarantoolctl.test.lua
@@ -327,17 +327,17 @@ do
             test_i:plan(25)
             check_ok(test_i, dir, 'start', 'filler', 0)
             check_ctlcat_xlog(test_i, dir, nil, "---\n", 6)
-            check_ctlcat_xlog(test_i, dir, "--show-system", "---\n", 9)
+            check_ctlcat_xlog(test_i, dir, "--show-system", "---\n", 10)
             check_ctlcat_xlog(test_i, dir, "--format=json", "\n", 6)
             check_ctlcat_xlog(test_i, dir, "--format=lua",  "\n", 6)
-            check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json", "\n", 2)
+            check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json", "\n", 1)
             check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json --show-system", "\n", 3)
             check_ctlcat_xlog(test_i, dir, "--from=6 --to=3 --format=json --show-system", "\n", 0)
             check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json --show-system --replica 1", "\n", 3)
             check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json --show-system --replica 1 --replica 2", "\n", 3)
             check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json --show-system --replica 2", "\n", 0)
-            check_ctlcat_snap(test_i, dir, "--space=280", "---\n", 12)
-            check_ctlcat_snap(test_i, dir, "--space=288", "---\n", 31)
+            check_ctlcat_snap(test_i, dir, "--space=280", "---\n", 13)
+            check_ctlcat_snap(test_i, dir, "--space=288", "---\n", 32)
         end)
     end)
 
@@ -392,13 +392,13 @@ do
             test_i:plan(6)
             check_ok(test_i, dir, 'start', 'filler', 0)
             local lsn_before = test_run:get_lsn("remote", 1)
-            test_i:is(lsn_before, 4, "check lsn before")
+            test_i:is(lsn_before, 5, "check lsn before")
             local res, stdout, stderr = run_command(dir, command_base)
             test_i:is(res, 0, "execution result")
-            test_i:is(test_run:get_lsn("remote", 1), 10, "check lsn after")
+            test_i:is(test_run:get_lsn("remote", 1), 11, "check lsn after")
             local res, stdout, stderr = run_command(dir, command_base)
             test_i:is(res, 0, "execution result")
-            test_i:is(test_run:get_lsn("remote", 1), 16, "check lsn after")
+            test_i:is(test_run:get_lsn("remote", 1), 17, "check lsn after")
         end)
     end)
 
diff --git a/test/box-py/bootstrap.result b/test/box-py/bootstrap.result
index 9ffe4ee956ca3685e69497c4a48479209094e449..e73466c013c4762b8b5c62d019b7e463add642c0 100644
--- a/test/box-py/bootstrap.result
+++ b/test/box-py/bootstrap.result
@@ -5,7 +5,7 @@ box.space._schema:select{}
 ---
 - - ['cluster', '<cluster uuid>']
   - ['max_id', 511]
-  - ['version', 1, 7, 2]
+  - ['version', 1, 7, 4]
 ...
 box.space._cluster:select{}
 ---
@@ -46,6 +46,8 @@ box.space._space:select{}
         'type': 'num'}, {'name': 'privilege', 'type': 'num'}]]
   - [320, 1, '_cluster', 'memtx', 0, {}, [{'name': 'id', 'type': 'num'}, {'name': 'uuid',
         'type': 'str'}]]
+  - [330, 1, '_truncate', 'memtx', 0, {}, [{'name': 'id', 'type': 'num'}, {'name': 'count',
+        'type': 'num'}]]
 ...
 box.space._index:select{}
 ---
@@ -82,6 +84,7 @@ box.space._index:select{}
   - [313, 2, 'object', 'tree', {'unique': false}, [[2, 'string'], [3, 'unsigned']]]
   - [320, 0, 'primary', 'tree', {'unique': true}, [[0, 'unsigned']]]
   - [320, 1, 'uuid', 'tree', {'unique': true}, [[1, 'string']]]
+  - [330, 0, 'primary', 'tree', {'unique': true}, [[0, 'unsigned']]]
 ...
 box.space._user:select{}
 ---
diff --git a/test/box/access_misc.result b/test/box/access_misc.result
index 7423518faff085537031dbce51204a97621763fa..917178d23a2fd2ce4ae8b876d2ef64729b8b3fd6 100644
--- a/test/box/access_misc.result
+++ b/test/box/access_misc.result
@@ -613,6 +613,8 @@ box.space._space:select()
         'type': 'num'}, {'name': 'privilege', 'type': 'num'}]]
   - [320, 1, '_cluster', 'memtx', 0, {}, [{'name': 'id', 'type': 'num'}, {'name': 'uuid',
         'type': 'str'}]]
+  - [330, 1, '_truncate', 'memtx', 0, {}, [{'name': 'id', 'type': 'num'}, {'name': 'count',
+        'type': 'num'}]]
 ...
 box.space._func:select()
 ---
diff --git a/test/box/access_sysview.result b/test/box/access_sysview.result
index 178776a6f00df1bfb82929faa857c5f9748176ae..4bf593e2e24f19459b865fca4cea7216ccfc5020 100644
--- a/test/box/access_sysview.result
+++ b/test/box/access_sysview.result
@@ -230,11 +230,11 @@ box.session.su('guest')
 ...
 #box.space._vspace:select{}
 ---
-- 13
+- 14
 ...
 #box.space._vindex:select{}
 ---
-- 32
+- 33
 ...
 #box.space._vuser:select{}
 ---
@@ -262,7 +262,7 @@ box.session.su('guest')
 ...
 #box.space._vindex:select{}
 ---
-- 32
+- 33
 ...
 #box.space._vuser:select{}
 ---
diff --git a/test/box/alter.result b/test/box/alter.result
index c1fa5835f3f6e1be9db91241b4a57305ee3804a3..6571d04a166575a08db339bece26303446d69a2c 100644
--- a/test/box/alter.result
+++ b/test/box/alter.result
@@ -101,7 +101,7 @@ space = box.space[t[1]]
 ...
 space.id
 ---
-- 321
+- 331
 ...
 space.field_count
 ---
@@ -146,7 +146,7 @@ space_deleted
 ...
 space:replace{0}
 ---
-- error: Space '321' does not exist
+- error: Space '331' does not exist
 ...
 _index:insert{_space.id, 0, 'primary', 'tree', 1, 1, 0, 'unsigned'}
 ---
@@ -199,6 +199,7 @@ _index:select{}
   - [313, 2, 'object', 'tree', {'unique': false}, [[2, 'string'], [3, 'unsigned']]]
   - [320, 0, 'primary', 'tree', {'unique': true}, [[0, 'unsigned']]]
   - [320, 1, 'uuid', 'tree', {'unique': true}, [[1, 'string']]]
+  - [330, 0, 'primary', 'tree', {'unique': true}, [[0, 'unsigned']]]
 ...
 -- modify indexes of a system space
 _index:delete{_index.id, 0}
@@ -541,31 +542,6 @@ space:drop()
 ---
 ...
 -- data dictionary compatibility is checked by upgrade.test.lua
--- ------------------------------------------------------------------
--- gh-1966 Index lsn does not get updated on space truncate
--- ------------------------------------------------------------------
-space = box.schema.space.create('test')
----
-...
-_ = space:create_index('primary', { parts = {1, 'unsigned'} })
----
-...
-lsn = box.space._index:select{box.space.test.id, 0}[1][5].lsn
----
-...
-space:truncate()
----
-...
-indexes = box.space._index:select{space.id}
----
-...
-lsn < box.space._index:select{box.space.test.id, 0}[1][5].lsn
----
-- true
-...
-space:drop()
----
-...
 test_run:cmd("clear filter")
 ---
 - true
diff --git a/test/box/alter.test.lua b/test/box/alter.test.lua
index ccfea68e09ad6bde6595094bf887ee90bcdfd2d3..421c751e0caeaab6d769c58af28fc9df35552bdb 100644
--- a/test/box/alter.test.lua
+++ b/test/box/alter.test.lua
@@ -219,18 +219,6 @@ space:drop()
 
 -- data dictionary compatibility is checked by upgrade.test.lua
 
--- ------------------------------------------------------------------
--- gh-1966 Index lsn does not get updated on space truncate
--- ------------------------------------------------------------------
-
-space = box.schema.space.create('test')
-_ = space:create_index('primary', { parts = {1, 'unsigned'} })
-lsn = box.space._index:select{box.space.test.id, 0}[1][5].lsn
-space:truncate()
-indexes = box.space._index:select{space.id}
-lsn < box.space._index:select{box.space.test.id, 0}[1][5].lsn
-space:drop()
-
 test_run:cmd("clear filter")
 --
 -- create_index() does not modify index options
diff --git a/test/box/stat.result b/test/box/stat.result
index 15a5f89e6acdfadb2ba6e5ce96eaffd0e449cc0b..dae759d5343d38d14222af317ca286420a0007c5 100644
--- a/test/box/stat.result
+++ b/test/box/stat.result
@@ -43,7 +43,7 @@ for i=1,10 do space:insert{i, 'tuple'..tostring(i)} end
 ...
 box.stat.INSERT.total
 ---
-- 12
+- 13
 ...
 box.stat.DELETE.total
 ---
diff --git a/test/vinyl/info.result b/test/vinyl/info.result
index b5b5fed3d29c6cd9947f125242b8ce067ad554f5..36e0c5ce14991776ec0a0f89769db8effcc6b848 100644
--- a/test/vinyl/info.result
+++ b/test/vinyl/info.result
@@ -61,7 +61,7 @@ box.snapshot()
 ...
 box_info_sort(box.info.vinyl())
 ---
-- - lsn: 5
+- - lsn: 6
   - memory:
     - limit: 536870912
     - ratio: 0%
diff --git a/test/vinyl/layout.result b/test/vinyl/layout.result
index 1e0f3a7d8058815672e67f4d3eb986e0d0f4c27e..2c61f7a9543a962bf8084250c1da6fee34925162 100644
--- a/test/vinyl/layout.result
+++ b/test/vinyl/layout.result
@@ -117,23 +117,23 @@ test_run:cmd("push filter 'offset: .*' to 'offset: <offset>'")
 ...
 result
 ---
-- - - 00000000000000000008.vylog
+- - - 00000000000000000009.vylog
     - - HEADER:
           type: INSERT
         BODY:
-          tuple: [0, {0: 2, 7: [[0, 'unsigned']], 6: 512}]
+          tuple: [0, {0: 3, 7: [[0, 'unsigned']], 6: 512}]
       - HEADER:
           type: INSERT
         BODY:
-          tuple: [10, {0: 2, 9: 8}]
+          tuple: [10, {0: 3, 9: 9}]
       - HEADER:
           type: INSERT
         BODY:
-          tuple: [5, {0: 2, 2: 2, 9: 8}]
+          tuple: [5, {0: 3, 2: 2, 9: 9}]
       - HEADER:
           type: INSERT
         BODY:
-          tuple: [4, {0: 2, 2: 1}]
+          tuple: [4, {0: 3, 2: 1}]
       - HEADER:
           type: INSERT
         BODY:
@@ -141,7 +141,7 @@ result
       - HEADER:
           type: INSERT
         BODY:
-          tuple: [2, {0: 2}]
+          tuple: [2, {0: 3}]
       - HEADER:
           type: INSERT
         BODY:
@@ -154,12 +154,12 @@ result
           timestamp: <timestamp>
           type: INSERT
         BODY:
-          tuple: [4, {0: 2, 2: 4}]
+          tuple: [4, {0: 3, 2: 4}]
       - HEADER:
           timestamp: <timestamp>
           type: INSERT
         BODY:
-          tuple: [5, {0: 2, 2: 4, 9: 11}]
+          tuple: [5, {0: 3, 2: 4, 9: 12}]
       - HEADER:
           timestamp: <timestamp>
           type: INSERT
@@ -169,16 +169,16 @@ result
           timestamp: <timestamp>
           type: INSERT
         BODY:
-          tuple: [10, {0: 2, 9: 11}]
+          tuple: [10, {0: 3, 9: 12}]
   - - 00000000000000000002.index
     - - HEADER:
           type: RUNINFO
         BODY:
-          min_lsn: 6
+          min_lsn: 7
           max_key: [3]
           page_count: 1
           bloom_filter: [0, 64, 5, "\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\v\x01\x04\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\v\x01\x04\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\v\x01\x04\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0"]
-          max_lsn: 8
+          max_lsn: 9
           min_key: [1]
       - HEADER:
           type: PAGEINFO
@@ -191,17 +191,17 @@ result
           page_index_offset: <offset>
   - - 00000000000000000002.run
     - - HEADER:
-          lsn: 6
+          lsn: 7
           type: REPLACE
         BODY:
           tuple: [1]
       - HEADER:
-          lsn: 7
+          lsn: 8
           type: REPLACE
         BODY:
           tuple: [2]
       - HEADER:
-          lsn: 8
+          lsn: 9
           type: REPLACE
         BODY:
           tuple: [3]
@@ -213,11 +213,11 @@ result
     - - HEADER:
           type: RUNINFO
         BODY:
-          min_lsn: 9
+          min_lsn: 10
           max_key: [6]
           page_count: 1
           bloom_filter: [0, 64, 5, "\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\v\x01\x04\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\v\x01\x04\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\v\x01\x04\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0"]
-          max_lsn: 11
+          max_lsn: 12
           min_key: [4]
       - HEADER:
           type: PAGEINFO
@@ -230,17 +230,17 @@ result
           page_index_offset: <offset>
   - - 00000000000000000004.run
     - - HEADER:
-          lsn: 9
+          lsn: 10
           type: REPLACE
         BODY:
           tuple: [4]
       - HEADER:
-          lsn: 10
+          lsn: 11
           type: REPLACE
         BODY:
           tuple: [5]
       - HEADER:
-          lsn: 11
+          lsn: 12
           type: REPLACE
         BODY:
           tuple: [6]
diff --git a/test/wal_off/alter.result b/test/wal_off/alter.result
index 3b5481c0d5fd00415f52cb2398ff4c531d992458..c48294b3c8ef99544400384651eb38f7688ee02c 100644
--- a/test/wal_off/alter.result
+++ b/test/wal_off/alter.result
@@ -28,7 +28,7 @@ end;
 ...
 #spaces;
 ---
-- 65516
+- 65515
 ...
 -- cleanup
 for k, v in pairs(spaces) do
diff --git a/test/xlog-py/misc.test.py b/test/xlog-py/misc.test.py
index 6dfc73c9bfd4e18df4f8a2f82892f5161596f65d..86e01287939a2c64b0cdff9bbb5d6b30b7e7d809 100644
--- a/test/xlog-py/misc.test.py
+++ b/test/xlog-py/misc.test.py
@@ -26,7 +26,7 @@ if os.access(wal, os.F_OK):
 server.admin("index = space:create_index('primary', { type = 'hash' })")
 
 server.stop()
-lsn += 2
+lsn += 3
 
 print """
 # a new xlog must be opened after regular termination.
diff --git a/test/xlog/panic_on_wal_error.result b/test/xlog/panic_on_wal_error.result
index 267b53406c41df7b80cb92ba34cc2c9875864488..e34b81e4f7d69af379d999f25de5867eebb316cd 100644
--- a/test/xlog/panic_on_wal_error.result
+++ b/test/xlog/panic_on_wal_error.result
@@ -162,7 +162,7 @@ box.info.replication[1].upstream.status
 ...
 box.info.replication[1].upstream.message
 ---
-- 'Missing .xlog file between LSN 6 {1: 6} and 8 {1: 8}'
+- 'Missing .xlog file between LSN 7 {1: 7} and 9 {1: 9}'
 ...
 box.space.test:select{}
 ---
diff --git a/test/xlog/upgrade.result b/test/xlog/upgrade.result
index ed2a90286a47f200df62bada3832fb31aa501638..0d16a90231990baaff9dc2cf94be92d731b5b66c 100644
--- a/test/xlog/upgrade.result
+++ b/test/xlog/upgrade.result
@@ -36,7 +36,7 @@ box.space._schema:select()
 ---
 - - ['cluster', '<server_uuid>']
   - ['max_id', 513]
-  - ['version', 1, 7, 2]
+  - ['version', 1, 7, 4]
 ...
 box.space._space:select()
 ---
@@ -73,6 +73,8 @@ box.space._space:select()
         'type': 'num'}, {'name': 'privilege', 'type': 'num'}]]
   - [320, 1, '_cluster', 'memtx', 0, {}, [{'name': 'id', 'type': 'num'}, {'name': 'uuid',
         'type': 'str'}]]
+  - [330, 1, '_truncate', 'memtx', 0, {}, [{'name': 'id', 'type': 'num'}, {'name': 'count',
+        'type': 'num'}]]
   - [512, 1, 'distro', 'memtx', 0, {}, [{'name': 'os', 'type': 'str'}, {'name': 'dist',
         'type': 'str'}, {'name': 'version', 'type': 'num'}, {'name': 'time', 'type': 'num'}]]
   - [513, 1, 'temporary', 'memtx', 0, {'temporary': true}, []]
@@ -112,6 +114,7 @@ box.space._index:select()
   - [313, 2, 'object', 'tree', {'unique': false}, [[2, 'string'], [3, 'unsigned']]]
   - [320, 0, 'primary', 'tree', {'unique': true}, [[0, 'unsigned']]]
   - [320, 1, 'uuid', 'tree', {'unique': true}, [[1, 'string']]]
+  - [330, 0, 'primary', 'tree', {'unique': true}, [[0, 'unsigned']]]
   - [512, 0, 'primary', 'hash', {'unique': true}, [[0, 'string'], [1, 'string'], [
         2, 'unsigned']]]
   - [512, 1, 'codename', 'hash', {'unique': true}, [[1, 'string']]]