diff --git a/cmake/module.cmake b/cmake/module.cmake
index c8015a0417483c0bd65484edc643b8abfecec50e..8338be7b6c1f1a2944bcc633113a93d997046994 100644
--- a/cmake/module.cmake
+++ b/cmake/module.cmake
@@ -15,10 +15,14 @@ function(rebuild_module_api)
     add_custom_command(OUTPUT ${dstfile}
         COMMAND cat ${CMAKE_CURRENT_SOURCE_DIR}/tarantool_header.h > ${tmpfile}
         COMMAND cat ${headers} | ${CMAKE_SOURCE_DIR}/extra/apigen >> ${tmpfile}
+        COMMAND ${CMAKE_C_COMPILER}
+            -I ${CMAKE_SOURCE_DIR}/src -I ${CMAKE_BINARY_DIR}/src
+            -E ${CMAKE_SOURCE_DIR}/src/box/errcode.h |
+            grep "enum box_error_code" >> ${tmpfile}
         COMMAND cat ${CMAKE_CURRENT_SOURCE_DIR}/tarantool_footer.h >> ${tmpfile}
         COMMAND ${CMAKE_COMMAND} -E copy_if_different ${tmpfile} ${dstfile}
         COMMAND ${CMAKE_COMMAND} -E remove ${tmpfile}
-        DEPENDS ${srcfiles}
+        DEPENDS ${srcfiles} ${CMAKE_SOURCE_DIR}/src/box/errcode.h
                 ${CMAKE_CURRENT_SOURCE_DIR}/tarantool_header.h
                 ${CMAKE_CURRENT_SOURCE_DIR}/tarantool_footer.h
         )
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index bfdb80469f58ac693374dad63f367b216982340e..ed9aa3d8e3a7d9e22c4a8717032f92148c609e11 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -72,6 +72,7 @@ set (server_sources
      pickle.cc
      stat.cc
      ipc.cc
+     latch.cc
      errinj.cc
      fio.c
      crc32.c
diff --git a/src/box/alter.cc b/src/box/alter.cc
index 374bce01609c0944577c0ed5c76b1b152f8fcafa..e759e00ba55804f245378fe55f6a14e7c3bdaf99 100644
--- a/src/box/alter.cc
+++ b/src/box/alter.cc
@@ -1189,7 +1189,7 @@ space_has_data(uint32_t id, uint32_t iid, uint32_t uid)
 		return false;
 	struct iterator *it = index->position();
 	char key[6];
-	assert(mp_sizeof_uint(SC_SYSTEM_ID_MIN) <= sizeof(key));
+	assert(mp_sizeof_uint(BOX_SYSTEM_ID_MIN) <= sizeof(key));
 	mp_encode_uint(key, uid);
 
 	index->initIterator(it, ITER_EQ, key, 1);
@@ -1203,7 +1203,7 @@ bool
 user_has_data(struct user *user)
 {
 	uint32_t uid = user->uid;
-	uint32_t spaces[] = { SC_SPACE_ID, SC_FUNC_ID, SC_PRIV_ID, SC_PRIV_ID };
+	uint32_t spaces[] = { BOX_SPACE_ID, BOX_FUNC_ID, BOX_PRIV_ID, BOX_PRIV_ID };
 	/*
 	 * owner index id #1 for _space and _func and _priv.
 	 * For _priv also check that the user has no grants.
diff --git a/src/box/box.cc b/src/box/box.cc
index cba32f16a6ce61c29c4137c89974f868839f19fe..9f872196dfeb4872bf57d62bee555c34108be49d 100644
--- a/src/box/box.cc
+++ b/src/box/box.cc
@@ -303,6 +303,190 @@ boxk(enum iproto_type type, uint32_t space_id, const char *format, ...)
 	process_rw(&req, &null_port);
 }
 
+int
+box_return_tuple(box_function_ctx_t *ctx, box_tuple_t *tuple)
+{
+	try {
+		port_add_tuple(ctx->port, tuple);
+		return 0;
+	} catch (Exception *e) {
+		return -1;
+	}
+}
+
+/* schema_find_id()-like method using only public API */
+uint32_t
+box_space_id_by_name(const char *name, uint32_t len)
+{
+	char buf[1 + 5 + BOX_NAME_MAX + 5];
+	if (len > BOX_NAME_MAX)
+		return BOX_ID_NIL;
+
+	char *p = buf;
+	p = mp_encode_array(p, 1);
+	p = mp_encode_str(p, name, len);
+	assert(p < buf + sizeof(buf));
+
+	/* NOTE: error and missing key cases are indistinguishable */
+	box_tuple_t *tuple;
+	if (box_index_get(BOX_VSPACE_ID, 2, buf, p, &tuple) != 0)
+		return BOX_ID_NIL;
+	if (tuple == NULL)
+		return BOX_ID_NIL;
+	return box_tuple_field_u32(tuple, 0, BOX_ID_NIL);
+}
+
+uint32_t
+box_index_id_by_name(uint32_t space_id, const char *name, uint32_t len)
+{
+	char buf[1 + 5 + BOX_NAME_MAX + 5];
+	if (len > BOX_NAME_MAX)
+		return BOX_ID_NIL;
+
+	char *p = buf;
+	p = mp_encode_array(p, 2);
+	p = mp_encode_uint(p, space_id);
+	p = mp_encode_str(p, name, len);
+	assert(p < buf + sizeof(buf));
+
+	/* NOTE: error and missing key cases are indistinguishable */
+	box_tuple_t *tuple;
+	if (box_index_get(BOX_VINDEX_ID, 2, buf, p, &tuple) != 0)
+		return BOX_ID_NIL;
+	if (tuple == NULL)
+		return BOX_ID_NIL;
+	return box_tuple_field_u32(tuple, 1, BOX_ID_NIL);
+}
+/** \endcond public */
+
+static inline int
+box_process1(struct request *request, box_tuple_t **result)
+{
+	struct port_buf port_buf;
+	port_buf_create(&port_buf);
+	try {
+		box_process(request, &port_buf.base);
+		box_tuple_t *tuple = NULL;
+		/* Sophia: always bless tuple even if result is NULL */
+		if (port_buf.first != NULL)
+			tuple = tuple_bless(port_buf.first->tuple);
+		port_buf_destroy(&port_buf);
+		if (result)
+			*result = tuple;
+		return 0;
+	} catch (Exception *e) {
+		port_buf_destroy(&port_buf);
+		return -1;
+	}
+}
+
+int
+box_select(struct port *port, uint32_t space_id, uint32_t index_id,
+	   int iterator, uint32_t offset, uint32_t limit,
+	   const char *key, const char *key_end)
+{
+	struct request request;
+	request_create(&request, IPROTO_SELECT);
+	request.space_id = space_id;
+	request.index_id = index_id;
+	request.limit = limit;
+	request.offset = offset;
+	request.iterator = iterator;
+	request.key = key;
+	request.key_end = key_end;
+
+	try {
+		box_process(&request, port);
+		return 0;
+	} catch (Exception *e) {
+		/* will be hanled by box.error() in Lua */
+		return -1;
+	}
+}
+
+int
+box_insert(uint32_t space_id, const char *tuple, const char *tuple_end,
+	   box_tuple_t **result)
+{
+	mp_tuple_assert(tuple, tuple_end);
+	struct request request;
+	request_create(&request, IPROTO_INSERT);
+	request.space_id = space_id;
+	request.tuple = tuple;
+	request.tuple_end = tuple_end;
+	return box_process1(&request, result);
+}
+
+int
+box_replace(uint32_t space_id, const char *tuple, const char *tuple_end,
+	    box_tuple_t **result)
+{
+	mp_tuple_assert(tuple, tuple_end);
+	struct request request;
+	request_create(&request, IPROTO_REPLACE);
+	request.space_id = space_id;
+	request.tuple = tuple;
+	request.tuple_end = tuple_end;
+	return box_process1(&request, result);
+}
+
+int
+box_delete(uint32_t space_id, uint32_t index_id, const char *key,
+	   const char *key_end, box_tuple_t **result)
+{
+	mp_tuple_assert(key, key_end);
+	struct request request;
+	request_create(&request, IPROTO_DELETE);
+	request.space_id = space_id;
+	request.index_id = index_id;
+	request.key = key;
+	request.key_end = key_end;
+	return box_process1(&request, result);
+}
+
+int
+box_update(uint32_t space_id, uint32_t index_id, const char *key,
+	   const char *key_end, const char *ops, const char *ops_end,
+	   int index_base, box_tuple_t **result)
+{
+	mp_tuple_assert(key, key_end);
+	mp_tuple_assert(ops, ops_end);
+	struct request request;
+	request_create(&request, IPROTO_UPDATE);
+	request.space_id = space_id;
+	request.index_id = index_id;
+	request.key = key;
+	request.key_end = key_end;
+	request.index_base = index_base;
+	/** Legacy: in case of update, ops are passed in in request tuple */
+	request.tuple = ops;
+	request.tuple_end = ops_end;
+	return box_process1(&request, result);
+}
+
+int
+box_upsert(uint32_t space_id, uint32_t index_id, const char *key,
+	   const char *key_end, const char *ops, const char *ops_end,
+	   const char *tuple, const char *tuple_end, int index_base,
+	   box_tuple_t **result)
+{
+	mp_tuple_assert(key, key_end);
+	mp_tuple_assert(ops, ops_end);
+	mp_tuple_assert(tuple, tuple_end);
+	struct request request;
+	request_create(&request, IPROTO_UPSERT);
+	request.space_id = space_id;
+	request.index_id = index_id;
+	request.key = key;
+	request.key_end = key_end;
+	request.ops = ops;
+	request.ops_end = ops_end;
+	request.tuple = tuple;
+	request.tuple_end = tuple_end;
+	request.index_base = index_base;
+	return box_process1(&request, result);
+}
+
 /**
  * @brief Called when recovery/replication wants to add a new server
  * to cluster.
@@ -314,7 +498,7 @@ static void
 box_on_cluster_join(const tt_uuid *server_uuid)
 {
 	/** Find the largest existing server id. */
-	struct space *space = space_cache_find(SC_CLUSTER_ID);
+	struct space *space = space_cache_find(BOX_CLUSTER_ID);
 	class Index *index = index_find(space, 0);
 	struct iterator *it = index->position();
 	index->initIterator(it, ITER_LE, NULL, 0);
@@ -325,7 +509,7 @@ box_on_cluster_join(const tt_uuid *server_uuid)
 	if (server_id >= VCLOCK_MAX)
 		tnt_raise(LoggedError, ER_REPLICA_MAX, server_id);
 
-	boxk(IPROTO_INSERT, SC_CLUSTER_ID, "%u%s",
+	boxk(IPROTO_INSERT, BOX_CLUSTER_ID, "%u%s",
 	     (unsigned) server_id, tt_uuid_str(server_uuid));
 }
 
@@ -334,7 +518,7 @@ box_process_join(int fd, struct xrow_header *header)
 {
 	/* Check permissions */
 	access_check_universe(PRIV_R);
-	access_check_space(space_cache_find(SC_CLUSTER_ID), PRIV_W);
+	access_check_space(space_cache_find(BOX_CLUSTER_ID), PRIV_W);
 
 	assert(header->type == IPROTO_JOIN);
 
@@ -361,7 +545,7 @@ box_set_server_uuid()
 	assert(r->server_id == 0);
 	if (vclock_has(&r->vclock, 1))
 		vclock_del_server(&r->vclock, 1);
-	boxk(IPROTO_REPLACE, SC_CLUSTER_ID, "%u%s",
+	boxk(IPROTO_REPLACE, BOX_CLUSTER_ID, "%u%s",
 	     1, tt_uuid_str(&r->server_uuid));
 	/* Remove surrogate server */
 	vclock_del_server(&r->vclock, 0);
@@ -377,7 +561,7 @@ box_set_cluster_uuid()
 	/* Generate a new cluster UUID */
 	tt_uuid_create(&uu);
 	/* Save cluster UUID in _schema */
-	boxk(IPROTO_REPLACE, SC_SCHEMA_ID, "%s%s", "cluster",
+	boxk(IPROTO_REPLACE, BOX_SCHEMA_ID, "%s%s", "cluster",
 	     tt_uuid_str(&uu));
 }
 
diff --git a/src/box/box.h b/src/box/box.h
index 231da1adbf421a924b02387eef58a8fceab1a683..3ca093176c1f0b6d57c2d033a47cdcf8a4df493c 100644
--- a/src/box/box.h
+++ b/src/box/box.h
@@ -117,4 +117,55 @@ extern struct recovery_state *recovery;
 }
 #endif /* defined(__cplusplus) */
 
+struct request;
+struct port;
+struct box_function_ctx {
+	struct request *request;
+	struct port *port;
+};
+
+typedef struct tuple box_tuple_t;
+
+/* box_select is private and used only by FFI */
+API_EXPORT int
+box_select(struct port *port, uint32_t space_id, uint32_t index_id,
+	   int iterator, uint32_t offset, uint32_t limit,
+	   const char *key, const char *key_end);
+
+/** \cond public */
+typedef struct box_function_ctx box_function_ctx_t;
+API_EXPORT int
+box_return_tuple(box_function_ctx_t *ctx, box_tuple_t *tuple);
+
+API_EXPORT uint32_t
+box_space_id_by_name(const char *name, uint32_t len);
+
+API_EXPORT uint32_t
+box_index_id_by_name(uint32_t space_id, const char *name, uint32_t len);
+
+API_EXPORT int
+box_insert(uint32_t space_id, const char *tuple, const char *tuple_end,
+	   box_tuple_t **result);
+
+API_EXPORT int
+box_replace(uint32_t space_id, const char *tuple, const char *tuple_end,
+	    box_tuple_t **result);
+
+API_EXPORT int
+box_delete(uint32_t space_id, uint32_t index_id, const char *key,
+	   const char *key_end, box_tuple_t **result);
+
+API_EXPORT int
+box_update(uint32_t space_id, uint32_t index_id, const char *key,
+	   const char *key_end, const char *ops, const char *ops_end,
+	   int index_base, box_tuple_t **result);
+
+API_EXPORT int
+box_upsert(uint32_t space_id, uint32_t index_id, const char *key,
+	   const char *key_end, const char *ops, const char *ops_end,
+	   const char *tuple, const char *tuple_end, int index_base,
+	   box_tuple_t **result);
+
+/** \endcond public */
+
 #endif /* INCLUDES_TARANTOOL_BOX_H */
diff --git a/src/box/errcode.c b/src/box/errcode.c
index 0d7c8e53e5f00891e9bdeb11e7e6da6e849115ff..53eacf9dc48368081dfa5ef972955d76287deacc 100644
--- a/src/box/errcode.c
+++ b/src/box/errcode.c
@@ -37,7 +37,7 @@
 	.errdesc = d				\
 },
 
-struct errcode_record tnt_error_codes[tnt_error_codes_enum_MAX] = {
+struct errcode_record box_error_codes[box_error_code_MAX] = {
 	ERROR_CODES(ERRCODE_RECORD_MEMBER)
 };
 
diff --git a/src/box/errcode.h b/src/box/errcode.h
index 05dca1420782f38460aae8029125de4c43ffbf10..13a776a7242f59cba8fcc84c4f939ec0f5041597 100644
--- a/src/box/errcode.h
+++ b/src/box/errcode.h
@@ -154,35 +154,36 @@ struct errcode_record {
 	/* 98 */_(ER_UNSUPPORTED_ROLE_PRIV,     2, "Unsupported role privilege '%s'") \
 	/* 99 */_(ER_LOAD_FUNCTION,		2, "Failed to dynamically load function '%s': %s") \
 	/*100 */_(ER_FUNCTION_LANGUAGE,		2, "Unsupported language '%s' specified for function '%s'") \
-	/*101 */_(ER_RTREE_RECT_ERROR,		2, "RTree: %s must be an array with %u (point) or %u (rectangle/box) numeric coordinates") \
+	/*101 */_(ER_RTREE_RECT,		2, "RTree: %s must be an array with %u (point) or %u (rectangle/box) numeric coordinates") \
+	/*102 */_(ER_PROC_C,			2, "%s") \
 
 /*
  * !IMPORTANT! Please follow instructions at start of the file
  * when adding new errors.
  */
 
-ENUM0(tnt_error_codes_enum, ERROR_CODES);
-extern struct errcode_record tnt_error_codes[];
+ENUM0(box_error_code, ERROR_CODES);
+extern struct errcode_record box_error_codes[];
 
 /** Return a string representation of error name, e.g. "ER_OK".
  */
 
 static inline const char *tnt_errcode_str(uint32_t errcode)
 {
-	if (errcode >= tnt_error_codes_enum_MAX) {
+	if (errcode >= box_error_code_MAX) {
 		/* Unknown error code - can be triggered using box.error() */
 		return "ER_UNKNOWN";
 	}
-	return tnt_error_codes[errcode].errstr;
+	return box_error_codes[errcode].errstr;
 }
 
 /** Return a description of the error. */
 static inline const char *tnt_errcode_desc(uint32_t errcode)
 {
-	if (errcode >= tnt_error_codes_enum_MAX)
+	if (errcode >= box_error_code_MAX)
 		return "Unknown error";
 
-	return tnt_error_codes[errcode].errdesc;
+	return box_error_codes[errcode].errdesc;
 }
 
 #ifdef __cplusplus
diff --git a/src/box/error.cc b/src/box/error.cc
index 2812623acf1f2312d2f5946bff2260601a4ec42e..789ffc6f6287941fb7fbb7e0d06b9f9180d14e6f 100644
--- a/src/box/error.cc
+++ b/src/box/error.cc
@@ -31,6 +31,8 @@
 #include "error.h"
 #include <stdio.h>
 
+#include <fiber.h>
+
 static struct method clienterror_methods[] = {
 	make_method(&type_ClientError, "code", &ClientError::errcode),
 	METHODS_SENTINEL
@@ -83,3 +85,49 @@ ErrorInjection::ErrorInjection(const char *file, unsigned line, const char *msg)
 	/* nothing */
 }
 
+const char *
+box_error_type(const box_error_t *error)
+{
+	Exception *e = (Exception *) error;
+	return e->type->name;
+}
+
+uint32_t
+box_error_code(const box_error_t *error)
+{
+	Exception *e = (Exception *) error;
+	return ClientError::get_errcode(e);
+}
+
+const char *
+box_error_message(const box_error_t *error)
+{
+	Exception *e = (Exception *) error;
+	return e->errmsg();
+}
+
+const box_error_t *
+box_error_last(void)
+{
+	return (box_error_t *) diag_last_error(&fiber()->diag);
+}
+
+void
+box_error_clear(void)
+{
+	diag_clear(&fiber()->diag);
+}
+
+int
+box_error_raise(uint32_t code, const char *fmt, ...)
+{
+	char msg[EXCEPTION_ERRMSG_MAX];
+
+	va_list ap;
+	va_start(ap, fmt);
+	vsnprintf(msg, sizeof(msg), fmt, ap);
+	va_end(ap);
+
+	tnt_error(ClientError, msg, code);
+	return -1;
+}
diff --git a/src/box/error.h b/src/box/error.h
index 32e5da12aa27bfc23d368d080189ceb783d929dd..1e3deab61be4ee4e725fff40a113ef180a165c5b 100644
--- a/src/box/error.h
+++ b/src/box/error.h
@@ -85,4 +85,46 @@ class ErrorInjection: public LoggedError {
 	ErrorInjection(const char *file, unsigned line, const char *msg);
 };
 
+/** \cond public */
+struct box_error;
+typedef struct box_error box_error_t;
+
+/**
+ * Return error type, e.g. "ClientError", "SocketError", etc.
+ */
+API_EXPORT const char *
+box_error_type(const box_error_t *error);
+
+/*
+ * Return IPROTO error code
+ */
+API_EXPORT uint32_t
+box_error_code(const box_error_t *error);
+
+/*
+ * Return error message
+ */
+API_EXPORT const char *
+box_error_message(const box_error_t *error);
+
+/**
+ * Return last error
+ */
+API_EXPORT const box_error_t *
+box_error_last(void);
+
+/*
+ * Clear last error
+ */
+API_EXPORT void
+box_error_clear(void);
+
+/*
+ * Set last error
+ * \param code IPROTO error code
+ */
+API_EXPORT int
+box_error_raise(uint32_t code, const char *fmt, ...);
+/** \endcond public */
+
 #endif /* TARANTOOL_BOX_ERROR_H_INCLUDED */
diff --git a/src/box/func.h b/src/box/func.h
index 22d0b5a17833f2d81d7471c5ecfc25fe0f9b2b7b..99bc9d49fe6091ee18d4cb50525b79ae422be810 100644
--- a/src/box/func.h
+++ b/src/box/func.h
@@ -31,22 +31,7 @@
  * SUCH DAMAGE.
  */
 #include "key_def.h"
-
-extern "C" {
-
-/** \cond public */
-struct port;
-struct request;
-
-/**
- * API of C stored function.
- */
-typedef int (*box_function_f)(struct request *request,
-			      struct port *port);
-
-/** \endcond public */
-} /* extern "C" */
-
+#include "request.h"
 
 /**
  * Stored function.
diff --git a/src/box/index.cc b/src/box/index.cc
index 243b1bf20fba649ab0a461d7f219f9dbc7145d51..c5d70d48492e1d3dc079cf8f9e331aa072c84e2c 100644
--- a/src/box/index.cc
+++ b/src/box/index.cc
@@ -32,8 +32,26 @@
 #include "tuple.h"
 #include "say.h"
 #include "schema.h"
-
-STRS(iterator_type, ITERATOR_TYPE);
+#include "user_def.h"
+#include "space.h"
+
+const char *iterator_type_strs[] = {
+	/* [ITER_EQ]  = */ "EQ",
+	/* [ITER_REQ]  = */ "REQ",
+	/* [ITER_ALL] = */ "ALL",
+	/* [ITER_LT]  = */ "LT",
+	/* [ITER_LE]  = */ "LE",
+	/* [ITER_GE]  = */ "GE",
+	/* [ITER_GT]  = */ "GT",
+	/* [ITER_BITS_ALL_SET] = */ "BITS_ALL_SET",
+	/* [ITER_BITS_ANY_SET] = */ "BITS_ANY_SET",
+	/* [ITER_BITS_ALL_NOT_SET] = */ "BITS_ALL_NOT_SET",
+	/* [ITER_OVERLAPS] = */ "OVERLAPS",
+	/* [ITER_NEIGHBOR] = */ "NEIGHBOR",
+};
+
+static_assert(sizeof(iterator_type_strs) / sizeof(const char *) ==
+	iterator_type_MAX, "iterator_type_str constants");
 
 /* {{{ Utilities. **********************************************/
 
@@ -81,7 +99,7 @@ key_validate(struct key_def *key_def, enum iterator_type type, const char *key,
 			key_mp_type_validate(ARRAY, mp_type, ER_KEY_PART_TYPE, 0);
 			uint32_t array_size = mp_decode_array(&key);
 			if (array_size != d && array_size != d * 2)
-				tnt_raise(ClientError, ER_RTREE_RECT_ERROR,
+				tnt_raise(ClientError, ER_RTREE_RECT,
 					  "Key", d, d * 2);
 			for (uint32_t part = 0; part < array_size; part++) {
 				enum mp_type mp_type = mp_typeof(*key);
@@ -275,4 +293,207 @@ index_build(Index *index, Index *pk)
 	index->endBuild();
 }
 
+static inline Index *
+check_index(uint32_t space_id, uint32_t index_id)
+{
+	struct space *space = space_cache_find(space_id);
+	access_check_space(space, PRIV_R);
+	return index_find(space, index_id);
+}
+
+static inline box_tuple_t *
+tuple_bless_null(struct tuple *tuple)
+{
+	if (tuple != NULL)
+		return tuple_bless(tuple);
+	return NULL;
+}
+
+size_t
+box_index_len(uint32_t space_id, uint32_t index_id)
+{
+	try {
+		return check_index(space_id, index_id)->size();
+	} catch (Exception *) {
+		return (size_t) -1; /* handled by box.error() in Lua */
+	}
+}
+
+size_t
+box_index_bsize(uint32_t space_id, uint32_t index_id)
+{
+       try {
+               return check_index(space_id, index_id)->bsize();
+       } catch (Exception *) {
+               return (size_t) -1; /* handled by box.error() in Lua */
+       }
+}
+
+int
+box_index_random(uint32_t space_id, uint32_t index_id, uint32_t rnd,
+		box_tuple_t **result)
+{
+	assert(result != NULL);
+	try {
+		Index *index = check_index(space_id, index_id);
+		struct tuple *tuple = index->random(rnd);
+		*result = tuple_bless_null(tuple);
+		return 0;
+	}  catch (Exception *) {
+		return -1;
+	}
+}
+
+int
+box_index_get(uint32_t space_id, uint32_t index_id, const char *key,
+	      const char *key_end, box_tuple_t **result)
+{
+	mp_tuple_assert(key, key_end);
+	assert(result != NULL);
+	try {
+		Index *index = check_index(space_id, index_id);
+		if (!index->key_def->opts.is_unique)
+			tnt_raise(ClientError, ER_MORE_THAN_ONE_TUPLE);
+		uint32_t part_count = key ? mp_decode_array(&key) : 0;
+		primary_key_validate(index->key_def, key, part_count);
+		struct tuple *tuple = index->findByKey(key, part_count);
+		*result = tuple_bless_null(tuple);
+		return 0;
+	}  catch (Exception *) {
+		return -1;
+	}
+}
+
+int
+box_index_min(uint32_t space_id, uint32_t index_id, const char *key,
+	      const char *key_end, box_tuple_t **result)
+{
+	mp_tuple_assert(key, key_end);
+	assert(result != NULL);
+	try {
+		Index *index = check_index(space_id, index_id);
+		if (index->key_def->type != TREE) {
+			/* Show nice error messages in Lua */
+			tnt_raise(ClientError, ER_UNSUPPORTED,
+				  index_type_strs[index->key_def->type],
+				  "min()");
+		}
+		uint32_t part_count = key ? mp_decode_array(&key) : 0;
+		key_validate(index->key_def, ITER_GE, key, part_count);
+		struct tuple *tuple = index->min(key, part_count);
+		*result = tuple_bless_null(tuple);
+		return 0;
+	}  catch (Exception *) {
+		return -1;
+	}
+}
+
+int
+box_index_max(uint32_t space_id, uint32_t index_id, const char *key,
+	      const char *key_end, box_tuple_t **result)
+{
+	mp_tuple_assert(key, key_end);
+	assert(result != NULL);
+	try {
+		Index *index = check_index(space_id, index_id);
+		if (index->key_def->type != TREE) {
+			/* Show nice error messages in Lua */
+			tnt_raise(ClientError, ER_UNSUPPORTED,
+				  index_type_strs[index->key_def->type],
+				  "max()");
+		}
+		uint32_t part_count = key ? mp_decode_array(&key) : 0;
+		key_validate(index->key_def, ITER_LE, key, part_count);
+		struct tuple *tuple = index->max(key, part_count);
+		*result = tuple_bless_null(tuple);
+		return 0;
+	}  catch (Exception *) {
+		return -1;
+	}
+}
+
+ssize_t
+box_index_count(uint32_t space_id, uint32_t index_id, int type,
+		const char *key, const char *key_end)
+{
+	mp_tuple_assert(key, key_end);
+	enum iterator_type itype = (enum iterator_type) type;
+	try {
+		Index *index = check_index(space_id, index_id);
+		uint32_t part_count = key ? mp_decode_array(&key) : 0;
+		key_validate(index->key_def, itype, key, part_count);
+		return index->count(itype, key, part_count);
+	} catch (Exception *) {
+		return -1; /* handled by box.error() in Lua */
+	}
+}
+
+/* }}} */
+
+/* {{{ Iterators ************************************************/
+
+box_iterator_t *
+box_index_iterator(uint32_t space_id, uint32_t index_id, int type,
+                   const char *key, const char *key_end)
+{
+	mp_tuple_assert(key, key_end);
+	struct iterator *it = NULL;
+	enum iterator_type itype = (enum iterator_type) type;
+	try {
+		Index *index = check_index(space_id, index_id);
+		assert(mp_typeof(*key) == MP_ARRAY); /* checked by Lua */
+		uint32_t part_count = mp_decode_array(&key);
+		key_validate(index->key_def, itype, key, part_count);
+		it = index->allocIterator();
+		index->initIterator(it, itype, key, part_count);
+		it->sc_version = sc_version;
+		it->space_id = space_id;
+		it->index_id = index_id;
+		it->index = index;
+		return it;
+	} catch (Exception *) {
+		if (it)
+			it->free(it);
+		/* will be hanled by box.error() in Lua */
+		return NULL;
+	}
+}
+
+int
+box_iterator_next(box_iterator_t *itr, box_tuple_t **result)
+{
+	assert(result != NULL);
+	try {
+		if (itr->sc_version != sc_version) {
+			Index *index = check_index(itr->space_id, itr->index_id);
+			if (index != itr->index) {
+				*result = NULL;
+				return 0;
+			}
+			if (index->sc_version > itr->sc_version) {
+				*result = NULL; /* invalidate iterator */
+				return 0;
+			}
+			itr->sc_version = sc_version;
+		}
+	} catch (Exception *) {
+		*result = NULL;
+		return 0; /* invalidate iterator */
+	}
+	try {
+		struct tuple *tuple = itr->next(itr);
+		*result = tuple_bless_null(tuple);
+		return 0;
+	} catch (Exception *) {
+		return -1;
+	}
+}
+
+void
+box_iterator_free(box_iterator_t *itr)
+{
+	if (itr->free)
+		itr->free(itr);
+}
+
 /* }}} */
diff --git a/src/box/index.h b/src/box/index.h
index f57dc337bfd12819643d7d46c70b5ec87070cbee..1e03b8f728e338a7554e235cd6aebb05d37b4412 100644
--- a/src/box/index.h
+++ b/src/box/index.h
@@ -36,7 +36,9 @@
 #include "object.h"
 #include "key_def.h"
 
-struct tuple;
+/** \cond public */
+typedef struct tuple box_tuple_t;
+typedef struct iterator box_iterator_t;
 
 /**
  * @abstract Iterator type
@@ -59,29 +61,36 @@ struct tuple;
  * to NULL.
  * For ITER_EQ, the key must not be NULL.
  */
-#define ITERATOR_TYPE(_)                                             \
-	/* ITER_EQ must be the first member for request_create  */   \
-	_(ITER_EQ,  0)       /* key == x ASC order              */   \
-	_(ITER_REQ, 1)       /* key == x DESC order             */   \
-	_(ITER_ALL, 2)       /* all tuples                      */   \
-	_(ITER_LT,  3)       /* key <  x                        */   \
-	_(ITER_LE,  4)       /* key <= x                        */   \
-	_(ITER_GE,  5)       /* key >= x                        */   \
-	_(ITER_GT,  6)       /* key >  x                        */   \
-	_(ITER_BITS_ALL_SET,     7) /* all bits from x are set in key      */ \
-	_(ITER_BITS_ANY_SET,     8) /* at least one x's bit is set         */ \
-	_(ITER_BITS_ALL_NOT_SET, 9) /* all bits are not set                */ \
-	_(ITER_OVERLAPS, 10) /* key overlaps x */ \
-	_(ITER_NEIGHBOR, 11) /* typles in distance ascending order from specified point */ \
-
-ENUM(iterator_type, ITERATOR_TYPE);
-extern const char *iterator_type_strs[];
 
-static inline bool
-iterator_type_is_reverse(enum iterator_type type)
-{
-	return type == ITER_REQ || type == ITER_LT || type == ITER_LE;
-}
+enum iterator_type {
+	/* ITER_EQ must be the first member for request_create  */
+	ITER_EQ               =  0, /* key == x ASC order                  */
+	ITER_REQ              =  1, /* key == x DESC order                 */
+	ITER_ALL              =  2, /* all tuples                          */
+	ITER_LT               =  3, /* key <  x                            */
+	ITER_LE               =  4, /* key <= x                            */
+	ITER_GE               =  5, /* key >= x                            */
+	ITER_GT               =  6, /* key >  x                            */
+	ITER_BITS_ALL_SET     =  7, /* all bits from x are set in key      */
+	ITER_BITS_ANY_SET     =  8, /* at least one x's bit is set         */
+	ITER_BITS_ALL_NOT_SET =  9, /* all bits are not set                */
+	ITER_OVERLAPS         = 10, /* key overlaps x                      */
+	ITER_NEIGHBOR         = 11, /* typles in distance ascending order from specified point */
+	iterator_type_MAX     = ITER_NEIGHBOR + 1
+};
+
+API_EXPORT box_iterator_t *
+box_index_iterator(uint32_t space_id, uint32_t index_id, int type,
+		   const char *key, const char *key_end);
+API_EXPORT int
+box_iterator_next(box_iterator_t *itr, box_tuple_t **result);
+
+API_EXPORT void
+box_iterator_free(box_iterator_t *itr);
+
+/** \endcond public */
+
+extern const char *iterator_type_strs[];
 
 struct iterator {
 	struct tuple *(*next)(struct iterator *);
@@ -94,10 +103,10 @@ struct iterator {
 	class Index *index;
 };
 
-static inline void
-iterator_close(struct iterator *it) {
-	if (it->close)
-		it->close(it);
+static inline bool
+iterator_type_is_reverse(enum iterator_type type)
+{
+	return type == ITER_REQ || type == ITER_LT || type == ITER_LE;
 }
 
 /**
@@ -228,7 +237,8 @@ struct IteratorGuard: public Object {
 
 	~IteratorGuard()
 	{
-		iterator_close(it);
+		if (it->close)
+			it->close(it);
 	}
 };
 
@@ -288,4 +298,34 @@ index_is_primary(const Index *index)
 void
 index_build(Index *index, Index *pk);
 
+/** \cond public */
+
+API_EXPORT size_t
+box_index_len(uint32_t space_id, uint32_t index_id);
+
+API_EXPORT size_t
+box_index_bsize(uint32_t space_id, uint32_t index_id);
+
+API_EXPORT int
+box_index_random(uint32_t space_id, uint32_t index_id, uint32_t rnd,
+		box_tuple_t **result);
+
+API_EXPORT int
+box_index_get(uint32_t space_id, uint32_t index_id, const char *key,
+	      const char *key_end, box_tuple_t **result);
+
+API_EXPORT int
+box_index_min(uint32_t space_id, uint32_t index_id, const char *key,
+	      const char *key_end, box_tuple_t **result);
+
+API_EXPORT int
+box_index_max(uint32_t space_id, uint32_t index_id, const char *key,
+	      const char *key_end, box_tuple_t **result);
+
+API_EXPORT ssize_t
+box_index_count(uint32_t space_id, uint32_t index_id, int type,
+		const char *key, const char *key_end);
+
+/** \endcond public */
+
 #endif /* TARANTOOL_BOX_INDEX_H_INCLUDED */
diff --git a/src/box/lua/call.cc b/src/box/lua/call.cc
index b44986449047b8c465931bfc21bdecc37cdb3bf1..87e26cf4c0ca8c4d18d45d1db02b349ccdfe790d 100644
--- a/src/box/lua/call.cc
+++ b/src/box/lua/call.cc
@@ -188,93 +188,31 @@ lbox_process(lua_State *L)
 	return 1;
 }
 
-void
-lbox_request_create(struct request *request,
-		    struct lua_State *L, enum iproto_type type,
-		    int key, int tuple, int ops)
-{
-	request_create(request, type);
-	request->space_id = lua_tointeger(L, 1);
-	struct region *gc = &fiber()->gc;
-	struct mpstream stream;
-	mpstream_init(&stream, gc, region_reserve_cb, region_alloc_cb);
-
-	if (key > 0) {
-		size_t used = region_used(gc);
-		luamp_encode_tuple(L, luaL_msgpack_default, &stream, key);
-		mpstream_flush(&stream);
-		size_t key_len = region_used(gc) - used;
-		request->key = (char *) region_join(gc, key_len);
-		request->key_end = request->key + key_len;
-	}
-	if (tuple > 0) {
-		size_t used = region_used(gc);
-		/*
-		 * region_join() above could have allocated memory and
-		 * invalidated stream write position. Reset the
-		 * stream to avoid overwriting the key.
-		 */
-		mpstream_reset(&stream);
-		luamp_encode_tuple(L, luaL_msgpack_default, &stream, tuple);
-		mpstream_flush(&stream);
-		size_t tuple_len = region_used(gc) - used;
-		request->tuple = (char *) region_join(gc, tuple_len);
-		request->tuple_end = request->tuple + tuple_len;
-	}
-	if (ops > 0) {
-		size_t used = region_used(gc);
-		mpstream_reset(&stream);
-		luamp_encode_tuple(L, luaL_msgpack_default, &stream, ops);
-		mpstream_flush(&stream);
-		size_t ops_len = region_used(gc) - used;
-		request->ops = (char *) region_join(gc, ops_len);
-		request->ops_end = request->ops + ops_len;
-	}
-}
-
-int
-boxffi_select(struct port *port, uint32_t space_id, uint32_t index_id,
-	      int iterator, uint32_t offset, uint32_t limit,
-	      const char *key, const char *key_end)
-{
-	struct request request;
-	request_create(&request, IPROTO_SELECT);
-	request.space_id = space_id;
-	request.index_id = index_id;
-	request.limit = limit;
-	request.offset = offset;
-	request.iterator = iterator;
-	request.key = key;
-	request.key_end = key_end;
-
-	try {
-		box_process(&request, port);
-		return 0;
-	} catch (Exception *e) {
-		/* will be hanled by box.error() in Lua */
-		return -1;
-	}
-}
-
 static int
 lbox_select(lua_State *L)
 {
 	if (lua_gettop(L) != 6 || !lua_isnumber(L, 1) || !lua_isnumber(L, 2) ||
 		!lua_isnumber(L, 3) || !lua_isnumber(L, 4) || !lua_isnumber(L, 5)) {
-		return luaL_error(L, "Usage index:select(space_id, index_id,"
-			"iterator, offset, limit, key)");
+		return luaL_error(L, "Usage index:select(iterator, offset, "
+				  "limit, key)");
 	}
 
-	struct request request;
+	uint32_t space_id = lua_tointeger(L, 1);
+	uint32_t index_id = lua_tointeger(L, 2);
+	int iterator = lua_tointeger(L, 3);
+	uint32_t offset = lua_tointeger(L, 4);
+	uint32_t limit = lua_tointeger(L, 5);
+
+	size_t key_len;
+	const char *key = lbox_encode_tuple_on_gc(L, 6, &key_len);
+
+	int top = lua_gettop(L);
 	struct port_lua port;
-	lbox_request_create(&request, L, IPROTO_SELECT, 6, -1, -1);
-	request.index_id = lua_tointeger(L, 2);
-	request.iterator = lua_tointeger(L, 3);
-	request.offset = lua_tointeger(L, 4);
-	request.limit = lua_tointeger(L, 5);
 	port_lua_table_create(&port, L);
-	box_process(&request, (struct port *) &port);
-	return 1;
+	if (box_select((struct port *) &port, space_id, index_id, iterator,
+			offset, limit, key, key + key_len) != 0)
+		return lbox_error(L);
+	return lua_gettop(L) - top;
 }
 
 static int
@@ -283,12 +221,14 @@ lbox_insert(lua_State *L)
 	if (lua_gettop(L) != 2 || !lua_isnumber(L, 1))
 		return luaL_error(L, "Usage space:insert(tuple)");
 
-	struct request request;
-	struct port_lua port;
-	lbox_request_create(&request, L, IPROTO_INSERT, -1, 2, -1);
-	port_lua_create(&port, L);
-	box_process(&request, (struct port *) &port);
-	return lua_gettop(L) - 2;
+	uint32_t space_id = lua_tointeger(L, 1);
+	size_t tuple_len;
+	const char *tuple = lbox_encode_tuple_on_gc(L, 2, &tuple_len);
+
+	box_tuple_t *result;
+	if (box_insert(space_id, tuple, tuple + tuple_len, &result) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, result);
 }
 
 static int
@@ -297,12 +237,14 @@ lbox_replace(lua_State *L)
 	if (lua_gettop(L) != 2 || !lua_isnumber(L, 1))
 		return luaL_error(L, "Usage space:replace(tuple)");
 
-	struct request request;
-	struct port_lua port;
-	lbox_request_create(&request, L, IPROTO_REPLACE, -1, 2, -1);
-	port_lua_create(&port, L);
-	box_process(&request, (struct port *) &port);
-	return lua_gettop(L) - 2;
+	uint32_t space_id = lua_tointeger(L, 1);
+	size_t tuple_len;
+	const char *tuple = lbox_encode_tuple_on_gc(L, 2, &tuple_len);
+
+	box_tuple_t *result;
+	if (box_replace(space_id, tuple, tuple + tuple_len, &result) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, result);
 }
 
 static int
@@ -310,17 +252,20 @@ lbox_update(lua_State *L)
 {
 	if (lua_gettop(L) != 4 || !lua_isnumber(L, 1) || !lua_isnumber(L, 2) ||
 	    lua_type(L, 3) != LUA_TTABLE || lua_type(L, 4) != LUA_TTABLE)
-		return luaL_error(L, "Usage space:update(key, ops)");
-
-	struct request request;
-	struct port_lua port;
-	/** Legacy: in case of update, ops are passed in in request tuple */
-	lbox_request_create(&request, L, IPROTO_UPDATE, 3, 4, -1);
-	request.index_base = 1; /* field ids are one-indexed */
-	port_lua_create(&port, L);
-	/* Ignore index_id for now */
-	box_process(&request, (struct port *) &port);
-	return lua_gettop(L) - 4;
+		return luaL_error(L, "Usage index:update(key, ops)");
+
+	uint32_t space_id = lua_tointeger(L, 1);
+	uint32_t index_id = lua_tointeger(L, 2);
+	size_t key_len;
+	const char *key = lbox_encode_tuple_on_gc(L, 3, &key_len);
+	size_t ops_len;
+	const char *ops = lbox_encode_tuple_on_gc(L, 4, &ops_len);
+
+	box_tuple_t *result;
+	if (box_update(space_id, index_id, key, key + key_len,
+		       ops, ops + ops_len, 1, &result) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, result);
 }
 
 static int
@@ -329,51 +274,48 @@ lbox_upsert(lua_State *L)
 	if (lua_gettop(L) != 5 || !lua_isnumber(L, 1) || !lua_isnumber(L, 2) ||
 	    lua_type(L, 3) != LUA_TTABLE || lua_type(L, 4) != LUA_TTABLE ||
 	    lua_type(L, 5) != LUA_TTABLE)
-		return luaL_error(L, "Usage space:upsert(key, ops, tuple)");
-
-	struct request request;
-	struct port_lua port;
-	lbox_request_create(&request, L, IPROTO_UPSERT, 3, 5, 4);
-	request.index_base = 1; /* field ids are one-indexed */
-	port_lua_create(&port, L);
-	/* Ignore index_id for now */
-	box_process(&request, (struct port *) &port);
-	return lua_gettop(L) - 5;
+		return luaL_error(L, "Usage index:upsert(key, ops, tuple)");
+
+	uint32_t space_id = lua_tointeger(L, 1);
+	uint32_t index_id = lua_tointeger(L, 2);
+	size_t key_len;
+	const char *key = lbox_encode_tuple_on_gc(L, 3, &key_len);
+	size_t ops_len;
+	const char *ops = lbox_encode_tuple_on_gc(L, 4, &ops_len);
+	size_t tuple_len;
+	const char *tuple = lbox_encode_tuple_on_gc(L, 5, &tuple_len);
+
+	box_tuple_t *result;
+	if (box_upsert(space_id, index_id, key, key + key_len,
+		       ops, ops + ops_len, tuple, tuple + tuple_len, 1,
+		       &result) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, result);
 }
 
 static int
 lbox_delete(lua_State *L)
 {
-	if (lua_gettop(L) != 3 || !lua_isnumber(L, 1) || !lua_isnumber(L, 2))
+	if (lua_gettop(L) != 3 || !lua_isnumber(L, 1) || !lua_isnumber(L, 2) ||
+	    lua_type(L, 3) != LUA_TTABLE)
 		return luaL_error(L, "Usage space:delete(key)");
 
-	struct request request;
-	struct port_lua port;
-	lbox_request_create(&request, L, IPROTO_DELETE, 3, -1, -1);
-	port_lua_create(&port, L);
-	/* Ignore index_id for now */
-	box_process(&request, (struct port *) &port);
-	return lua_gettop(L) - 3;
+	uint32_t space_id = lua_tointeger(L, 1);
+	uint32_t index_id = lua_tointeger(L, 2);
+	size_t key_len;
+	const char *key = lbox_encode_tuple_on_gc(L, 3, &key_len);
+
+	box_tuple_t *result;
+	if (box_delete(space_id, index_id, key, key + key_len, &result) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, result);
 }
 
 static int
-lbox_commit(lua_State * /* L */)
+lbox_commit(lua_State *L)
 {
-	struct txn *txn = in_txn();
-	/**
-	 * COMMIT is like BEGIN or ROLLBACK
-	 * a "transaction-initiating statement".
-	 * Do nothing if transaction is not started,
-	 * it's the same as BEGIN + COMMIT.
-	*/
-	if (! txn)
-		return 0;
-	try {
-		txn_commit(txn);
-	} catch (...) {
-		txn_rollback();
-		throw;
-	}
+	if (box_txn_commit() != 0)
+		return lbox_error(L);
 	return 0;
 }
 
@@ -563,13 +505,25 @@ execute_c_call(struct func *func, struct request *request, struct obuf *out)
 		port_buf_destroy(&port_buf);
 	});
 
-	func->func(request, &port_buf.base);
-
+	box_function_ctx_t ctx = { request, &port_buf.base };
+	int rc = 0;
+	try {
+		rc = func->func(&ctx, request->tuple, request->tuple_end);
+	} catch (...) {
+		panic("C++ exception thrown from stored C function");
+	}
 	if (in_txn()) {
 		say_warn("a transaction is active at CALL return");
 		txn_rollback();
 	}
 
+	if (rc != 0) {
+		Exception *e = diag_last_error(&fiber()->diag);
+		if (e != NULL)
+			e->raise();
+		tnt_raise(ClientError, ER_PROC_C, "unknown procedure error");
+	}
+
 	struct obuf_svp svp = iproto_prepare_select(out);
 	try {
 		for (struct port_buf_entry *entry = port_buf.first;
diff --git a/src/box/lua/call.h b/src/box/lua/call.h
index 942342ac09722abf8c4976858029fe1b47fcd8d7..f0df15b9a3b6e8a8ba144873982ee580faa4eea7 100644
--- a/src/box/lua/call.h
+++ b/src/box/lua/call.h
@@ -32,9 +32,10 @@
  */
 
 #include <stdint.h>
+#include "trivia/util.h"
 
 struct request;
-struct port;
+struct obuf;
 
 /**
  * Invoke a Lua stored procedure from the binary protocol
@@ -46,13 +47,9 @@ box_lua_call(struct request *request, struct obuf *out);
 void
 box_lua_eval(struct request *request, struct obuf *out);
 
-extern "C" {
-
-int
+API_EXPORT int
 boxffi_select(struct port *port, uint32_t space_id, uint32_t index_id,
 	      int iterator, uint32_t offset, uint32_t limit,
 	      const char *key, const char *key_end);
 
-} /* extern "C" */
-
 #endif /* INCLUDES_TARANTOOL_MOD_BOX_LUA_CALL_H */
diff --git a/src/box/lua/error.cc b/src/box/lua/error.cc
index 4b7c57fdb5da31f2b69d3c0392a158a7f498aa88..95b4ff0a96a5984281648d9881315e6c9053f725 100644
--- a/src/box/lua/error.cc
+++ b/src/box/lua/error.cc
@@ -201,8 +201,8 @@ box_lua_error_init(struct lua_State *L) {
 		{NULL, NULL}
 	};
 	luaL_register_module(L, "box.error", errorlib);
-	for (int i = 0; i < tnt_error_codes_enum_MAX; i++) {
-		const char *name = tnt_error_codes[i].errstr;
+	for (int i = 0; i < box_error_code_MAX; i++) {
+		const char *name = box_error_codes[i].errstr;
 		if (strstr(name, "UNUSED") || strstr(name, "RESERVED"))
 			continue;
 		assert(strncmp(name, "ER_", 3) == 0);
diff --git a/src/box/lua/index.cc b/src/box/lua/index.cc
index 17e82f7909b8758fa69b51476c61ab9dd468259e..ddc8045c1fa9ee5af32729f5fa80ac00dda29110 100644
--- a/src/box/lua/index.cc
+++ b/src/box/lua/index.cc
@@ -32,10 +32,6 @@
 #include "lua/utils.h"
 #include "lua/msgpack.h"
 #include "box/index.h"
-#include "box/space.h"
-#include "box/schema.h"
-#include "box/user_def.h"
-#include "box/tuple.h"
 #include "box/lua/error.h"
 #include "box/lua/tuple.h"
 #include "fiber.h"
@@ -46,82 +42,6 @@
 
 static int CTID_STRUCT_ITERATOR_REF = 0;
 
-static inline Index *
-check_index(uint32_t space_id, uint32_t index_id)
-{
-	struct space *space = space_cache_find(space_id);
-	access_check_space(space, PRIV_R);
-	return index_find(space, index_id);
-}
-
-size_t
-boxffi_index_bsize(uint32_t space_id, uint32_t index_id)
-{
-       try {
-               return check_index(space_id, index_id)->bsize();
-       } catch (Exception *) {
-               return (size_t) -1; /* handled by box.error() in Lua */
-       }
-}
-
-size_t
-boxffi_index_len(uint32_t space_id, uint32_t index_id)
-{
-	try {
-		return check_index(space_id, index_id)->size();
-	} catch (Exception *) {
-		return (size_t) -1; /* handled by box.error() in Lua */
-	}
-}
-
-static inline int
-lbox_returntuple(lua_State *L, struct tuple *tuple)
-{
-	if (tuple == (struct tuple *) -1) {
-		return lbox_error(L);
-	} else if (tuple == NULL) {
-		lua_pushnil(L);
-		return 1;
-	} else {
-		lbox_pushtuple_noref(L, tuple);
-		return 1;
-	}
-}
-
-static inline struct tuple *
-boxffi_returntuple(struct tuple *tuple)
-{
-	if (tuple == NULL)
-		return NULL;
-	tuple_ref(tuple);
-	return tuple;
-}
-
-static inline char *
-lbox_tokey(lua_State *L, int idx)
-{
-	struct region *gc = &fiber()->gc;
-	size_t used = region_used(gc);
-	struct mpstream stream;
-	mpstream_init(&stream, gc, region_reserve_cb, region_alloc_cb);
-	luamp_encode_tuple(L, luaL_msgpack_default, &stream, idx);
-	mpstream_flush(&stream);
-	size_t key_len = region_used(gc) - used;
-	return (char *) region_join(gc, key_len);
-}
-
-struct tuple *
-boxffi_index_random(uint32_t space_id, uint32_t index_id, uint32_t rnd)
-{
-	try {
-		Index *index = check_index(space_id, index_id);
-		struct tuple *tuple = index->random(rnd);
-		return boxffi_returntuple(tuple);
-	}  catch (Exception *) {
-		return (struct tuple *) -1; /* handled by box.error() in Lua */
-	}
-}
-
 static int
 lbox_index_random(lua_State *L)
 {
@@ -133,24 +53,10 @@ lbox_index_random(lua_State *L)
 	uint32_t index_id = lua_tointeger(L, 2);
 	uint32_t rnd = lua_tointeger(L, 3);
 
-	struct tuple *tuple = boxffi_index_random(space_id, index_id, rnd);
-	return lbox_returntuple(L, tuple);
-}
-
-struct tuple *
-boxffi_index_get(uint32_t space_id, uint32_t index_id, const char *key)
-{
-	try {
-		Index *index = check_index(space_id, index_id);
-		if (!index->key_def->opts.is_unique)
-			tnt_raise(ClientError, ER_MORE_THAN_ONE_TUPLE);
-		uint32_t part_count = key ? mp_decode_array(&key) : 0;
-		primary_key_validate(index->key_def, key, part_count);
-		struct tuple *tuple = index->findByKey(key, part_count);
-		return boxffi_returntuple(tuple);
-	}  catch (Exception *) {
-		return (struct tuple *) -1; /* handled by box.error() in Lua */
-	}
+	box_tuple_t *tuple;
+	if (box_index_random(space_id, index_id, rnd, &tuple) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, tuple);
 }
 
 static int
@@ -162,30 +68,13 @@ lbox_index_get(lua_State *L)
 	RegionGuard region_guard(&fiber()->gc);
 	uint32_t space_id = lua_tointeger(L, 1);
 	uint32_t index_id = lua_tointeger(L, 2);
-	const char *key = lbox_tokey(L, 3);
+	size_t key_len;
+	const char *key = lbox_encode_tuple_on_gc(L, 3, &key_len);
 
-	struct tuple *tuple = boxffi_index_get(space_id, index_id, key);
-	return lbox_returntuple(L, tuple);
-}
-
-struct tuple *
-boxffi_index_min(uint32_t space_id, uint32_t index_id, const char *key)
-{
-	try {
-		Index *index = check_index(space_id, index_id);
-		if (index->key_def->type != TREE) {
-			/* Show nice error messages in Lua */
-			tnt_raise(ClientError, ER_UNSUPPORTED,
-				  index_type_strs[index->key_def->type],
-				  "min()");
-		}
-		uint32_t part_count = key ? mp_decode_array(&key) : 0;
-		key_validate(index->key_def, ITER_GE, key, part_count);
-		struct tuple *tuple = index->min(key, part_count);
-		return boxffi_returntuple(tuple);
-	}  catch (Exception *) {
-		return (struct tuple *) -1; /* handled by box.error() in Lua */
-	}
+	box_tuple_t *tuple;
+	if (box_index_get(space_id, index_id, key, key + key_len, &tuple) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, tuple);
 }
 
 static int
@@ -197,30 +86,13 @@ lbox_index_min(lua_State *L)
 	RegionGuard region_guard(&fiber()->gc);
 	uint32_t space_id = lua_tointeger(L, 1);
 	uint32_t index_id = lua_tointeger(L, 2);
-	const char *key = lbox_tokey(L, 3);
-
-	struct tuple *tuple = boxffi_index_min(space_id, index_id, key);
-	return lbox_returntuple(L, tuple);
-}
+	size_t key_len;
+	const char *key = lbox_encode_tuple_on_gc(L, 3, &key_len);
 
-struct tuple *
-boxffi_index_max(uint32_t space_id, uint32_t index_id, const char *key)
-{
-	try {
-		Index *index = check_index(space_id, index_id);
-		if (index->key_def->type != TREE) {
-			/* Show nice error messages in Lua */
-			tnt_raise(ClientError, ER_UNSUPPORTED,
-				  index_type_strs[index->key_def->type],
-				  "max()");
-		}
-		uint32_t part_count = key ? mp_decode_array(&key) : 0;
-		key_validate(index->key_def, ITER_LE, key, part_count);
-		struct tuple *tuple = index->max(key, part_count);
-		return boxffi_returntuple(tuple);
-	}  catch (Exception *) {
-		return (struct tuple *) -1; /* handled by box.error() in Lua */
-	}
+	box_tuple_t *tuple;
+	if (box_index_min(space_id, index_id, key, key + key_len, &tuple) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, tuple);
 }
 
 static int
@@ -232,25 +104,15 @@ lbox_index_max(lua_State *L)
 	RegionGuard region_guard(&fiber()->gc);
 	uint32_t space_id = lua_tointeger(L, 1);
 	uint32_t index_id = lua_tointeger(L, 2);
-	const char *key = lbox_tokey(L, 3);
+	size_t key_len;
+	const char *key = lbox_encode_tuple_on_gc(L, 3, &key_len);
 
-	struct tuple *tuple = boxffi_index_max(space_id, index_id, key);
-	return lbox_returntuple(L, tuple);
+	box_tuple_t *tuple;
+	if (box_index_max(space_id, index_id, key, key + key_len, &tuple) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, tuple);
 }
 
-ssize_t
-boxffi_index_count(uint32_t space_id, uint32_t index_id, int type, const char *key)
-{
-	enum iterator_type itype = (enum iterator_type) type;
-	try {
-		Index *index = check_index(space_id, index_id);
-		uint32_t part_count = key ? mp_decode_array(&key) : 0;
-		key_validate(index->key_def, itype, key, part_count);
-		return index->count(itype, key, part_count);
-	} catch (Exception *) {
-		return -1; /* handled by box.error() in Lua */
-	}
-}
 static int
 lbox_index_count(lua_State *L)
 {
@@ -264,10 +126,11 @@ lbox_index_count(lua_State *L)
 	uint32_t space_id = lua_tointeger(L, 1);
 	uint32_t index_id = lua_tointeger(L, 2);
 	uint32_t iterator = lua_tointeger(L, 3);
-	const char *key = lbox_tokey(L, 4);
+	size_t key_len;
+	const char *key = lbox_encode_tuple_on_gc(L, 4, &key_len);
 
-	ssize_t count = boxffi_index_count(space_id, index_id,
-		iterator, key);
+	ssize_t count = box_index_count(space_id, index_id, iterator, key,
+					key + key_len);
 	if (count == -1)
 		return lbox_error(L);
 	lua_pushinteger(L, count);
@@ -278,10 +141,8 @@ static void
 box_index_init_iterator_types(struct lua_State *L, int idx)
 {
 	for (int i = 0; i < iterator_type_MAX; i++) {
-		assert(strncmp(iterator_type_strs[i], "ITER_", 5) == 0);
 		lua_pushnumber(L, i);
-		/* cut ITER_ prefix from enum name */
-		lua_setfield(L, idx, iterator_type_strs[i] + 5);
+		lua_setfield(L, idx, iterator_type_strs[i]);
 	}
 }
 
@@ -289,32 +150,6 @@ box_index_init_iterator_types(struct lua_State *L, int idx)
 
 /* {{{ box.index.iterator Lua library: index iterators */
 
-struct iterator *
-boxffi_index_iterator(uint32_t space_id, uint32_t index_id, int type,
-		      const char *key)
-{
-	struct iterator *it = NULL;
-	enum iterator_type itype = (enum iterator_type) type;
-	try {
-		Index *index = check_index(space_id, index_id);
-		assert(mp_typeof(*key) == MP_ARRAY); /* checked by Lua */
-		uint32_t part_count = mp_decode_array(&key);
-		key_validate(index->key_def, itype, key, part_count);
-		it = index->allocIterator();
-		index->initIterator(it, itype, key, part_count);
-		it->sc_version = sc_version;
-		it->space_id = space_id;
-		it->index_id = index_id;
-		it->index = index;
-		return it;
-	} catch (Exception *) {
-		if (it)
-			it->free(it);
-		/* will be handled by box.error() in Lua */
-		return NULL;
-	}
-}
-
 static int
 lbox_index_iterator(lua_State *L)
 {
@@ -325,10 +160,11 @@ lbox_index_iterator(lua_State *L)
 	uint32_t space_id = lua_tointeger(L, 1);
 	uint32_t index_id = lua_tointeger(L, 2);
 	uint32_t iterator = lua_tointeger(L, 3);
-	/* const char *key = lbox_tokey(L, 4); */
-	const char *mpkey = lua_tolstring(L, 4, NULL); /* Key encoded by Lua */
-	struct iterator *it = boxffi_index_iterator(space_id, index_id,
-		iterator, mpkey);
+	size_t mpkey_len;
+	const char *mpkey = lua_tolstring(L, 4, &mpkey_len); /* Key encoded by Lua */
+	/* const char *key = lbox_encode_tuple_on_gc(L, 4, key_len); */
+	struct iterator *it = box_index_iterator(space_id, index_id, iterator,
+						 mpkey, mpkey + mpkey_len);
 	if (it == NULL)
 		return lbox_error(L);
 
@@ -339,32 +175,6 @@ lbox_index_iterator(lua_State *L)
 	return 1;
 }
 
-struct tuple*
-boxffi_iterator_next(struct iterator *itr)
-{
-	try {
-		if (itr->sc_version != sc_version) {
-			Index *index = check_index(itr->space_id, itr->index_id);
-			if (index != itr->index)
-				return NULL;
-			if (index->sc_version > itr->sc_version)
-				return NULL;
-			itr->sc_version = sc_version;
-		}
-	} catch (Exception *) {
-		return NULL; /* invalidate iterator */
-	}
-	try {
-		struct tuple *tuple = itr->next(itr);
-		if (tuple == NULL)
-			return NULL;
-		tuple_ref(tuple); /* must not throw in this case */
-		return tuple;
-	} catch (Exception *) {
-		return (struct tuple *) -1; /* handled by box.error() in Lua */
-	}
-}
-
 static int
 lbox_iterator_next(lua_State *L)
 {
@@ -379,8 +189,10 @@ lbox_iterator_next(lua_State *L)
 		return luaL_error(L, "usage: next(state)");
 
 	struct iterator *itr = *(struct iterator **) data;
-	struct tuple *tuple = boxffi_iterator_next(itr);
-	return lbox_returntuple(L, tuple);
+	box_tuple_t *tuple;
+	if (box_iterator_next(itr, &tuple) != 0)
+		return lbox_error(L);
+	return lbox_pushtupleornil(L, tuple);
 }
 
 /* }}} */
diff --git a/src/box/lua/index.h b/src/box/lua/index.h
index 6837cae4cb575987621947776f2817779ee71f02..f4f5c2c9f2f98b2aacd887a7a93dd447214f9c8c 100644
--- a/src/box/lua/index.h
+++ b/src/box/lua/index.h
@@ -41,41 +41,4 @@ struct iterator;
 void
 box_lua_index_init(struct lua_State *L);
 
-#if defined(__cplusplus)
-extern "C" {
-#endif /* defined(__cplusplus) */
-
-size_t
-boxffi_index_len(uint32_t space_id, uint32_t index_id);
-
-struct tuple *
-boxffi_index_random(uint32_t space_id, uint32_t index_id, uint32_t rnd);
-
-struct tuple *
-boxffi_index_get(uint32_t space_id, uint32_t index_id, const char *key);
-
-struct tuple *
-boxffi_index_min(uint32_t space_id, uint32_t index_id, const char *key);
-
-struct tuple *
-boxffi_index_max(uint32_t space_id, uint32_t index_id, const char *key);
-
-ssize_t
-boxffi_index_count(uint32_t space_id, uint32_t index_id, int type,
-		   const char *key);
-
-struct iterator *
-boxffi_index_iterator(uint32_t space_id, uint32_t index_id, int type,
-		      const char *key);
-
-size_t
-boxffi_index_bsize(uint32_t space_id, uint32_t index_id);
-
-struct tuple*
-boxffi_iterator_next(struct iterator *itr);
-
-#if defined(__cplusplus)
-} /* extern "C" */
-#endif /* defined(__cplusplus) */
-
 #endif /* INCLUDES_TARANTOOL_BOX_LUA_INDEX_H */
diff --git a/src/box/lua/schema.lua b/src/box/lua/schema.lua
index addc045bf68988314f0fe24b1abd7a3a0d227ebe..e1772af58a024654196c34efcd708ce91aeac97a 100644
--- a/src/box/lua/schema.lua
+++ b/src/box/lua/schema.lua
@@ -13,34 +13,45 @@ ffi.cdef[[
     struct space *space_by_id(uint32_t id);
     void space_run_triggers(struct space *space, bool yesno);
 
-    struct iterator {
-        struct tuple *(*next)(struct iterator *);
-        void (*free)(struct iterator *);
-        void (*close)(struct iterator *);
-        int sc_version;
-        uint32_t space_id;
-        uint32_t index_id;
-    };
+    typedef struct tuple box_tuple_t;
+    typedef struct iterator box_iterator_t;
+
+    /** \cond public */
+    box_iterator_t *
+    box_index_iterator(uint32_t space_id, uint32_t index_id, int type,
+                       const char *key, const char *key_end);
+    int
+    box_iterator_next(box_iterator_t *itr, box_tuple_t **result);
+    void
+    box_iterator_free(box_iterator_t *itr);
+    /** \endcond public */
+    /** \cond public */
     size_t
-    boxffi_index_len(uint32_t space_id, uint32_t index_id);
+    box_index_len(uint32_t space_id, uint32_t index_id);
     size_t
-    boxffi_index_bsize(uint32_t space_id, uint32_t index_id);
-    struct tuple *
-    boxffi_index_random(uint32_t space_id, uint32_t index_id, uint32_t rnd);
-    struct tuple *
-    boxffi_index_get(uint32_t space_id, uint32_t index_id, const char *key);
-    struct tuple *
-    boxffi_index_min(uint32_t space_id, uint32_t index_id, const char *key);
-    struct tuple *
-    boxffi_index_max(uint32_t space_id, uint32_t index_id, const char *key);
+    box_index_bsize(uint32_t space_id, uint32_t index_id);
+    int
+    box_index_random(uint32_t space_id, uint32_t index_id, uint32_t rnd,
+                     box_tuple_t **result);
+    int
+    box_index_get(uint32_t space_id, uint32_t index_id, const char *key,
+                  const char *key_end, box_tuple_t **result);
+    int
+    box_index_min(uint32_t space_id, uint32_t index_id, const char *key,
+                  const char *key_end, box_tuple_t **result);
+    int
+    box_index_max(uint32_t space_id, uint32_t index_id, const char *key,
+                  const char *key_end, box_tuple_t **result);
     ssize_t
-    boxffi_index_count(uint32_t space_id, uint32_t index_id, int type,
-                       const char *key);
-    struct iterator *
-    boxffi_index_iterator(uint32_t space_id, uint32_t index_id, int type,
-                  const char *key);
-    struct tuple *
-    boxffi_iterator_next(struct iterator *itr);
+    box_index_count(uint32_t space_id, uint32_t index_id, int type,
+                    const char *key, const char *key_end);
+    /** \endcond public */
+    /** \cond public */
+    int
+    box_txn_begin();
+    void
+    box_txn_rollback();
+    /** \endcond public */
 
     struct port
     {
@@ -70,16 +81,11 @@ ffi.cdef[[
     port_buf_transfer(struct port_buf *port_buf);
 
     int
-    boxffi_select(struct port_buf *port, uint32_t space_id, uint32_t index_id,
-              int iterator, uint32_t offset, uint32_t limit,
-              const char *key, const char *key_end);
+    box_select(struct port_buf *port, uint32_t space_id, uint32_t index_id,
+               int iterator, uint32_t offset, uint32_t limit,
+               const char *key, const char *key_end);
     void password_prepare(const char *password, int len,
-		                  char *out, int out_len);
-    int
-    boxffi_txn_begin();
-
-    void
-    boxffi_txn_rollback();
+                          char *out, int out_len);
 ]]
 
 local function user_or_role_resolve(user)
@@ -220,13 +226,13 @@ local function update_param_table(table, defaults)
 end
 
 box.begin = function()
-    if ffi.C.boxffi_txn_begin() == -1 then
+    if builtin.box_txn_begin() == -1 then
         box.error()
     end
 end
 -- box.commit yields, so it's defined in call.cc
 
-box.rollback = ffi.C.boxffi_txn_rollback;
+box.rollback = builtin.box_txn_rollback;
 
 box.schema.space = {}
 box.schema.space.create = function(name, options)
@@ -529,6 +535,9 @@ box.schema.index.alter = function(space_id, index_id, options)
                    key_opts, parts}
 end
 
+-- a static box_tuple_t ** instance for calling box_index_* API
+local ptuple = ffi.new('box_tuple_t *[1]')
+
 local function keify(key)
     if key == nil then
         return {}
@@ -572,11 +581,10 @@ local iterator_gen = function(param, state)
         error('usage: next(param, state)')
     end
     -- next() modifies state in-place
-    local tuple = builtin.boxffi_iterator_next(state)
-    if tuple == ffi.cast('void *', -1) then
+    if builtin.box_iterator_next(state, ptuple) ~= 0 then
         return box.error() -- error
-    elseif tuple ~= nil then
-        return state, box.tuple.bless(tuple) -- new state, value
+    elseif ptuple[0] ~= nil then
+        return state, box.tuple.bless(ptuple[0]) -- new state, value
     else
         return nil
     end
@@ -591,10 +599,6 @@ local iterator_gen_luac = function(param, state)
     end
 end
 
-local iterator_cdata_gc = function(iterator)
-    return iterator.free(iterator)
-end
-
 -- global struct port instance to use by select()/get()
 local port_buf = ffi.new('struct port_buf')
 local port_buf_entry_t = ffi.typeof('struct port_buf_entry')
@@ -638,7 +642,7 @@ function box.schema.space.bless(space)
     local index_mt = {}
     -- __len and __index
     index_mt.len = function(index)
-        local ret = builtin.boxffi_index_len(index.space_id, index.id)
+        local ret = builtin.box_index_len(index.space_id, index.id)
         if ret == -1 then
             box.error()
         end
@@ -646,7 +650,7 @@ function box.schema.space.bless(space)
     end
     -- index.bsize
     index_mt.bsize = function(index)
-        local ret = builtin.boxffi_index_bsize(index.space_id, index.id)
+        local ret = builtin.box_index_bsize(index.space_id, index.id)
         if ret == -1 then
             box.error()
         end
@@ -658,12 +662,12 @@ function box.schema.space.bless(space)
     index_mt.__index = index_mt
     -- min and max
     index_mt.min_ffi = function(index, key)
-        local pkey = msgpackffi.encode_tuple(key)
-        local tuple = builtin.boxffi_index_min(index.space_id, index.id, pkey)
-        if tuple == ffi.cast('void *', -1) then
+        local pkey, pkey_end = msgpackffi.encode_tuple(key)
+        if builtin.box_index_min(index.space_id, index.id,
+                                 pkey, pkey_end, ptuple) ~= 0 then
             box.error() -- error
-        elseif tuple ~= nil then
-            return box.tuple.bless(tuple)
+        elseif ptuple[0] ~= nil then
+            return box.tuple.bless(ptuple[0])
         else
             return
         end
@@ -673,12 +677,12 @@ function box.schema.space.bless(space)
         return internal.min(index.space_id, index.id, key);
     end
     index_mt.max_ffi = function(index, key)
-        local pkey = msgpackffi.encode_tuple(key)
-        local tuple = builtin.boxffi_index_max(index.space_id, index.id, pkey)
-        if tuple == ffi.cast('void *', -1) then
+        local pkey, pkey_end = msgpackffi.encode_tuple(key)
+        if builtin.box_index_max(index.space_id, index.id,
+                                 pkey, pkey_end, ptuple) ~= 0 then
             box.error() -- error
-        elseif tuple ~= nil then
-            return box.tuple.bless(tuple)
+        elseif ptuple[0] ~= nil then
+            return box.tuple.bless(ptuple[0])
         else
             return
         end
@@ -689,11 +693,11 @@ function box.schema.space.bless(space)
     end
     index_mt.random_ffi = function(index, rnd)
         rnd = rnd or math.random()
-        local tuple = builtin.boxffi_index_random(index.space_id, index.id, rnd)
-        if tuple == ffi.cast('void *', -1) then
+        if builtin.box_index_random(index.space_id, index.id, rnd,
+                                    ptuple) ~= 0 then
             box.error() -- error
-        elseif tuple ~= nil then
-            return box.tuple.bless(tuple)
+        elseif ptuple[0] ~= nil then
+            return box.tuple.bless(ptuple[0])
         else
             return
         end
@@ -708,12 +712,14 @@ function box.schema.space.bless(space)
         local itype = check_iterator_type(opts, pkey + 1 >= pkey_end);
 
         local keybuf = ffi.string(pkey, pkey_end - pkey)
-        local cdata = builtin.boxffi_index_iterator(index.space_id, index.id,
-            itype, keybuf);
+        local pkeybuf = ffi.cast('const char *', keybuf)
+        local cdata = builtin.box_index_iterator(index.space_id, index.id,
+            itype, pkeybuf, pkeybuf + #keybuf);
         if cdata == nil then
             box.error()
         end
-        return fun.wrap(iterator_gen, keybuf, ffi.gc(cdata, iterator_cdata_gc))
+        return fun.wrap(iterator_gen, keybuf,
+            ffi.gc(cdata, builtin.box_iterator_free))
     end
     index_mt.pairs_luac = function(index, key, opts)
         key = keify(key)
@@ -722,15 +728,15 @@ function box.schema.space.bless(space)
         local keybuf = ffi.string(keymp, #keymp)
         local cdata = internal.iterator(index.space_id, index.id, itype, keymp);
         return fun.wrap(iterator_gen_luac, keybuf,
-            ffi.gc(cdata, iterator_cdata_gc))
+            ffi.gc(cdata, builtin.box_iterator_free))
     end
 
     -- index subtree size
     index_mt.count_ffi = function(index, key, opts)
         local pkey, pkey_end = msgpackffi.encode_tuple(key)
         local itype = check_iterator_type(opts, pkey + 1 >= pkey_end);
-        local count = builtin.boxffi_index_count(index.space_id, index.id,
-            itype, pkey);
+        local count = builtin.box_index_count(index.space_id, index.id,
+            itype, pkey, pkey_end);
         if count == -1 then
             box.error()
         end
@@ -750,11 +756,11 @@ function box.schema.space.bless(space)
 
     index_mt.get_ffi = function(index, key)
         local key, key_end = msgpackffi.encode_tuple(key)
-        local tuple = builtin.boxffi_index_get(index.space_id, index.id, key)
-        if tuple == ffi.cast('void *', -1) then
+        if builtin.box_index_get(index.space_id, index.id,
+                                 key, key_end, ptuple) ~= 0 then
             return box.error() -- error
-        elseif tuple ~= nil then
-            return box.tuple.bless(tuple)
+        elseif ptuple[0] ~= nil then
+            return box.tuple.bless(ptuple[0])
         else
             return
         end
@@ -784,7 +790,7 @@ function box.schema.space.bless(space)
         local iterator, offset, limit = check_select_opts(opts, key + 1 >= key_end)
 
         builtin.port_buf_create(port_buf)
-        if builtin.boxffi_select(port_buf, index.space_id,
+        if builtin.box_select(port_buf, index.space_id,
             index.id, iterator, offset, limit, key, key_end) ~=0 then
             builtin.port_buf_destroy(port_buf);
             return box.error()
@@ -794,12 +800,11 @@ function box.schema.space.bless(space)
         local entry = port_buf.first
         local i = 1
         while entry ~= nil do
-            -- tuple.bless must never fail
             ret[i] = box.tuple.bless(entry.tuple)
             entry = entry.next
             i = i + 1
         end
-        builtin.port_buf_transfer(port_buf);
+        builtin.port_buf_destroy(port_buf);
         return ret
     end
 
@@ -976,11 +981,11 @@ function box.schema.space.bless(space)
         return box.schema.index.create(space.id, name, options)
     end
     space_mt.run_triggers = function(space, yesno)
-        local s = ffi.C.space_by_id(space.id)
+        local s = builtin.space_by_id(space.id)
         if s == nil then
             box.error(box.error.NO_SUCH_SPACE, space.name)
         end
-        ffi.C.space_run_triggers(s, yesno)
+        builtin.space_run_triggers(s, yesno)
     end
     space_mt.__index = space_mt
 
@@ -1158,7 +1163,7 @@ box.schema.user = {}
 box.schema.user.password = function(password)
     local BUF_SIZE = 128
     local buf = ffi.new("char[?]", BUF_SIZE)
-    ffi.C.password_prepare(password, #password, buf, BUF_SIZE)
+    builtin.password_prepare(password, #password, buf, BUF_SIZE)
     return ffi.string(buf)
 end
 
diff --git a/src/box/lua/space.cc b/src/box/lua/space.cc
index bd8e4063c8c230d1213957663a68806962579d28..09aef7d29abede76a1e7bbf405637b9607a35990 100644
--- a/src/box/lua/space.cc
+++ b/src/box/lua/space.cc
@@ -290,33 +290,33 @@ box_lua_space_init(struct lua_State *L)
 	lua_newtable(L);
 	lua_setfield(L, -2, "schema");
 	lua_getfield(L, -1, "schema");
-	lua_pushnumber(L, SC_SCHEMA_ID);
+	lua_pushnumber(L, BOX_SCHEMA_ID);
 	lua_setfield(L, -2, "SCHEMA_ID");
-	lua_pushnumber(L, SC_SPACE_ID);
+	lua_pushnumber(L, BOX_SPACE_ID);
 	lua_setfield(L, -2, "SPACE_ID");
-	lua_pushnumber(L, SC_VSPACE_ID);
+	lua_pushnumber(L, BOX_VSPACE_ID);
 	lua_setfield(L, -2, "VSPACE_ID");
-	lua_pushnumber(L, SC_INDEX_ID);
+	lua_pushnumber(L, BOX_INDEX_ID);
 	lua_setfield(L, -2, "INDEX_ID");
-	lua_pushnumber(L, SC_VINDEX_ID);
+	lua_pushnumber(L, BOX_VINDEX_ID);
 	lua_setfield(L, -2, "VINDEX_ID");
-	lua_pushnumber(L, SC_USER_ID);
+	lua_pushnumber(L, BOX_USER_ID);
 	lua_setfield(L, -2, "USER_ID");
-	lua_pushnumber(L, SC_VUSER_ID);
+	lua_pushnumber(L, BOX_VUSER_ID);
 	lua_setfield(L, -2, "VUSER_ID");
-	lua_pushnumber(L, SC_FUNC_ID);
+	lua_pushnumber(L, BOX_FUNC_ID);
 	lua_setfield(L, -2, "FUNC_ID");
-	lua_pushnumber(L, SC_VFUNC_ID);
+	lua_pushnumber(L, BOX_VFUNC_ID);
 	lua_setfield(L, -2, "VFUNC_ID");
-	lua_pushnumber(L, SC_PRIV_ID);
+	lua_pushnumber(L, BOX_PRIV_ID);
 	lua_setfield(L, -2, "PRIV_ID");
-	lua_pushnumber(L, SC_VPRIV_ID);
+	lua_pushnumber(L, BOX_VPRIV_ID);
 	lua_setfield(L, -2, "VPRIV_ID");
-	lua_pushnumber(L, SC_CLUSTER_ID);
+	lua_pushnumber(L, BOX_CLUSTER_ID);
 	lua_setfield(L, -2, "CLUSTER_ID");
-	lua_pushnumber(L, SC_SYSTEM_ID_MIN);
+	lua_pushnumber(L, BOX_SYSTEM_ID_MIN);
 	lua_setfield(L, -2, "SYSTEM_ID_MIN");
-	lua_pushnumber(L, SC_SYSTEM_ID_MAX);
+	lua_pushnumber(L, BOX_SYSTEM_ID_MAX);
 	lua_setfield(L, -2, "SYSTEM_ID_MAX");
 	lua_pushnumber(L, BOX_INDEX_MAX);
 	lua_setfield(L, -2, "INDEX_MAX");
diff --git a/src/box/lua/tuple.cc b/src/box/lua/tuple.cc
index a1639830fe063fcfcb93b3f7e509ce1859676302..ac75b438f287e56e3cebd44b65f6ff6e08ff63bb 100644
--- a/src/box/lua/tuple.cc
+++ b/src/box/lua/tuple.cc
@@ -90,39 +90,6 @@ lua_istuple(struct lua_State *L, int narg)
 	return t;
 }
 
-static int
-lbox_tuple_new(lua_State *L)
-{
-	int argc = lua_gettop(L);
-	if (unlikely(argc < 1)) {
-		lua_newtable(L); /* create an empty tuple */
-		++argc;
-	}
-
-	struct region *gc = &fiber()->gc;
-	RegionGuard guard(gc);
-	struct mpstream stream;
-	mpstream_init(&stream, gc, region_reserve_cb, region_alloc_cb);
-
-	if (argc == 1 && (lua_istable(L, 1) || lua_istuple(L, 1))) {
-		/* New format: box.tuple.new({1, 2, 3}) */
-		luamp_encode_tuple(L, luaL_msgpack_default, &stream, 1);
-	} else {
-		/* Backward-compatible format: box.tuple.new(1, 2, 3). */
-		luamp_encode_array(luaL_msgpack_default, &stream, argc);
-		for (int k = 1; k <= argc; ++k) {
-			luamp_encode(L, luaL_msgpack_default, &stream, k);
-		}
-	}
-	mpstream_flush(&stream);
-
-	size_t tuple_len = region_used(gc) - guard.used;
-	const char *data = (char *) region_join(gc, tuple_len);
-	struct tuple *tuple = tuple_new(tuple_format_ber, data, data + tuple_len);
-	lbox_pushtuple(L, tuple);
-	return 1;
-}
-
 static int
 lbox_tuple_gc(struct lua_State *L)
 {
@@ -196,7 +163,7 @@ luamp_encode_extension_box(struct lua_State *L, int idx,
 	struct tuple *tuple = lua_istuple(L, idx);
 	if (tuple != NULL) {
 		char *ptr = mpstream_reserve(stream, tuple->bsize);
-		tuple_to_buf(tuple, ptr);
+		tuple_to_buf(tuple, ptr, tuple->bsize);
 		mpstream_advance(stream, tuple->bsize);
 		return MP_ARRAY;
 	}
@@ -212,6 +179,19 @@ luamp_encode_tuple(struct lua_State *L, struct luaL_serializer *cfg,
 		tnt_raise(ClientError, ER_TUPLE_NOT_ARRAY);
 }
 
+char *
+lbox_encode_tuple_on_gc(lua_State *L, int idx, size_t *p_len)
+{
+	struct region *gc = &fiber()->gc;
+	size_t used = region_used(gc);
+	struct mpstream stream;
+	mpstream_init(&stream, gc, region_reserve_cb, region_alloc_cb);
+	luamp_encode_tuple(L, luaL_msgpack_default, &stream, idx);
+	mpstream_flush(&stream);
+	*p_len = region_used(gc) - used;
+	return (char *) region_join(gc, *p_len);
+}
+
 /**
  * Tuple transforming function.
  *
@@ -303,12 +283,14 @@ lbox_tuple_transform(struct lua_State *L)
 }
 
 void
-lbox_pushtuple_noref(struct lua_State *L, struct tuple *tuple)
+lbox_pushtuple(struct lua_State *L, struct tuple *tuple)
 {
 	assert(CTID_CONST_STRUCT_TUPLE_REF != 0);
 	struct tuple **ptr = (struct tuple **) luaL_pushcdata(L,
 		CTID_CONST_STRUCT_TUPLE_REF, sizeof(struct tuple *));
 	*ptr = tuple;
+	/* The order is important - first reference tuple, next set gc */
+	tuple_ref(tuple);
 	lua_pushcfunction(L, lbox_tuple_gc);
 	luaL_setcdatagc(L, -2);
 }
@@ -320,11 +302,6 @@ static const struct luaL_reg lbox_tuple_meta[] = {
 	{NULL, NULL}
 };
 
-static const struct luaL_reg lbox_tuplelib[] = {
-	{"new", lbox_tuple_new},
-	{NULL, NULL}
-};
-
 static const struct luaL_reg lbox_tuple_iterator_meta[] = {
 	{NULL, NULL}
 };
@@ -341,8 +318,6 @@ box_lua_tuple_init(struct lua_State *L)
 	lua_setglobal(L, "cfuncs");
 	luaL_register_type(L, tuple_iteratorlib_name,
 			   lbox_tuple_iterator_meta);
-	luaL_register_module(L, tuplelib_name, lbox_tuplelib);
-	lua_pop(L, 1);
 
 	luamp_set_encode_extension(luamp_encode_extension_box);
 
diff --git a/src/box/lua/tuple.h b/src/box/lua/tuple.h
index 92a5371c3cff8b192bc5bad1214fc5d3ccee7f28..def6d88d8aede83df4b059d900a0e6f8fb973894 100644
--- a/src/box/lua/tuple.h
+++ b/src/box/lua/tuple.h
@@ -41,14 +41,15 @@ struct tuple;
  * Push tuple on lua stack
  */
 void
-lbox_pushtuple_noref(struct lua_State *L, struct tuple *tuple);
+lbox_pushtuple(struct lua_State *L, struct tuple *tuple);
 
-static inline void
-lbox_pushtuple(struct lua_State *L, struct tuple *tuple)
+static inline int
+lbox_pushtupleornil(lua_State *L, box_tuple_t *tuple)
 {
-	assert(tuple != NULL);
-	lbox_pushtuple_noref(L, tuple);
-	tuple_ref(tuple);
+	if (tuple == NULL)
+		return 0;
+	lbox_pushtuple(L, tuple);
+	return 1;
 }
 
 struct tuple *lua_istuple(struct lua_State *L, int narg);
@@ -57,6 +58,9 @@ void
 luamp_encode_tuple(struct lua_State *L, struct luaL_serializer *cfg,
 		  struct mpstream *stream, int index);
 
+char *
+lbox_encode_tuple_on_gc(lua_State *L, int idx, size_t *p_len);
+
 void
 box_lua_tuple_init(struct lua_State *L);
 
diff --git a/src/box/lua/tuple.lua b/src/box/lua/tuple.lua
index 717c6d4fc0224f465750dbcc663167a634372c65..a394abad97cd1b4189f500bfbb258a457dc6b8d5 100644
--- a/src/box/lua/tuple.lua
+++ b/src/box/lua/tuple.lua
@@ -7,63 +7,124 @@ local fun = require('fun')
 local internal = require('box.internal')
 
 ffi.cdef([[
-struct tuple
-{
-    uint32_t _version;
-    uint16_t _refs;
-    uint16_t _format_id;
-    uint32_t _bsize;
-    char data[0];
-} __attribute__((packed));
+/** \cond public */
+typedef struct tuple_format box_tuple_format_t;
+
+box_tuple_format_t *
+box_tuple_format_default(void);
+
+typedef struct tuple box_tuple_t;
+
+box_tuple_t *
+box_tuple_new(box_tuple_format_t *format, const char *data, const char *end);
+
+int
+box_tuple_ref(box_tuple_t *tuple);
 
 void
-tuple_unref(struct tuple *tuple);
+box_tuple_unref(box_tuple_t *tuple);
+
 uint32_t
-tuple_field_count(const struct tuple *tuple);
+box_tuple_field_count(const box_tuple_t *tuple);
+
+size_t
+box_tuple_bsize(const box_tuple_t *tuple);
+
+ssize_t
+box_tuple_to_buf(const box_tuple_t *tuple, char *buf, size_t size);
+
+box_tuple_format_t *
+box_tuple_format(const box_tuple_t *tuple);
+
 const char *
-tuple_field(const struct tuple *tuple, uint32_t i);
+box_tuple_field(const box_tuple_t *tuple, uint32_t i);
+
+typedef struct tuple_iterator box_tuple_iterator_t;
 
-struct tuple_iterator {
-    const struct tuple *tuple;
-    const char *pos;
-    int fieldno;
-};
+box_tuple_iterator_t *
+box_tuple_iterator(box_tuple_t *tuple);
 
 void
-tuple_rewind(struct tuple_iterator *it, const struct tuple *tuple);
+box_tuple_iterator_free(box_tuple_iterator_t *it);
+
+uint32_t
+box_tuple_position(box_tuple_iterator_t *it);
+
+void
+box_tuple_rewind(box_tuple_iterator_t *it);
 
 const char *
-tuple_seek(struct tuple_iterator *it, uint32_t field_no);
+box_tuple_seek(box_tuple_iterator_t *it, uint32_t field_no);
 
 const char *
-tuple_next(struct tuple_iterator *it);
+box_tuple_next(box_tuple_iterator_t *it);
 
-void
-tuple_to_buf(struct tuple *tuple, char *buf);
+/** \endcond public */
 
-struct tuple *
-boxffi_tuple_update(struct tuple *tuple, const char *expr,
-                    const char *expr_end);
+box_tuple_t *
+boxffi_tuple_update(box_tuple_t *tuple, const char *expr, const char *expr_end);
 
-struct tuple *
-boxffi_tuple_upsert(struct tuple *tuple, const char *expr,
+box_tuple_t *
+boxffi_tuple_upsert(box_tuple_t *tuple, const char *expr,
                     const char *expr_end);
 ]])
 
 local builtin = ffi.C
 
-local const_struct_tuple_ref_t = ffi.typeof('const struct tuple&')
+local tuple_t = ffi.typeof('box_tuple_t')
+local const_tuple_ref_t = ffi.typeof('const box_tuple_t&')
 
 local tuple_gc = function(tuple)
-    builtin.tuple_unref(tuple)
+    builtin.box_tuple_unref(tuple)
 end
 
 local tuple_bless = function(tuple)
+    -- overflow checked by tuple_bless() in C
+    builtin.box_tuple_ref(tuple)
     -- must never fail:
-    return ffi.gc(ffi.cast(const_struct_tuple_ref_t, tuple), tuple_gc)
+    return ffi.gc(ffi.cast(const_tuple_ref_t, tuple), tuple_gc)
 end
 
-local tuple_iterator_t = ffi.typeof('struct tuple_iterator')
+local format_lua = nil -- cached box_tuple_format
+local function tuple_new(...)
+    if format_lua == nil then
+        format_lua = builtin.box_tuple_format_default()
+    end
+
+    local obj = ...
+    if select('#', ...) > 1 or type(obj) ~= 'table' then
+        -- Backward-compatible format: box.tuple.new(1, 2, 3).
+        obj = {}
+        for i=1,select('#', ...) do
+            local val = select(i, ...)
+            if val == nil then
+                val = msgpackffi.NULL
+            end
+            obj[i] = val
+        end
+    end
+    local data, data_end = msgpackffi.encode_tuple(obj)
+    local tuple = builtin.box_tuple_new(format_lua, data, data_end)
+    if tuple == nil then
+        return box.error()
+    end
+    return tuple_bless(tuple)
+end
+
+local tuple_iterator_t = ffi.typeof('box_tuple_iterator_t')
+local tuple_iterator_ref_t = ffi.typeof('box_tuple_iterator_t &')
+
+local function tuple_iterator(tuple)
+    if tuple == nil then
+        error("Invalid tuple for iterator")
+    end
+    local it = builtin.box_tuple_iterator(tuple)
+    if it == nil then
+        box.error()
+    end
+    return ffi.gc(ffi.cast(tuple_iterator_ref_t, it),
+        builtin.box_tuple_iterator_free)
+end
 
 local function tuple_iterator_next(it, tuple, pos)
     if pos == nil then
@@ -71,14 +132,15 @@ local function tuple_iterator_next(it, tuple, pos)
     elseif type(pos) ~= "number" then
          error("error: invalid key to 'next'")
     end
+    local curpos = builtin.box_tuple_position(it)
     local field
-    if it.tuple == tuple and it.fieldno == pos then
+    if curpos == pos then
         -- Sequential iteration
-        field = builtin.tuple_next(it)
+        field = builtin.box_tuple_next(it)
     else
         -- Seek
-        builtin.tuple_rewind(it, tuple)
-        field = builtin.tuple_seek(it, pos);
+        builtin.box_tuple_rewind(it)
+        field = builtin.box_tuple_seek(it, pos);
     end
     if field == nil then
         if #tuple == pos then
@@ -90,35 +152,39 @@ local function tuple_iterator_next(it, tuple, pos)
         end
     end
     -- () used to shrink the return stack to one value
-    return it.fieldno, (msgpackffi.decode_unchecked(field))
+    return pos + 1, (msgpackffi.decode_unchecked(field))
 end;
 
--- precreated iterator for tuple_next
-local next_it = ffi.new(tuple_iterator_t)
-
 -- See http://www.lua.org/manual/5.2/manual.html#pdf-next
 local function tuple_next(tuple, pos)
-    return tuple_iterator_next(next_it, tuple, pos);
+    if pos == nil then
+        pos = 0
+    end
+    local field = builtin.box_tuple_field(tuple, pos)
+    if field == nil then
+        return nil
+    end
+    return pos + 1, (msgpackffi.decode_unchecked(field))
 end
 
 -- See http://www.lua.org/manual/5.2/manual.html#pdf-ipairs
 local function tuple_ipairs(tuple, pos)
-    local it = ffi.new(tuple_iterator_t)
+    local it = tuple_iterator(tuple)
     return fun.wrap(it, tuple, pos)
 end
 
 local function tuple_totable(tuple, i, j)
-    -- use a precreated iterator for tuple_next
-    builtin.tuple_rewind(next_it, tuple)
+    local it = tuple_iterator(tuple)
+    builtin.box_tuple_rewind(it)
     local field
     if i ~= nil then
         if i < 1 then
             error('tuple.totable: invalid second argument')
         end
-        field = builtin.tuple_seek(next_it, i - 1)
+        field = builtin.box_tuple_seek(it, i - 1)
     else
         i = 1
-        field = builtin.tuple_next(next_it)
+        field = builtin.box_tuple_next(it)
     end
     if j ~= nil then
         if j <= 0 then
@@ -132,7 +198,7 @@ local function tuple_totable(tuple, i, j)
         local val = msgpackffi.decode_unchecked(field)
         table.insert(ret, val)
         i = i + 1
-        field = builtin.tuple_next(next_it)
+        field = builtin.box_tuple_next(it)
     end
     return setmetatable(ret, msgpackffi.array_mt)
 end
@@ -186,11 +252,13 @@ end
 
 -- Set encode hooks for msgpackffi
 local function tuple_to_msgpack(buf, tuple)
-    local data = buf:alloc(tuple._bsize)
-    builtin.tuple_to_buf(tuple, data)
+    local bsize = tuple:bsize()
+    buf:reserve(bsize)
+    builtin.box_tuple_to_buf(tuple, buf.wpos, bsize)
+    buf.wpos = buf.wpos + bsize
 end
 
-msgpackffi.on_encode(ffi.typeof('const struct tuple &'), tuple_to_msgpack)
+msgpackffi.on_encode(const_tuple_ref_t, tuple_to_msgpack)
 
 
 -- cfuncs table is set by C part
@@ -208,13 +276,13 @@ local methods = {
     ["update"]      = tuple_update;
     ["upsert"]      = tuple_upsert;
     ["bsize"]       = function(tuple)
-        return tonumber(tuple._bsize)
+        return tonumber(builtin.box_tuple_bsize(tuple))
     end;
     ["__serialize"] = tuple_totable; -- encode hook for msgpack/yaml/json
 }
 
 local tuple_field = function(tuple, field_n)
-    local field = builtin.tuple_field(tuple, field_n - 1)
+    local field = builtin.box_tuple_field(tuple, field_n - 1)
     if field == nil then
         return nil
     end
@@ -223,9 +291,9 @@ local tuple_field = function(tuple, field_n)
 end
 
 
-ffi.metatype('struct tuple', {
+ffi.metatype(tuple_t, {
     __len = function(tuple)
-        return builtin.tuple_field_count(tuple)
+        return builtin.box_tuple_field_count(tuple)
     end;
     __tostring = function(tuple)
         -- Unpack tuple, call yaml.encode, remove yaml header and footer
@@ -254,5 +322,8 @@ ffi.metatype(tuple_iterator_t, {
 -- Remove the global variable
 cfuncs = nil
 
--- internal api for box.select and iterators
-box.tuple.bless = tuple_bless
+box.tuple = {
+    new = tuple_new;
+    -- internal api for box.select and iterators
+    bless = tuple_bless;
+}
diff --git a/src/box/memtx_rtree.cc b/src/box/memtx_rtree.cc
index 123567d5beb373c7f5360839cab3824d2649ef07..3d6c7e38bd22a8130f4d9bfbecf6989ea47bf2a6 100644
--- a/src/box/memtx_rtree.cc
+++ b/src/box/memtx_rtree.cc
@@ -80,7 +80,7 @@ extract_rectangle(struct rtree_rect *rect, const struct tuple *tuple,
 	const char *elems = tuple_field(tuple, key_def->parts[0].fieldno);
 	unsigned dimension = key_def->opts.dimension;
 	if (mp_decode_rect(rect, dimension, elems)) {
-		tnt_raise(ClientError, ER_RTREE_RECT_ERROR,
+		tnt_raise(ClientError, ER_RTREE_RECT,
 			  "Field", dimension, dimension * 2);
 	}
 }
@@ -218,7 +218,7 @@ MemtxRTree::initIterator(struct iterator *iterator, enum iterator_type type,
 				  "key only for ITER_ALL");
 		}
 	} else if (mp_decode_rect(&rect, m_dimension, key, part_count)) {
-		tnt_raise(ClientError, ER_RTREE_RECT_ERROR,
+		tnt_raise(ClientError, ER_RTREE_RECT,
 			  "Key", m_dimension, m_dimension * 2);
 	}
 
diff --git a/src/box/relay.cc b/src/box/relay.cc
index ef107deb90e5985836a285acdd710592b7787aa2..a5beec36ba59d80ad77e04c295715b5aa0e61a73 100644
--- a/src/box/relay.cc
+++ b/src/box/relay.cc
@@ -209,9 +209,9 @@ replication_subscribe(int fd, struct xrow_header *packet)
 	}
 
 	/* Check server uuid */
-	r->server_id = schema_find_id(SC_CLUSTER_ID, 1,
+	r->server_id = schema_find_id(BOX_CLUSTER_ID, 1,
 				   tt_uuid_str(&server_uuid), UUID_STR_LEN);
-	if (r->server_id == SC_ID_NIL) {
+	if (r->server_id == BOX_ID_NIL) {
 		tnt_raise(ClientError, ER_UNKNOWN_SERVER,
 			  tt_uuid_str(&server_uuid));
 	}
diff --git a/src/box/request.h b/src/box/request.h
index 4c8ce05dbfabe83af31e3aaab515d88d462e0b61..606f084e05474dc3803c736da489638763613759 100644
--- a/src/box/request.h
+++ b/src/box/request.h
@@ -88,4 +88,11 @@ request_decode(struct request *request, const char *data, uint32_t len);
 int
 request_encode(struct request *request, struct iovec *iov);
 
+/**
+ * API of C stored function.
+ */
+typedef struct box_function_ctx box_function_ctx_t;
+typedef int (*box_function_f)(box_function_ctx_t *ctx,
+	     const char *args, const char *args_end);
+
 #endif /* TARANTOOL_BOX_REQUEST_H_INCLUDED */
diff --git a/src/box/schema.cc b/src/box/schema.cc
index d3314928fa810e7c3dc637e2fedcb58c5f96c5f6..33e7a40a3a19a8188c5290be9469ab255a5c5e75 100644
--- a/src/box/schema.cc
+++ b/src/box/schema.cc
@@ -65,8 +65,8 @@ int sc_version;
 bool
 space_is_system(struct space *space)
 {
-	return space->def.id > SC_SYSTEM_ID_MIN &&
-		space->def.id < SC_SYSTEM_ID_MAX;
+	return space->def.id > BOX_SYSTEM_ID_MIN &&
+		space->def.id < BOX_SYSTEM_ID_MAX;
 }
 
 /** Return space by its number */
@@ -95,8 +95,8 @@ space_foreach(void (*func)(struct space *sp, void *udata), void *udata)
 	mh_int_t i;
 	struct space *space;
 	char key[6];
-	assert(mp_sizeof_uint(SC_SYSTEM_ID_MIN) <= sizeof(key));
-	mp_encode_uint(key, SC_SYSTEM_ID_MIN);
+	assert(mp_sizeof_uint(BOX_SYSTEM_ID_MIN) <= sizeof(key));
+	mp_encode_uint(key, BOX_SYSTEM_ID_MIN);
 
 	/*
 	 * Make sure we always visit system spaces first,
@@ -104,7 +104,7 @@ space_foreach(void (*func)(struct space *sp, void *udata), void *udata)
 	 * This is essential for correctly recovery from the
 	 * snapshot, and harmless otherwise.
 	 */
-	space = space_by_id(SC_SPACE_ID);
+	space = space_by_id(BOX_SPACE_ID);
 	Index *pk = space ? space_index(space, 0) : NULL;
 	if (pk) {
 		struct iterator *it = pk->allocIterator();
@@ -208,7 +208,7 @@ schema_find_id(uint32_t system_space_id, uint32_t index_id,
 	 * max length in advance.
 	 */
 	if (len + 5 > sizeof(buf))
-		return SC_ID_NIL;
+		return BOX_ID_NIL;
 
 	mp_encode_str(buf, name, len);
 
@@ -221,7 +221,7 @@ schema_find_id(uint32_t system_space_id, uint32_t index_id,
 		/* id is always field #1 */
 		return tuple_field_u32(tuple, 0);
 	}
-	return SC_ID_NIL;
+	return BOX_ID_NIL;
 }
 
 /**
@@ -249,7 +249,7 @@ schema_init()
 	 */
 	/* _schema - key/value space with schema description */
 	struct space_def def = {
-		SC_SCHEMA_ID, ADMIN, 0, "_schema", "memtx", false
+		BOX_SCHEMA_ID, ADMIN, 0, "_schema", "memtx", false
 	};
 	struct key_opts opts = { true /* is_unique */, 0 /* dimension */ };
 	struct key_def *key_def = key_def_new(def.id,
@@ -262,39 +262,39 @@ schema_init()
 	(void) sc_space_new(&def, key_def, &on_replace_schema);
 
 	/* _space - home for all spaces. */
-	key_def->space_id = def.id = SC_SPACE_ID;
+	key_def->space_id = def.id = BOX_SPACE_ID;
 	snprintf(def.name, sizeof(def.name), "_space");
 	key_def_set_part(key_def, 0 /* part no */, 0 /* field no */, NUM);
 
 	(void) sc_space_new(&def, key_def, &alter_space_on_replace_space);
 
 	/* _user - all existing users */
-	key_def->space_id = def.id = SC_USER_ID;
+	key_def->space_id = def.id = BOX_USER_ID;
 	snprintf(def.name, sizeof(def.name), "_user");
 	(void) sc_space_new(&def, key_def, &on_replace_user);
 
 	/* _func - all executable objects on which one can have grants */
-	key_def->space_id = def.id = SC_FUNC_ID;
+	key_def->space_id = def.id = BOX_FUNC_ID;
 	snprintf(def.name, sizeof(def.name), "_func");
 	(void) sc_space_new(&def, key_def, &on_replace_func);
 	/*
 	 * _priv - association user <-> object
 	 * The real index is defined in the snapshot.
 	 */
-	key_def->space_id = def.id = SC_PRIV_ID;
+	key_def->space_id = def.id = BOX_PRIV_ID;
 	snprintf(def.name, sizeof(def.name), "_priv");
 	(void) sc_space_new(&def, key_def, &on_replace_priv);
 	/*
 	 * _cluster - association server uuid <-> server id
 	 * The real index is defined in the snapshot.
 	 */
-	key_def->space_id = def.id = SC_CLUSTER_ID;
+	key_def->space_id = def.id = BOX_CLUSTER_ID;
 	snprintf(def.name, sizeof(def.name), "_cluster");
 	(void) sc_space_new(&def, key_def, &on_replace_cluster);
 	key_def_delete(key_def);
 
 	/* _index - definition of indexes in all spaces */
-	def.id = SC_INDEX_ID;
+	def.id = BOX_INDEX_ID;
 	snprintf(def.name, sizeof(def.name), "_index");
 	key_def = key_def_new(def.id,
 			      0 /* index id */,
@@ -406,7 +406,7 @@ func_by_name(const char *name, uint32_t name_len)
 bool
 schema_find_grants(const char *type, uint32_t id)
 {
-	struct space *priv = space_cache_find(SC_PRIV_ID);
+	struct space *priv = space_cache_find(BOX_PRIV_ID);
 	/** "object" index */
 	Index *index = index_find(priv, 2);
 	struct iterator *it = index->position();
diff --git a/src/box/schema.h b/src/box/schema.h
index 6494458337b9423d094bfb50877fb4d0648b0b92..89096a7362aa29485ec113eb6e7986e2de35fa40 100644
--- a/src/box/schema.h
+++ b/src/box/schema.h
@@ -33,33 +33,34 @@
 #include "error.h"
 #include <stdio.h> /* snprintf */
 
-enum schema_id {
+/** \cond public */
+enum {
 	/** Start of the reserved range of system spaces. */
-	SC_SYSTEM_ID_MIN = 256,
+	BOX_SYSTEM_ID_MIN = 256,
 	/** Space id of _schema. */
-	SC_SCHEMA_ID = 272,
+	BOX_SCHEMA_ID = 272,
 	/** Space id of _space. */
-	SC_SPACE_ID = 280,
-	SC_VSPACE_ID = 281,
+	BOX_SPACE_ID = 280,
+	BOX_VSPACE_ID = 281,
 	/** Space id of _index. */
-	SC_INDEX_ID = 288,
-	SC_VINDEX_ID = 289,
+	BOX_INDEX_ID = 288,
+	BOX_VINDEX_ID = 289,
 	/** Space id of _func. */
-	SC_FUNC_ID = 296,
-	SC_VFUNC_ID = 297,
+	BOX_FUNC_ID = 296,
+	BOX_VFUNC_ID = 297,
 	/** Space id of _user. */
-	SC_USER_ID = 304,
-	SC_VUSER_ID = 305,
+	BOX_USER_ID = 304,
+	BOX_VUSER_ID = 305,
 	/** Space id of _priv. */
-	SC_PRIV_ID = 312,
-	SC_VPRIV_ID = 313,
+	BOX_PRIV_ID = 312,
+	BOX_VPRIV_ID = 313,
 	/** Space id of _cluster. */
-	SC_CLUSTER_ID = 320,
+	BOX_CLUSTER_ID = 320,
 	/** End of the reserved range of system spaces. */
-	SC_SYSTEM_ID_MAX = 511,
-	SC_ID_NIL = 2147483647
+	BOX_SYSTEM_ID_MAX = 511,
+	BOX_ID_NIL = 2147483647
 };
-
+/** \endcond public */
 
 extern int sc_version;
 
diff --git a/src/box/sysview_engine.cc b/src/box/sysview_engine.cc
index 5021f72a1ac9c32ba1c5df08e07062ff4d9739bc..5bead15e2a5c84affa3ed296e2260404fd9d2ac2 100644
--- a/src/box/sysview_engine.cc
+++ b/src/box/sysview_engine.cc
@@ -83,15 +83,15 @@ SysviewEngine::createIndex(struct key_def *key_def)
 {
 	assert(key_def->type == TREE);
 	switch (key_def->space_id) {
-	case SC_VSPACE_ID:
+	case BOX_VSPACE_ID:
 		return new SysviewVspaceIndex(key_def);
-	case SC_VINDEX_ID:
+	case BOX_VINDEX_ID:
 		return new SysviewVindexIndex(key_def);
-	case SC_VUSER_ID:
+	case BOX_VUSER_ID:
 		return new SysviewVuserIndex(key_def);
-	case SC_VFUNC_ID:
+	case BOX_VFUNC_ID:
 		return new SysviewVfuncIndex(key_def);
-	case SC_VPRIV_ID:
+	case BOX_VPRIV_ID:
 		return new SysviewVprivIndex(key_def);
 	default:
 		struct space *space = space_cache_find(key_def->space_id);
diff --git a/src/box/sysview_index.cc b/src/box/sysview_index.cc
index 1caa0bbf7ebd1ce139d151c412afb5eba6055c39..69e51f3a9ece46330277c990de6e0f53ae13745f 100644
--- a/src/box/sysview_index.cc
+++ b/src/box/sysview_index.cc
@@ -169,12 +169,12 @@ vspace_filter(struct space *source, struct tuple *tuple)
 }
 
 SysviewVspaceIndex::SysviewVspaceIndex(struct key_def *key_def)
-	: SysviewIndex(key_def, SC_SPACE_ID, key_def->iid, vspace_filter)
+	: SysviewIndex(key_def, BOX_SPACE_ID, key_def->iid, vspace_filter)
 {
 }
 
 SysviewVindexIndex::SysviewVindexIndex(struct key_def *key_def)
-	: SysviewIndex(key_def, SC_INDEX_ID, key_def->iid, vspace_filter)
+	: SysviewIndex(key_def, BOX_INDEX_ID, key_def->iid, vspace_filter)
 {
 }
 
@@ -193,7 +193,7 @@ vuser_filter(struct space *source, struct tuple *tuple)
 }
 
 SysviewVuserIndex::SysviewVuserIndex(struct key_def *key_def)
-	: SysviewIndex(key_def, SC_USER_ID, key_def->iid, vuser_filter)
+	: SysviewIndex(key_def, BOX_USER_ID, key_def->iid, vuser_filter)
 {
 }
 
@@ -212,7 +212,7 @@ vpriv_filter(struct space *source, struct tuple *tuple)
 }
 
 SysviewVprivIndex::SysviewVprivIndex(struct key_def *key_def)
-	: SysviewIndex(key_def, SC_PRIV_ID, key_def->iid, vpriv_filter)
+	: SysviewIndex(key_def, BOX_PRIV_ID, key_def->iid, vpriv_filter)
 {
 }
 
@@ -236,6 +236,6 @@ vfunc_filter(struct space *source, struct tuple *tuple)
 }
 
 SysviewVfuncIndex::SysviewVfuncIndex(struct key_def *key_def)
-	: SysviewIndex(key_def, SC_FUNC_ID, key_def->iid, vfunc_filter)
+	: SysviewIndex(key_def, BOX_FUNC_ID, key_def->iid, vfunc_filter)
 {
 }
diff --git a/src/box/tuple.cc b/src/box/tuple.cc
index 7d10607aea574808f8429fd76e87e18fc27a9e92..4710d4cf108c617c18f97546468acbf439a84baf 100644
--- a/src/box/tuple.cc
+++ b/src/box/tuple.cc
@@ -33,11 +33,13 @@
 #include <stdio.h>
 
 #include "small/small.h"
+#include "small/mempool.h"
 #include "small/quota.h"
 
 #include "key_def.h"
 #include "tuple_update.h"
 #include "errinj.h"
+#include "fiber.h"
 
 /** Global table of tuple formats */
 struct tuple_format **tuple_formats;
@@ -63,6 +65,14 @@ enum {
 	SLAB_SIZE_MIN = 1024 * 1024
 };
 
+static struct mempool tuple_iterator_pool;
+
+/**
+ * Last tuple returned by public C API
+ * \sa tuple_bless()
+ */
+struct tuple *box_tuple_last;
+
 /** Extract all available type info from keys. */
 void
 field_type_create(enum field_type *types, uint32_t field_count,
@@ -622,11 +632,23 @@ tuple_init(float tuple_arena_max_size, uint32_t objsize_min,
 	slab_cache_create(&memtx_slab_cache, &memtx_arena);
 	small_alloc_create(&memtx_alloc, &memtx_slab_cache,
 			   objsize_min, alloc_factor);
+	mempool_create(&tuple_iterator_pool, &cord()->slabc,
+		       sizeof(struct tuple_iterator));
+
+	box_tuple_last = NULL;
 }
 
 void
 tuple_free()
 {
+	/* Unref last tuple returned by public C API */
+	if (box_tuple_last != NULL) {
+		tuple_unref(box_tuple_last);
+		box_tuple_last = NULL;
+	}
+
+	mempool_destroy(&tuple_iterator_pool);
+
 	/* Clear recycled ids. */
 	while (recycled_format_ids != FORMAT_ID_NIL) {
 
@@ -677,3 +699,121 @@ double mp_decode_num(const char **data, uint32_t i)
 	return val;
 }
 
+box_tuple_format_t *
+box_tuple_format_default(void)
+{
+	return tuple_format_ber;
+}
+
+box_tuple_t *
+box_tuple_new(box_tuple_format_t *format, const char *data, const char *end)
+{
+	try {
+		return tuple_bless(tuple_new(format, data, end));
+	} catch (Exception *e) {
+		return NULL;
+	}
+}
+
+int
+box_tuple_ref(box_tuple_t *tuple)
+{
+	assert(tuple != NULL);
+	try {
+		tuple_ref(tuple);
+		return 0;
+	} catch (Exception *e) {
+		return -1;
+	}
+}
+
+void
+box_tuple_unref(box_tuple_t *tuple)
+{
+	assert(tuple != NULL);
+	return tuple_unref(tuple);
+}
+
+uint32_t
+box_tuple_field_count(const box_tuple_t *tuple)
+{
+	assert(tuple != NULL);
+	return tuple_field_count(tuple);
+}
+
+size_t
+box_tuple_bsize(const box_tuple_t *tuple)
+{
+	assert(tuple != NULL);
+	return tuple->bsize;
+}
+
+ssize_t
+box_tuple_to_buf(const box_tuple_t *tuple, char *buf, size_t size)
+{
+	assert(tuple != NULL);
+	return tuple_to_buf(tuple, buf, size);
+}
+
+box_tuple_format_t *
+box_tuple_format(const box_tuple_t *tuple)
+{
+	assert(tuple != NULL);
+	return tuple_format(tuple);
+}
+
+const char *
+box_tuple_field(const box_tuple_t *tuple, uint32_t i)
+{
+	assert(tuple != NULL);
+	return tuple_field(tuple, i);
+}
+
+typedef struct tuple_iterator box_tuple_iterator_t;
+
+box_tuple_iterator_t *
+box_tuple_iterator(box_tuple_t *tuple)
+{
+	assert(tuple != NULL);
+	struct tuple_iterator *it;
+	try {
+		it = (struct tuple_iterator *)
+			mempool_alloc0(&tuple_iterator_pool);
+	} catch (Exception *e) {
+		return NULL;
+	}
+	tuple_ref(tuple);
+	tuple_rewind(it, tuple);
+	return it;
+}
+
+void
+box_tuple_iterator_free(box_tuple_iterator_t *it)
+{
+	tuple_unref(it->tuple);
+	mempool_free(&tuple_iterator_pool, it);
+}
+
+uint32_t
+box_tuple_position(box_tuple_iterator_t *it)
+{
+	return it->fieldno;
+}
+
+void
+box_tuple_rewind(box_tuple_iterator_t *it)
+{
+	tuple_rewind(it, it->tuple);
+}
+
+const char *
+box_tuple_seek(box_tuple_iterator_t *it, uint32_t field_no)
+{
+	return tuple_seek(it, field_no);
+}
+
+const char *
+box_tuple_next(box_tuple_iterator_t *it)
+{
+	return tuple_next(it);
+}
diff --git a/src/box/tuple.h b/src/box/tuple.h
index bc1e5374d2d6557c821346aa259a2f9eaf2883da..fb86c2a40f103a3b5b3b3657f4b92fce9b2969e4 100644
--- a/src/box/tuple.h
+++ b/src/box/tuple.h
@@ -206,7 +206,7 @@ tuple_ref_exception();
  *
  * @pre tuple->refs + count >= 0
  */
-extern "C" inline void
+inline void
 tuple_ref(struct tuple *tuple)
 {
 	if (tuple->refs + 1 > TUPLE_REF_MAX)
@@ -220,7 +220,7 @@ tuple_ref(struct tuple *tuple)
  *
  * @pre tuple->refs + count >= 0
  */
-extern "C" inline void
+inline void
 tuple_unref(struct tuple *tuple)
 {
 	assert(tuple->refs - 1 >= 0);
@@ -258,7 +258,7 @@ tuple_format(const struct tuple *tuple)
  * @param tuple
  * @return the number of fields in tuple
  */
-extern "C" inline uint32_t
+inline uint32_t
 tuple_field_count(const struct tuple *tuple)
 {
 	const char *data = tuple->data;
@@ -372,7 +372,7 @@ tuple_field_cstr(struct tuple *tuple, uint32_t i);
 struct tuple_iterator {
 	/** @cond false **/
 	/* State */
-	const struct tuple *tuple;
+	struct tuple *tuple;
 	/** Always points to the beginning of the next field. */
 	const char *pos;
 	/** @endcond **/
@@ -397,8 +397,8 @@ struct tuple_iterator {
  * @param[out] it tuple iterator
  * @param[in]  tuple tuple
  */
-extern "C" inline void
-tuple_rewind(struct tuple_iterator *it, const struct tuple *tuple)
+inline void
+tuple_rewind(struct tuple_iterator *it, struct tuple *tuple)
 {
 	it->tuple = tuple;
 	it->pos = tuple->data;
@@ -412,7 +412,7 @@ tuple_rewind(struct tuple_iterator *it, const struct tuple *tuple)
  * @retval field  if the iterator has the requested field
  * @retval NULL   otherwise (iteration is out of range)
  */
-extern "C" const char *
+const char *
 tuple_seek(struct tuple_iterator *it, uint32_t field_no);
 
 /**
@@ -420,7 +420,7 @@ tuple_seek(struct tuple_iterator *it, uint32_t field_no);
  * @param it tuple iterator
  * @return next field or NULL if the iteration is out of range
  */
-extern "C" const char *
+const char *
 tuple_next(struct tuple_iterator *it);
 
 /**
@@ -531,11 +531,15 @@ void
 tuple_to_obuf(struct tuple *tuple, struct obuf *buf);
 
 /**
- * Store tuple fields in the memory buffer. Buffer must have at least
- * tuple->bsize bytes.
+ * Store tuple fields in the memory buffer.
+ * \retval -1 on error.
+ * \retval number of bytes written on success.
+ * Upon successful return, the function returns the number of bytes written.
+ * If buffer size is not enough then the return value is the number of bytes
+ * which would have been written if enough space had been available.
  */
-extern "C" void
-tuple_to_buf(struct tuple *tuple, char *buf);
+ssize_t
+tuple_to_buf(const struct tuple *tuple, char *buf, size_t size);
 
 /** Initialize tuple library */
 void
@@ -551,5 +555,102 @@ tuple_begin_snapshot();
 
 void
 tuple_end_snapshot();
+
+/** \cond public */
+typedef struct tuple_format box_tuple_format_t;
+
+API_EXPORT box_tuple_format_t *
+box_tuple_format_default(void);
+
+typedef struct tuple box_tuple_t;
+
+API_EXPORT box_tuple_t *
+box_tuple_new(box_tuple_format_t *format, const char *data, const char *end);
+
+API_EXPORT int
+box_tuple_ref(box_tuple_t *tuple);
+
+API_EXPORT void
+box_tuple_unref(box_tuple_t *tuple);
+
+API_EXPORT uint32_t
+box_tuple_field_count(const box_tuple_t *tuple);
+
+API_EXPORT size_t
+box_tuple_bsize(const box_tuple_t *tuple);
+
+API_EXPORT ssize_t
+box_tuple_to_buf(const box_tuple_t *tuple, char *buf, size_t size);
+
+API_EXPORT box_tuple_format_t *
+box_tuple_format(const box_tuple_t *tuple);
+
+API_EXPORT const char *
+box_tuple_field(const box_tuple_t *tuple, uint32_t i);
+
+typedef struct tuple_iterator box_tuple_iterator_t;
+
+API_EXPORT box_tuple_iterator_t *
+box_tuple_iterator(box_tuple_t *tuple);
+
+API_EXPORT void
+box_tuple_iterator_free(box_tuple_iterator_t *it);
+
+API_EXPORT uint32_t
+box_tuple_position(box_tuple_iterator_t *it);
+
+API_EXPORT void
+box_tuple_rewind(box_tuple_iterator_t *it);
+
+API_EXPORT const char *
+box_tuple_seek(box_tuple_iterator_t *it, uint32_t field_no);
+
+API_EXPORT const char *
+box_tuple_next(box_tuple_iterator_t *it);
+
+/** \endcond public */
+
+extern struct tuple *box_tuple_last;
+
+/**
+ * Convert internal `struct tuple` to public `box_tuple_t`.
+ * \post \a tuple ref counted until the next call.
+ * \post tuple_ref() doesn't fail at least once
+ * \sa tuple_ref
+ * \throw ER_TUPLE_REF_OVERFLOW
+ */
+static inline box_tuple_t *
+tuple_bless(struct tuple *tuple)
+{
+	assert(tuple != NULL);
+	/* Ensure tuple can be referenced at least once after return */
+	if (tuple->refs + 2 > TUPLE_REF_MAX)
+		tuple_ref_exception();
+	tuple->refs++;
+	/* Remove previous tuple */
+	if (likely(box_tuple_last != NULL))
+		tuple_unref(box_tuple_last); /* do not throw */
+	/* Remember current tuple */
+	box_tuple_last = tuple;
+	return tuple;
+}
+
+static inline void
+mp_tuple_assert(const char *tuple, const char *tuple_end)
+{
+	assert(mp_typeof(*tuple) == MP_ARRAY);
+	mp_next(&tuple);
+	assert(tuple == tuple_end);
+}
+
+static inline uint32_t
+box_tuple_field_u32(box_tuple_t *tuple, uint32_t field_no, uint32_t deflt)
+{
+	const char *field = box_tuple_field(tuple, field_no);
+	if (field != NULL && mp_typeof(*field) == MP_UINT)
+		return mp_decode_uint(&field);
+	return deflt;
+}
+
 #endif /* TARANTOOL_BOX_TUPLE_H_INCLUDED */
 
diff --git a/src/box/tuple_convert.cc b/src/box/tuple_convert.cc
index 7456505f4d7bd9104f7aa79baa072ca9744ce8ce..9c490442ad769be1d221cc354bff26570c1b0329 100644
--- a/src/box/tuple_convert.cc
+++ b/src/box/tuple_convert.cc
@@ -37,8 +37,11 @@ tuple_to_obuf(struct tuple *tuple, struct obuf *buf)
 	obuf_dup(buf, tuple->data, tuple->bsize);
 }
 
-void
-tuple_to_buf(struct tuple *tuple, char *buf)
+ssize_t
+tuple_to_buf(const struct tuple *tuple, char *buf, size_t size)
 {
-	memcpy(buf, tuple->data, tuple->bsize);
+	if (likely(tuple->bsize <= size)) {
+		memcpy(buf, tuple->data, tuple->bsize);
+	}
+	return tuple->bsize;
 }
diff --git a/src/box/txn.cc b/src/box/txn.cc
index f6286f1e3d32a3d1d6ce9a340c6946e904784307..83cfbcecbb9001da9fcab7cd6f567ec5fa51ea99 100644
--- a/src/box/txn.cc
+++ b/src/box/txn.cc
@@ -236,7 +236,7 @@ txn_check_autocommit(struct txn *txn, const char *where)
 extern "C" {
 
 int
-boxffi_txn_begin()
+box_txn_begin()
 {
 	try {
 		if (in_txn())
@@ -248,10 +248,37 @@ boxffi_txn_begin()
 	return 0;
 }
 
+int
+box_txn_commit()
+{
+	struct txn *txn = in_txn();
+	/**
+	 * COMMIT is like BEGIN or ROLLBACK
+	 * a "transaction-initiating statement".
+	 * Do nothing if transaction is not started,
+	 * it's the same as BEGIN + COMMIT.
+	*/
+	if (! txn)
+		return 0;
+	try {
+		txn_commit(txn);
+	} catch (...) {
+		txn_rollback();
+		return -1;
+	}
+	return 0;
+}
+
 void
-boxffi_txn_rollback()
+box_txn_rollback()
 {
 	txn_rollback(); /* doesn't throw */
 }
 
+void *
+box_txn_alloc(size_t size)
+{
+	return region_alloc_nothrow(&fiber()->gc, size);
+}
+
 } /* extern "C" */
diff --git a/src/box/txn.h b/src/box/txn.h
index 8c5e29f14988a79c2f0ff77d3ecedeb08a898369..cc8f61aa603f918b8171a53f4ebc2738edd8b964 100644
--- a/src/box/txn.h
+++ b/src/box/txn.h
@@ -158,19 +158,33 @@ txn_stmt(struct txn *txn)
  * FFI bindings: do not throw exceptions, do not accept extra
  * arguments
  */
-extern "C" {
+
+/** \cond public */
 
 /**
  * @retval 0 - success
  * @retval -1 - failed, perhaps a transaction has already been
  * started
  */
-int
-boxffi_txn_begin();
+API_EXPORT int
+box_txn_begin(void);
 
-void
-boxffi_txn_rollback();
+API_EXPORT int
+box_txn_commit(void);
+
+API_EXPORT void
+box_txn_rollback(void);
+
+/**
+ * Allocate memory on txn memory pool.
+ * The memory is automatically deallocated when the transaction
+ * is committed or rolled back.
+ *
+ * @retval 0  out of memory
+ */
+API_EXPORT void *
+box_txn_alloc(size_t size);
 
-} /* extern  "C" */
+/** \endcond public */
 
 #endif /* TARANTOOL_BOX_TXN_H_INCLUDED */
diff --git a/src/box/user.cc b/src/box/user.cc
index a20e158726b7eee0be9220e59391aab963db664f..2e028c575dbd3dad871b203996adb5c467b01400 100644
--- a/src/box/user.cc
+++ b/src/box/user.cc
@@ -276,7 +276,7 @@ user_reload_privs(struct user *user)
 	privset_new(&user->privs);
 	/* Load granted privs from _priv space. */
 	{
-		struct space *space = space_cache_find(SC_PRIV_ID);
+		struct space *space = space_cache_find(BOX_PRIV_ID);
 		char key[6];
 		/** Primary key - by user id */
 		Index *index = index_find(space, 0);
@@ -436,7 +436,7 @@ user_cache_find(uint32_t uid)
 struct user *
 user_cache_find_by_name(const char *name, uint32_t len)
 {
-	uint32_t uid = schema_find_id(SC_USER_ID, 2, name, len);
+	uint32_t uid = schema_find_id(BOX_USER_ID, 2, name, len);
 	struct user *user = user_by_id(uid);
 	if (user == NULL || user->type != SC_USER) {
 		char name_buf[BOX_NAME_MAX + 1];
diff --git a/src/ffisyms.cc b/src/ffisyms.cc
index e2edf632796fe4fa2b31233c38473535927b7897..980d279380b05beb40384c44fd1a1d4688fa128e 100644
--- a/src/ffisyms.cc
+++ b/src/ffisyms.cc
@@ -3,10 +3,12 @@
 #include "scramble.h"
 #include <box/box.h>
 #include <box/tuple.h>
-#include <box/lua/index.h>
+#include <box/index.h>
+#include <box/func.h>
 #include <box/lua/tuple.h>
 #include <box/lua/call.h>
 #include <box/sophia_engine.h>
+#include <box/request.h>
 #include <box/port.h>
 #include <lua/init.h>
 #include "main.h"
@@ -17,6 +19,7 @@
 #include "random.h"
 #include "iobuf.h"
 #include <lib/salad/guava.h>
+#include "latch.h"
 
 /*
  * A special hack to cc/ld to keep symbols in an optimized binary.
@@ -28,23 +31,30 @@ void *ffi_symbols[] = {
 	(void *) bswap_u64,
 	(void *) mp_bswap_float,
 	(void *) mp_bswap_double,
-	(void *) tuple_field_count,
-	(void *) tuple_field,
-	(void *) tuple_rewind,
-	(void *) tuple_seek,
-	(void *) tuple_next,
-	(void *) tuple_unref,
-	(void *) boxffi_index_len,
-	(void *) boxffi_index_bsize,
-	(void *) boxffi_index_random,
-	(void *) boxffi_index_get,
-	(void *) boxffi_index_min,
-	(void *) boxffi_index_max,
-	(void *) boxffi_index_count,
-	(void *) boxffi_index_iterator,
+	(void *) box_select,
+	(void *) box_insert,
+	(void *) box_replace,
+	(void *) box_delete,
+	(void *) box_update,
+	(void *) box_upsert,
+	(void *) box_tuple_field_count,
+	(void *) box_tuple_field,
+	(void *) box_tuple_rewind,
+	(void *) box_tuple_seek,
+	(void *) box_tuple_next,
+	(void *) box_tuple_ref,
+	(void *) box_tuple_unref,
+	(void *) box_tuple_to_buf,
+	(void *) box_index_len,
+	(void *) box_index_bsize,
+	(void *) box_index_random,
+	(void *) box_index_get,
+	(void *) box_index_min,
+	(void *) box_index_max,
+	(void *) box_index_count,
+	(void *) box_index_iterator,
+	(void *) box_iterator_next,
 	(void *) boxffi_tuple_update,
-	(void *) boxffi_iterator_next,
-	(void *) boxffi_select,
 	(void *) password_prepare,
 	(void *) tarantool_error_message,
 	(void *) load_cfg,
@@ -72,5 +82,16 @@ void *ffi_symbols[] = {
 	(void *) ibuf_reserve_nothrow_slow,
 	(void *) port_buf_create,
 	(void *) port_buf_destroy,
-	(void *) port_buf_transfer
+	(void *) port_buf_transfer,
+	(void *) box_return_tuple,
+	(void *) box_error_type,
+	(void *) box_error_code,
+	(void *) box_error_message,
+	(void *) box_error_clear,
+	(void *) box_error_last,
+	(void *) box_latch_new,
+	(void *) box_latch_delete,
+	(void *) box_latch_lock,
+	(void *) box_latch_trylock,
+	(void *) box_latch_unlock
 };
diff --git a/src/latch.cc b/src/latch.cc
new file mode 100644
index 0000000000000000000000000000000000000000..a4be37f517aae9d8458abfe24ebd1c462f07b0af
--- /dev/null
+++ b/src/latch.cc
@@ -0,0 +1,71 @@
+/*
+ * Redistribution and use in source and binary forms, with or
+ * without modification, are permitted provided that the following
+ * conditions are met:
+ *
+ * 1. Redistributions of source code must retain the above
+ *    copyright notice, this list of conditions and the
+ *    following disclaimer.
+ *
+ * 2. Redistributions in binary form must reproduce the above
+ *    copyright notice, this list of conditions and the following
+ *    disclaimer in the documentation and/or other materials
+ *    provided with the distribution.
+ *
+ * THIS SOFTWARE IS PROVIDED BY <COPYRIGHT HOLDER> ``AS IS'' AND
+ * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+ * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+ * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL
+ * <COPYRIGHT HOLDER> OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT,
+ * INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+ * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
+ * BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+ * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF
+ * THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+ * SUCH DAMAGE.
+ */
+
+#include "latch.h"
+
+struct box_latch
+{
+    struct latch l;
+};
+
+box_latch_t*
+box_latch_new(void)
+{
+	box_latch_t* bl = (box_latch_t*) malloc(sizeof(box_latch_t));
+	if (bl)
+		latch_create(&bl->l);
+	return bl;
+}
+
+void
+box_latch_delete(box_latch_t* bl)
+{
+	if (bl) {
+		latch_destroy(&bl->l);
+		free(bl);
+	}
+}
+
+void
+box_latch_lock(box_latch_t* bl)
+{
+	latch_lock(&bl->l);
+}
+
+int
+box_latch_trylock(box_latch_t* bl)
+{
+	return latch_trylock(&bl->l);
+}
+
+void
+box_latch_unlock(box_latch_t* bl)
+{
+	latch_unlock(&bl->l);
+}
diff --git a/src/latch.h b/src/latch.h
index b1dcf0dbca3601d4db09828c5690ec3a303a8c42..2d56399126c476fd6c649d209d9882096c1a6125 100644
--- a/src/latch.h
+++ b/src/latch.h
@@ -157,4 +157,28 @@ latch_unlock(struct latch *l)
 	}
 }
 
+/** \cond public */
+/**
+ * API of C stored function.
+ */
+
+typedef struct box_latch box_latch_t;
+
+API_EXPORT box_latch_t*
+box_latch_new(void);
+
+API_EXPORT void
+box_latch_delete(box_latch_t* bl);
+
+API_EXPORT void
+box_latch_lock(box_latch_t* bl);
+
+API_EXPORT int
+box_latch_trylock(box_latch_t* bl);
+
+API_EXPORT void
+box_latch_unlock(box_latch_t* bl);
+
+/** \endcond public */
+
 #endif /* TARANTOOL_LATCH_H_INCLUDED */
diff --git a/src/trivia/CMakeLists.txt b/src/trivia/CMakeLists.txt
index e81b4a585b886d5a0ec2958bf5e128984869262b..f92d73e8a16c6f2115501567fc0ab162e4fb30d3 100644
--- a/src/trivia/CMakeLists.txt
+++ b/src/trivia/CMakeLists.txt
@@ -3,5 +3,14 @@ set(api_headers
     ${CMAKE_SOURCE_DIR}/src/say.h
     ${CMAKE_SOURCE_DIR}/src/coeio.h
     ${CMAKE_SOURCE_DIR}/src/lua/utils.h
-    ${CMAKE_SOURCE_DIR}/src/box/func.h)
+    ${CMAKE_SOURCE_DIR}/src/box/txn.h
+    ${CMAKE_SOURCE_DIR}/src/box/tuple.h
+    ${CMAKE_SOURCE_DIR}/src/box/schema.h
+    ${CMAKE_SOURCE_DIR}/src/box/box.h
+    ${CMAKE_SOURCE_DIR}/src/box/index.h
+    ${CMAKE_SOURCE_DIR}/src/box/func.h
+    ${CMAKE_SOURCE_DIR}/src/box/error.h
+    ${CMAKE_SOURCE_DIR}/src/box/lua/call.h
+    ${CMAKE_SOURCE_DIR}/src/latch.h
+)
 rebuild_module_api(${api_headers})
diff --git a/src/trivia/tarantool_header.h b/src/trivia/tarantool_header.h
index 934ffab8c7f22b466eef3a59fcad5af54afaaf70..c6944e5820347a6f01f352c82df6e5eb0062dcba 100644
--- a/src/trivia/tarantool_header.h
+++ b/src/trivia/tarantool_header.h
@@ -14,6 +14,12 @@
  * Tarantool Module API
  */
 
+#if defined(__cplusplus)
+#define API_EXPORT extern "C" __attribute__ ((visibility ("default")))
+#else
+#define API_EXPORT extern __attribute__ ((visibility ("default")))
+#endif
+
 #if defined(__cplusplus)
 extern "C" {
 #endif /* defined(__cplusplus) */
diff --git a/src/trivia/util.h b/src/trivia/util.h
index 767777f7e94c440c296791d5a62ca9a9d1b508be..d0e9d8f9e6956ec6b68a7ce1e3b7ccd5e847341b 100644
--- a/src/trivia/util.h
+++ b/src/trivia/util.h
@@ -57,8 +57,8 @@ extern "C" {
 /* Macros to define enum and corresponding strings. */
 #define ENUM0_MEMBER(s, ...) s,
 #define ENUM_MEMBER(s, v, ...) s = v,
-#define ENUM0(enum_name, enum_members) enum enum_name {enum_members(ENUM0_MEMBER) enum_name##_MAX}
-#define ENUM(enum_name, enum_members) enum enum_name {enum_members(ENUM_MEMBER) enum_name##_MAX}
+#define ENUM0(enum_name, enum_members) enum enum_name { enum_members(ENUM0_MEMBER) enum_name##_MAX }
+#define ENUM(enum_name, enum_members) enum enum_name { enum_members(ENUM_MEMBER) enum_name##_MAX }
 #if defined(__cplusplus)
 #define ENUM_STRS_MEMBER(s, v, ...) names[s] = #s;
 /* A special hack to emulate C99 designated initializers */
@@ -206,4 +206,11 @@ fmemopen(void *buf, size_t size, const char *mode);
 } /* extern "C" */
 #endif /* defined(__cplusplus) */
 
+
+#if defined(__cplusplus)
+#define API_EXPORT extern "C" __attribute__ ((visibility ("default")))
+#else
+#define API_EXPORT extern __attribute__ ((visibility ("default")))
+#endif
+
 #endif /* TARANTOOL_UTIL_H_INCLUDED */
diff --git a/test/CMakeLists.txt b/test/CMakeLists.txt
index 75ad740b097bed7951dece6051a938afe325d7fe..4cf0ba46b990146170c9132c1c7185e857a2b6ac 100644
--- a/test/CMakeLists.txt
+++ b/test/CMakeLists.txt
@@ -1,5 +1,16 @@
 enable_tnt_compile_flags()
 
+include_directories(${CMAKE_BINARY_DIR}/src/trivia)
+function(build_module module files)
+    add_library(${module} SHARED ${files})
+    set_target_properties(${module} PROPERTIES PREFIX "")
+    add_dependencies(${module} rebuild_module_api)
+    if(TARGET_OS_DARWIN)
+        set_target_properties(${module} PROPERTIES LINK_FLAGS "-undefined dynamic_lookup")
+    endif(TARGET_OS_DARWIN)
+endfunction()
+
+
 add_compile_flags("C;CXX"
     "-Wno-unused-parameter")
 
@@ -14,6 +25,7 @@ add_custom_target(test-force
     COMMAND ${PROJECT_SOURCE_DIR}/test/test-run.py --builddir=${PROJECT_BINARY_DIR} --force --vardir=${PROJECT_BINARY_DIR}/test/var)
 
 add_subdirectory(app)
+add_subdirectory(box)
 add_subdirectory(unit)
 
 # Move tarantoolctl config
diff --git a/test/app/CMakeLists.txt b/test/app/CMakeLists.txt
index 804698325c69722700810776a13eda74cb01b7ea..ee67cf533af5d984f2dddf1a5d84789af0e1125f 100644
--- a/test/app/CMakeLists.txt
+++ b/test/app/CMakeLists.txt
@@ -1,14 +1 @@
-include_directories(${CMAKE_BINARY_DIR}/src/trivia)
-
-function(build_module module files)
-    add_library(${module} SHARED ${files})
-    set_target_properties(${module} PROPERTIES PREFIX "")
-    add_dependencies(${module} rebuild_module_api)
-    if(TARGET_OS_DARWIN)
-        set_target_properties(${module} PROPERTIES LINK_FLAGS "-undefined dynamic_lookup")
-    endif(TARGET_OS_DARWIN)
-endfunction()
-
-
 build_module(module_api module_api.c)
-build_module(function1 function1.c)
diff --git a/test/app/function1.c b/test/app/function1.c
deleted file mode 100644
index d0308c662bfe1eb1780a4eafffe0dbc36c2b0216..0000000000000000000000000000000000000000
--- a/test/app/function1.c
+++ /dev/null
@@ -1,18 +0,0 @@
-#include "tarantool.h"
-#include <stdio.h>
-
-int
-function1(struct request *request, struct port *port)
-{
-	say_info("-- function1 -  called --");
-	printf("ok - function1\n");
-	return 0;
-}
-
-int
-test(struct request *request, struct port *port)
-{
-	say_info("-- test  -  called --");
-	printf("ok - test\n");
-	return 0;
-}
diff --git a/test/app/function1.result b/test/app/function1.result
deleted file mode 100644
index 208183413473d241189f187817cf0d9f79201ca6..0000000000000000000000000000000000000000
--- a/test/app/function1.result
+++ /dev/null
@@ -1,2 +0,0 @@
-ok - function1
-ok - test
diff --git a/test/app/function1.test.lua b/test/app/function1.test.lua
deleted file mode 100755
index f63a02f9ae43c26ccd05db7170439295a649c5f0..0000000000000000000000000000000000000000
--- a/test/app/function1.test.lua
+++ /dev/null
@@ -1,25 +0,0 @@
-#!/usr/bin/env tarantool
-
-box.cfg{
-    listen              = os.getenv("LISTEN"),
-    slab_alloc_arena    = 0.1,
-    pid_file            = "tarantool.pid",
-    rows_per_wal        = 50,
-    logger = "tarantool.log"
-}
-
-package.cpath = '../app/?.so;../app/?.dylib;'..package.cpath
-
-log = require('log')
-net = require('net.box')
-
-box.schema.func.create('function1', {language = "C"})
-box.schema.user.grant('guest', 'execute', 'function', 'function1')
-box.schema.func.create('function1.test', {language = "C"})
-box.schema.user.grant('guest', 'execute', 'function', 'function1.test')
-
-c = net:new(os.getenv("LISTEN"))
-c:call('function1')
-c:call('function1.test')
-
-os.exit(0)
diff --git a/test/big/lua.result b/test/big/lua.result
index 07b817ce80f7cb9ca16f630bc232fcd92612d24b..4274109f5f0a03b99d32d98800b86e398d8f15fd 100644
--- a/test/big/lua.result
+++ b/test/big/lua.result
@@ -734,7 +734,7 @@ t:find(2, '2')
 ...
 t:find(89, '2')
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- error: '[string "-- tuple.lua (internal file)..."]:151: error: invalid key to ''next'''
 ...
 t:findall(4, '3')
 ---
diff --git a/test/box/CMakeLists.txt b/test/box/CMakeLists.txt
new file mode 100644
index 0000000000000000000000000000000000000000..96d411b346eb424503417c5e5805f17ca1725a24
--- /dev/null
+++ b/test/box/CMakeLists.txt
@@ -0,0 +1 @@
+build_module(function1 function1.c)
diff --git a/test/box/function1.c b/test/box/function1.c
new file mode 100644
index 0000000000000000000000000000000000000000..3af9773c199464e665d77077413fbf7bdc87be3e
--- /dev/null
+++ b/test/box/function1.c
@@ -0,0 +1,123 @@
+#include "tarantool.h"
+
+#include <stdio.h>
+
+#include "msgpuck/msgpuck.h"
+
+int
+function1(box_function_ctx_t *ctx, const char *args, const char *args_end)
+{
+	say_info("-- function1 -  called --");
+	printf("ok - function1\n");
+	return 0;
+}
+
+int
+args(box_function_ctx_t *ctx, const char *args, const char *args_end)
+{
+	uint32_t arg_count = mp_decode_array(&args);
+	if (arg_count < 1) {
+		return box_error_raise(ER_PROC_C, "%s",
+			"invalid argument count");
+	}
+
+	if (mp_typeof(*args) != MP_UINT) {
+		return box_error_raise(ER_PROC_C, "%s",
+			"first tuple field must be uint");
+	}
+
+	uint32_t num = mp_decode_uint(&args);
+
+	char tuple_buf[512];
+	char *d = tuple_buf;
+	d = mp_encode_array(d, 2);
+	d = mp_encode_uint(d, num);
+	d = mp_encode_str(d, "hello", strlen("hello"));
+	assert(d <= tuple_buf + sizeof(tuple_buf));
+
+	box_tuple_format_t *fmt = box_tuple_format_default();
+	box_tuple_t *tuple = box_tuple_new(fmt, tuple_buf, d);
+	if (tuple == NULL)
+		return -1;
+	return box_return_tuple(ctx, tuple);
+}
+
+/*
+ * For each UINT key in arguments create or increment counter in
+ * box.space.test space.
+ */
+int
+multi_inc(box_function_ctx_t *ctx, const char *args, const char *args_end)
+{
+	static const char *SPACE_NAME = "test";
+	static const char *INDEX_NAME = "primary";
+
+	uint32_t space_id = box_space_id_by_name(SPACE_NAME, strlen(SPACE_NAME));
+	uint32_t index_id = box_index_id_by_name(space_id, INDEX_NAME,
+		strlen(INDEX_NAME));
+	if (space_id == BOX_ID_NIL || index_id == BOX_ID_NIL) {
+		return box_error_raise(ER_PROC_C,
+			"Can't find index %s in space %s",
+			INDEX_NAME, SPACE_NAME);
+	}
+	say_debug("space_id = %u, index_id = %u", space_id, index_id);
+
+	uint32_t arg_count = mp_decode_array(&args);
+	box_txn_begin();
+	for (uint32_t i = 0; i < arg_count; i++) {
+		/* Decode next argument */
+		if (mp_typeof(*args) != MP_UINT)
+			return box_error_raise(ER_PROC_C, "Expected uint keys");
+		uint32_t key = mp_decode_uint(&args);
+		(void) key;
+
+		/* Prepare MsgPack key for search */
+		char key_buf[16];
+		char *key_end = key_buf;
+		key_end = mp_encode_array(key_end, 1);
+		key_end = mp_encode_uint(key_end, key);
+		assert(key_end < key_buf + sizeof(key_buf));
+
+		/* Get current value from space */
+		uint64_t counter = 0;
+		box_tuple_t *tuple;
+		if (box_index_get(space_id, index_id, key_buf, key_end,
+				  &tuple) != 0) {
+			return -1; /* error */
+		} else if (tuple != NULL) {
+			const char *field = box_tuple_field(tuple, 1);
+			if (field == NULL || mp_typeof(*field) != MP_UINT)
+				return box_error_raise(ER_PROC_LUA, "Invalid tuple");
+			counter = mp_decode_uint(&field) + 1;
+		}
+
+		/* Replace value */
+		char tuple_buf[16];
+		char *tuple_end = tuple_buf;
+		tuple_end = mp_encode_array(tuple_end, 2);
+		tuple_end = mp_encode_uint(tuple_end, key); /* key */
+		tuple_end = mp_encode_uint(tuple_end, counter); /* counter */
+		assert(tuple_end <= tuple_buf + sizeof(tuple_buf));
+
+		if (box_replace(space_id, tuple_buf, tuple_end, NULL) != 0)
+			return -1;
+	}
+	box_txn_commit();
+	return 0;
+}
+
+int
+errors(box_function_ctx_t *ctx, const char *args, const char *args_end)
+{
+	box_error_raise(ER_PROC_C, "%s", "Proc error");
+
+	const box_error_t *error = box_error_last();
+	assert(strcmp(box_error_type(error), "ClientError") == 0);
+	assert(box_error_code(error) == ER_PROC_C);
+	assert(strcmp(box_error_message(error), "Proc error") == 0);
+
+	box_error_clear();
+	assert(box_error_last() == NULL);
+
+	return -1; /* raises "Unknown procedure error" */
+}
diff --git a/test/box/function1.result b/test/box/function1.result
new file mode 100644
index 0000000000000000000000000000000000000000..6622f0cb59366c3a3044f8ff04cb8a351c30aa03
--- /dev/null
+++ b/test/box/function1.result
@@ -0,0 +1,140 @@
+package.cpath = '../box/?.so;../box/?.dylib;'..package.cpath
+---
+...
+log = require('log')
+---
+...
+net = require('net.box')
+---
+...
+c = net:new(os.getenv("LISTEN"))
+---
+...
+box.schema.func.create('function1', {language = "C"})
+---
+...
+box.schema.user.grant('guest', 'execute', 'function', 'function1')
+---
+...
+_ = box.schema.space.create('test')
+---
+...
+_ = box.space.test:create_index('primary')
+---
+...
+box.schema.user.grant('guest', 'read,write', 'space', 'test')
+---
+...
+c:call('function1')
+---
+- []
+...
+box.schema.func.drop("function1")
+---
+...
+box.schema.func.create('function1.args', {language = "C"})
+---
+...
+box.schema.user.grant('guest', 'execute', 'function', 'function1.args')
+---
+...
+c:call('function1.args')
+---
+- error: invalid argument count
+...
+c:call('function1.args', "xx")
+---
+- error: first tuple field must be uint
+...
+c:call('function1.args', 15)
+---
+- - [15, 'hello']
+...
+box.schema.func.drop("function1.args")
+---
+...
+box.schema.func.create('function1.multi_inc', {language = "C"})
+---
+...
+box.schema.user.grant('guest', 'execute', 'function', 'function1.multi_inc')
+---
+...
+c:call('function1.multi_inc')
+---
+- []
+...
+box.space.test:select{}
+---
+- []
+...
+c:call('function1.multi_inc', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)
+---
+- []
+...
+box.space.test:select{}
+---
+- - [1, 0]
+  - [2, 0]
+  - [3, 0]
+  - [4, 0]
+  - [5, 0]
+  - [6, 0]
+  - [7, 0]
+  - [8, 0]
+  - [9, 0]
+  - [10, 0]
+...
+c:call('function1.multi_inc', 2, 4, 6, 8, 10)
+---
+- []
+...
+box.space.test:select{}
+---
+- - [1, 0]
+  - [2, 1]
+  - [3, 0]
+  - [4, 1]
+  - [5, 0]
+  - [6, 1]
+  - [7, 0]
+  - [8, 1]
+  - [9, 0]
+  - [10, 1]
+...
+c:call('function1.multi_inc', 0, 2, 4)
+---
+- []
+...
+box.space.test:select{}
+---
+- - [0, 0]
+  - [1, 0]
+  - [2, 2]
+  - [3, 0]
+  - [4, 2]
+  - [5, 0]
+  - [6, 1]
+  - [7, 0]
+  - [8, 1]
+  - [9, 0]
+  - [10, 1]
+...
+box.schema.func.drop("function1.multi_inc")
+---
+...
+box.schema.func.create('function1.errors', {language = "C"})
+---
+...
+box.schema.user.grant('guest', 'execute', 'function', 'function1.errors')
+---
+...
+c:call('function1.errors')
+---
+- error: unknown procedure error
+...
+box.schema.func.drop("function1.errors")
+---
+...
+box.space.test:drop()
+---
+...
diff --git a/test/box/function1.test.lua b/test/box/function1.test.lua
new file mode 100644
index 0000000000000000000000000000000000000000..5b873c90495de148e5312c106855cc8ffb61bd80
--- /dev/null
+++ b/test/box/function1.test.lua
@@ -0,0 +1,43 @@
+package.cpath = '../box/?.so;../box/?.dylib;'..package.cpath
+
+log = require('log')
+net = require('net.box')
+
+c = net:new(os.getenv("LISTEN"))
+
+box.schema.func.create('function1', {language = "C"})
+box.schema.user.grant('guest', 'execute', 'function', 'function1')
+_ = box.schema.space.create('test')
+_ = box.space.test:create_index('primary')
+box.schema.user.grant('guest', 'read,write', 'space', 'test')
+
+c:call('function1')
+box.schema.func.drop("function1")
+
+box.schema.func.create('function1.args', {language = "C"})
+box.schema.user.grant('guest', 'execute', 'function', 'function1.args')
+c:call('function1.args')
+c:call('function1.args', "xx")
+c:call('function1.args', 15)
+box.schema.func.drop("function1.args")
+
+box.schema.func.create('function1.multi_inc', {language = "C"})
+box.schema.user.grant('guest', 'execute', 'function', 'function1.multi_inc')
+
+c:call('function1.multi_inc')
+box.space.test:select{}
+c:call('function1.multi_inc', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)
+box.space.test:select{}
+c:call('function1.multi_inc', 2, 4, 6, 8, 10)
+box.space.test:select{}
+c:call('function1.multi_inc', 0, 2, 4)
+box.space.test:select{}
+
+box.schema.func.drop("function1.multi_inc")
+
+box.schema.func.create('function1.errors', {language = "C"})
+box.schema.user.grant('guest', 'execute', 'function', 'function1.errors')
+c:call('function1.errors')
+box.schema.func.drop("function1.errors")
+
+box.space.test:drop()
diff --git a/test/box/misc.result b/test/box/misc.result
index 7a3030305d43d8e7f76d51a604d3a76ff21ebba1..c7b5e72976bcd0105f7f920617e6feb4eb7951bc 100644
--- a/test/box/misc.result
+++ b/test/box/misc.result
@@ -257,7 +257,8 @@ t;
   - 'box.error.DROP_USER : 44'
   - 'box.error.CROSS_ENGINE_TRANSACTION : 81'
   - 'box.error.injection : table: <address>
-  - 'box.error.RTREE_RECT_ERROR : 101'
+  - 'box.error.REPLICA_MAX : 73'
+  - 'box.error.RTREE_RECT : 101'
   - 'box.error.FUNCTION_LANGUAGE : 100'
   - 'box.error.MODIFY_INDEX : 14'
   - 'box.error.TUPLE_FOUND : 3'
@@ -297,7 +298,7 @@ t;
   - 'box.error.GUEST_USER_PASSWORD : 96'
   - 'box.error.INVALID_XLOG_NAME : 75'
   - 'box.error.INVALID_XLOG : 74'
-  - 'box.error.REPLICA_MAX : 73'
+  - 'box.error.PROC_C : 102'
   - 'box.error.ITERATOR_TYPE : 72'
   - 'box.error.NONMASTER : 6'
   - 'box.error.SPACE_EXISTS : 10'
diff --git a/test/box/select.result b/test/box/select.result
index e621bcfb0ce102b73036a8fb5fc03c8ea9617b7b..54d0b5836f93158ad4c1d0ef1fa19f7744c6e73d 100644
--- a/test/box/select.result
+++ b/test/box/select.result
@@ -638,7 +638,7 @@ while (true) do table.insert(lots_of_links, s:get{0}) ref_count = ref_count + 1
 ...
 ref_count
 ---
-- 65534
+- 65532
 ...
 lots_of_links = {}
 ---
diff --git a/test/box/tuple.result b/test/box/tuple.result
index 9bb0e2fde2bd31ded68c240e0e20be7471fa122c..b7eb8ad455172a142c1962a0617ea50b6f6c498e 100644
--- a/test/box/tuple.result
+++ b/test/box/tuple.result
@@ -305,12 +305,12 @@ t:unpack(2, 1)
 ...
 t:totable(0)
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:116: tuple.totable: invalid second
+- error: '[string "-- tuple.lua (internal file)..."]:182: tuple.totable: invalid second
     argument'
 ...
 t:totable(1, 0)
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:125: tuple.totable: invalid third
+- error: '[string "-- tuple.lua (internal file)..."]:191: tuple.totable: invalid third
     argument'
 ...
 --
@@ -435,15 +435,16 @@ t:next(3)
 ...
 t:next(4)
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- null
 ...
 t:next(-1)
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- null
 ...
 t:next("fdsaf")
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:72: error: invalid key to ''next'''
+- error: '[string "-- tuple.lua (internal file)..."]:163: bad argument #2 to ''box_tuple_field''
+    (cannot convert ''string'' to ''unsigned int'')'
 ...
 box.tuple.new({'x', 'y', 'z'}):next()
 ---
@@ -455,7 +456,7 @@ t=space:insert{1953719668}
 ...
 t:next(1684234849)
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- null
 ...
 t:next(1)
 ---
@@ -611,7 +612,7 @@ r = {}
 ...
 for _state, val in t:pairs(10) do table.insert(r, val) end
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- error: '[string "-- tuple.lua (internal file)..."]:151: error: invalid key to ''next'''
 ...
 r
 ---
@@ -697,19 +698,19 @@ t:findall(1, 'xxxxx')
 ...
 t:find(100, 'a')
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- error: '[string "-- tuple.lua (internal file)..."]:151: error: invalid key to ''next'''
 ...
 t:findall(100, 'a')
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- error: '[string "-- tuple.lua (internal file)..."]:151: error: invalid key to ''next'''
 ...
 t:find(100, 'xxxxx')
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- error: '[string "-- tuple.lua (internal file)..."]:151: error: invalid key to ''next'''
 ...
 t:findall(100, 'xxxxx')
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:89: error: invalid key to ''next'''
+- error: '[string "-- tuple.lua (internal file)..."]:151: error: invalid key to ''next'''
 ...
 ---
 -- Lua type coercion
@@ -803,12 +804,12 @@ t = box.tuple.new({'a', 'b', 'c', 'd', 'e'})
 ...
 t:update()
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:165: Usage: tuple:update({ {
+- error: '[string "-- tuple.lua (internal file)..."]:231: Usage: tuple:update({ {
     op, field, arg}+ })'
 ...
 t:update(10)
 ---
-- error: '[string "-- tuple.lua (internal file)..."]:165: Usage: tuple:update({ {
+- error: '[string "-- tuple.lua (internal file)..."]:231: Usage: tuple:update({ {
     op, field, arg}+ })'
 ...
 t:update({})
diff --git a/test/box/update.result b/test/box/update.result
index 30be9e9fe3b35ab656a998ca4137f7394cc277a4..94763b76fd8d7fd3738fdf35a538d54a255db4b8 100644
--- a/test/box/update.result
+++ b/test/box/update.result
@@ -454,7 +454,7 @@ s:insert{1, 2, 3}
 ...
 s:update({1})
 ---
-- error: Usage space:update(key, ops)
+- error: Usage index:update(key, ops)
 ...
 s:update({1}, {'=', 1, 1})
 ---
@@ -767,7 +767,7 @@ s:delete{0}
 ...
 s:upsert({0}, {{'+', 2, 2}}) -- wrong usage
 ---
-- error: Usage space:upsert(key, ops, tuple)
+- error: Usage index:upsert(key, ops, tuple)
 ...
 s:select{0}
 ---