* [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE
@ 2018-07-27 11:29 Vladimir Davydov
2018-07-27 11:29 ` [PATCH 01/25] vinyl: make point lookup always return the latest tuple version Vladimir Davydov
` (24 more replies)
0 siblings, 25 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
This patch set optimizes REPLACE and DELETE operations in vinyl in
presence of secondary indexes: now they don't need to read the primary
key in order to delete the overwritten/deleted tuple from secondary
indexes, instead this job is handed over to primary index compaction
task, while the read iterator is tought to filter out overwritten
tuples that haven't been purged yet.
The primary difference of this patch set from the RFC version, which can
be found by the link
https://www.freelists.org/post/tarantool-patches/RFC-PATCH-0023-vinyl-eliminate-read-on-REPLACEDELETE
is that it resolves all the problems the RFC suffered from, namely
potential OOM during deferred DELETE generation, lost DELETEs on
restart, severe performance degradation in case a lot of tuples get
overwritten in memory, missing functional tests.
For more information about the algorithm and implementation details,
please see the final patch of the series.
https://github.com/tarantool/tarantool/issues/2129
https://github.com/tarantool/tarantool/commits/dv/gh-2129-vy-eliminate-read-on-replace-delete
Vladimir Davydov (25):
vinyl: make point lookup always return the latest tuple version
vinyl: simplify vy_squash_process
vinyl: always get full tuple from pk after reading from secondary
index
vinyl: fold vy_replace_one and vy_replace_impl
vinyl: fold vy_delete_impl
vinyl: refactor unique check
vinyl: check key uniqueness before modifying tx write set
vinyl: remove env argument of vy_check_is_unique_{primary,secondary}
vinyl: store full tuples in secondary index cache
vinyl: do not free pending tasks on shutdown
vinyl: store pointer to scheduler in struct vy_task
vinyl: rename some members of vy_scheduler and vy_task struct
vinyl: use cbus for communication between scheduler and worker threads
vinyl: zap vy_scheduler::is_worker_pool_running
vinyl: rename vy_task::status to is_failed
xrow: allow to store flags in DML requests
vinyl: pin last statement returned by write iterator explicitly
vinyl: teach write iterator to return overwritten tuples
vinyl: prepare write iterator heap comparator for deferred DELETEs
vinyl: allow to skip certain statements on read
vinyl: add function to create surrogate deletes from raw msgpack
vinyl: remove pointless assertion from vy_stmt_new_surrogate_delete
txn: add helper to detect transaction boundaries
Introduce _vinyl_deferred_delete system space
vinyl: eliminate disk read on REPLACE/DELETE
src/box/bootstrap.snap | Bin 1540 -> 1605 bytes
src/box/iproto_constants.c | 4 +-
src/box/iproto_constants.h | 3 +-
src/box/lua/space.cc | 2 +
src/box/lua/upgrade.lua | 21 +
src/box/schema.cc | 72 ++-
src/box/schema_def.h | 2 +
src/box/txn.c | 3 +-
src/box/txn.h | 11 +
src/box/vinyl.c | 1019 ++++++++++++++++++-----------------
src/box/vy_lsm.h | 5 +
src/box/vy_mem.c | 20 +-
src/box/vy_mem.h | 5 +
src/box/vy_point_lookup.c | 72 ++-
src/box/vy_point_lookup.h | 27 +-
src/box/vy_read_iterator.c | 61 +--
src/box/vy_read_iterator.h | 24 +
src/box/vy_run.c | 7 +-
src/box/vy_scheduler.c | 685 +++++++++++++++++------
src/box/vy_scheduler.h | 33 +-
src/box/vy_stmt.c | 12 +-
src/box/vy_stmt.h | 56 +-
src/box/vy_tx.c | 133 +++++
src/box/vy_write_iterator.c | 168 +++++-
src/box/vy_write_iterator.h | 27 +-
src/box/xrow.c | 8 +
src/box/xrow.h | 2 +
test/app-tap/tarantoolctl.test.lua | 2 +-
test/box-py/bootstrap.result | 5 +-
test/box/access_misc.result | 3 +
test/box/access_sysview.result | 2 +-
test/box/alter.result | 4 +-
test/unit/vy_iterators_helper.c | 5 +
test/unit/vy_iterators_helper.h | 12 +-
test/unit/vy_point_lookup.c | 6 +-
test/unit/vy_write_iterator.c | 232 +++++++-
test/unit/vy_write_iterator.result | 22 +-
test/vinyl/deferred_delete.result | 677 +++++++++++++++++++++++
test/vinyl/deferred_delete.test.lua | 261 +++++++++
test/vinyl/info.result | 18 +-
test/vinyl/info.test.lua | 9 +-
test/vinyl/layout.result | 46 +-
test/vinyl/quota.result | 2 +-
test/vinyl/tx_gap_lock.result | 16 +-
test/vinyl/tx_gap_lock.test.lua | 10 +-
test/vinyl/write_iterator.result | 5 +
test/vinyl/write_iterator.test.lua | 3 +
test/wal_off/alter.result | 2 +-
test/xlog/upgrade.result | 5 +-
49 files changed, 2953 insertions(+), 876 deletions(-)
create mode 100644 test/vinyl/deferred_delete.result
create mode 100644 test/vinyl/deferred_delete.test.lua
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 01/25] vinyl: make point lookup always return the latest tuple version
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 02/25] vinyl: simplify vy_squash_process Vladimir Davydov
` (23 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Currently, vy_point_lookup(), in contrast to vy_read_iterator, doesn't
rescan the memory level after reading disk, so if the caller doesn't
track the key before calling this function, the caller won't be sent to
a read view in case the key gets updated during yield and hence will
be returned a stale tuple. This is OK now, because we always track the
key before calling vy_point_lookup(), either in the primary or in a
secondary index. However, for #2129 we need it to always return the
latest tuple version, no matter if the key is tracked or not.
The point is in the scope of #2129 we won't write DELETE statements to
secondary indexes corresponding to a tuple replaced in the primary
index. Instead after reading a tuple from a secondary index we will
check whether it matches the tuple corresponding to it in the primary
index: if it is not, it means that the tuple read from the secondary
index was overwritten and should be skipped. E.g. suppose we have the
primary index over the first field and a secondary index over the second
field and the following statements in the space:
REPLACE{1, 10}
REPLACE{1, 20}
Then reading {10} from the secondary index will return REPLACE{1, 10}, but
lookup of {1} in the primary index will return REPLACE{1, 20} which
doesn't match REPLACE{1, 10} read from the secondary index hence the
latter was overwritten and should be skipped.
The problem is in the example above we don't want to track key {1} in
the primary index before lookup, because we don't actually read its
value. So for the check to work correctly, we need the point lookup to
guarantee that the returned tuple is always the newest one. It's fairly
easy to do - we just need to rescan the memory level after yielding on
disk if its version changed.
Needed for #2129
---
src/box/vy_point_lookup.c | 35 +++++++++++++++++++++++++++++------
src/box/vy_point_lookup.h | 9 +++------
2 files changed, 32 insertions(+), 12 deletions(-)
diff --git a/src/box/vy_point_lookup.c b/src/box/vy_point_lookup.c
index 504a8e80..f2261fdf 100644
--- a/src/box/vy_point_lookup.c
+++ b/src/box/vy_point_lookup.c
@@ -203,10 +203,13 @@ vy_point_lookup(struct vy_lsm *lsm, struct vy_tx *tx,
int rc = 0;
lsm->stat.lookup++;
+
/* History list */
- struct vy_history history;
+ struct vy_history history, mem_history, disk_history;
vy_history_create(&history, &lsm->env->history_node_pool);
-restart:
+ vy_history_create(&mem_history, &lsm->env->history_node_pool);
+ vy_history_create(&disk_history, &lsm->env->history_node_pool);
+
rc = vy_point_lookup_scan_txw(lsm, tx, key, &history);
if (rc != 0 || vy_history_is_terminal(&history))
goto done;
@@ -215,14 +218,16 @@ restart:
if (rc != 0 || vy_history_is_terminal(&history))
goto done;
- rc = vy_point_lookup_scan_mems(lsm, rv, key, &history);
- if (rc != 0 || vy_history_is_terminal(&history))
+restart:
+ rc = vy_point_lookup_scan_mems(lsm, rv, key, &mem_history);
+ if (rc != 0 || vy_history_is_terminal(&mem_history))
goto done;
/* Save version before yield */
+ uint32_t mem_version = lsm->mem->version;
uint32_t mem_list_version = lsm->mem_list_version;
- rc = vy_point_lookup_scan_slices(lsm, rv, key, &history);
+ rc = vy_point_lookup_scan_slices(lsm, rv, key, &disk_history);
if (rc != 0)
goto done;
@@ -241,11 +246,29 @@ restart:
* This in unnecessary in case of rotation but since we
* cannot distinguish these two cases we always restart.
*/
- vy_history_cleanup(&history);
+ vy_history_cleanup(&mem_history);
+ vy_history_cleanup(&disk_history);
goto restart;
}
+ if (mem_version != lsm->mem->version) {
+ /*
+ * Rescan the memory level if its version changed while we
+ * were reading disk, because there may be new statements
+ * matching the search key.
+ */
+ vy_history_cleanup(&mem_history);
+ rc = vy_point_lookup_scan_mems(lsm, rv, key, &mem_history);
+ if (rc != 0)
+ goto done;
+ if (vy_history_is_terminal(&mem_history))
+ vy_history_cleanup(&disk_history);
+ }
+
done:
+ vy_history_splice(&history, &mem_history);
+ vy_history_splice(&history, &disk_history);
+
if (rc == 0) {
int upserts_applied;
rc = vy_history_apply(&history, lsm->cmp_def, lsm->mem_format,
diff --git a/src/box/vy_point_lookup.h b/src/box/vy_point_lookup.h
index d74be9a9..3b7c5a04 100644
--- a/src/box/vy_point_lookup.h
+++ b/src/box/vy_point_lookup.h
@@ -62,12 +62,9 @@ struct tuple;
* tuple in the LSM tree. The tuple is returned in @ret with its
* reference counter elevated.
*
- * The caller must guarantee that if the tuple looked up by this
- * function is modified, the transaction will be sent to read view.
- * This is needed to avoid inserting a stale value into the cache.
- * In other words, vy_tx_track() must be called for the search key
- * before calling this function unless this is a primary index and
- * the tuple is already tracked in a secondary index.
+ * Note, this function doesn't track the result in the transaction
+ * read set, i.e. it is up to the caller to call vy_tx_track() if
+ * necessary.
*/
int
vy_point_lookup(struct vy_lsm *lsm, struct vy_tx *tx,
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 02/25] vinyl: simplify vy_squash_process
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
2018-07-27 11:29 ` [PATCH 01/25] vinyl: make point lookup always return the latest tuple version Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 03/25] vinyl: always get full tuple from pk after reading from secondary index Vladimir Davydov
` (22 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Since vy_point_lookup() now guarantees that it returns the newest
tuple version, we can remove the code that squashes UPSERTs from
vy_squash_process().
---
src/box/vinyl.c | 115 ++++++--------------------------------------------------
1 file changed, 12 insertions(+), 103 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index 374c5252..530820e8 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -3532,11 +3532,6 @@ vy_squash_process(struct vy_squash *squash)
struct vy_lsm *lsm = squash->lsm;
struct vy_env *env = squash->env;
- /*
- * vy_apply_upsert() is used for primary key only,
- * so this is the same as lsm->key_def
- */
- struct key_def *def = lsm->cmp_def;
/* Upserts enabled only in the primary index LSM tree. */
assert(lsm->index_id == 0);
@@ -3554,8 +3549,10 @@ vy_squash_process(struct vy_squash *squash)
/*
* While we were reading on-disk runs, new statements could
- * have been inserted into the in-memory tree. Apply them to
- * the result.
+ * have been prepared for the squashed key. We mustn't apply
+ * them, because they may be rolled back, but we must adjust
+ * their n_upserts counter so that they will get squashed by
+ * vy_lsm_commit_upsert().
*/
struct vy_mem *mem = lsm->mem;
struct tree_mem_key tree_key = {
@@ -3572,108 +3569,20 @@ vy_squash_process(struct vy_squash *squash)
tuple_unref(result);
return 0;
}
- /**
- * Algorithm of the squashing.
- * Assume, during building the non-UPSERT statement
- * 'result' in the mem some new UPSERTs were inserted, and
- * some of them were commited, while the other were just
- * prepared. And lets UPSERT_THRESHOLD to be equal to 3,
- * for example.
- * Mem
- * -------------------------------------+
- * UPSERT, lsn = 1, n_ups = 0 |
- * UPSERT, lsn = 2, n_ups = 1 | Commited
- * UPSERT, lsn = 3, n_ups = 2 |
- * -------------------------------------+
- * UPSERT, lsn = MAX, n_ups = 3 |
- * UPSERT, lsn = MAX + 1, n_ups = 4 | Prepared
- * UPSERT, lsn = MAX + 2, n_ups = 5 |
- * -------------------------------------+
- * In such a case the UPSERT statements with
- * lsns = {1, 2, 3} are squashed. But now the n_upsert
- * values in the prepared statements are not correct.
- * If we will not update values, then the
- * vy_lsm_commit_upsert will not be able to squash them.
- *
- * So after squashing it is necessary to update n_upsert
- * value in the prepared statements:
- * Mem
- * -------------------------------------+
- * UPSERT, lsn = 1, n_ups = 0 |
- * UPSERT, lsn = 2, n_ups = 1 | Commited
- * REPLACE, lsn = 3 |
- * -------------------------------------+
- * UPSERT, lsn = MAX, n_ups = 0 !!! |
- * UPSERT, lsn = MAX + 1, n_ups = 1 !!! | Prepared
- * UPSERT, lsn = MAX + 2, n_ups = 2 !!! |
- * -------------------------------------+
- */
vy_mem_tree_iterator_prev(&mem->tree, &mem_itr);
- const struct tuple *mem_stmt;
- int64_t stmt_lsn;
- /*
- * According to the described algorithm, squash the
- * commited UPSERTs at first.
- */
+ uint8_t n_upserts = 0;
while (!vy_mem_tree_iterator_is_invalid(&mem_itr)) {
+ const struct tuple *mem_stmt;
mem_stmt = *vy_mem_tree_iterator_get_elem(&mem->tree, &mem_itr);
- stmt_lsn = vy_stmt_lsn(mem_stmt);
- if (vy_tuple_compare(result, mem_stmt, def) != 0)
- break;
- /**
- * Leave alone prepared statements; they will be handled
- * in vy_range_commit_stmt.
- */
- if (stmt_lsn >= MAX_LSN)
+ if (vy_tuple_compare(result, mem_stmt, lsm->cmp_def) != 0 ||
+ vy_stmt_type(mem_stmt) != IPROTO_UPSERT)
break;
- if (vy_stmt_type(mem_stmt) != IPROTO_UPSERT) {
- /**
- * Somebody inserted non-upsert statement,
- * squashing is useless.
- */
- tuple_unref(result);
- return 0;
- }
- assert(lsm->index_id == 0);
- struct tuple *applied = vy_apply_upsert(mem_stmt, result, def,
- mem->format, true);
- lsm->stat.upsert.applied++;
- tuple_unref(result);
- if (applied == NULL)
- return -1;
- result = applied;
- /**
- * In normal cases we get a result with the same lsn as
- * in mem_stmt.
- * But if there are buggy upserts that do wrong things,
- * they are ignored and the result has lower lsn.
- * We should fix the lsn in any case to replace
- * exactly mem_stmt in general and the buggy upsert
- * in particular.
- */
- vy_stmt_set_lsn(result, stmt_lsn);
+ assert(vy_stmt_lsn(mem_stmt) >= MAX_LSN);
+ vy_stmt_set_n_upserts((struct tuple *)mem_stmt, n_upserts);
+ if (n_upserts <= VY_UPSERT_THRESHOLD)
+ ++n_upserts;
vy_mem_tree_iterator_prev(&mem->tree, &mem_itr);
}
- /*
- * The second step of the algorithm above is updating of
- * n_upsert values of the prepared UPSERTs.
- */
- if (stmt_lsn >= MAX_LSN) {
- uint8_t n_upserts = 0;
- while (!vy_mem_tree_iterator_is_invalid(&mem_itr)) {
- mem_stmt = *vy_mem_tree_iterator_get_elem(&mem->tree,
- &mem_itr);
- if (vy_tuple_compare(result, mem_stmt, def) != 0 ||
- vy_stmt_type(mem_stmt) != IPROTO_UPSERT)
- break;
- assert(vy_stmt_lsn(mem_stmt) >= MAX_LSN);
- vy_stmt_set_n_upserts((struct tuple *)mem_stmt,
- n_upserts);
- if (n_upserts <= VY_UPSERT_THRESHOLD)
- ++n_upserts;
- vy_mem_tree_iterator_prev(&mem->tree, &mem_itr);
- }
- }
lsm->stat.upsert.squashed++;
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 03/25] vinyl: always get full tuple from pk after reading from secondary index
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
2018-07-27 11:29 ` [PATCH 01/25] vinyl: make point lookup always return the latest tuple version Vladimir Davydov
2018-07-27 11:29 ` [PATCH 02/25] vinyl: simplify vy_squash_process Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 04/25] vinyl: fold vy_replace_one and vy_replace_impl Vladimir Davydov
` (21 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Currently, we don't always need a full tuple. Sometimes (e.g. for
checking uniqueness constraint), a partial tuple read from a secondary
index is enough. So we have vy_lsm_get() which reads a partial tuple
from an index. However, once the optimization described in #2129 is
implemented, it might happen that a tuple read from a secondary index
was overwritten or deleted in the primary index, but DELETE statement
hasn't been propagated to the secondary index yet, i.e. we will have to
read the primary index anyway, even if we don't need a full tuple.
That said, let us:
- Make vy_lsm_get() always fetch a full tuple, even for secondary
indexes, and rename it to vy_get().
- Rewrite vy_lsm_full_by_key() as a wrapper around vy_get() and rename
it to vy_get_by_raw_key().
- Introduce vy_get_by_secondary_tuple() which gets a full tuple given a
tuple read from a secondary index. For now, it's basically a call to
vy_point_lookup(), but it'll become a bit more complex once #2129 is
implemented.
- Prepare vy_get() for the fact that a tuple read from a secondary
index may be absent in the primary index, in which case it should
try the next matching one.
Needed for #2129
---
src/box/vinyl.c | 204 ++++++++++++++++++++++++++++++++------------------------
1 file changed, 118 insertions(+), 86 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index 530820e8..a2e2a447 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -1253,7 +1253,48 @@ vy_is_committed(struct vy_env *env, struct space *space)
}
/**
- * Get a vinyl tuple from the LSM tree by the key.
+ * Get a full tuple by a tuple read from a secondary index.
+ * @param lsm LSM tree from which the tuple was read.
+ * @param tx Current transaction.
+ * @param rv Read view.
+ * @param tuple Tuple read from a secondary index.
+ * @param[out] result The found tuple is stored here. Must be
+ * unreferenced after usage.
+ *
+ * @param 0 Success.
+ * @param -1 Memory error or read error.
+ */
+static int
+vy_get_by_secondary_tuple(struct vy_lsm *lsm, struct vy_tx *tx,
+ const struct vy_read_view **rv,
+ struct tuple *tuple, struct tuple **result)
+{
+ assert(lsm->index_id > 0);
+ /*
+ * No need in vy_tx_track() as the tuple must already be
+ * tracked in the secondary index LSM tree.
+ */
+ if (vy_point_lookup(lsm->pk, tx, rv, tuple, result) != 0)
+ return -1;
+
+ if (*result == NULL) {
+ /*
+ * All indexes of a space must be consistent, i.e.
+ * if a tuple is present in one index, it must be
+ * present in all other indexes as well, so we can
+ * get here only if there's a bug somewhere in vinyl.
+ * Don't abort as core dump won't really help us in
+ * this case. Just warn the user and proceed to the
+ * next tuple.
+ */
+ say_warn("%s: key %s missing in primary index",
+ vy_lsm_name(lsm), vy_stmt_str(tuple));
+ }
+ return 0;
+}
+
+/**
+ * Get a tuple from a vinyl space by key.
* @param lsm LSM tree in which search.
* @param tx Current transaction.
* @param rv Read view.
@@ -1264,10 +1305,10 @@ vy_is_committed(struct vy_env *env, struct space *space)
* @param 0 Success.
* @param -1 Memory error or read error.
*/
-static inline int
-vy_lsm_get(struct vy_lsm *lsm, struct vy_tx *tx,
- const struct vy_read_view **rv,
- struct tuple *key, struct tuple **result)
+static int
+vy_get(struct vy_lsm *lsm, struct vy_tx *tx,
+ const struct vy_read_view **rv,
+ struct tuple *key, struct tuple **result)
{
/*
* tx can be NULL, for example, if an user calls
@@ -1275,22 +1316,75 @@ vy_lsm_get(struct vy_lsm *lsm, struct vy_tx *tx,
*/
assert(tx == NULL || tx->state == VINYL_TX_READY);
+ int rc;
+ struct tuple *tuple;
+
if (tuple_field_count(key) >= lsm->cmp_def->part_count) {
+ /*
+ * Use point lookup for a full key.
+ */
if (tx != NULL && vy_tx_track_point(tx, lsm, key) != 0)
return -1;
- return vy_point_lookup(lsm, tx, rv, key, result);
+ if (vy_point_lookup(lsm, tx, rv, key, &tuple) != 0)
+ return -1;
+ if (lsm->index_id > 0 && tuple != NULL) {
+ rc = vy_get_by_secondary_tuple(lsm, tx, rv,
+ tuple, result);
+ tuple_unref(tuple);
+ if (rc != 0)
+ return -1;
+ } else {
+ *result = tuple;
+ }
+ return 0;
}
struct vy_read_iterator itr;
vy_read_iterator_open(&itr, lsm, tx, ITER_EQ, key, rv);
- int rc = vy_read_iterator_next(&itr, result);
- if (*result != NULL)
- tuple_ref(*result);
+ while ((rc = vy_read_iterator_next(&itr, &tuple)) == 0) {
+ if (lsm->index_id == 0 || tuple == NULL) {
+ *result = tuple;
+ if (tuple != NULL)
+ tuple_ref(tuple);
+ break;
+ }
+ rc = vy_get_by_secondary_tuple(lsm, tx, rv, tuple, result);
+ if (rc != 0 || *result != NULL)
+ break;
+ }
vy_read_iterator_close(&itr);
return rc;
}
/**
+ * Get a tuple from a vinyl space by raw key.
+ * @param lsm LSM tree in which search.
+ * @param tx Current transaction.
+ * @param rv Read view.
+ * @param key_raw MsgPack array of key fields.
+ * @param part_count Count of parts in the key.
+ * @param[out] result The found tuple is stored here. Must be
+ * unreferenced after usage.
+ *
+ * @param 0 Success.
+ * @param -1 Memory error or read error.
+ */
+static int
+vy_get_by_raw_key(struct vy_lsm *lsm, struct vy_tx *tx,
+ const struct vy_read_view **rv,
+ const char *key_raw, uint32_t part_count,
+ struct tuple **result)
+{
+ struct tuple *key = vy_stmt_new_select(lsm->env->key_format,
+ key_raw, part_count);
+ if (key == NULL)
+ return -1;
+ int rc = vy_get(lsm, tx, rv, key, result);
+ tuple_unref(key);
+ return rc;
+}
+
+/**
* Check if the LSM tree contains the key. If true, then set
* a duplicate key error in the diagnostics area.
* @param env Vinyl environment.
@@ -1317,7 +1411,7 @@ vy_check_is_unique(struct vy_env *env, struct vy_tx *tx,
*/
if (env->status != VINYL_ONLINE)
return 0;
- if (vy_lsm_get(lsm, tx, rv, key, &found))
+ if (vy_get(lsm, tx, rv, key, &found))
return -1;
if (found) {
@@ -1468,8 +1562,8 @@ vy_replace_one(struct vy_env *env, struct vy_tx *tx, struct space *space,
* old tuple to pass it to the trigger.
*/
if (stmt != NULL && !rlist_empty(&space->on_replace)) {
- if (vy_lsm_get(pk, tx, vy_tx_read_view(tx),
- new_tuple, &stmt->old_tuple) != 0)
+ if (vy_get(pk, tx, vy_tx_read_view(tx),
+ new_tuple, &stmt->old_tuple) != 0)
goto error_unref;
}
if (vy_tx_set(tx, pk, new_tuple))
@@ -1524,8 +1618,7 @@ vy_replace_impl(struct vy_env *env, struct vy_tx *tx, struct space *space,
return -1;
/* Get full tuple from the primary index. */
- if (vy_lsm_get(pk, tx, vy_tx_read_view(tx),
- new_stmt, &old_stmt) != 0)
+ if (vy_get(pk, tx, vy_tx_read_view(tx), new_stmt, &old_stmt) != 0)
goto error;
if (old_stmt == NULL) {
@@ -1624,51 +1717,6 @@ vy_unique_key_validate(struct vy_lsm *lsm, const char *key,
}
/**
- * Find a tuple in the primary index LSM tree by the key of the
- * specified LSM tree.
- * @param lsm LSM tree for which the key is specified.
- * Can be both primary and secondary.
- * @param tx Current transaction.
- * @param rv Read view.
- * @param key_raw MessagePack'ed data, the array without a
- * header.
- * @param part_count Count of parts in the key.
- * @param[out] result The found statement is stored here. Must be
- * unreferenced after usage.
- *
- * @retval 0 Success.
- * @retval -1 Memory error.
- */
-static inline int
-vy_lsm_full_by_key(struct vy_lsm *lsm, struct vy_tx *tx,
- const struct vy_read_view **rv,
- const char *key_raw, uint32_t part_count,
- struct tuple **result)
-{
- int rc;
- struct tuple *key = vy_stmt_new_select(lsm->env->key_format,
- key_raw, part_count);
- if (key == NULL)
- return -1;
- struct tuple *found;
- rc = vy_lsm_get(lsm, tx, rv, key, &found);
- tuple_unref(key);
- if (rc != 0)
- return -1;
- if (lsm->index_id == 0 || found == NULL) {
- *result = found;
- return 0;
- }
- /*
- * No need in vy_tx_track() as the tuple is already
- * tracked in the secondary index LSM tree.
- */
- rc = vy_point_lookup(lsm->pk, tx, rv, found, result);
- tuple_unref(found);
- return rc;
-}
-
-/**
* Delete the tuple from all LSM trees of the vinyl space.
* @param env Vinyl environment.
* @param tx Current transaction.
@@ -1750,8 +1798,8 @@ vy_delete(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
* and pass them to indexes for deletion.
*/
if (has_secondary || !rlist_empty(&space->on_replace)) {
- if (vy_lsm_full_by_key(lsm, tx, vy_tx_read_view(tx),
- key, part_count, &stmt->old_tuple) != 0)
+ if (vy_get_by_raw_key(lsm, tx, vy_tx_read_view(tx),
+ key, part_count, &stmt->old_tuple) != 0)
return -1;
if (stmt->old_tuple == NULL)
return 0;
@@ -1832,8 +1880,8 @@ vy_update(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
if (vy_unique_key_validate(lsm, key, part_count))
return -1;
- if (vy_lsm_full_by_key(lsm, tx, vy_tx_read_view(tx),
- key, part_count, &stmt->old_tuple) != 0)
+ if (vy_get_by_raw_key(lsm, tx, vy_tx_read_view(tx),
+ key, part_count, &stmt->old_tuple) != 0)
return -1;
/* Nothing to update. */
if (stmt->old_tuple == NULL)
@@ -2106,8 +2154,7 @@ vy_upsert(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
pk->key_def, pk->env->key_format);
if (key == NULL)
return -1;
- int rc = vy_lsm_get(pk, tx, vy_tx_read_view(tx),
- key, &stmt->old_tuple);
+ int rc = vy_get(pk, tx, vy_tx_read_view(tx), key, &stmt->old_tuple);
tuple_unref(key);
if (rc != 0)
return -1;
@@ -3809,28 +3856,13 @@ next:
fiber_sleep(0.01);
}
#endif
- /*
- * Get the full tuple from the primary index.
- * Note, there's no need in vy_tx_track() as the
- * tuple is already tracked in the secondary index.
- */
- if (vy_point_lookup(it->lsm->pk, it->tx, vy_tx_read_view(it->tx),
- tuple, ret) != 0)
+ /* Get the full tuple from the primary index. */
+ if (vy_get_by_secondary_tuple(it->lsm, it->tx,
+ vy_tx_read_view(it->tx),
+ tuple, ret) != 0)
goto fail;
- if (*ret == NULL) {
- /*
- * All indexes of a space must be consistent, i.e.
- * if a tuple is present in one index, it must be
- * present in all other indexes as well, so we can
- * get here only if there's a bug somewhere in vinyl.
- * Don't abort as core dump won't really help us in
- * this case. Just warn the user and proceed to the
- * next tuple.
- */
- say_warn("%s: key %s missing in primary index",
- vy_lsm_name(it->lsm), vy_stmt_str(tuple));
+ if (*ret == NULL)
goto next;
- }
tuple_bless(*ret);
tuple_unref(*ret);
return 0;
@@ -3919,7 +3951,7 @@ vinyl_index_get(struct index *index, const char *key,
const struct vy_read_view **rv = (tx != NULL ? vy_tx_read_view(tx) :
&env->xm->p_global_read_view);
- if (vy_lsm_full_by_key(lsm, tx, rv, key, part_count, ret) != 0)
+ if (vy_get_by_raw_key(lsm, tx, rv, key, part_count, ret) != 0)
return -1;
if (*ret != NULL) {
tuple_bless(*ret);
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 04/25] vinyl: fold vy_replace_one and vy_replace_impl
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (2 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 03/25] vinyl: always get full tuple from pk after reading from secondary index Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 05/25] vinyl: fold vy_delete_impl Vladimir Davydov
` (20 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
There's no point in separating REPLACE path between the cases when
the space has secondary indexes and when it only has the primary
index, because they are quite similar. Let's fold vy_replace_one
and vy_replace_impl into vy_replace to remove code duplication.
---
src/box/vinyl.c | 234 ++++++++++++++++++--------------------------------------
1 file changed, 74 insertions(+), 160 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index a2e2a447..89ca1e78 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -1527,160 +1527,6 @@ vy_insert_secondary(struct vy_env *env, struct vy_tx *tx, struct space *space,
}
/**
- * Execute REPLACE in a space with a single index, possibly with
- * lookup for an old tuple if the space has at least one
- * on_replace trigger.
- * @param env Vinyl environment.
- * @param tx Current transaction.
- * @param space Space in which replace.
- * @param request Request with the tuple data.
- * @param stmt Statement for triggers is filled with old
- * statement.
- *
- * @retval 0 Success.
- * @retval -1 Memory error OR duplicate key error OR the primary
- * index is not found OR a tuple reference increment
- * error.
- */
-static inline int
-vy_replace_one(struct vy_env *env, struct vy_tx *tx, struct space *space,
- struct request *request, struct txn_stmt *stmt)
-{
- (void)env;
- assert(tx != NULL && tx->state == VINYL_TX_READY);
- struct vy_lsm *pk = vy_lsm(space->index[0]);
- assert(pk->index_id == 0);
- if (tuple_validate_raw(pk->mem_format, request->tuple))
- return -1;
- struct tuple *new_tuple =
- vy_stmt_new_replace(pk->mem_format, request->tuple,
- request->tuple_end);
- if (new_tuple == NULL)
- return -1;
- /**
- * If the space has triggers, then we need to fetch the
- * old tuple to pass it to the trigger.
- */
- if (stmt != NULL && !rlist_empty(&space->on_replace)) {
- if (vy_get(pk, tx, vy_tx_read_view(tx),
- new_tuple, &stmt->old_tuple) != 0)
- goto error_unref;
- }
- if (vy_tx_set(tx, pk, new_tuple))
- goto error_unref;
-
- if (stmt != NULL)
- stmt->new_tuple = new_tuple;
- else
- tuple_unref(new_tuple);
- return 0;
-
-error_unref:
- tuple_unref(new_tuple);
- return -1;
-}
-
-/**
- * Execute REPLACE in a space with multiple indexes and lookup for
- * an old tuple, that should has been set in \p stmt->old_tuple if
- * the space has at least one on_replace trigger.
- * @param env Vinyl environment.
- * @param tx Current transaction.
- * @param space Vinyl space.
- * @param request Request with the tuple data.
- * @param stmt Statement for triggers filled with old
- * statement.
- *
- * @retval 0 Success
- * @retval -1 Memory error OR duplicate key error OR the primary
- * index is not found OR a tuple reference increment
- * error.
- */
-static inline int
-vy_replace_impl(struct vy_env *env, struct vy_tx *tx, struct space *space,
- struct request *request, struct txn_stmt *stmt)
-{
- assert(tx != NULL && tx->state == VINYL_TX_READY);
- struct tuple *old_stmt = NULL;
- struct tuple *new_stmt = NULL;
- struct tuple *delete = NULL;
- struct vy_lsm *pk = vy_lsm_find(space, 0);
- if (pk == NULL) /* space has no primary key */
- return -1;
- /* Primary key is dumped last. */
- assert(!vy_is_committed_one(env, pk));
- assert(pk->index_id == 0);
- if (tuple_validate_raw(pk->mem_format, request->tuple))
- return -1;
- new_stmt = vy_stmt_new_replace(pk->mem_format, request->tuple,
- request->tuple_end);
- if (new_stmt == NULL)
- return -1;
-
- /* Get full tuple from the primary index. */
- if (vy_get(pk, tx, vy_tx_read_view(tx), new_stmt, &old_stmt) != 0)
- goto error;
-
- if (old_stmt == NULL) {
- /*
- * We can turn REPLACE into INSERT if the new key
- * does not have history.
- */
- vy_stmt_set_type(new_stmt, IPROTO_INSERT);
- }
-
- /*
- * Replace in the primary index without explicit deletion
- * of the old tuple.
- */
- if (vy_tx_set(tx, pk, new_stmt) != 0)
- goto error;
-
- if (space->index_count > 1 && old_stmt != NULL) {
- delete = vy_stmt_new_surrogate_delete(pk->mem_format, old_stmt);
- if (delete == NULL)
- goto error;
- }
-
- /* Update secondary keys, avoid duplicates. */
- for (uint32_t iid = 1; iid < space->index_count; ++iid) {
- struct vy_lsm *lsm = vy_lsm(space->index[iid]);
- if (vy_is_committed_one(env, lsm))
- continue;
- /*
- * Delete goes first, so if old and new keys
- * fully match, there is no look up beyond the
- * transaction index.
- */
- if (old_stmt != NULL) {
- if (vy_tx_set(tx, lsm, delete) != 0)
- goto error;
- }
- if (vy_insert_secondary(env, tx, space, lsm, new_stmt) != 0)
- goto error;
- }
- if (delete != NULL)
- tuple_unref(delete);
- /*
- * The old tuple is used if there is an on_replace
- * trigger.
- */
- if (stmt != NULL) {
- stmt->new_tuple = new_stmt;
- stmt->old_tuple = old_stmt;
- }
- return 0;
-error:
- if (delete != NULL)
- tuple_unref(delete);
- if (old_stmt != NULL)
- tuple_unref(old_stmt);
- if (new_stmt != NULL)
- tuple_unref(new_stmt);
- return -1;
-}
-
-/**
* Check that the key can be used for search in a unique index
* LSM tree.
* @param lsm LSM tree for checking.
@@ -2303,18 +2149,86 @@ static int
vy_replace(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
struct space *space, struct request *request)
{
+ assert(tx != NULL && tx->state == VINYL_TX_READY);
if (vy_is_committed(env, space))
return 0;
if (request->type == IPROTO_INSERT)
return vy_insert(env, tx, stmt, space, request);
- if (space->index_count == 1) {
- /* Replace in a space with a single index. */
- return vy_replace_one(env, tx, space, request, stmt);
- } else {
- /* Replace in a space with secondary indexes. */
- return vy_replace_impl(env, tx, space, request, stmt);
+ struct vy_lsm *pk = vy_lsm_find(space, 0);
+ if (pk == NULL)
+ return -1;
+ /* Primary key is dumped last. */
+ assert(!vy_is_committed_one(env, pk));
+
+ /* Validate and create a statement for the new tuple. */
+ if (tuple_validate_raw(pk->mem_format, request->tuple))
+ return -1;
+ stmt->new_tuple = vy_stmt_new_replace(pk->mem_format, request->tuple,
+ request->tuple_end);
+ if (stmt->new_tuple == NULL)
+ return -1;
+ /*
+ * Get the overwritten tuple from the primary index if
+ * the space has on_replace triggers, in which case we
+ * need to pass the old tuple to trigger callbacks, or
+ * if the space has secondary indexes and so we need
+ * the old tuple to delete it from them.
+ */
+ if (space->index_count > 1 || !rlist_empty(&space->on_replace)) {
+ if (vy_get(pk, tx, vy_tx_read_view(tx),
+ stmt->new_tuple, &stmt->old_tuple) != 0)
+ return -1;
+ if (stmt->old_tuple == NULL) {
+ /*
+ * We can turn REPLACE into INSERT if the
+ * new key does not have history.
+ */
+ vy_stmt_set_type(stmt->new_tuple, IPROTO_INSERT);
+ }
+ }
+ /*
+ * Replace in the primary index without explicit deletion
+ * of the old tuple.
+ */
+ if (vy_tx_set(tx, pk, stmt->new_tuple) != 0)
+ return -1;
+ if (space->index_count == 1)
+ return 0;
+ /*
+ * Replace in secondary indexes with explicit deletion
+ * of the old tuple, if any.
+ */
+ int rc = 0;
+ struct tuple *delete = NULL;
+ if (stmt->old_tuple != NULL) {
+ delete = vy_stmt_new_surrogate_delete(pk->mem_format,
+ stmt->old_tuple);
+ if (delete == NULL)
+ return -1;
+ }
+ for (uint32_t i = 1; i < space->index_count; i++) {
+ struct vy_lsm *lsm = vy_lsm(space->index[i]);
+ if (vy_is_committed_one(env, lsm))
+ continue;
+ /*
+ * DELETE goes first, so if old and new keys
+ * fully match, there is no look up beyond the
+ * transaction write set.
+ */
+ if (delete != NULL) {
+ rc = vy_tx_set(tx, lsm, delete);
+ if (rc != 0)
+ break;
+ }
+ rc = vy_insert_secondary(env, tx, space, lsm,
+ stmt->new_tuple);
+ if (rc != 0)
+ break;
}
+ if (delete != NULL)
+ tuple_unref(delete);
+ return rc;
}
static int
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 05/25] vinyl: fold vy_delete_impl
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (3 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 04/25] vinyl: fold vy_replace_one and vy_replace_impl Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 06/25] vinyl: refactor unique check Vladimir Davydov
` (19 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
vy_delete_impl helper is only used once in vy_delete and it is rather
small so inlining it definitely won't hurt. On the contrary, it will
consolidate DELETE logic in one place, making the code easier to follow.
---
src/box/vinyl.c | 68 ++++++++++++++++-----------------------------------------
1 file changed, 19 insertions(+), 49 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index 89ca1e78..8c0afd28 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -1563,47 +1563,6 @@ vy_unique_key_validate(struct vy_lsm *lsm, const char *key,
}
/**
- * Delete the tuple from all LSM trees of the vinyl space.
- * @param env Vinyl environment.
- * @param tx Current transaction.
- * @param space Vinyl space.
- * @param tuple Tuple to delete.
- *
- * @retval 0 Success
- * @retval -1 Memory error or the index is not found.
- */
-static inline int
-vy_delete_impl(struct vy_env *env, struct vy_tx *tx, struct space *space,
- const struct tuple *tuple)
-{
- struct vy_lsm *pk = vy_lsm_find(space, 0);
- if (pk == NULL)
- return -1;
- /* Primary key is dumped last. */
- assert(!vy_is_committed_one(env, pk));
- struct tuple *delete =
- vy_stmt_new_surrogate_delete(pk->mem_format, tuple);
- if (delete == NULL)
- return -1;
- if (vy_tx_set(tx, pk, delete) != 0)
- goto error;
-
- /* At second, delete from seconary indexes. */
- for (uint32_t i = 1; i < space->index_count; ++i) {
- struct vy_lsm *lsm = vy_lsm(space->index[i]);
- if (vy_is_committed_one(env, lsm))
- continue;
- if (vy_tx_set(tx, lsm, delete) != 0)
- goto error;
- }
- tuple_unref(delete);
- return 0;
-error:
- tuple_unref(delete);
- return -1;
-}
-
-/**
* Execute DELETE in a vinyl space.
* @param env Vinyl environment.
* @param tx Current transaction.
@@ -1650,21 +1609,32 @@ vy_delete(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
if (stmt->old_tuple == NULL)
return 0;
}
+ int rc = 0;
+ struct tuple *delete;
if (has_secondary) {
assert(stmt->old_tuple != NULL);
- return vy_delete_impl(env, tx, space, stmt->old_tuple);
+ delete = vy_stmt_new_surrogate_delete(pk->mem_format,
+ stmt->old_tuple);
+ if (delete == NULL)
+ return -1;
+ for (uint32_t i = 0; i < space->index_count; i++) {
+ struct vy_lsm *lsm = vy_lsm(space->index[i]);
+ if (vy_is_committed_one(env, lsm))
+ continue;
+ rc = vy_tx_set(tx, lsm, delete);
+ if (rc != 0)
+ break;
+ }
} else { /* Primary is the single index in the space. */
assert(lsm->index_id == 0);
- struct tuple *delete =
- vy_stmt_new_surrogate_delete_from_key(request->key,
- pk->key_def,
- pk->mem_format);
+ delete = vy_stmt_new_surrogate_delete_from_key(request->key,
+ pk->key_def, pk->mem_format);
if (delete == NULL)
return -1;
- int rc = vy_tx_set(tx, pk, delete);
- tuple_unref(delete);
- return rc;
+ rc = vy_tx_set(tx, pk, delete);
}
+ tuple_unref(delete);
+ return rc;
}
/**
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 06/25] vinyl: refactor unique check
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (4 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 05/25] vinyl: fold vy_delete_impl Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 07/25] vinyl: check key uniqueness before modifying tx write set Vladimir Davydov
` (18 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
For the sake of further patches, let's do some refactoring:
- Rename vy_check_is_unique to vy_check_is_unique_primary and use it
only for checking the unique constraint of primary indexes. Also,
make it return immediately if the primary index doesn't need
uniqueness check, like vy_check_is_unique_secondary does.
- Open-code uniqueness check in vy_check_is_unique_secondary instead of
using vy_check_is_unique.
- Reduce indentation level of vy_check_is_unique_secondary by inverting
the if statement.
---
src/box/vinyl.c | 82 +++++++++++++++++++++++++++++++++++----------------------
1 file changed, 51 insertions(+), 31 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index 8c0afd28..5da1c4bc 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -1385,36 +1385,39 @@ vy_get_by_raw_key(struct vy_lsm *lsm, struct vy_tx *tx,
}
/**
- * Check if the LSM tree contains the key. If true, then set
- * a duplicate key error in the diagnostics area.
+ * Check if insertion of a new tuple violates unique constraint
+ * of the primary index.
* @param env Vinyl environment.
* @param tx Current transaction.
* @param rv Read view.
* @param space_name Space name.
* @param index_name Index name.
- * @param lsm LSM tree in which to search.
- * @param key Key statement.
+ * @param lsm LSM tree corresponding to the index.
+ * @param stmt New tuple.
*
- * @retval 0 Success, the key isn't found.
- * @retval -1 Memory error or the key is found.
+ * @retval 0 Success, unique constraint is satisfied.
+ * @retval -1 Duplicate is found or read error occurred.
*/
static inline int
-vy_check_is_unique(struct vy_env *env, struct vy_tx *tx,
- const struct vy_read_view **rv,
- const char *space_name, const char *index_name,
- struct vy_lsm *lsm, struct tuple *key)
+vy_check_is_unique_primary(struct vy_env *env, struct vy_tx *tx,
+ const struct vy_read_view **rv,
+ const char *space_name, const char *index_name,
+ struct vy_lsm *lsm, struct tuple *stmt)
{
- struct tuple *found;
+ assert(lsm->index_id == 0);
+ assert(vy_stmt_type(stmt) == IPROTO_INSERT);
/*
* During recovery we apply rows that were successfully
* applied before restart so no conflict is possible.
*/
if (env->status != VINYL_ONLINE)
return 0;
- if (vy_get(lsm, tx, rv, key, &found))
+ if (!lsm->check_is_unique)
+ return 0;
+ struct tuple *found;
+ if (vy_get(lsm, tx, rv, stmt, &found))
return -1;
-
- if (found) {
+ if (found != NULL) {
tuple_unref(found);
diag_set(ClientError, ER_TUPLE_FOUND,
index_name, space_name);
@@ -1444,19 +1447,36 @@ vy_check_is_unique_secondary(struct vy_env *env, struct vy_tx *tx,
struct vy_lsm *lsm, const struct tuple *stmt)
{
assert(lsm->index_id > 0);
- struct key_def *def = lsm->key_def;
- if (lsm->check_is_unique &&
- !key_update_can_be_skipped(def->column_mask,
- vy_stmt_column_mask(stmt)) &&
- (!def->is_nullable || !vy_tuple_key_contains_null(stmt, def))) {
- struct tuple *key = vy_stmt_extract_key(stmt, def,
- lsm->env->key_format);
- if (key == NULL)
- return -1;
- int rc = vy_check_is_unique(env, tx, rv, space_name,
- index_name, lsm, key);
- tuple_unref(key);
- return rc;
+ assert(vy_stmt_type(stmt) == IPROTO_INSERT ||
+ vy_stmt_type(stmt) == IPROTO_REPLACE);
+ /*
+ * During recovery we apply rows that were successfully
+ * applied before restart so no conflict is possible.
+ */
+ if (env->status != VINYL_ONLINE)
+ return 0;
+ if (!lsm->check_is_unique)
+ return 0;
+ if (key_update_can_be_skipped(lsm->key_def->column_mask,
+ vy_stmt_column_mask(stmt)))
+ return 0;
+ if (lsm->key_def->is_nullable &&
+ vy_tuple_key_contains_null(stmt, lsm->key_def))
+ return 0;
+ struct tuple *key = vy_stmt_extract_key(stmt, lsm->key_def,
+ lsm->env->key_format);
+ if (key == NULL)
+ return -1;
+ struct tuple *found;
+ int rc = vy_get(lsm, tx, rv, key, &found);
+ tuple_unref(key);
+ if (rc != 0)
+ return -1;
+ if (found != NULL) {
+ tuple_unref(found);
+ diag_set(ClientError, ER_TUPLE_FOUND,
+ index_name, space_name);
+ return -1;
}
return 0;
}
@@ -1483,10 +1503,10 @@ vy_insert_primary(struct vy_env *env, struct vy_tx *tx, struct space *space,
* A primary index is always unique and the new tuple must not
* conflict with existing tuples.
*/
- if (pk->check_is_unique &&
- vy_check_is_unique(env, tx, vy_tx_read_view(tx), space_name(space),
- index_name_by_id(space, pk->index_id),
- pk, stmt) != 0)
+ if (vy_check_is_unique_primary(env, tx, vy_tx_read_view(tx),
+ space_name(space),
+ index_name_by_id(space, pk->index_id),
+ pk, stmt) != 0)
return -1;
return vy_tx_set(tx, pk, stmt);
}
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 07/25] vinyl: check key uniqueness before modifying tx write set
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (5 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 06/25] vinyl: refactor unique check Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 08/25] vinyl: remove env argument of vy_check_is_unique_{primary,secondary} Vladimir Davydov
` (17 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Currently, we handle INSERT/REPLACE/UPDATE requests by iterating over
all space indexes starting from the primary and inserting the
corresponding statements to tx write set, checking key uniqueness if
necessary. This means that by the time we write a REPLACE to the write
set of a secondary index, it has already been written to the primary
index write set. This is OK, and vy_tx_prepare() relies on that to
implement the common memory level. However, this also means that when we
check uniqueness of a secondary index, the new REPLACE can be found via
the primary index. This is OK now, because all indexes are fully
independent, but it isn't going to fly after #2129 is implemented. The
problem is in order to check if a tuple is present in a secondary index,
we will have to look up the corresponding full tuple in the primary
index. To illustrate the problem, consider the following situation:
Primary index covers field 1.
Secondary index covers field 2.
Committed statements:
REPLACE{1, 10, lsn=1} - present in both indexes
DELETE{1, lsn=2} - present only in the primary index
Transaction:
REPLACE{1, 10}
When we check uniqueness of the secondary index, we find committed
statement REPLACE{1, 10, lsn=1}, then look up the corresponding full
tuple in the primary index and find REPLACE{1, 10}. Since the two tuples
match, we mistakenly assume that there's a conflict.
To avoid a situation like that, let's check uniqueness before modifying
the write set of any index.
Needed for #2129
---
src/box/vinyl.c | 128 +++++++++++++++++++++++++++-----------------------------
1 file changed, 62 insertions(+), 66 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index 5da1c4bc..1d086439 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -1472,6 +1472,17 @@ vy_check_is_unique_secondary(struct vy_env *env, struct vy_tx *tx,
tuple_unref(key);
if (rc != 0)
return -1;
+ if (found != NULL && vy_tuple_compare(stmt, found,
+ lsm->pk->key_def) == 0) {
+ /*
+ * If the old and new tuples are the same in
+ * terms of the primary key definition, the
+ * statement doesn't modify the secondary key
+ * and so there's actually no conflict.
+ */
+ tuple_unref(found);
+ return 0;
+ }
if (found != NULL) {
tuple_unref(found);
diag_set(ClientError, ER_TUPLE_FOUND,
@@ -1482,68 +1493,51 @@ vy_check_is_unique_secondary(struct vy_env *env, struct vy_tx *tx,
}
/**
- * Insert a tuple in a primary index LSM tree.
- * @param env Vinyl environment.
- * @param tx Current transaction.
- * @param space Target space.
- * @param pk Primary index LSM tree.
- * @param stmt Tuple to insert.
- *
- * @retval 0 Success.
- * @retval -1 Memory error or duplicate key error.
- */
-static inline int
-vy_insert_primary(struct vy_env *env, struct vy_tx *tx, struct space *space,
- struct vy_lsm *pk, struct tuple *stmt)
-{
- assert(vy_stmt_type(stmt) == IPROTO_INSERT);
- assert(tx != NULL && tx->state == VINYL_TX_READY);
- assert(pk->index_id == 0);
- /*
- * A primary index is always unique and the new tuple must not
- * conflict with existing tuples.
- */
- if (vy_check_is_unique_primary(env, tx, vy_tx_read_view(tx),
- space_name(space),
- index_name_by_id(space, pk->index_id),
- pk, stmt) != 0)
- return -1;
- return vy_tx_set(tx, pk, stmt);
-}
-
-/**
- * Insert a tuple in a secondary index LSM tree.
- * @param env Vinyl environment.
- * @param tx Current transaction.
- * @param space Target space.
- * @param lsm Secondary index LSM tree.
- * @param stmt Tuple to replace.
+ * Check if insertion of a new tuple violates unique constraint
+ * of any index of the space.
+ * @param env Vinyl environment.
+ * @param tx Current transaction.
+ * @param space Space to check.
+ * @param stmt New tuple.
*
- * @retval 0 Success.
- * @retval -1 Memory error or duplicate key error.
+ * @retval 0 Success, unique constraint is satisfied.
+ * @retval -1 Duplicate is found or read error occurred.
*/
static int
-vy_insert_secondary(struct vy_env *env, struct vy_tx *tx, struct space *space,
- struct vy_lsm *lsm, struct tuple *stmt)
+vy_check_is_unique(struct vy_env *env, struct vy_tx *tx,
+ struct space *space, struct tuple *stmt)
{
+ assert(space->index_count > 0);
assert(vy_stmt_type(stmt) == IPROTO_INSERT ||
vy_stmt_type(stmt) == IPROTO_REPLACE);
- assert(tx != NULL && tx->state == VINYL_TX_READY);
- assert(lsm->index_id > 0);
- if (vy_check_is_unique_secondary(env, tx, vy_tx_read_view(tx),
- space_name(space),
- index_name_by_id(space, lsm->index_id),
- lsm, stmt) != 0)
- return -1;
+ const struct vy_read_view **rv = vy_tx_read_view(tx);
/*
- * We must always append the statement to transaction write set
- * of each LSM tree, even if operation itself does not update
- * the LSM tree, e.g. it's an UPDATE, to ensure we read our
- * own writes.
+ * We only need to check the uniqueness of the primary index
+ * if this is INSERT, because REPLACE will silently overwrite
+ * the existing tuple, if any.
*/
- return vy_tx_set(tx, lsm, stmt);
+ if (vy_stmt_type(stmt) == IPROTO_INSERT) {
+ struct vy_lsm *lsm = vy_lsm(space->index[0]);
+ if (vy_check_is_unique_primary(env, tx, rv, space_name(space),
+ index_name_by_id(space, 0),
+ lsm, stmt) != 0)
+ return -1;
+ }
+
+ /*
+ * For secondary indexes, uniqueness must be checked on both
+ * INSERT and REPLACE.
+ */
+ for (uint32_t i = 1; i < space->index_count; i++) {
+ struct vy_lsm *lsm = vy_lsm(space->index[i]);
+ if (vy_check_is_unique_secondary(env, tx, rv, space_name(space),
+ index_name_by_id(space, i),
+ lsm, stmt) != 0)
+ return -1;
+ }
+ return 0;
}
/**
@@ -1764,6 +1758,8 @@ vy_update(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
if (vy_check_update(space, pk, stmt->old_tuple, stmt->new_tuple,
column_mask) != 0)
return -1;
+ if (vy_check_is_unique(env, tx, space, stmt->new_tuple) != 0)
+ return -1;
/*
* In the primary index the tuple can be replaced without
@@ -1786,7 +1782,7 @@ vy_update(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
continue;
if (vy_tx_set(tx, lsm, delete) != 0)
goto error;
- if (vy_insert_secondary(env, tx, space, lsm, stmt->new_tuple))
+ if (vy_tx_set(tx, lsm, stmt->new_tuple) != 0)
goto error;
}
tuple_unref(delete);
@@ -1814,13 +1810,15 @@ vy_insert_first_upsert(struct vy_env *env, struct vy_tx *tx,
assert(tx != NULL && tx->state == VINYL_TX_READY);
assert(space->index_count > 0);
assert(vy_stmt_type(stmt) == IPROTO_INSERT);
+ if (vy_check_is_unique(env, tx, space, stmt) != 0)
+ return -1;
struct vy_lsm *pk = vy_lsm(space->index[0]);
assert(pk->index_id == 0);
if (vy_tx_set(tx, pk, stmt) != 0)
return -1;
for (uint32_t i = 1; i < space->index_count; ++i) {
struct vy_lsm *lsm = vy_lsm(space->index[i]);
- if (vy_insert_secondary(env, tx, space, lsm, stmt) != 0)
+ if (vy_tx_set(tx, lsm, stmt) != 0)
return -1;
}
return 0;
@@ -2045,6 +2043,8 @@ vy_upsert(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
*/
return 0;
}
+ if (vy_check_is_unique(env, tx, space, stmt->new_tuple) != 0)
+ return -1;
if (vy_tx_set(tx, pk, stmt->new_tuple))
return -1;
if (space->index_count == 1)
@@ -2063,8 +2063,7 @@ vy_upsert(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
continue;
if (vy_tx_set(tx, lsm, delete) != 0)
goto error;
- if (vy_insert_secondary(env, tx, space, lsm,
- stmt->new_tuple) != 0)
+ if (vy_tx_set(tx, lsm, stmt->new_tuple) != 0)
goto error;
}
tuple_unref(delete);
@@ -2107,15 +2106,16 @@ vy_insert(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
request->tuple_end);
if (stmt->new_tuple == NULL)
return -1;
- if (vy_insert_primary(env, tx, space, pk, stmt->new_tuple) != 0)
+ if (vy_check_is_unique(env, tx, space, stmt->new_tuple) != 0)
+ return -1;
+ if (vy_tx_set(tx, pk, stmt->new_tuple) != 0)
return -1;
for (uint32_t iid = 1; iid < space->index_count; ++iid) {
struct vy_lsm *lsm = vy_lsm(space->index[iid]);
if (vy_is_committed_one(env, lsm))
continue;
- if (vy_insert_secondary(env, tx, space, lsm,
- stmt->new_tuple) != 0)
+ if (vy_tx_set(tx, lsm, stmt->new_tuple) != 0)
return -1;
}
return 0;
@@ -2158,6 +2158,8 @@ vy_replace(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
request->tuple_end);
if (stmt->new_tuple == NULL)
return -1;
+ if (vy_check_is_unique(env, tx, space, stmt->new_tuple) != 0)
+ return -1;
/*
* Get the overwritten tuple from the primary index if
* the space has on_replace triggers, in which case we
@@ -2201,18 +2203,12 @@ vy_replace(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
struct vy_lsm *lsm = vy_lsm(space->index[i]);
if (vy_is_committed_one(env, lsm))
continue;
- /*
- * DELETE goes first, so if old and new keys
- * fully match, there is no look up beyond the
- * transaction write set.
- */
if (delete != NULL) {
rc = vy_tx_set(tx, lsm, delete);
if (rc != 0)
break;
}
- rc = vy_insert_secondary(env, tx, space, lsm,
- stmt->new_tuple);
+ rc = vy_tx_set(tx, lsm, stmt->new_tuple);
if (rc != 0)
break;
}
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 08/25] vinyl: remove env argument of vy_check_is_unique_{primary,secondary}
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (6 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 07/25] vinyl: check key uniqueness before modifying tx write set Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-31 20:45 ` [tarantool-patches] " Konstantin Osipov
2018-07-27 11:29 ` [PATCH 09/25] vinyl: store full tuples in secondary index cache Vladimir Davydov
` (16 subsequent siblings)
24 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Besides vy_check_is_unique, other callers of vy_check_is_unique_primary
and vy_check_is_unique_secondary are only called when vinyl engine is
online. So let's move the optimization that skips uniqueness check on
recovery to vy_check_is_unique and remove the env argument.
---
src/box/vinyl.c | 42 +++++++++++++++---------------------------
1 file changed, 15 insertions(+), 27 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index 1d086439..c29f139e 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -1387,7 +1387,6 @@ vy_get_by_raw_key(struct vy_lsm *lsm, struct vy_tx *tx,
/**
* Check if insertion of a new tuple violates unique constraint
* of the primary index.
- * @param env Vinyl environment.
* @param tx Current transaction.
* @param rv Read view.
* @param space_name Space name.
@@ -1399,19 +1398,13 @@ vy_get_by_raw_key(struct vy_lsm *lsm, struct vy_tx *tx,
* @retval -1 Duplicate is found or read error occurred.
*/
static inline int
-vy_check_is_unique_primary(struct vy_env *env, struct vy_tx *tx,
- const struct vy_read_view **rv,
+vy_check_is_unique_primary(struct vy_tx *tx, const struct vy_read_view **rv,
const char *space_name, const char *index_name,
struct vy_lsm *lsm, struct tuple *stmt)
{
assert(lsm->index_id == 0);
assert(vy_stmt_type(stmt) == IPROTO_INSERT);
- /*
- * During recovery we apply rows that were successfully
- * applied before restart so no conflict is possible.
- */
- if (env->status != VINYL_ONLINE)
- return 0;
+
if (!lsm->check_is_unique)
return 0;
struct tuple *found;
@@ -1429,7 +1422,6 @@ vy_check_is_unique_primary(struct vy_env *env, struct vy_tx *tx,
/**
* Check if insertion of a new tuple violates unique constraint
* of a secondary index.
- * @param env Vinyl environment.
* @param tx Current transaction.
* @param rv Read view.
* @param space_name Space name.
@@ -1441,20 +1433,14 @@ vy_check_is_unique_primary(struct vy_env *env, struct vy_tx *tx,
* @retval -1 Duplicate is found or read error occurred.
*/
static int
-vy_check_is_unique_secondary(struct vy_env *env, struct vy_tx *tx,
- const struct vy_read_view **rv,
+vy_check_is_unique_secondary(struct vy_tx *tx, const struct vy_read_view **rv,
const char *space_name, const char *index_name,
struct vy_lsm *lsm, const struct tuple *stmt)
{
assert(lsm->index_id > 0);
assert(vy_stmt_type(stmt) == IPROTO_INSERT ||
vy_stmt_type(stmt) == IPROTO_REPLACE);
- /*
- * During recovery we apply rows that were successfully
- * applied before restart so no conflict is possible.
- */
- if (env->status != VINYL_ONLINE)
- return 0;
+
if (!lsm->check_is_unique)
return 0;
if (key_update_can_be_skipped(lsm->key_def->column_mask,
@@ -1510,6 +1496,12 @@ vy_check_is_unique(struct vy_env *env, struct vy_tx *tx,
assert(space->index_count > 0);
assert(vy_stmt_type(stmt) == IPROTO_INSERT ||
vy_stmt_type(stmt) == IPROTO_REPLACE);
+ /*
+ * During recovery we apply rows that were successfully
+ * applied before restart so no conflict is possible.
+ */
+ if (env->status != VINYL_ONLINE)
+ return 0;
const struct vy_read_view **rv = vy_tx_read_view(tx);
@@ -1520,7 +1512,7 @@ vy_check_is_unique(struct vy_env *env, struct vy_tx *tx,
*/
if (vy_stmt_type(stmt) == IPROTO_INSERT) {
struct vy_lsm *lsm = vy_lsm(space->index[0]);
- if (vy_check_is_unique_primary(env, tx, rv, space_name(space),
+ if (vy_check_is_unique_primary(tx, rv, space_name(space),
index_name_by_id(space, 0),
lsm, stmt) != 0)
return -1;
@@ -1532,7 +1524,7 @@ vy_check_is_unique(struct vy_env *env, struct vy_tx *tx,
*/
for (uint32_t i = 1; i < space->index_count; i++) {
struct vy_lsm *lsm = vy_lsm(space->index[i]);
- if (vy_check_is_unique_secondary(env, tx, rv, space_name(space),
+ if (vy_check_is_unique_secondary(tx, rv, space_name(space),
index_name_by_id(space, i),
lsm, stmt) != 0)
return -1;
@@ -3866,8 +3858,6 @@ vinyl_index_get(struct index *index, const char *key,
/** Argument passed to vy_build_on_replace(). */
struct vy_build_ctx {
- /** Vinyl environment. */
- struct vy_env *env;
/** LSM tree under construction. */
struct vy_lsm *lsm;
/** Format to check new tuples against. */
@@ -3908,7 +3898,7 @@ vy_build_on_replace(struct trigger *trigger, void *event)
/* Check key uniqueness if necessary. */
if (stmt->new_tuple != NULL &&
- vy_check_is_unique_secondary(ctx->env, tx, vy_tx_read_view(tx),
+ vy_check_is_unique_secondary(tx, vy_tx_read_view(tx),
ctx->space_name, ctx->index_name,
lsm, stmt->new_tuple) != 0)
goto err;
@@ -3994,9 +3984,8 @@ vy_build_insert_tuple(struct vy_env *env, struct vy_lsm *lsm,
* into it after the yield.
*/
vy_mem_pin(mem);
- rc = vy_check_is_unique_secondary(env, NULL,
- &env->xm->p_committed_read_view,
- space_name, index_name, lsm, tuple);
+ rc = vy_check_is_unique_secondary(NULL, &env->xm->p_committed_read_view,
+ space_name, index_name, lsm, tuple);
vy_mem_unpin(mem);
if (rc != 0)
return -1;
@@ -4189,7 +4178,6 @@ vinyl_space_build_index(struct space *src_space, struct index *new_index,
struct trigger on_replace;
struct vy_build_ctx ctx;
- ctx.env = env;
ctx.lsm = new_lsm;
ctx.format = new_format;
ctx.space_name = space_name(src_space);
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 09/25] vinyl: store full tuples in secondary index cache
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (7 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 08/25] vinyl: remove env argument of vy_check_is_unique_{primary,secondary} Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-31 20:47 ` Konstantin Osipov
2018-07-27 11:29 ` [PATCH 10/25] vinyl: do not free pending tasks on shutdown Vladimir Davydov
` (15 subsequent siblings)
24 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Currently, both vy_read_iterator_next() and vy_point_lookup() add the
returned tuple to the tuple cache. As a result, we store partial tuples
in a secondary index tuple cache although we could store full tuples
(we have to retrieve them anyway when reading a secondary index). This
means wasting memory. Besides, when the #2129 gets implemented, there
will be tuples in a secondary index that have to be skipped as they have
been overwritten in the primary index. Caching them would be inefficient
and error prone. So let's call vy_cache_add() from the upper level and
add only full tuples to the cache.
Closes #3478
Needed for #2129
---
src/box/vinyl.c | 11 +++++++++
src/box/vy_point_lookup.c | 5 +---
src/box/vy_read_iterator.c | 61 ++++++++++++++++++++++------------------------
src/box/vy_read_iterator.h | 24 ++++++++++++++++++
4 files changed, 65 insertions(+), 36 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index c29f139e..f2f93736 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -1290,6 +1290,10 @@ vy_get_by_secondary_tuple(struct vy_lsm *lsm, struct vy_tx *tx,
say_warn("%s: key %s missing in primary index",
vy_lsm_name(lsm), vy_stmt_str(tuple));
}
+
+ if ((*rv)->vlsn == INT64_MAX)
+ vy_cache_add(&lsm->pk->cache, *result, NULL, tuple, ITER_EQ);
+
return 0;
}
@@ -1336,6 +1340,8 @@ vy_get(struct vy_lsm *lsm, struct vy_tx *tx,
} else {
*result = tuple;
}
+ if ((*rv)->vlsn == INT64_MAX)
+ vy_cache_add(&lsm->cache, *result, NULL, key, ITER_EQ);
return 0;
}
@@ -1352,6 +1358,8 @@ vy_get(struct vy_lsm *lsm, struct vy_tx *tx,
if (rc != 0 || *result != NULL)
break;
}
+ if (rc == 0)
+ vy_read_iterator_cache_add(&itr, *result);
vy_read_iterator_close(&itr);
return rc;
}
@@ -3707,6 +3715,7 @@ vinyl_iterator_primary_next(struct iterator *base, struct tuple **ret)
goto fail;
if (vy_read_iterator_next(&it->iterator, ret) != 0)
goto fail;
+ vy_read_iterator_cache_add(&it->iterator, *ret);
if (*ret == NULL) {
/* EOF. Close the iterator immediately. */
vinyl_iterator_close(it);
@@ -3736,6 +3745,7 @@ next:
if (tuple == NULL) {
/* EOF. Close the iterator immediately. */
+ vy_read_iterator_cache_add(&it->iterator, NULL);
vinyl_iterator_close(it);
*ret = NULL;
return 0;
@@ -3755,6 +3765,7 @@ next:
goto fail;
if (*ret == NULL)
goto next;
+ vy_read_iterator_cache_add(&it->iterator, *ret);
tuple_bless(*ret);
tuple_unref(*ret);
return 0;
diff --git a/src/box/vy_point_lookup.c b/src/box/vy_point_lookup.c
index f2261fdf..5e43340b 100644
--- a/src/box/vy_point_lookup.c
+++ b/src/box/vy_point_lookup.c
@@ -280,11 +280,8 @@ done:
if (rc != 0)
return -1;
- if (*ret != NULL) {
+ if (*ret != NULL)
vy_stmt_counter_acct_tuple(&lsm->stat.get, *ret);
- if ((*rv)->vlsn == INT64_MAX)
- vy_cache_add(&lsm->cache, *ret, NULL, key, ITER_EQ);
- }
double latency = ev_monotonic_now(loop()) - start_time;
latency_collect(&lsm->stat.latency, latency);
diff --git a/src/box/vy_read_iterator.c b/src/box/vy_read_iterator.c
index 160bb899..954fc0df 100644
--- a/src/box/vy_read_iterator.c
+++ b/src/box/vy_read_iterator.c
@@ -845,24 +845,17 @@ vy_read_iterator_next(struct vy_read_iterator *itr, struct tuple **result)
ev_tstamp start_time = ev_monotonic_now(loop());
struct vy_lsm *lsm = itr->lsm;
- struct tuple *stmt, *prev_stmt;
+ struct tuple *stmt;
- /*
- * Remember the statement returned by the last iteration.
- * We will need it to update the cache.
- */
- prev_stmt = itr->last_stmt;
- if (prev_stmt != NULL)
- tuple_ref(prev_stmt);
- else /* first iteration */
- lsm->stat.lookup++;
+ if (itr->last_stmt == NULL)
+ lsm->stat.lookup++; /* first iteration */
next_key:
if (vy_read_iterator_advance(itr) != 0)
- goto err;
+ return -1;
if (vy_read_iterator_apply_history(itr, &stmt) != 0)
- goto err;
+ return -1;
if (vy_read_iterator_track_read(itr, stmt) != 0)
- goto err;
+ return -1;
if (itr->last_stmt != NULL)
tuple_unref(itr->last_stmt);
@@ -877,9 +870,9 @@ next_key:
* previous + current tuple as an unbroken chain.
*/
if (vy_stmt_lsn(stmt) == INT64_MAX) {
- if (prev_stmt != NULL)
- tuple_unref(prev_stmt);
- prev_stmt = NULL;
+ if (itr->last_cached_stmt != NULL)
+ tuple_unref(itr->last_cached_stmt);
+ itr->last_cached_stmt = NULL;
}
goto next_key;
}
@@ -887,18 +880,6 @@ next_key:
vy_stmt_type(stmt) == IPROTO_INSERT ||
vy_stmt_type(stmt) == IPROTO_REPLACE);
- /*
- * Store the result in the cache provided we are reading
- * the latest data.
- */
- if ((**itr->read_view).vlsn == INT64_MAX) {
- vy_cache_add(&lsm->cache, stmt, prev_stmt,
- itr->key, itr->iterator_type);
- }
- if (prev_stmt != NULL)
- tuple_unref(prev_stmt);
-
- /* Update LSM tree stats. */
if (stmt != NULL)
vy_stmt_counter_acct_tuple(&lsm->stat.get, stmt);
@@ -914,10 +895,24 @@ next_key:
*result = stmt;
return 0;
-err:
- if (prev_stmt != NULL)
- tuple_unref(prev_stmt);
- return -1;
+}
+
+void
+vy_read_iterator_cache_add(struct vy_read_iterator *itr, struct tuple *stmt)
+{
+ if ((**itr->read_view).vlsn != INT64_MAX) {
+ if (itr->last_cached_stmt != NULL)
+ tuple_unref(itr->last_cached_stmt);
+ itr->last_cached_stmt = NULL;
+ return;
+ }
+ vy_cache_add(&itr->lsm->cache, stmt, itr->last_cached_stmt,
+ itr->key, itr->iterator_type);
+ if (stmt != NULL)
+ tuple_ref(stmt);
+ if (itr->last_cached_stmt != NULL)
+ tuple_unref(itr->last_cached_stmt);
+ itr->last_cached_stmt = stmt;
}
/**
@@ -928,6 +923,8 @@ vy_read_iterator_close(struct vy_read_iterator *itr)
{
if (itr->last_stmt != NULL)
tuple_unref(itr->last_stmt);
+ if (itr->last_cached_stmt != NULL)
+ tuple_unref(itr->last_cached_stmt);
vy_read_iterator_cleanup(itr);
free(itr->src);
TRASH(itr);
diff --git a/src/box/vy_read_iterator.h b/src/box/vy_read_iterator.h
index 2cac1087..baab8859 100644
--- a/src/box/vy_read_iterator.h
+++ b/src/box/vy_read_iterator.h
@@ -65,6 +65,11 @@ struct vy_read_iterator {
/** Last statement returned by vy_read_iterator_next(). */
struct tuple *last_stmt;
/**
+ * Last statement added to the tuple cache by
+ * vy_read_iterator_cache_add().
+ */
+ struct tuple *last_cached_stmt;
+ /**
* Copy of lsm->range_tree_version.
* Used for detecting range tree changes.
*/
@@ -142,6 +147,25 @@ NODISCARD int
vy_read_iterator_next(struct vy_read_iterator *itr, struct tuple **result);
/**
+ * Add the last tuple returned by the read iterator to the cache.
+ * @param itr Read iterator
+ * @param stmt Last tuple returned by the iterator.
+ *
+ * We use a separate function for populating the cache rather than
+ * doing that right in vy_read_iterator_next() so that we can store
+ * full tuples in a secondary index cache, thus saving some memory.
+ *
+ * Usage pattern:
+ * - Call vy_read_iterator_next() to get a partial tuple.
+ * - Call vy_point_lookup() to get the full tuple corresponding
+ * to the partial tuple returned by the iterator.
+ * - Call vy_read_iterator_cache_add() on the full tuple to add
+ * the result to the cache.
+ */
+void
+vy_read_iterator_cache_add(struct vy_read_iterator *itr, struct tuple *stmt);
+
+/**
* Close the iterator and free resources.
*/
void
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 10/25] vinyl: do not free pending tasks on shutdown
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (8 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 09/25] vinyl: store full tuples in secondary index cache Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-31 20:48 ` Konstantin Osipov
2018-07-27 11:29 ` [PATCH 11/25] vinyl: store pointer to scheduler in struct vy_task Vladimir Davydov
` (14 subsequent siblings)
24 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
This is a prerequisite for switching scheduler-worker communication from
pthread mutex/cond to cbus, which in turn is needed to generate and send
deferred DELETEs from workers back to tx (#2129).
After this patch, pending tasks will be leaked on shutdown. This is OK,
as we leak a lot of objects on shutdown anyway. The proper way of fixing
this leak would be to rework shutdown without atexit() so that we can
use cbus till the very end.
Needed for #2129
---
src/box/vy_scheduler.c | 47 ++++++++++-------------------------------------
1 file changed, 10 insertions(+), 37 deletions(-)
diff --git a/src/box/vy_scheduler.c b/src/box/vy_scheduler.c
index a82fe9f2..561e7b37 100644
--- a/src/box/vy_scheduler.c
+++ b/src/box/vy_scheduler.c
@@ -84,12 +84,8 @@ struct vy_task_ops {
* This function is called by the scheduler if either ->execute
* or ->complete failed. It may be used to undo changes done to
* the LSM tree when preparing the task.
- *
- * If @in_shutdown is set, the callback is invoked from the
- * engine destructor.
*/
- void (*abort)(struct vy_scheduler *scheduler, struct vy_task *task,
- bool in_shutdown);
+ void (*abort)(struct vy_scheduler *scheduler, struct vy_task *task);
};
struct vy_task {
@@ -279,15 +275,11 @@ vy_scheduler_start_workers(struct vy_scheduler *scheduler)
static void
vy_scheduler_stop_workers(struct vy_scheduler *scheduler)
{
- struct stailq task_queue;
- stailq_create(&task_queue);
-
assert(scheduler->is_worker_pool_running);
scheduler->is_worker_pool_running = false;
- /* Clear the input queue and wake up worker threads. */
+ /* Wake up worker threads. */
tt_pthread_mutex_lock(&scheduler->mutex);
- stailq_concat(&task_queue, &scheduler->input_queue);
pthread_cond_broadcast(&scheduler->worker_cond);
tt_pthread_mutex_unlock(&scheduler->mutex);
@@ -298,15 +290,6 @@ vy_scheduler_stop_workers(struct vy_scheduler *scheduler)
free(scheduler->worker_pool);
scheduler->worker_pool = NULL;
-
- /* Abort all pending tasks. */
- struct vy_task *task, *next;
- stailq_concat(&task_queue, &scheduler->output_queue);
- stailq_foreach_entry_safe(task, next, &task_queue, link) {
- if (task->ops->abort != NULL)
- task->ops->abort(scheduler, task, true);
- vy_task_delete(&scheduler->task_pool, task);
- }
}
void
@@ -888,8 +871,7 @@ fail:
}
static void
-vy_task_dump_abort(struct vy_scheduler *scheduler, struct vy_task *task,
- bool in_shutdown)
+vy_task_dump_abort(struct vy_scheduler *scheduler, struct vy_task *task)
{
struct vy_lsm *lsm = task->lsm;
@@ -902,17 +884,13 @@ vy_task_dump_abort(struct vy_scheduler *scheduler, struct vy_task *task,
* It's no use alerting the user if the server is
* shutting down or the LSM tree was dropped.
*/
- if (!in_shutdown && !lsm->is_dropped) {
+ if (!lsm->is_dropped) {
struct error *e = diag_last_error(&task->diag);
error_log(e);
say_error("%s: dump failed", vy_lsm_name(lsm));
}
- /* The metadata log is unavailable on shutdown. */
- if (!in_shutdown)
- vy_run_discard(task->new_run);
- else
- vy_run_unref(task->new_run);
+ vy_run_discard(task->new_run);
lsm->is_dumping = false;
vy_scheduler_update_lsm(scheduler, lsm);
@@ -1213,8 +1191,7 @@ vy_task_compact_complete(struct vy_scheduler *scheduler, struct vy_task *task)
}
static void
-vy_task_compact_abort(struct vy_scheduler *scheduler, struct vy_task *task,
- bool in_shutdown)
+vy_task_compact_abort(struct vy_scheduler *scheduler, struct vy_task *task)
{
struct vy_lsm *lsm = task->lsm;
struct vy_range *range = task->range;
@@ -1226,18 +1203,14 @@ vy_task_compact_abort(struct vy_scheduler *scheduler, struct vy_task *task,
* It's no use alerting the user if the server is
* shutting down or the LSM tree was dropped.
*/
- if (!in_shutdown && !lsm->is_dropped) {
+ if (!lsm->is_dropped) {
struct error *e = diag_last_error(&task->diag);
error_log(e);
say_error("%s: failed to compact range %s",
vy_lsm_name(lsm), vy_range_str(range));
}
- /* The metadata log is unavailable on shutdown. */
- if (!in_shutdown)
- vy_run_discard(task->new_run);
- else
- vy_run_unref(task->new_run);
+ vy_run_discard(task->new_run);
assert(range->heap_node.pos == UINT32_MAX);
vy_range_heap_insert(&lsm->range_heap, &range->heap_node);
@@ -1476,7 +1449,7 @@ vy_scheduler_complete_task(struct vy_scheduler *scheduler,
{
if (task->lsm->is_dropped) {
if (task->ops->abort)
- task->ops->abort(scheduler, task, false);
+ task->ops->abort(scheduler, task);
return 0;
}
@@ -1499,7 +1472,7 @@ vy_scheduler_complete_task(struct vy_scheduler *scheduler,
return 0;
fail:
if (task->ops->abort)
- task->ops->abort(scheduler, task, false);
+ task->ops->abort(scheduler, task);
diag_move(diag, &scheduler->diag);
return -1;
}
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 11/25] vinyl: store pointer to scheduler in struct vy_task
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (9 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 10/25] vinyl: do not free pending tasks on shutdown Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-31 20:49 ` Konstantin Osipov
2018-07-27 11:29 ` [PATCH 12/25] vinyl: rename some members of vy_scheduler and vy_task struct Vladimir Davydov
` (13 subsequent siblings)
24 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Currently, we don't really need it, but once we switch communication
channel between the scheduler and workers from pthread mutex/cond to
cbus (needed for #2129), tasks won't be completed on behalf of the
scheduler fiber and hence we will need a back pointer from vy_task to
vy_scheduler.
Needed for #2129
---
src/box/vy_scheduler.c | 74 ++++++++++++++++++++++++++------------------------
1 file changed, 39 insertions(+), 35 deletions(-)
diff --git a/src/box/vy_scheduler.c b/src/box/vy_scheduler.c
index 561e7b37..6a6fa1fc 100644
--- a/src/box/vy_scheduler.c
+++ b/src/box/vy_scheduler.c
@@ -72,24 +72,27 @@ struct vy_task_ops {
* which is too heavy for the tx thread (like IO or compression).
* Returns 0 on success. On failure returns -1 and sets diag.
*/
- int (*execute)(struct vy_scheduler *scheduler, struct vy_task *task);
+ int (*execute)(struct vy_task *task);
/**
* This function is called by the scheduler upon task completion.
* It may be used to finish the task from the tx thread context.
*
* Returns 0 on success. On failure returns -1 and sets diag.
*/
- int (*complete)(struct vy_scheduler *scheduler, struct vy_task *task);
+ int (*complete)(struct vy_task *task);
/**
* This function is called by the scheduler if either ->execute
* or ->complete failed. It may be used to undo changes done to
* the LSM tree when preparing the task.
*/
- void (*abort)(struct vy_scheduler *scheduler, struct vy_task *task);
+ void (*abort)(struct vy_task *task);
};
struct vy_task {
+ /** Virtual method table. */
const struct vy_task_ops *ops;
+ /** Pointer to the scheduler. */
+ struct vy_scheduler *scheduler;
/** Return code of ->execute. */
int status;
/** If ->execute fails, the error is stored here. */
@@ -138,10 +141,10 @@ struct vy_task {
* does not free it from under us.
*/
static struct vy_task *
-vy_task_new(struct mempool *pool, struct vy_lsm *lsm,
+vy_task_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
const struct vy_task_ops *ops)
{
- struct vy_task *task = mempool_alloc(pool);
+ struct vy_task *task = mempool_alloc(&scheduler->task_pool);
if (task == NULL) {
diag_set(OutOfMemory, sizeof(*task),
"mempool", "struct vy_task");
@@ -149,16 +152,17 @@ vy_task_new(struct mempool *pool, struct vy_lsm *lsm,
}
memset(task, 0, sizeof(*task));
task->ops = ops;
+ task->scheduler = scheduler;
task->lsm = lsm;
task->cmp_def = key_def_dup(lsm->cmp_def);
if (task->cmp_def == NULL) {
- mempool_free(pool, task);
+ mempool_free(&scheduler->task_pool, task);
return NULL;
}
task->key_def = key_def_dup(lsm->key_def);
if (task->key_def == NULL) {
key_def_delete(task->cmp_def);
- mempool_free(pool, task);
+ mempool_free(&scheduler->task_pool, task);
return NULL;
}
vy_lsm_ref(lsm);
@@ -168,14 +172,13 @@ vy_task_new(struct mempool *pool, struct vy_lsm *lsm,
/** Free a task allocated with vy_task_new(). */
static void
-vy_task_delete(struct mempool *pool, struct vy_task *task)
+vy_task_delete(struct vy_task *task)
{
key_def_delete(task->cmp_def);
key_def_delete(task->key_def);
vy_lsm_unref(task->lsm);
diag_destroy(&task->diag);
- TRASH(task);
- mempool_free(pool, task);
+ mempool_free(&task->scheduler->task_pool, task);
}
static bool
@@ -643,7 +646,7 @@ vy_run_discard(struct vy_run *run)
}
static int
-vy_task_write_run(struct vy_scheduler *scheduler, struct vy_task *task)
+vy_task_write_run(struct vy_task *task)
{
struct vy_lsm *lsm = task->lsm;
struct vy_stmt_stream *wi = task->wi;
@@ -676,7 +679,7 @@ vy_task_write_run(struct vy_scheduler *scheduler, struct vy_task *task)
if (rc != 0)
break;
- if (!scheduler->is_worker_pool_running) {
+ if (!task->scheduler->is_worker_pool_running) {
diag_set(FiberIsCancelled);
rc = -1;
break;
@@ -698,14 +701,15 @@ fail:
}
static int
-vy_task_dump_execute(struct vy_scheduler *scheduler, struct vy_task *task)
+vy_task_dump_execute(struct vy_task *task)
{
- return vy_task_write_run(scheduler, task);
+ return vy_task_write_run(task);
}
static int
-vy_task_dump_complete(struct vy_scheduler *scheduler, struct vy_task *task)
+vy_task_dump_complete(struct vy_task *task)
{
+ struct vy_scheduler *scheduler = task->scheduler;
struct vy_lsm *lsm = task->lsm;
struct vy_run *new_run = task->new_run;
int64_t dump_lsn = new_run->dump_lsn;
@@ -871,8 +875,9 @@ fail:
}
static void
-vy_task_dump_abort(struct vy_scheduler *scheduler, struct vy_task *task)
+vy_task_dump_abort(struct vy_task *task)
{
+ struct vy_scheduler *scheduler = task->scheduler;
struct vy_lsm *lsm = task->lsm;
assert(lsm->is_dumping);
@@ -975,8 +980,7 @@ vy_task_dump_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
return 0;
}
- struct vy_task *task = vy_task_new(&scheduler->task_pool,
- lsm, &dump_ops);
+ struct vy_task *task = vy_task_new(scheduler, lsm, &dump_ops);
if (task == NULL)
goto err;
@@ -1031,7 +1035,7 @@ err_wi_sub:
err_wi:
vy_run_discard(new_run);
err_run:
- vy_task_delete(&scheduler->task_pool, task);
+ vy_task_delete(task);
err:
diag_log();
say_error("%s: could not start dump", vy_lsm_name(lsm));
@@ -1039,14 +1043,15 @@ err:
}
static int
-vy_task_compact_execute(struct vy_scheduler *scheduler, struct vy_task *task)
+vy_task_compact_execute(struct vy_task *task)
{
- return vy_task_write_run(scheduler, task);
+ return vy_task_write_run(task);
}
static int
-vy_task_compact_complete(struct vy_scheduler *scheduler, struct vy_task *task)
+vy_task_compact_complete(struct vy_task *task)
{
+ struct vy_scheduler *scheduler = task->scheduler;
struct vy_lsm *lsm = task->lsm;
struct vy_range *range = task->range;
struct vy_run *new_run = task->new_run;
@@ -1191,8 +1196,9 @@ vy_task_compact_complete(struct vy_scheduler *scheduler, struct vy_task *task)
}
static void
-vy_task_compact_abort(struct vy_scheduler *scheduler, struct vy_task *task)
+vy_task_compact_abort(struct vy_task *task)
{
+ struct vy_scheduler *scheduler = task->scheduler;
struct vy_lsm *lsm = task->lsm;
struct vy_range *range = task->range;
@@ -1243,8 +1249,7 @@ vy_task_compact_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
return 0;
}
- struct vy_task *task = vy_task_new(&scheduler->task_pool,
- lsm, &compact_ops);
+ struct vy_task *task = vy_task_new(scheduler, lsm, &compact_ops);
if (task == NULL)
goto err_task;
@@ -1303,7 +1308,7 @@ err_wi_sub:
err_wi:
vy_run_discard(new_run);
err_run:
- vy_task_delete(&scheduler->task_pool, task);
+ vy_task_delete(task);
err_task:
diag_log();
say_error("%s: could not start compacting range %s: %s",
@@ -1444,12 +1449,11 @@ fail:
}
static int
-vy_scheduler_complete_task(struct vy_scheduler *scheduler,
- struct vy_task *task)
+vy_task_complete(struct vy_task *task)
{
if (task->lsm->is_dropped) {
if (task->ops->abort)
- task->ops->abort(scheduler, task);
+ task->ops->abort(task);
return 0;
}
@@ -1464,7 +1468,7 @@ vy_scheduler_complete_task(struct vy_scheduler *scheduler,
diag_move(diag_get(), diag);
goto fail; });
if (task->ops->complete &&
- task->ops->complete(scheduler, task) != 0) {
+ task->ops->complete(task) != 0) {
assert(!diag_is_empty(diag_get()));
diag_move(diag_get(), diag);
goto fail;
@@ -1472,8 +1476,8 @@ vy_scheduler_complete_task(struct vy_scheduler *scheduler,
return 0;
fail:
if (task->ops->abort)
- task->ops->abort(scheduler, task);
- diag_move(diag, &scheduler->diag);
+ task->ops->abort(task);
+ diag_move(diag, &task->scheduler->diag);
return -1;
}
@@ -1510,11 +1514,11 @@ vy_scheduler_f(va_list va)
/* Complete and delete all processed tasks. */
stailq_foreach_entry_safe(task, next, &output_queue, link) {
- if (vy_scheduler_complete_task(scheduler, task) != 0)
+ if (vy_task_complete(task) != 0)
tasks_failed++;
else
tasks_done++;
- vy_task_delete(&scheduler->task_pool, task);
+ vy_task_delete(task);
scheduler->workers_available++;
assert(scheduler->workers_available <=
scheduler->worker_pool_size);
@@ -1615,7 +1619,7 @@ vy_worker_f(void *arg)
assert(task != NULL);
/* Execute task */
- task->status = task->ops->execute(scheduler, task);
+ task->status = task->ops->execute(task);
if (task->status != 0) {
struct diag *diag = diag_get();
assert(!diag_is_empty(diag));
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 12/25] vinyl: rename some members of vy_scheduler and vy_task struct
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (10 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 11/25] vinyl: store pointer to scheduler in struct vy_task Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 13/25] vinyl: use cbus for communication between scheduler and worker threads Vladimir Davydov
` (12 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
I'm planning to add some new members and remove some old members from
those structs. For this to play nicely, let's do some renames:
vy_scheduler::workers_available => idle_worker_count
vy_scheduler::input_queue => pending_tasks
vy_scheduler::output_queue => processed_tasks
vy_task::link => in_pending, in_processed
---
src/box/vy_scheduler.c | 50 ++++++++++++++++++++++++++------------------------
src/box/vy_scheduler.h | 10 +++++-----
2 files changed, 31 insertions(+), 29 deletions(-)
diff --git a/src/box/vy_scheduler.c b/src/box/vy_scheduler.c
index 6a6fa1fc..1ae6dd02 100644
--- a/src/box/vy_scheduler.c
+++ b/src/box/vy_scheduler.c
@@ -120,17 +120,16 @@ struct vy_task {
*/
struct vy_slice *first_slice, *last_slice;
/**
- * Link in the list of pending or processed tasks.
- * See vy_scheduler::input_queue, output_queue.
- */
- struct stailq_entry link;
- /**
* Index options may be modified while a task is in
* progress so we save them here to safely access them
* from another thread.
*/
double bloom_fpr;
int64_t page_size;
+ /** Link in vy_scheduler::pending_tasks. */
+ struct stailq_entry in_pending;
+ /** Link in vy_scheduler::processed_tasks. */
+ struct stailq_entry in_processed;
};
/**
@@ -259,7 +258,7 @@ vy_scheduler_start_workers(struct vy_scheduler *scheduler)
assert(scheduler->worker_pool_size >= 2);
scheduler->is_worker_pool_running = true;
- scheduler->workers_available = scheduler->worker_pool_size;
+ scheduler->idle_worker_count = scheduler->worker_pool_size;
scheduler->worker_pool = calloc(scheduler->worker_pool_size,
sizeof(struct cord));
if (scheduler->worker_pool == NULL)
@@ -318,8 +317,8 @@ vy_scheduler_create(struct vy_scheduler *scheduler, int write_threads,
scheduler->worker_pool_size = write_threads;
mempool_create(&scheduler->task_pool, cord_slab_cache(),
sizeof(struct vy_task));
- stailq_create(&scheduler->input_queue);
- stailq_create(&scheduler->output_queue);
+ stailq_create(&scheduler->pending_tasks);
+ stailq_create(&scheduler->processed_tasks);
tt_pthread_cond_init(&scheduler->worker_cond, NULL);
tt_pthread_mutex_init(&scheduler->mutex, NULL);
@@ -1422,7 +1421,7 @@ vy_schedule(struct vy_scheduler *scheduler, struct vy_task **ptask)
if (*ptask != NULL)
return 0;
- if (scheduler->workers_available <= 1) {
+ if (scheduler->idle_worker_count <= 1) {
/*
* If all worker threads are busy doing compaction
* when we run out of quota, ongoing transactions will
@@ -1501,26 +1500,27 @@ vy_scheduler_f(va_list va)
vy_scheduler_start_workers(scheduler);
while (scheduler->scheduler_fiber != NULL) {
- struct stailq output_queue;
+ struct stailq processed_tasks;
struct vy_task *task, *next;
int tasks_failed = 0, tasks_done = 0;
bool was_empty;
/* Get the list of processed tasks. */
- stailq_create(&output_queue);
+ stailq_create(&processed_tasks);
tt_pthread_mutex_lock(&scheduler->mutex);
- stailq_concat(&output_queue, &scheduler->output_queue);
+ stailq_concat(&processed_tasks, &scheduler->processed_tasks);
tt_pthread_mutex_unlock(&scheduler->mutex);
/* Complete and delete all processed tasks. */
- stailq_foreach_entry_safe(task, next, &output_queue, link) {
+ stailq_foreach_entry_safe(task, next, &processed_tasks,
+ in_processed) {
if (vy_task_complete(task) != 0)
tasks_failed++;
else
tasks_done++;
vy_task_delete(task);
- scheduler->workers_available++;
- assert(scheduler->workers_available <=
+ scheduler->idle_worker_count++;
+ assert(scheduler->idle_worker_count <=
scheduler->worker_pool_size);
}
/*
@@ -1534,7 +1534,7 @@ vy_scheduler_f(va_list va)
* opens a time window for a worker to submit
* a processed task and wake up the scheduler
* (via scheduler_async). Hence we should go
- * and recheck the output_queue in order not
+ * and recheck the processed_tasks in order not
* to lose a wakeup event and hang for good.
*/
continue;
@@ -1543,7 +1543,7 @@ vy_scheduler_f(va_list va)
if (tasks_failed > 0)
goto error;
/* All worker threads are busy. */
- if (scheduler->workers_available == 0)
+ if (scheduler->idle_worker_count == 0)
goto wait;
/* Get a task to schedule. */
if (vy_schedule(scheduler, &task) != 0)
@@ -1554,13 +1554,14 @@ vy_scheduler_f(va_list va)
/* Queue the task and notify workers if necessary. */
tt_pthread_mutex_lock(&scheduler->mutex);
- was_empty = stailq_empty(&scheduler->input_queue);
- stailq_add_tail_entry(&scheduler->input_queue, task, link);
+ was_empty = stailq_empty(&scheduler->pending_tasks);
+ stailq_add_tail_entry(&scheduler->pending_tasks,
+ task, in_pending);
if (was_empty)
tt_pthread_cond_signal(&scheduler->worker_cond);
tt_pthread_mutex_unlock(&scheduler->mutex);
- scheduler->workers_available--;
+ scheduler->idle_worker_count--;
fiber_reschedule();
continue;
error:
@@ -1605,7 +1606,7 @@ vy_worker_f(void *arg)
tt_pthread_mutex_lock(&scheduler->mutex);
while (scheduler->is_worker_pool_running) {
/* Wait for a task */
- if (stailq_empty(&scheduler->input_queue)) {
+ if (stailq_empty(&scheduler->pending_tasks)) {
/* Wake scheduler up if there are no more tasks */
ev_async_send(scheduler->scheduler_loop,
&scheduler->scheduler_async);
@@ -1613,8 +1614,8 @@ vy_worker_f(void *arg)
&scheduler->mutex);
continue;
}
- task = stailq_shift_entry(&scheduler->input_queue,
- struct vy_task, link);
+ task = stailq_shift_entry(&scheduler->pending_tasks,
+ struct vy_task, in_pending);
tt_pthread_mutex_unlock(&scheduler->mutex);
assert(task != NULL);
@@ -1628,7 +1629,8 @@ vy_worker_f(void *arg)
/* Return processed task to scheduler */
tt_pthread_mutex_lock(&scheduler->mutex);
- stailq_add_tail_entry(&scheduler->output_queue, task, link);
+ stailq_add_tail_entry(&scheduler->processed_tasks,
+ task, in_processed);
}
tt_pthread_mutex_unlock(&scheduler->mutex);
return NULL;
diff --git a/src/box/vy_scheduler.h b/src/box/vy_scheduler.h
index 777756c0..284f666e 100644
--- a/src/box/vy_scheduler.h
+++ b/src/box/vy_scheduler.h
@@ -77,13 +77,13 @@ struct vy_scheduler {
/** Total number of worker threads. */
int worker_pool_size;
/** Number worker threads that are currently idle. */
- int workers_available;
+ int idle_worker_count;
/** Memory pool used for allocating vy_task objects. */
struct mempool task_pool;
- /** Queue of pending tasks, linked by vy_task::link. */
- struct stailq input_queue;
- /** Queue of processed tasks, linked by vy_task::link. */
- struct stailq output_queue;
+ /** Queue of pending tasks, linked by vy_task::in_pending. */
+ struct stailq pending_tasks;
+ /** Queue of processed tasks, linked by vy_task::in_processed. */
+ struct stailq processed_tasks;
/**
* Signaled to wake up a worker when there is
* a pending task in the input queue. Also used
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 13/25] vinyl: use cbus for communication between scheduler and worker threads
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (11 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 12/25] vinyl: rename some members of vy_scheduler and vy_task struct Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 14/25] vinyl: zap vy_scheduler::is_worker_pool_running Vladimir Davydov
` (11 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
We need cbus for forwarding deferred DELETE statements generated in a
worker thread during primary index compaction to the tx thread where
they can be inserted into secondary indexes. Since pthread mutex/cond
and cbus are incompatible by their nature, let's rework communication
channel between the tx and worker threads using cbus.
Needed for #2129
---
src/box/vy_scheduler.c | 215 ++++++++++++++++++++++++++++++-------------------
src/box/vy_scheduler.h | 25 +-----
2 files changed, 134 insertions(+), 106 deletions(-)
diff --git a/src/box/vy_scheduler.c b/src/box/vy_scheduler.c
index 1ae6dd02..4813f6f4 100644
--- a/src/box/vy_scheduler.c
+++ b/src/box/vy_scheduler.c
@@ -46,6 +46,7 @@
#include "errinj.h"
#include "fiber.h"
#include "fiber_cond.h"
+#include "cbus.h"
#include "salad/stailq.h"
#include "say.h"
#include "vy_lsm.h"
@@ -55,14 +56,34 @@
#include "vy_run.h"
#include "vy_write_iterator.h"
#include "trivia/util.h"
-#include "tt_pthread.h"
/* Min and max values for vy_scheduler::timeout. */
#define VY_SCHEDULER_TIMEOUT_MIN 1
#define VY_SCHEDULER_TIMEOUT_MAX 60
-static void *vy_worker_f(void *);
+static int vy_worker_f(va_list);
static int vy_scheduler_f(va_list);
+static void vy_task_execute_f(struct cmsg *);
+static void vy_task_complete_f(struct cmsg *);
+
+static const struct cmsg_hop vy_task_execute_route[] = {
+ { vy_task_execute_f, NULL },
+};
+
+static const struct cmsg_hop vy_task_complete_route[] = {
+ { vy_task_complete_f, NULL },
+};
+
+/** Vinyl worker thread. */
+struct vy_worker {
+ struct cord cord;
+ /** Pipe from tx to the worker thread. */
+ struct cpipe worker_pipe;
+ /** Pipe from the worker thread to tx. */
+ struct cpipe tx_pipe;
+ /** Link in vy_scheduler::idle_workers. */
+ struct stailq_entry in_idle;
+};
struct vy_task;
@@ -89,10 +110,22 @@ struct vy_task_ops {
};
struct vy_task {
+ /**
+ * CBus message used for sending the task to/from
+ * a worker thread.
+ */
+ struct cmsg cmsg;
/** Virtual method table. */
const struct vy_task_ops *ops;
/** Pointer to the scheduler. */
struct vy_scheduler *scheduler;
+ /** Worker thread this task is assigned to. */
+ struct vy_worker *worker;
+ /**
+ * Fiber that is currently executing this task in
+ * a worker thread.
+ */
+ struct fiber *fiber;
/** Return code of ->execute. */
int status;
/** If ->execute fails, the error is stored here. */
@@ -126,8 +159,6 @@ struct vy_task {
*/
double bloom_fpr;
int64_t page_size;
- /** Link in vy_scheduler::pending_tasks. */
- struct stailq_entry in_pending;
/** Link in vy_scheduler::processed_tasks. */
struct stailq_entry in_processed;
};
@@ -241,16 +272,6 @@ vy_compact_heap_less(struct heap_node *a, struct heap_node *b)
#undef HEAP_NAME
static void
-vy_scheduler_async_cb(ev_loop *loop, struct ev_async *watcher, int events)
-{
- (void)loop;
- (void)events;
- struct vy_scheduler *scheduler = container_of(watcher,
- struct vy_scheduler, scheduler_async);
- fiber_cond_signal(&scheduler->scheduler_cond);
-}
-
-static void
vy_scheduler_start_workers(struct vy_scheduler *scheduler)
{
assert(!scheduler->is_worker_pool_running);
@@ -260,17 +281,19 @@ vy_scheduler_start_workers(struct vy_scheduler *scheduler)
scheduler->is_worker_pool_running = true;
scheduler->idle_worker_count = scheduler->worker_pool_size;
scheduler->worker_pool = calloc(scheduler->worker_pool_size,
- sizeof(struct cord));
+ sizeof(*scheduler->worker_pool));
if (scheduler->worker_pool == NULL)
panic("failed to allocate vinyl worker pool");
- ev_async_start(scheduler->scheduler_loop, &scheduler->scheduler_async);
for (int i = 0; i < scheduler->worker_pool_size; i++) {
char name[FIBER_NAME_MAX];
snprintf(name, sizeof(name), "vinyl.writer.%d", i);
- if (cord_start(&scheduler->worker_pool[i], name,
- vy_worker_f, scheduler) != 0)
+ struct vy_worker *worker = &scheduler->worker_pool[i];
+ if (cord_costart(&worker->cord, name, vy_worker_f, worker) != 0)
panic("failed to start vinyl worker thread");
+ cpipe_create(&worker->worker_pipe, name);
+ stailq_add_tail_entry(&scheduler->idle_workers,
+ worker, in_idle);
}
}
@@ -280,16 +303,12 @@ vy_scheduler_stop_workers(struct vy_scheduler *scheduler)
assert(scheduler->is_worker_pool_running);
scheduler->is_worker_pool_running = false;
- /* Wake up worker threads. */
- tt_pthread_mutex_lock(&scheduler->mutex);
- pthread_cond_broadcast(&scheduler->worker_cond);
- tt_pthread_mutex_unlock(&scheduler->mutex);
-
- /* Wait for worker threads to exit. */
- for (int i = 0; i < scheduler->worker_pool_size; i++)
- cord_join(&scheduler->worker_pool[i]);
- ev_async_stop(scheduler->scheduler_loop, &scheduler->scheduler_async);
-
+ for (int i = 0; i < scheduler->worker_pool_size; i++) {
+ struct vy_worker *worker = &scheduler->worker_pool[i];
+ cbus_stop_loop(&worker->worker_pipe);
+ cpipe_destroy(&worker->worker_pipe);
+ cord_join(&worker->cord);
+ }
free(scheduler->worker_pool);
scheduler->worker_pool = NULL;
}
@@ -310,19 +329,14 @@ vy_scheduler_create(struct vy_scheduler *scheduler, int write_threads,
if (scheduler->scheduler_fiber == NULL)
panic("failed to allocate vinyl scheduler fiber");
- scheduler->scheduler_loop = loop();
fiber_cond_create(&scheduler->scheduler_cond);
- ev_async_init(&scheduler->scheduler_async, vy_scheduler_async_cb);
scheduler->worker_pool_size = write_threads;
mempool_create(&scheduler->task_pool, cord_slab_cache(),
sizeof(struct vy_task));
- stailq_create(&scheduler->pending_tasks);
+ stailq_create(&scheduler->idle_workers);
stailq_create(&scheduler->processed_tasks);
- tt_pthread_cond_init(&scheduler->worker_cond, NULL);
- tt_pthread_mutex_init(&scheduler->mutex, NULL);
-
vy_dump_heap_create(&scheduler->dump_heap);
vy_compact_heap_create(&scheduler->compact_heap);
@@ -344,9 +358,6 @@ vy_scheduler_destroy(struct vy_scheduler *scheduler)
if (scheduler->is_worker_pool_running)
vy_scheduler_stop_workers(scheduler);
- tt_pthread_cond_destroy(&scheduler->worker_cond);
- tt_pthread_mutex_destroy(&scheduler->mutex);
-
diag_destroy(&scheduler->diag);
mempool_destroy(&scheduler->task_pool);
fiber_cond_destroy(&scheduler->dump_cond);
@@ -647,6 +658,8 @@ vy_run_discard(struct vy_run *run)
static int
vy_task_write_run(struct vy_task *task)
{
+ enum { YIELD_LOOPS = 32 };
+
struct vy_lsm *lsm = task->lsm;
struct vy_stmt_stream *wi = task->wi;
@@ -668,6 +681,7 @@ vy_task_write_run(struct vy_task *task)
if (wi->iface->start(wi) != 0)
goto fail_abort_writer;
int rc;
+ int loops = 0;
struct tuple *stmt = NULL;
while ((rc = wi->iface->next(wi, &stmt)) == 0 && stmt != NULL) {
inj = errinj(ERRINJ_VY_RUN_WRITE_STMT_TIMEOUT, ERRINJ_DOUBLE);
@@ -678,7 +692,9 @@ vy_task_write_run(struct vy_task *task)
if (rc != 0)
break;
- if (!task->scheduler->is_worker_pool_running) {
+ if (++loops % YIELD_LOOPS == 0)
+ fiber_sleep(0);
+ if (fiber_is_cancelled()) {
diag_set(FiberIsCancelled);
rc = -1;
break;
@@ -1316,6 +1332,62 @@ err_task:
}
/**
+ * Fiber function that actually executes a vinyl task.
+ * After finishing a task, it sends it back to tx.
+ */
+static int
+vy_task_f(va_list va)
+{
+ struct vy_task *task = va_arg(va, struct vy_task *);
+ task->status = task->ops->execute(task);
+ if (task->status != 0) {
+ struct diag *diag = diag_get();
+ assert(!diag_is_empty(diag));
+ diag_move(diag, &task->diag);
+ }
+ cmsg_init(&task->cmsg, vy_task_complete_route);
+ cpipe_push(&task->worker->tx_pipe, &task->cmsg);
+ task->fiber = NULL;
+ return 0;
+}
+
+/**
+ * Callback invoked by a worker thread upon receiving a task.
+ * It schedules a fiber which actually executes the task, so
+ * as not to block the event loop.
+ */
+static void
+vy_task_execute_f(struct cmsg *cmsg)
+{
+ struct vy_task *task = container_of(cmsg, struct vy_task, cmsg);
+ assert(task->fiber == NULL);
+ task->fiber = fiber_new("task", vy_task_f);
+ if (task->fiber == NULL) {
+ task->status = -1;
+ diag_move(diag_get(), &task->diag);
+ cmsg_init(&task->cmsg, vy_task_complete_route);
+ cpipe_push(&task->worker->tx_pipe, &task->cmsg);
+ } else {
+ fiber_start(task->fiber, task);
+ }
+}
+
+/**
+ * Callback invoked by the tx thread upon receiving an executed
+ * task from a worker thread. It adds the task to the processed
+ * task queue and wakes up the scheduler so that it can complete
+ * it.
+ */
+static void
+vy_task_complete_f(struct cmsg *cmsg)
+{
+ struct vy_task *task = container_of(cmsg, struct vy_task, cmsg);
+ stailq_add_tail_entry(&task->scheduler->processed_tasks,
+ task, in_processed);
+ fiber_cond_signal(&task->scheduler->scheduler_cond);
+}
+
+/**
* Create a task for dumping an LSM tree. The new task is returned
* in @ptask. If there's no LSM tree that needs to be dumped @ptask
* is set to NULL.
@@ -1503,13 +1575,10 @@ vy_scheduler_f(va_list va)
struct stailq processed_tasks;
struct vy_task *task, *next;
int tasks_failed = 0, tasks_done = 0;
- bool was_empty;
/* Get the list of processed tasks. */
stailq_create(&processed_tasks);
- tt_pthread_mutex_lock(&scheduler->mutex);
stailq_concat(&processed_tasks, &scheduler->processed_tasks);
- tt_pthread_mutex_unlock(&scheduler->mutex);
/* Complete and delete all processed tasks. */
stailq_foreach_entry_safe(task, next, &processed_tasks,
@@ -1518,6 +1587,8 @@ vy_scheduler_f(va_list va)
tasks_failed++;
else
tasks_done++;
+ stailq_add_entry(&scheduler->idle_workers,
+ task->worker, in_idle);
vy_task_delete(task);
scheduler->idle_worker_count++;
assert(scheduler->idle_worker_count <=
@@ -1553,15 +1624,13 @@ vy_scheduler_f(va_list va)
goto wait;
/* Queue the task and notify workers if necessary. */
- tt_pthread_mutex_lock(&scheduler->mutex);
- was_empty = stailq_empty(&scheduler->pending_tasks);
- stailq_add_tail_entry(&scheduler->pending_tasks,
- task, in_pending);
- if (was_empty)
- tt_pthread_cond_signal(&scheduler->worker_cond);
- tt_pthread_mutex_unlock(&scheduler->mutex);
-
+ assert(!stailq_empty(&scheduler->idle_workers));
+ task->worker = stailq_shift_entry(&scheduler->idle_workers,
+ struct vy_worker, in_idle);
scheduler->idle_worker_count--;
+ cmsg_init(&task->cmsg, vy_task_execute_route);
+ cpipe_push(&task->worker->worker_pipe, &task->cmsg);
+
fiber_reschedule();
continue;
error:
@@ -1597,41 +1666,17 @@ wait:
return 0;
}
-static void *
-vy_worker_f(void *arg)
+static int
+vy_worker_f(va_list ap)
{
- struct vy_scheduler *scheduler = arg;
- struct vy_task *task = NULL;
-
- tt_pthread_mutex_lock(&scheduler->mutex);
- while (scheduler->is_worker_pool_running) {
- /* Wait for a task */
- if (stailq_empty(&scheduler->pending_tasks)) {
- /* Wake scheduler up if there are no more tasks */
- ev_async_send(scheduler->scheduler_loop,
- &scheduler->scheduler_async);
- tt_pthread_cond_wait(&scheduler->worker_cond,
- &scheduler->mutex);
- continue;
- }
- task = stailq_shift_entry(&scheduler->pending_tasks,
- struct vy_task, in_pending);
- tt_pthread_mutex_unlock(&scheduler->mutex);
- assert(task != NULL);
-
- /* Execute task */
- task->status = task->ops->execute(task);
- if (task->status != 0) {
- struct diag *diag = diag_get();
- assert(!diag_is_empty(diag));
- diag_move(diag, &task->diag);
- }
-
- /* Return processed task to scheduler */
- tt_pthread_mutex_lock(&scheduler->mutex);
- stailq_add_tail_entry(&scheduler->processed_tasks,
- task, in_processed);
- }
- tt_pthread_mutex_unlock(&scheduler->mutex);
- return NULL;
+ struct vy_worker *worker = va_arg(ap, struct vy_worker *);
+ struct cbus_endpoint endpoint;
+
+ cpipe_create(&worker->tx_pipe, "tx");
+ cbus_endpoint_create(&endpoint, cord_name(&worker->cord),
+ fiber_schedule_cb, fiber());
+ cbus_loop(&endpoint);
+ cbus_endpoint_destroy(&endpoint, cbus_process);
+ cpipe_destroy(&worker->tx_pipe);
+ return 0;
}
diff --git a/src/box/vy_scheduler.h b/src/box/vy_scheduler.h
index 284f666e..a235aa6f 100644
--- a/src/box/vy_scheduler.h
+++ b/src/box/vy_scheduler.h
@@ -42,16 +42,15 @@
#define HEAP_FORWARD_DECLARATION
#include "salad/heap.h"
#include "salad/stailq.h"
-#include "tt_pthread.h"
#if defined(__cplusplus)
extern "C" {
#endif /* defined(__cplusplus) */
-struct cord;
struct fiber;
struct vy_lsm;
struct vy_run_env;
+struct vy_worker;
struct vy_scheduler;
typedef void
@@ -61,42 +60,26 @@ typedef void
struct vy_scheduler {
/** Scheduler fiber. */
struct fiber *scheduler_fiber;
- /** Scheduler event loop. */
- struct ev_loop *scheduler_loop;
/** Used to wake up the scheduler fiber from TX. */
struct fiber_cond scheduler_cond;
- /** Used to wake up the scheduler from a worker thread. */
- struct ev_async scheduler_async;
/**
* Array of worker threads used for performing
* dump/compaction tasks.
*/
- struct cord *worker_pool;
+ struct vy_worker *worker_pool;
/** Set if the worker threads are running. */
bool is_worker_pool_running;
/** Total number of worker threads. */
int worker_pool_size;
/** Number worker threads that are currently idle. */
int idle_worker_count;
+ /** List of idle workers, linked by vy_worker::in_idle. */
+ struct stailq idle_workers;
/** Memory pool used for allocating vy_task objects. */
struct mempool task_pool;
- /** Queue of pending tasks, linked by vy_task::in_pending. */
- struct stailq pending_tasks;
/** Queue of processed tasks, linked by vy_task::in_processed. */
struct stailq processed_tasks;
/**
- * Signaled to wake up a worker when there is
- * a pending task in the input queue. Also used
- * to stop worker threads on shutdown.
- */
- pthread_cond_t worker_cond;
- /**
- * Mutex protecting input and output queues and
- * the condition variable used to wake up worker
- * threads.
- */
- pthread_mutex_t mutex;
- /**
* Heap of LSM trees, ordered by dump priority,
* linked by vy_lsm::in_dump.
*/
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 14/25] vinyl: zap vy_scheduler::is_worker_pool_running
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (12 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 13/25] vinyl: use cbus for communication between scheduler and worker threads Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 15/25] vinyl: rename vy_task::status to is_failed Vladimir Davydov
` (10 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
This flag is set iff worker_pool != NULL hence it is pointless.
---
src/box/vy_scheduler.c | 9 +++------
src/box/vy_scheduler.h | 2 --
2 files changed, 3 insertions(+), 8 deletions(-)
diff --git a/src/box/vy_scheduler.c b/src/box/vy_scheduler.c
index 4813f6f4..5752ae91 100644
--- a/src/box/vy_scheduler.c
+++ b/src/box/vy_scheduler.c
@@ -274,11 +274,10 @@ vy_compact_heap_less(struct heap_node *a, struct heap_node *b)
static void
vy_scheduler_start_workers(struct vy_scheduler *scheduler)
{
- assert(!scheduler->is_worker_pool_running);
+ assert(scheduler->worker_pool == NULL);
/* One thread is reserved for dumps, see vy_schedule(). */
assert(scheduler->worker_pool_size >= 2);
- scheduler->is_worker_pool_running = true;
scheduler->idle_worker_count = scheduler->worker_pool_size;
scheduler->worker_pool = calloc(scheduler->worker_pool_size,
sizeof(*scheduler->worker_pool));
@@ -300,9 +299,7 @@ vy_scheduler_start_workers(struct vy_scheduler *scheduler)
static void
vy_scheduler_stop_workers(struct vy_scheduler *scheduler)
{
- assert(scheduler->is_worker_pool_running);
- scheduler->is_worker_pool_running = false;
-
+ assert(scheduler->worker_pool != NULL);
for (int i = 0; i < scheduler->worker_pool_size; i++) {
struct vy_worker *worker = &scheduler->worker_pool[i];
cbus_stop_loop(&worker->worker_pipe);
@@ -355,7 +352,7 @@ vy_scheduler_destroy(struct vy_scheduler *scheduler)
fiber_cond_signal(&scheduler->dump_cond);
fiber_cond_signal(&scheduler->scheduler_cond);
- if (scheduler->is_worker_pool_running)
+ if (scheduler->worker_pool != NULL)
vy_scheduler_stop_workers(scheduler);
diag_destroy(&scheduler->diag);
diff --git a/src/box/vy_scheduler.h b/src/box/vy_scheduler.h
index a235aa6f..deefacd7 100644
--- a/src/box/vy_scheduler.h
+++ b/src/box/vy_scheduler.h
@@ -67,8 +67,6 @@ struct vy_scheduler {
* dump/compaction tasks.
*/
struct vy_worker *worker_pool;
- /** Set if the worker threads are running. */
- bool is_worker_pool_running;
/** Total number of worker threads. */
int worker_pool_size;
/** Number worker threads that are currently idle. */
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 15/25] vinyl: rename vy_task::status to is_failed
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (13 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 14/25] vinyl: zap vy_scheduler::is_worker_pool_running Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 16/25] xrow: allow to store flags in DML requests Vladimir Davydov
` (9 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
vy_task::status stores the return code of the ->execute method. There
are only two codes in use: 0 - success and -1 - failure. So let's chage
this to a boolean flag.
---
src/box/vy_scheduler.c | 14 +++++++-------
1 file changed, 7 insertions(+), 7 deletions(-)
diff --git a/src/box/vy_scheduler.c b/src/box/vy_scheduler.c
index 5752ae91..b206a605 100644
--- a/src/box/vy_scheduler.c
+++ b/src/box/vy_scheduler.c
@@ -126,9 +126,9 @@ struct vy_task {
* a worker thread.
*/
struct fiber *fiber;
- /** Return code of ->execute. */
- int status;
- /** If ->execute fails, the error is stored here. */
+ /** Set if the task failed. */
+ bool is_failed;
+ /** In case of task failure the error is stored here. */
struct diag diag;
/** LSM tree this task is for. */
struct vy_lsm *lsm;
@@ -1336,10 +1336,10 @@ static int
vy_task_f(va_list va)
{
struct vy_task *task = va_arg(va, struct vy_task *);
- task->status = task->ops->execute(task);
- if (task->status != 0) {
+ if (task->ops->execute(task) != 0) {
struct diag *diag = diag_get();
assert(!diag_is_empty(diag));
+ task->is_failed = true;
diag_move(diag, &task->diag);
}
cmsg_init(&task->cmsg, vy_task_complete_route);
@@ -1360,7 +1360,7 @@ vy_task_execute_f(struct cmsg *cmsg)
assert(task->fiber == NULL);
task->fiber = fiber_new("task", vy_task_f);
if (task->fiber == NULL) {
- task->status = -1;
+ task->is_failed = true;
diag_move(diag_get(), &task->diag);
cmsg_init(&task->cmsg, vy_task_complete_route);
cpipe_push(&task->worker->tx_pipe, &task->cmsg);
@@ -1526,7 +1526,7 @@ vy_task_complete(struct vy_task *task)
}
struct diag *diag = &task->diag;
- if (task->status != 0) {
+ if (task->is_failed) {
assert(!diag_is_empty(diag));
goto fail; /* ->execute fialed */
}
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 16/25] xrow: allow to store flags in DML requests
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (14 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 15/25] vinyl: rename vy_task::status to is_failed Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 17/25] vinyl: pin last statement returned by write iterator explicitly Vladimir Davydov
` (8 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
In the scope of #2129 we need to mark REPLACE statements for which we
generated DELETE in secondary indexes so that we don't generate DELETE
again on compaction. We also need to mark DELETE statements that were
generated on compaction so that we can skip them on SELECT.
Let's add flags field to struct vy_stmt. Flags are stored both in memory
and on disk so to encode/decode them we also need to add a new iproto
key (IPROTO_FLAGS) and the corresponding field to struct request.
Needed for #2129
---
src/box/iproto_constants.c | 4 ++--
src/box/iproto_constants.h | 3 ++-
src/box/vy_stmt.c | 4 ++++
src/box/vy_stmt.h | 15 +++++++++++++++
src/box/xrow.c | 8 ++++++++
src/box/xrow.h | 2 ++
6 files changed, 33 insertions(+), 3 deletions(-)
diff --git a/src/box/iproto_constants.c b/src/box/iproto_constants.c
index e35738b4..ece99ca2 100644
--- a/src/box/iproto_constants.c
+++ b/src/box/iproto_constants.c
@@ -61,10 +61,10 @@ const unsigned char iproto_key_type[IPROTO_KEY_MAX] =
/* 0x13 */ MP_UINT, /* IPROTO_OFFSET */
/* 0x14 */ MP_UINT, /* IPROTO_ITERATOR */
/* 0x15 */ MP_UINT, /* IPROTO_INDEX_BASE */
+ /* 0x16 */ MP_UINT, /* IPROTO_FLAGS */
/* }}} */
/* {{{ unused */
- /* 0x16 */ MP_UINT,
/* 0x17 */ MP_UINT,
/* 0x18 */ MP_UINT,
/* 0x19 */ MP_UINT,
@@ -148,7 +148,7 @@ const char *iproto_key_strs[IPROTO_KEY_MAX] = {
"offset", /* 0x13 */
"iterator", /* 0x14 */
"index base", /* 0x15 */
- NULL, /* 0x16 */
+ "flags", /* 0x16 */
NULL, /* 0x17 */
NULL, /* 0x18 */
NULL, /* 0x19 */
diff --git a/src/box/iproto_constants.h b/src/box/iproto_constants.h
index f282a0b2..b7d73b9f 100644
--- a/src/box/iproto_constants.h
+++ b/src/box/iproto_constants.h
@@ -66,6 +66,7 @@ enum iproto_key {
IPROTO_OFFSET = 0x13,
IPROTO_ITERATOR = 0x14,
IPROTO_INDEX_BASE = 0x15,
+ IPROTO_FLAGS = 0x16,
/* Leave a gap between integer values and other keys */
IPROTO_KEY = 0x20,
IPROTO_TUPLE = 0x21,
@@ -96,7 +97,7 @@ enum iproto_ballot_key {
bit(LSN) | bit(SCHEMA_VERSION))
#define IPROTO_DML_BODY_BMAP (bit(SPACE_ID) | bit(INDEX_ID) | bit(LIMIT) |\
bit(OFFSET) | bit(ITERATOR) | bit(INDEX_BASE) |\
- bit(KEY) | bit(TUPLE) | bit(OPS))
+ bit(KEY) | bit(TUPLE) | bit(OPS) | bit(FLAGS))
static inline bool
xrow_header_has_key(const char *pos, const char *end)
diff --git a/src/box/vy_stmt.c b/src/box/vy_stmt.c
index a4b7975b..09daa7f4 100644
--- a/src/box/vy_stmt.c
+++ b/src/box/vy_stmt.c
@@ -112,6 +112,7 @@ vy_stmt_alloc(struct tuple_format *format, uint32_t bsize)
tuple->data_offset = sizeof(struct vy_stmt) + meta_size;;
vy_stmt_set_lsn(tuple, 0);
vy_stmt_set_type(tuple, 0);
+ vy_stmt_set_flags(tuple, 0);
return tuple;
}
@@ -498,6 +499,7 @@ vy_stmt_encode_primary(const struct tuple *value,
struct request request;
memset(&request, 0, sizeof(request));
request.type = type;
+ request.flags = vy_stmt_flags(value);
request.space_id = space_id;
uint32_t size;
const char *extracted = NULL;
@@ -544,6 +546,7 @@ vy_stmt_encode_secondary(const struct tuple *value,
struct request request;
memset(&request, 0, sizeof(request));
request.type = type;
+ request.flags = vy_stmt_flags(value);
uint32_t size;
const char *extracted = tuple_extract_key(value, cmp_def, &size);
if (extracted == NULL)
@@ -614,6 +617,7 @@ vy_stmt_decode(struct xrow_header *xrow, const struct key_def *key_def,
return NULL; /* OOM */
vy_stmt_set_lsn(stmt, xrow->lsn);
+ vy_stmt_set_flags(stmt, request.flags);
return stmt;
}
diff --git a/src/box/vy_stmt.h b/src/box/vy_stmt.h
index e53f98ce..bcf855dd 100644
--- a/src/box/vy_stmt.h
+++ b/src/box/vy_stmt.h
@@ -103,6 +103,7 @@ struct vy_stmt {
struct tuple base;
int64_t lsn;
uint8_t type; /* IPROTO_SELECT/REPLACE/UPSERT/DELETE */
+ uint8_t flags;
/**
* Offsets array concatenated with MessagePack fields
* array.
@@ -138,6 +139,20 @@ vy_stmt_set_type(struct tuple *stmt, enum iproto_type type)
((struct vy_stmt *) stmt)->type = type;
}
+/** Get flags of the vinyl statement. */
+static inline uint8_t
+vy_stmt_flags(const struct tuple *stmt)
+{
+ return ((const struct vy_stmt *)stmt)->flags;
+}
+
+/** Set flags of the vinyl statement. */
+static inline void
+vy_stmt_set_flags(struct tuple *stmt, uint8_t flags)
+{
+ ((struct vy_stmt *)stmt)->flags = flags;
+}
+
/**
* Get upserts count of the vinyl statement.
* Only for UPSERT statements allocated on lsregion.
diff --git a/src/box/xrow.c b/src/box/xrow.c
index 269a6e68..6cf235e0 100644
--- a/src/box/xrow.c
+++ b/src/box/xrow.c
@@ -514,6 +514,9 @@ error:
case IPROTO_INDEX_BASE:
request->index_base = mp_decode_uint(&value);
break;
+ case IPROTO_FLAGS:
+ request->flags = mp_decode_uint(&value);
+ break;
case IPROTO_LIMIT:
request->limit = mp_decode_uint(&value);
break;
@@ -608,6 +611,11 @@ xrow_encode_dml(const struct request *request, struct iovec *iov)
pos = mp_encode_uint(pos, request->index_base);
map_size++;
}
+ if (request->flags) {
+ pos = mp_encode_uint(pos, IPROTO_FLAGS);
+ pos = mp_encode_uint(pos, request->flags);
+ map_size++;
+ }
if (request->key) {
pos = mp_encode_uint(pos, IPROTO_KEY);
memcpy(pos, request->key, key_len);
diff --git a/src/box/xrow.h b/src/box/xrow.h
index 9887382c..1bb9096c 100644
--- a/src/box/xrow.h
+++ b/src/box/xrow.h
@@ -129,6 +129,8 @@ struct request {
const char *ops_end;
/** Base field offset for UPDATE/UPSERT, e.g. 0 for C and 1 for Lua. */
int index_base;
+ /** Engine-specific statement flags. */
+ uint32_t flags;
};
/**
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 17/25] vinyl: pin last statement returned by write iterator explicitly
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (15 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 16/25] xrow: allow to store flags in DML requests Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 18/25] vinyl: teach write iterator to return overwritten tuples Vladimir Davydov
` (7 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Currently, the last statement returned by the write iterator is pinned
indirectly, via a read view. This works, because the write iterator can
only return a statement if it corresponds to a certain read view.
However, in the scope of #2129, the write iterator will also have to
keep statements for which a deferred DELETE hasn't been generated yet,
even if no read view needs it. So let's make the write iterator pin the
last returned statement explicitly, i.e. via a dedicated member of the
write_iterator struct.
Needed for #2129
---
src/box/vy_write_iterator.c | 20 +++++++++++++-------
1 file changed, 13 insertions(+), 7 deletions(-)
diff --git a/src/box/vy_write_iterator.c b/src/box/vy_write_iterator.c
index 4e758be8..06ae342b 100644
--- a/src/box/vy_write_iterator.c
+++ b/src/box/vy_write_iterator.c
@@ -192,6 +192,10 @@ struct vy_write_iterator {
*/
int stmt_i;
/**
+ * Last statement returned to the caller, pinned in memory.
+ */
+ struct tuple *last_stmt;
+ /**
* Read views of the same key sorted by LSN in descending
* order, starting from INT64_MAX.
*
@@ -398,6 +402,10 @@ vy_write_iterator_stop(struct vy_stmt_stream *vstream)
struct vy_write_src *src, *tmp;
rlist_foreach_entry_safe(src, &stream->src_list, in_src_list, tmp)
vy_write_iterator_delete_src(stream, src);
+ if (stream->last_stmt != NULL) {
+ vy_stmt_unref_if_possible(stream->last_stmt);
+ stream->last_stmt = NULL;
+ }
}
/**
@@ -527,12 +535,6 @@ static inline struct tuple *
vy_write_iterator_pop_read_view_stmt(struct vy_write_iterator *stream)
{
struct vy_read_view_stmt *rv;
- if (stream->stmt_i >= 0) {
- /* Destroy the current before getting to the next. */
- rv = &stream->read_views[stream->stmt_i];
- assert(rv->history == NULL);
- vy_read_view_stmt_destroy(rv);
- }
if (stream->rv_used_count == 0)
return NULL;
/* Find a next non-empty history element. */
@@ -544,7 +546,11 @@ vy_write_iterator_pop_read_view_stmt(struct vy_write_iterator *stream)
} while (rv->tuple == NULL);
assert(stream->rv_used_count > 0);
stream->rv_used_count--;
- return rv->tuple;
+ if (stream->last_stmt != NULL)
+ vy_stmt_unref_if_possible(stream->last_stmt);
+ stream->last_stmt = rv->tuple;
+ rv->tuple = NULL;
+ return stream->last_stmt;
}
/**
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 18/25] vinyl: teach write iterator to return overwritten tuples
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (16 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 17/25] vinyl: pin last statement returned by write iterator explicitly Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:29 ` [PATCH 19/25] vinyl: prepare write iterator heap comparator for deferred DELETEs Vladimir Davydov
` (6 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
A REPLACE/DELETE request is supposed to delete the old tuple from all
indexes. In order to generate a DELETE statement for a secondary index,
we need to look up the old tuple in the primary index, which is costly
as it implies a random disk access. In the scope of #2129 we are
planning to optimize out the lookup by deferring generation of the
DELETE statement until primary index compaction.
To do that, we need to differentiate statements for which DELETE was
deferred from those for which it was inserted when the request was
executed (as it is the case for UPDATE). So this patch introduces a per
statement flag, VY_STMT_DEFERRED_DELETE. If set for a REPLACE or DELETE
statement, it will make the write iterator to return the overwritten
statement to the caller via a callback.
Needed for #2129
---
src/box/vinyl.c | 3 +-
src/box/vy_scheduler.c | 4 +-
src/box/vy_stmt.h | 19 +++
src/box/vy_write_iterator.c | 135 ++++++++++++++++++++-
src/box/vy_write_iterator.h | 27 ++++-
test/unit/vy_iterators_helper.c | 5 +
test/unit/vy_iterators_helper.h | 12 +-
test/unit/vy_point_lookup.c | 4 +-
test/unit/vy_write_iterator.c | 232 ++++++++++++++++++++++++++++++++++---
test/unit/vy_write_iterator.result | 22 +++-
10 files changed, 430 insertions(+), 33 deletions(-)
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index f2f93736..ddaa22bb 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -3007,7 +3007,8 @@ vy_send_range(struct vy_join_ctx *ctx,
struct rlist fake_read_views;
rlist_create(&fake_read_views);
ctx->wi = vy_write_iterator_new(ctx->key_def, ctx->format,
- true, true, &fake_read_views);
+ true, true, &fake_read_views,
+ NULL, NULL);
if (ctx->wi == NULL) {
rc = -1;
goto out;
diff --git a/src/box/vy_scheduler.c b/src/box/vy_scheduler.c
index b206a605..06dbb1f8 100644
--- a/src/box/vy_scheduler.c
+++ b/src/box/vy_scheduler.c
@@ -1006,7 +1006,7 @@ vy_task_dump_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
bool is_last_level = (lsm->run_count == 0);
wi = vy_write_iterator_new(task->cmp_def, lsm->disk_format,
lsm->index_id == 0, is_last_level,
- scheduler->read_views);
+ scheduler->read_views, NULL, NULL);
if (wi == NULL)
goto err_wi;
rlist_foreach_entry(mem, &lsm->sealed, in_sealed) {
@@ -1273,7 +1273,7 @@ vy_task_compact_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
bool is_last_level = (range->compact_priority == range->slice_count);
wi = vy_write_iterator_new(task->cmp_def, lsm->disk_format,
lsm->index_id == 0, is_last_level,
- scheduler->read_views);
+ scheduler->read_views, NULL, NULL);
if (wi == NULL)
goto err_wi;
diff --git a/src/box/vy_stmt.h b/src/box/vy_stmt.h
index bcf855dd..8de8aa84 100644
--- a/src/box/vy_stmt.h
+++ b/src/box/vy_stmt.h
@@ -70,6 +70,25 @@ extern struct tuple_format_vtab vy_tuple_format_vtab;
*/
extern size_t vy_max_tuple_size;
+/** Statement flags. */
+enum {
+ /**
+ * A REPLACE/DELETE request is supposed to delete the old
+ * tuple from all indexes. In order to generate a DELETE
+ * statement for a secondary index, we need to look up the
+ * old tuple in the primary index, which is expensive as
+ * it implies a random disk access. We can optimize out the
+ * lookup by deferring generation of the DELETE statement
+ * until primary index compaction.
+ *
+ * The following flag is set for those REPLACE and DELETE
+ * statements that skipped deletion of the old tuple from
+ * secondary indexes. It makes the write iterator generate
+ * DELETE statements for them during compaction.
+ */
+ VY_STMT_DEFERRED_DELETE = 1 << 0,
+};
+
/**
* There are two groups of statements:
*
diff --git a/src/box/vy_write_iterator.c b/src/box/vy_write_iterator.c
index 06ae342b..b76c2ccb 100644
--- a/src/box/vy_write_iterator.c
+++ b/src/box/vy_write_iterator.c
@@ -177,7 +177,16 @@ struct vy_write_iterator {
* key and its tuple format is different.
*/
bool is_primary;
-
+ /** Callback for generating deferred DELETE statements. */
+ vy_deferred_delete_f deferred_delete_cb;
+ /** Context passed to @deferred_delete_cb. */
+ void *deferred_delete_ctx;
+ /**
+ * Last scanned REPLACE or DELETE statement that was
+ * inserted into the primary index without deletion
+ * of the old tuple from secondary indexes.
+ */
+ struct tuple *deferred_delete_stmt;
/** Length of the @read_views. */
int rv_count;
/**
@@ -331,9 +340,10 @@ static const struct vy_stmt_stream_iface vy_slice_stream_iface;
*/
struct vy_stmt_stream *
vy_write_iterator_new(const struct key_def *cmp_def,
- struct tuple_format *format,
- bool is_primary, bool is_last_level,
- struct rlist *read_views)
+ struct tuple_format *format, bool is_primary,
+ bool is_last_level, struct rlist *read_views,
+ vy_deferred_delete_f deferred_delete_cb,
+ void *deferred_delete_ctx)
{
/*
* One is reserved for INT64_MAX - maximal read view.
@@ -368,6 +378,8 @@ vy_write_iterator_new(const struct key_def *cmp_def,
tuple_format_ref(stream->format);
stream->is_primary = is_primary;
stream->is_last_level = is_last_level;
+ stream->deferred_delete_cb = deferred_delete_cb;
+ stream->deferred_delete_ctx = deferred_delete_ctx;
return &stream->base;
}
@@ -406,6 +418,10 @@ vy_write_iterator_stop(struct vy_stmt_stream *vstream)
vy_stmt_unref_if_possible(stream->last_stmt);
stream->last_stmt = NULL;
}
+ if (stream->deferred_delete_stmt != NULL) {
+ vy_stmt_unref_if_possible(stream->deferred_delete_stmt);
+ stream->deferred_delete_stmt = NULL;
+ }
}
/**
@@ -554,6 +570,62 @@ vy_write_iterator_pop_read_view_stmt(struct vy_write_iterator *stream)
}
/**
+ * Generate a DELETE statement for the given tuple if its
+ * deletion from secondary indexes was deferred.
+ *
+ * @param stream Write iterator.
+ * @param stmt Current statement.
+ *
+ * @retval 0 Success.
+ * @retval -1 Error.
+ */
+static int
+vy_write_iterator_deferred_delete(struct vy_write_iterator *stream,
+ struct tuple *stmt)
+{
+ /*
+ * Nothing to do if the caller isn't interested in
+ * deferred DELETEs.
+ */
+ if (stream->deferred_delete_cb == NULL)
+ return 0;
+ /*
+ * UPSERTs cannot change secondary index parts neither
+ * can they produce deferred DELETEs, so we skip them.
+ */
+ if (vy_stmt_type(stmt) == IPROTO_UPSERT) {
+ assert((vy_stmt_flags(stmt) & VY_STMT_DEFERRED_DELETE) == 0);
+ return 0;
+ }
+ /*
+ * Invoke the callback to generate a deferred DELETE
+ * in case the current tuple was overwritten.
+ */
+ if (stream->deferred_delete_stmt != NULL) {
+ if (vy_stmt_type(stmt) != IPROTO_DELETE &&
+ stream->deferred_delete_cb(stmt,
+ stream->deferred_delete_stmt,
+ stream->deferred_delete_ctx) != 0)
+ return -1;
+ vy_stmt_unref_if_possible(stream->deferred_delete_stmt);
+ stream->deferred_delete_stmt = NULL;
+ }
+ /*
+ * Remember the current statement if it is marked with
+ * VY_STMT_DEFERRED_DELETE so that we can use it to
+ * generate a DELETE for the overwritten tuple when this
+ * function is called next time.
+ */
+ if ((vy_stmt_flags(stmt) & VY_STMT_DEFERRED_DELETE) != 0) {
+ assert(vy_stmt_type(stmt) == IPROTO_DELETE ||
+ vy_stmt_type(stmt) == IPROTO_REPLACE);
+ vy_stmt_ref_if_possible(stmt);
+ stream->deferred_delete_stmt = stmt;
+ }
+ return 0;
+}
+
+/**
* Build the history of the current key.
* Apply optimizations 1, 2 and 3 (@sa vy_write_iterator.h).
* When building a history, some statements can be
@@ -578,6 +650,7 @@ vy_write_iterator_build_history(struct vy_write_iterator *stream,
*count = 0;
*is_first_insert = false;
assert(stream->stmt_i == -1);
+ assert(stream->deferred_delete_stmt == NULL);
struct heap_node *node = vy_source_heap_top(&stream->src_heap);
if (node == NULL)
return 0; /* no more data */
@@ -630,6 +703,10 @@ vy_write_iterator_build_history(struct vy_write_iterator *stream,
*is_first_insert = true;
}
+ rc = vy_write_iterator_deferred_delete(stream, src->tuple);
+ if (rc != 0)
+ break;
+
if (vy_stmt_lsn(src->tuple) > current_rv_lsn) {
/*
* Skip statements invisible to the current read
@@ -710,6 +787,17 @@ next_lsn:
break;
}
+ /*
+ * No point in keeping the last VY_STMT_DEFERRED_DELETE
+ * statement around if this is major compaction, because
+ * there's no tuple it could overwrite.
+ */
+ if (rc == 0 && stream->is_last_level &&
+ stream->deferred_delete_stmt != NULL) {
+ vy_stmt_unref_if_possible(stream->deferred_delete_stmt);
+ stream->deferred_delete_stmt = NULL;
+ }
+
vy_source_heap_delete(&stream->src_heap, &end_of_key_src.heap_node);
vy_stmt_unref_if_possible(end_of_key_src.tuple);
return rc;
@@ -794,6 +882,23 @@ vy_read_view_merge(struct vy_write_iterator *stream, struct tuple *hint,
rv->history = NULL;
result->tuple = NULL;
assert(result->next == NULL);
+ /*
+ * The write iterator generates deferred DELETEs for all
+ * VY_STMT_DEFERRED_DELETE statements, except, may be,
+ * the last seen one. Clear the flag for all other output
+ * statements so as not to generate the same DELETEs on
+ * the next compaction.
+ */
+ uint8_t flags = vy_stmt_flags(rv->tuple);
+ if (rv->tuple != stream->deferred_delete_stmt &&
+ (flags & VY_STMT_DEFERRED_DELETE) != 0) {
+ if (!vy_stmt_is_refable(rv->tuple)) {
+ rv->tuple = vy_stmt_dup(rv->tuple);
+ if (rv->tuple == NULL)
+ return -1;
+ }
+ vy_stmt_set_flags(rv->tuple, flags & ~VY_STMT_DEFERRED_DELETE);
+ }
if (hint != NULL) {
/* Not the first statement. */
return 0;
@@ -918,6 +1023,28 @@ vy_write_iterator_next(struct vy_stmt_stream *vstream,
*ret = vy_write_iterator_pop_read_view_stmt(stream);
if (*ret != NULL)
return 0;
+ /*
+ * If we didn't generate a deferred DELETE corresponding to
+ * the last seen VY_STMT_DEFERRED_DELETE statement, we must
+ * include it into the output, because there still might be
+ * an overwritten tuple in an older source.
+ */
+ if (stream->deferred_delete_stmt != NULL) {
+ if (stream->deferred_delete_stmt == stream->last_stmt) {
+ /*
+ * The statement was returned via a read view.
+ * Nothing to do.
+ */
+ vy_stmt_unref_if_possible(stream->deferred_delete_stmt);
+ stream->deferred_delete_stmt = NULL;
+ } else {
+ if (stream->last_stmt != NULL)
+ vy_stmt_unref_if_possible(stream->last_stmt);
+ *ret = stream->last_stmt = stream->deferred_delete_stmt;
+ stream->deferred_delete_stmt = NULL;
+ return 0;
+ }
+ }
/* Build the next key sequence. */
stream->stmt_i = -1;
diff --git a/src/box/vy_write_iterator.h b/src/box/vy_write_iterator.h
index ea14b07a..3430bbd2 100644
--- a/src/box/vy_write_iterator.h
+++ b/src/box/vy_write_iterator.h
@@ -220,6 +220,24 @@ struct vy_mem;
struct vy_slice;
/**
+ * Callback invoked by the write iterator for tuples that were
+ * overwritten or deleted without generating DELETE statement
+ * for secondary indexes.
+ *
+ * @param old_stmt Overwritten tuple.
+ * @param new_stmt Statement that overwrote @old_stmt.
+ * @param ctx Callback context.
+ *
+ * @retval 0 Success.
+ * @retval -1 Error.
+ *
+ * @sa VY_STMT_DEFERRED_DELETE.
+ */
+typedef int
+(*vy_deferred_delete_f)(struct tuple *old_stmt,
+ struct tuple *new_stmt, void *ctx);
+
+/**
* Open an empty write iterator. To add sources to the iterator
* use vy_write_iterator_add_* functions.
* @param cmp_def - key definition for tuple compare.
@@ -227,13 +245,16 @@ struct vy_slice;
* @param LSM tree is_primary - set if this iterator is for a primary index.
* @param is_last_level - there is no older level than the one we're writing to.
* @param read_views - Opened read views.
+ * @param deferred_delete_cb - Callback for generating deferred DELETEs.
+ * @param deferred_delete_ctx - Context passed to @deferred_delete_cb.
* @return the iterator or NULL on error (diag is set).
*/
struct vy_stmt_stream *
vy_write_iterator_new(const struct key_def *cmp_def,
- struct tuple_format *format,
- bool is_primary, bool is_last_level,
- struct rlist *read_views);
+ struct tuple_format *format, bool is_primary,
+ bool is_last_level, struct rlist *read_views,
+ vy_deferred_delete_f deferred_delete_cb,
+ void *deferred_delete_ctx);
/**
* Add a mem as a source to the iterator.
diff --git a/test/unit/vy_iterators_helper.c b/test/unit/vy_iterators_helper.c
index 642d8bf2..89603376 100644
--- a/test/unit/vy_iterators_helper.c
+++ b/test/unit/vy_iterators_helper.c
@@ -136,6 +136,7 @@ vy_new_simple_stmt(struct tuple_format *format,
}
free(buf);
vy_stmt_set_lsn(ret, templ->lsn);
+ vy_stmt_set_flags(ret, templ->flags);
if (templ->optimize_update)
vy_stmt_set_column_mask(ret, 0);
return ret;
@@ -277,6 +278,10 @@ vy_stmt_are_same(const struct tuple *actual,
tuple_unref(tmp);
return false;
}
+ if (vy_stmt_flags(actual) != expected->flags) {
+ tuple_unref(tmp);
+ return false;
+ }
bool rc = memcmp(a, b, a_len) == 0;
tuple_unref(tmp);
return rc;
diff --git a/test/unit/vy_iterators_helper.h b/test/unit/vy_iterators_helper.h
index e38ec295..24641df3 100644
--- a/test/unit/vy_iterators_helper.h
+++ b/test/unit/vy_iterators_helper.h
@@ -43,10 +43,16 @@
#define vyend 99999999
#define MAX_FIELDS_COUNT 100
#define STMT_TEMPLATE(lsn, type, ...) \
-{ { __VA_ARGS__, vyend }, IPROTO_##type, lsn, false, 0, 0 }
+{ { __VA_ARGS__, vyend }, IPROTO_##type, lsn, false, 0, 0, 0 }
#define STMT_TEMPLATE_OPTIMIZED(lsn, type, ...) \
-{ { __VA_ARGS__, vyend }, IPROTO_##type, lsn, true, 0, 0 }
+{ { __VA_ARGS__, vyend }, IPROTO_##type, lsn, true, 0, 0, 0 }
+
+#define STMT_TEMPLATE_FLAGS(lsn, type, flags, ...) \
+{ { __VA_ARGS__, vyend }, IPROTO_##type, lsn, false, flags, 0, 0 }
+
+#define STMT_TEMPLATE_DEFERRED_DELETE(lsn, type, ...) \
+STMT_TEMPLATE_FLAGS(lsn, type, VY_STMT_DEFERRED_DELETE, __VA_ARGS__)
extern struct tuple_format_vtab vy_tuple_format_vtab;
extern struct tuple_format *vy_key_format;
@@ -82,6 +88,8 @@ struct vy_stmt_template {
* to skip it in the write_iterator.
*/
bool optimize_update;
+ /** Statement flags. */
+ uint8_t flags;
/*
* In case of upsert it is possible to use only one 'add' operation.
* This is the column number of the operation.
diff --git a/test/unit/vy_point_lookup.c b/test/unit/vy_point_lookup.c
index b9b7d6ff..54a34e98 100644
--- a/test/unit/vy_point_lookup.c
+++ b/test/unit/vy_point_lookup.c
@@ -192,7 +192,7 @@ test_basic()
}
struct vy_stmt_stream *write_stream
= vy_write_iterator_new(pk->cmp_def, pk->disk_format,
- true, true, &read_views);
+ true, true, &read_views, NULL, NULL);
vy_write_iterator_new_mem(write_stream, run_mem);
struct vy_run *run = vy_run_new(&run_env, 1);
isnt(run, NULL, "vy_run_new");
@@ -225,7 +225,7 @@ test_basic()
}
write_stream
= vy_write_iterator_new(pk->cmp_def, pk->disk_format,
- true, true, &read_views);
+ true, true, &read_views, NULL, NULL);
vy_write_iterator_new_mem(write_stream, run_mem);
run = vy_run_new(&run_env, 2);
isnt(run, NULL, "vy_run_new");
diff --git a/test/unit/vy_write_iterator.c b/test/unit/vy_write_iterator.c
index 25a346af..9f37bbf4 100644
--- a/test/unit/vy_write_iterator.c
+++ b/test/unit/vy_write_iterator.c
@@ -3,6 +3,41 @@
#include "vy_write_iterator.h"
#include "vy_iterators_helper.h"
+enum { MAX_DEFERRED_COUNT = 32 };
+
+/** Argument passed to @deferred_cb. */
+struct deferred_ctx {
+ /** Format to use for making DELETEs. */
+ struct tuple_format *format;
+ /** Deferred DELETEs generated by the write iterator. */
+ struct tuple *stmt[MAX_DEFERRED_COUNT];
+ /** Number of elements in @stmt array. */
+ int count;
+};
+
+/**
+ * Callback passed to the write iterator for generating deferred
+ * DELETE statements.
+ */
+static int
+deferred_cb(struct tuple *old_stmt, struct tuple *new_stmt, void *arg)
+{
+ struct deferred_ctx *ctx = arg;
+
+ fail_if(vy_stmt_type(old_stmt) == IPROTO_DELETE);
+ fail_if(vy_stmt_type(new_stmt) != IPROTO_DELETE &&
+ vy_stmt_type(new_stmt) != IPROTO_REPLACE);
+
+ struct tuple *delete = vy_stmt_new_surrogate_delete(ctx->format,
+ old_stmt);
+ fail_if(delete == NULL);
+ vy_stmt_set_lsn(delete, vy_stmt_lsn(new_stmt));
+
+ fail_if(ctx->count >= MAX_DEFERRED_COUNT);
+ ctx->stmt[ctx->count++] = delete;
+ return 0;
+}
+
/**
* Create a mem with the specified content, iterate over it with
* write_iterator and compare actual result statements with the
@@ -12,6 +47,8 @@
* @param content_count Size of the @content.
* @param expected Expected results of the iteration.
* @param expected_count Size of the @expected.
+ * @param deferred Expected deferred DELETEs returned by the iteration.
+ * @param deferred_count Size of @deferred.
* @param vlsns Read view lsns for the write iterator.
* @param vlsns_count Size of the @vlsns.
* @param is_primary True, if the new mem belongs to the primary
@@ -23,6 +60,8 @@ compare_write_iterator_results(const struct vy_stmt_template *content,
int content_count,
const struct vy_stmt_template *expected,
int expected_count,
+ const struct vy_stmt_template *deferred,
+ int deferred_count,
const int *vlsns, int vlsns_count,
bool is_primary, bool is_last_level)
{
@@ -37,9 +76,14 @@ compare_write_iterator_results(const struct vy_stmt_template *content,
struct vy_read_view *rv_array = malloc(sizeof(*rv_array) * vlsns_count);
fail_if(rv_array == NULL);
init_read_views_list(&rv_list, rv_array, vlsns, vlsns_count);
-
- struct vy_stmt_stream *wi = vy_write_iterator_new(key_def, mem->format,
- is_primary, is_last_level, &rv_list);
+ struct deferred_ctx deferred_ctx;
+ memset(&deferred_ctx, 0, sizeof(deferred_ctx));
+ deferred_ctx.format = mem->format;
+ struct vy_stmt_stream *wi;
+ wi = vy_write_iterator_new(key_def, mem->format, is_primary,
+ is_last_level, &rv_list,
+ deferred == NULL ? NULL :
+ deferred_cb, &deferred_ctx);
fail_if(wi == NULL);
fail_if(vy_write_iterator_new_mem(wi, mem) != 0);
@@ -57,6 +101,18 @@ compare_write_iterator_results(const struct vy_stmt_template *content,
++i;
} while (ret != NULL);
ok(i == expected_count, "correct results count");
+ wi->iface->stop(wi);
+
+ for (i = 0; i < MIN(deferred_ctx.count, deferred_count); i++) {
+ ok(vy_stmt_are_same(deferred_ctx.stmt[i], &deferred[i],
+ deferred_ctx.format, NULL),
+ "deferred stmt %d is correct", i);
+ tuple_unref(deferred_ctx.stmt[i]);
+ }
+ if (deferred != NULL) {
+ ok(deferred_ctx.count == deferred_count,
+ "correct deferred stmt count");
+ }
/* Clean up */
wi->iface->close(wi);
@@ -69,7 +125,7 @@ void
test_basic(void)
{
header();
- plan(46);
+ plan(66);
{
/*
* STATEMENT: REPL REPL REPL DEL REPL REPL REPL REPL REPL REPL
@@ -98,7 +154,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, true);
}
{
@@ -132,7 +188,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, false);
}
{
@@ -160,7 +216,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, true);
}
{
@@ -180,7 +236,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, true);
}
{
@@ -204,7 +260,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, true);
}
{
@@ -227,7 +283,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, false);
}
{
@@ -255,7 +311,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, false, true);
}
{
@@ -275,7 +331,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, false, false);
}
{
@@ -302,7 +358,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, false);
}
{
@@ -330,7 +386,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, true);
}
{
@@ -355,7 +411,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, false, false);
}
{
@@ -380,7 +436,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, false);
}
{
@@ -410,7 +466,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, false);
}
{
@@ -451,7 +507,7 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
- expected, expected_count,
+ expected, expected_count, NULL, 0,
vlsns, vlsns_count, true, false);
}
{
@@ -491,7 +547,147 @@ test_basic(void)
int expected_count = sizeof(expected) / sizeof(expected[0]);
int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
compare_write_iterator_results(content, content_count,
+ expected, expected_count, NULL, 0,
+ vlsns, vlsns_count, true, false);
+}
+{
+/*
+ * STATEMENT: REPL DEL REPL REPL DEL DEL DEL REPL DEL INS DEL INS REPL
+ * LSN: 4 5 6 7 8 9 10 11 12 13 14 15 16
+ * DEFERRED DEL: + + + + + + + +
+ * READ VIEW: * * *
+ *
+ * is_last_level = true
+ *
+ * Test generation of deferred DELETEs for various combinations
+ * of input statements.
+ */
+ const struct vy_stmt_template content[] = {
+ STMT_TEMPLATE_DEFERRED_DELETE(4, REPLACE, 1, 2),
+ STMT_TEMPLATE_DEFERRED_DELETE(5, DELETE, 1),
+ STMT_TEMPLATE_DEFERRED_DELETE(6, REPLACE, 1, 3),
+ STMT_TEMPLATE(7, REPLACE, 1, 4),
+ STMT_TEMPLATE_DEFERRED_DELETE(8, DELETE, 1),
+ STMT_TEMPLATE_DEFERRED_DELETE(9, DELETE, 1),
+ STMT_TEMPLATE(10, DELETE, 1),
+ STMT_TEMPLATE_DEFERRED_DELETE(11, REPLACE, 1, 5),
+ STMT_TEMPLATE(12, DELETE, 1),
+ STMT_TEMPLATE(13, INSERT, 1, 6),
+ STMT_TEMPLATE_DEFERRED_DELETE(14, DELETE, 1),
+ STMT_TEMPLATE(15, INSERT, 1, 7),
+ STMT_TEMPLATE_DEFERRED_DELETE(16, REPLACE, 1, 8),
+ };
+ const struct vy_stmt_template expected[] = {
+ STMT_TEMPLATE(16, REPLACE, 1, 8),
+ STMT_TEMPLATE(11, REPLACE, 1, 5),
+ STMT_TEMPLATE(7, REPLACE, 1, 4),
+ };
+ const struct vy_stmt_template deferred[] = {
+ STMT_TEMPLATE(16, DELETE, 1, 7),
+ STMT_TEMPLATE(14, DELETE, 1, 6),
+ STMT_TEMPLATE(8, DELETE, 1, 4),
+ STMT_TEMPLATE(5, DELETE, 1, 2),
+ };
+ const int vlsns[] = {5, 7, 11};
+ int content_count = sizeof(content) / sizeof(content[0]);
+ int expected_count = sizeof(expected) / sizeof(expected[0]);
+ int deferred_count = sizeof(deferred) / sizeof(deferred[0]);
+ int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
+ compare_write_iterator_results(content, content_count,
+ expected, expected_count,
+ deferred, deferred_count,
+ vlsns, vlsns_count, true, true);
+}
+{
+/*
+ * STATEMENT: REPL REPL DEL
+ * LSN: 7 8 9
+ * DEFERRED DEL: +
+ *
+ * is_last_level = false
+ *
+ * Check that the oldest VY_STMT_DEFERRED_DELETE statement is
+ * preserved in case it doesn't overwrite a terminal statement
+ * and this is not a major compaction.
+ */
+ const struct vy_stmt_template content[] = {
+ STMT_TEMPLATE_DEFERRED_DELETE(7, REPLACE, 1, 1),
+ STMT_TEMPLATE(8, REPLACE, 1, 2),
+ STMT_TEMPLATE(9, DELETE, 1, 3),
+ };
+ const struct vy_stmt_template expected[] = {
+ STMT_TEMPLATE(9, DELETE, 1, 1),
+ STMT_TEMPLATE_DEFERRED_DELETE(7, REPLACE, 1, 1),
+ };
+ const struct vy_stmt_template deferred[] = {};
+ const int vlsns[] = {};
+ int content_count = sizeof(content) / sizeof(content[0]);
+ int expected_count = sizeof(expected) / sizeof(expected[0]);
+ int deferred_count = sizeof(deferred) / sizeof(deferred[0]);
+ int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
+ compare_write_iterator_results(content, content_count,
+ expected, expected_count,
+ deferred, deferred_count,
+ vlsns, vlsns_count, true, false);
+}
+{
+/*
+ * STATEMENT: REPL REPL DEL
+ * LSN: 7 8 9
+ * DEFERRED DEL: +
+ * READ VIEW: *
+ *
+ * is_last_level = false
+ *
+ * Check that the oldest VY_STMT_DEFERRED_DELETE statement is
+ * not returned twice if it is referenced by a read view.
+ */
+ const struct vy_stmt_template content[] = {
+ STMT_TEMPLATE_DEFERRED_DELETE(7, REPLACE, 1, 1),
+ STMT_TEMPLATE(8, REPLACE, 1, 2),
+ STMT_TEMPLATE(9, DELETE, 1, 3),
+ };
+ const struct vy_stmt_template expected[] = {
+ STMT_TEMPLATE(9, DELETE, 1, 1),
+ STMT_TEMPLATE_DEFERRED_DELETE(7, REPLACE, 1, 1),
+ };
+ const struct vy_stmt_template deferred[] = {};
+ const int vlsns[] = {7};
+ int content_count = sizeof(content) / sizeof(content[0]);
+ int expected_count = sizeof(expected) / sizeof(expected[0]);
+ int deferred_count = sizeof(deferred) / sizeof(deferred[0]);
+ int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
+ compare_write_iterator_results(content, content_count,
+ expected, expected_count,
+ deferred, deferred_count,
+ vlsns, vlsns_count, true, false);
+}
+{
+/*
+ * STATEMENT: REPL
+ * LSN: 7
+ * DEFERRED DEL: +
+ *
+ * is_last_level = false
+ *
+ * Check that the oldest VY_STMT_DEFERRED_DELETE statement is
+ * not returned twice if it is the only statement in the output.
+ */
+ const struct vy_stmt_template content[] = {
+ STMT_TEMPLATE_DEFERRED_DELETE(7, REPLACE, 1, 1),
+ };
+ const struct vy_stmt_template expected[] = {
+ STMT_TEMPLATE_DEFERRED_DELETE(7, REPLACE, 1, 1),
+ };
+ const struct vy_stmt_template deferred[] = {};
+ const int vlsns[] = {};
+ int content_count = sizeof(content) / sizeof(content[0]);
+ int expected_count = sizeof(expected) / sizeof(expected[0]);
+ int deferred_count = sizeof(deferred) / sizeof(deferred[0]);
+ int vlsns_count = sizeof(vlsns) / sizeof(vlsns[0]);
+ compare_write_iterator_results(content, content_count,
expected, expected_count,
+ deferred, deferred_count,
vlsns, vlsns_count, true, false);
}
fiber_gc();
diff --git a/test/unit/vy_write_iterator.result b/test/unit/vy_write_iterator.result
index 56d8cb1f..4f95aeb9 100644
--- a/test/unit/vy_write_iterator.result
+++ b/test/unit/vy_write_iterator.result
@@ -1,5 +1,5 @@
*** test_basic ***
-1..46
+1..66
ok 1 - stmt 0 is correct
ok 2 - stmt 1 is correct
ok 3 - stmt 2 is correct
@@ -46,4 +46,24 @@ ok 43 - stmt 0 is correct
ok 44 - stmt 1 is correct
ok 45 - stmt 2 is correct
ok 46 - correct results count
+ok 47 - stmt 0 is correct
+ok 48 - stmt 1 is correct
+ok 49 - stmt 2 is correct
+ok 50 - correct results count
+ok 51 - deferred stmt 0 is correct
+ok 52 - deferred stmt 1 is correct
+ok 53 - deferred stmt 2 is correct
+ok 54 - deferred stmt 3 is correct
+ok 55 - correct deferred stmt count
+ok 56 - stmt 0 is correct
+ok 57 - stmt 1 is correct
+ok 58 - correct results count
+ok 59 - correct deferred stmt count
+ok 60 - stmt 0 is correct
+ok 61 - stmt 1 is correct
+ok 62 - correct results count
+ok 63 - correct deferred stmt count
+ok 64 - stmt 0 is correct
+ok 65 - correct results count
+ok 66 - correct deferred stmt count
*** test_basic: done ***
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 19/25] vinyl: prepare write iterator heap comparator for deferred DELETEs
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (17 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 18/25] vinyl: teach write iterator to return overwritten tuples Vladimir Davydov
@ 2018-07-27 11:29 ` Vladimir Davydov
2018-07-27 11:30 ` [PATCH 20/25] vinyl: allow to skip certain statements on read Vladimir Davydov
` (5 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:29 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
In the scope of #2129, we won't delete the overwritten tuple from
secondary indexes immediately on REPLACE. Instead we will defer
generation of the DELETE statement until the primary index compaction.
However, it may happen that the overwritten tuple and the tuple that
overwrote it have the same secondary key parts, in which case the
deferred DELETE is not needed and should be discarded on secondary
index compaction. This patch makes the write iterator heap comparator
function discard such useless deferred DELETEs.
Note, this patch also removes the code that prioritises terminal
statements over UPSERTs in the write iterator, which, according to the
comment, may happen only during forced recovery. I don't see why we
should do that, even during forced recovery, neither have I managed to
find the reason in the commit history, so I dropped this code in order
not to overburden the write iterator logic with some esoteric cases.
Needed for #2129
---
src/box/vy_write_iterator.c | 13 ++++++++-----
1 file changed, 8 insertions(+), 5 deletions(-)
diff --git a/src/box/vy_write_iterator.c b/src/box/vy_write_iterator.c
index b76c2ccb..7a3537f6 100644
--- a/src/box/vy_write_iterator.c
+++ b/src/box/vy_write_iterator.c
@@ -244,12 +244,15 @@ heap_less(heap_t *heap, struct heap_node *node1, struct heap_node *node2)
if (lsn1 != lsn2)
return lsn1 > lsn2;
- /**
- * LSNs are equal. This may happen only during forced recovery.
- * Prioritize terminal (non-UPSERT) statements
+ /*
+ * LSNs are equal. This may only happen if one of the statements
+ * is a deferred DELETE and the overwritten tuple which it is
+ * supposed to purge has the same key parts as the REPLACE that
+ * overwrote it. Discard the deferred DELETE as the overwritten
+ * tuple will be (or has already been) purged by the REPLACE.
*/
- return (vy_stmt_type(src1->tuple) == IPROTO_UPSERT ? 1 : 0) <
- (vy_stmt_type(src2->tuple) == IPROTO_UPSERT ? 1 : 0);
+ return (vy_stmt_type(src1->tuple) == IPROTO_DELETE ? 1 : 0) <
+ (vy_stmt_type(src2->tuple) == IPROTO_DELETE ? 1 : 0);
}
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 20/25] vinyl: allow to skip certain statements on read
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (18 preceding siblings ...)
2018-07-27 11:29 ` [PATCH 19/25] vinyl: prepare write iterator heap comparator for deferred DELETEs Vladimir Davydov
@ 2018-07-27 11:30 ` Vladimir Davydov
2018-07-27 11:30 ` [PATCH 21/25] vinyl: add function to create surrogate deletes from raw msgpack Vladimir Davydov
` (4 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:30 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
In the scope of #2129 we will defer insertion of certain DELETE
statements into secondary indexes until primary index compaction.
However, by the time we invoke compaction, new statements might
have been inserted into the space for the same set of keys.
If that happens, insertion of a deferred DELETE will break the
invariant which the read iterator relies upon: that for any key
older sources store older statements. To avoid that, let's add
a new per statement flag, VY_STMT_SKIP_READ, and make the read
iterator ignore statements marked with it.
Needed for #2129
---
src/box/vy_mem.c | 19 ++++++++++++-------
src/box/vy_run.c | 7 ++++++-
src/box/vy_stmt.h | 10 ++++++++++
3 files changed, 28 insertions(+), 8 deletions(-)
diff --git a/src/box/vy_mem.c b/src/box/vy_mem.c
index 3313ae54..0c46b93c 100644
--- a/src/box/vy_mem.c
+++ b/src/box/vy_mem.c
@@ -324,7 +324,8 @@ vy_mem_iterator_find_lsn(struct vy_mem_iterator *itr,
assert(!vy_mem_tree_iterator_is_invalid(&itr->curr_pos));
assert(itr->curr_stmt == vy_mem_iterator_curr_stmt(itr));
const struct key_def *cmp_def = itr->mem->cmp_def;
- while (vy_stmt_lsn(itr->curr_stmt) > (**itr->read_view).vlsn) {
+ while (vy_stmt_lsn(itr->curr_stmt) > (**itr->read_view).vlsn ||
+ vy_stmt_flags(itr->curr_stmt) & VY_STMT_SKIP_READ) {
if (vy_mem_iterator_step(itr, iterator_type) != 0 ||
(iterator_type == ITER_EQ &&
vy_stmt_compare(key, itr->curr_stmt, cmp_def))) {
@@ -341,6 +342,7 @@ vy_mem_iterator_find_lsn(struct vy_mem_iterator *itr,
*vy_mem_tree_iterator_get_elem(&itr->mem->tree,
&prev_pos);
if (vy_stmt_lsn(prev_stmt) > (**itr->read_view).vlsn ||
+ vy_stmt_flags(prev_stmt) & VY_STMT_SKIP_READ ||
vy_tuple_compare(itr->curr_stmt, prev_stmt,
cmp_def) != 0)
break;
@@ -496,18 +498,21 @@ vy_mem_iterator_next_lsn(struct vy_mem_iterator *itr)
const struct key_def *cmp_def = itr->mem->cmp_def;
struct vy_mem_tree_iterator next_pos = itr->curr_pos;
+next:
vy_mem_tree_iterator_next(&itr->mem->tree, &next_pos);
if (vy_mem_tree_iterator_is_invalid(&next_pos))
return 1; /* EOF */
const struct tuple *next_stmt;
next_stmt = *vy_mem_tree_iterator_get_elem(&itr->mem->tree, &next_pos);
- if (vy_tuple_compare(itr->curr_stmt, next_stmt, cmp_def) == 0) {
- itr->curr_pos = next_pos;
- itr->curr_stmt = next_stmt;
- return 0;
- }
- return 1;
+ if (vy_tuple_compare(itr->curr_stmt, next_stmt, cmp_def) != 0)
+ return 1;
+
+ itr->curr_pos = next_pos;
+ itr->curr_stmt = next_stmt;
+ if (vy_stmt_flags(itr->curr_stmt) & VY_STMT_SKIP_READ)
+ goto next;
+ return 0;
}
/**
diff --git a/src/box/vy_run.c b/src/box/vy_run.c
index eae3e74d..f107e3a9 100644
--- a/src/box/vy_run.c
+++ b/src/box/vy_run.c
@@ -1157,7 +1157,8 @@ vy_run_iterator_find_lsn(struct vy_run_iterator *itr,
assert(itr->curr_stmt != NULL);
assert(itr->curr_pos.page_no < slice->run->info.page_count);
- while (vy_stmt_lsn(itr->curr_stmt) > (**itr->read_view).vlsn) {
+ while (vy_stmt_lsn(itr->curr_stmt) > (**itr->read_view).vlsn ||
+ vy_stmt_flags(itr->curr_stmt) & VY_STMT_SKIP_READ) {
if (vy_run_iterator_next_pos(itr, iterator_type,
&itr->curr_pos) != 0) {
vy_run_iterator_stop(itr);
@@ -1183,6 +1184,7 @@ vy_run_iterator_find_lsn(struct vy_run_iterator *itr,
&test_stmt) != 0)
return -1;
if (vy_stmt_lsn(test_stmt) > (**itr->read_view).vlsn ||
+ vy_stmt_flags(test_stmt) & VY_STMT_SKIP_READ ||
vy_tuple_compare(itr->curr_stmt, test_stmt,
cmp_def) != 0) {
tuple_unref(test_stmt);
@@ -1478,6 +1480,7 @@ vy_run_iterator_next_lsn(struct vy_run_iterator *itr, struct tuple **ret)
assert(itr->curr_pos.page_no < itr->slice->run->info.page_count);
struct vy_run_iterator_pos next_pos;
+next:
if (vy_run_iterator_next_pos(itr, ITER_GE, &next_pos) != 0) {
vy_run_iterator_stop(itr);
return 0;
@@ -1495,6 +1498,8 @@ vy_run_iterator_next_lsn(struct vy_run_iterator *itr, struct tuple **ret)
tuple_unref(itr->curr_stmt);
itr->curr_stmt = next_key;
itr->curr_pos = next_pos;
+ if (vy_stmt_flags(itr->curr_stmt) & VY_STMT_SKIP_READ)
+ goto next;
vy_stmt_counter_acct_tuple(&itr->stat->get, itr->curr_stmt);
*ret = itr->curr_stmt;
diff --git a/src/box/vy_stmt.h b/src/box/vy_stmt.h
index 8de8aa84..878a27f7 100644
--- a/src/box/vy_stmt.h
+++ b/src/box/vy_stmt.h
@@ -87,6 +87,16 @@ enum {
* DELETE statements for them during compaction.
*/
VY_STMT_DEFERRED_DELETE = 1 << 0,
+ /**
+ * Statements that have this flag set are ignored by the
+ * read iterator.
+ *
+ * We set this flag for deferred DELETE statements, because
+ * they may violate the invariant which the read relies upon:
+ * the older a source, the older statements it stores for a
+ * particular key.
+ */
+ VY_STMT_SKIP_READ = 1 << 1,
};
/**
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 21/25] vinyl: add function to create surrogate deletes from raw msgpack
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (19 preceding siblings ...)
2018-07-27 11:30 ` [PATCH 20/25] vinyl: allow to skip certain statements on read Vladimir Davydov
@ 2018-07-27 11:30 ` Vladimir Davydov
2018-07-27 11:30 ` [PATCH 22/25] vinyl: remove pointless assertion from vy_stmt_new_surrogate_delete Vladimir Davydov
` (3 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:30 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Currently, there's only vy_stmt_new_surrogate_delete(), which takes a
tuple. Let's add vy_stmt_new_surrogate_delete_raw(), which takes raw
msgpack data.
Needed for #2129
---
src/box/vy_stmt.c | 7 +++----
src/box/vy_stmt.h | 12 +++++++++++-
2 files changed, 14 insertions(+), 5 deletions(-)
diff --git a/src/box/vy_stmt.c b/src/box/vy_stmt.c
index 09daa7f4..f4c3dd18 100644
--- a/src/box/vy_stmt.c
+++ b/src/box/vy_stmt.c
@@ -388,11 +388,10 @@ vy_stmt_new_surrogate_delete_from_key(const char *key,
}
struct tuple *
-vy_stmt_new_surrogate_delete(struct tuple_format *format,
- const struct tuple *src)
+vy_stmt_new_surrogate_delete_raw(struct tuple_format *format,
+ const char *src_data, const char *src_data_end)
{
- uint32_t src_size;
- const char *src_data = tuple_data_range(src, &src_size);
+ uint32_t src_size = src_data_end - src_data;
uint32_t total_size = src_size + format->field_map_size;
/* Surrogate tuple uses less memory than the original tuple */
char *data = region_alloc(&fiber()->gc, total_size);
diff --git a/src/box/vy_stmt.h b/src/box/vy_stmt.h
index 878a27f7..273d5e84 100644
--- a/src/box/vy_stmt.h
+++ b/src/box/vy_stmt.h
@@ -496,8 +496,18 @@ vy_stmt_new_surrogate_delete_from_key(const char *key,
* @retval NULL Memory or fields format error.
*/
struct tuple *
+vy_stmt_new_surrogate_delete_raw(struct tuple_format *format,
+ const char *data, const char *data_end);
+
+/** @copydoc vy_stmt_new_surrogate_delete_raw. */
+static inline struct tuple *
vy_stmt_new_surrogate_delete(struct tuple_format *format,
- const struct tuple *tuple);
+ const struct tuple *tuple)
+{
+ uint32_t size;
+ const char *data = tuple_data_range(tuple, &size);
+ return vy_stmt_new_surrogate_delete_raw(format, data, data + size);
+}
/**
* Create the REPLACE statement from raw MessagePack data.
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 22/25] vinyl: remove pointless assertion from vy_stmt_new_surrogate_delete
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (20 preceding siblings ...)
2018-07-27 11:30 ` [PATCH 21/25] vinyl: add function to create surrogate deletes from raw msgpack Vladimir Davydov
@ 2018-07-27 11:30 ` Vladimir Davydov
2018-07-27 11:30 ` [PATCH 23/25] txn: add helper to detect transaction boundaries Vladimir Davydov
` (2 subsequent siblings)
24 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:30 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
For some reason, vy_stmt_new_surrogate_delete() checks that the source
tuple has all fields mandated by the space format (min_field_count).
This is pointless, because to generate a surrogate DELETE statement, we
don't need all tuple fields - if a field is absent it will be replaced
with NULL. We haven't stepped on this assertion, because we always
create surrogate DELETEs from full tuples. However, to implement #2129
we need to be able to create surrogate DELETEs from tuples that only
have indexed fields. So let's remove this assertion.
Needed for #2129
---
src/box/vy_stmt.c | 1 -
1 file changed, 1 deletion(-)
diff --git a/src/box/vy_stmt.c b/src/box/vy_stmt.c
index f4c3dd18..0bf53cbf 100644
--- a/src/box/vy_stmt.c
+++ b/src/box/vy_stmt.c
@@ -404,7 +404,6 @@ vy_stmt_new_surrogate_delete_raw(struct tuple_format *format,
const char *src_pos = src_data;
uint32_t src_count = mp_decode_array(&src_pos);
- assert(src_count >= format->min_field_count);
uint32_t field_count;
if (src_count < format->index_field_count) {
field_count = src_count;
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 23/25] txn: add helper to detect transaction boundaries
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (21 preceding siblings ...)
2018-07-27 11:30 ` [PATCH 22/25] vinyl: remove pointless assertion from vy_stmt_new_surrogate_delete Vladimir Davydov
@ 2018-07-27 11:30 ` Vladimir Davydov
2018-07-31 20:52 ` [tarantool-patches] " Konstantin Osipov
2018-07-27 11:30 ` [PATCH 24/25] Introduce _vinyl_deferred_delete system space Vladimir Davydov
2018-07-27 11:30 ` [PATCH 25/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
24 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:30 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
Add txn_is_first_statement() function, which returns true if this is the
first statement of the transaction. The function is supposed to be used
from on_replace trigger to detect transaction boundaries.
Needed for #2129
---
src/box/txn.c | 3 +--
src/box/txn.h | 11 +++++++++++
2 files changed, 12 insertions(+), 2 deletions(-)
diff --git a/src/box/txn.c b/src/box/txn.c
index 8947ac35..17d97d76 100644
--- a/src/box/txn.c
+++ b/src/box/txn.c
@@ -398,8 +398,7 @@ txn_abort(struct txn *txn)
int
txn_check_singlestatement(struct txn *txn, const char *where)
{
- if (!txn->is_autocommit ||
- stailq_last(&txn->stmts) != stailq_first(&txn->stmts)) {
+ if (!txn->is_autocommit || !txn_is_first_statement(txn)) {
diag_set(ClientError, ER_UNSUPPORTED,
where, "multi-statement transactions");
return -1;
diff --git a/src/box/txn.h b/src/box/txn.h
index 9a1f175a..19330b1f 100644
--- a/src/box/txn.h
+++ b/src/box/txn.h
@@ -280,6 +280,17 @@ txn_rollback_stmt();
int
txn_check_singlestatement(struct txn *txn, const char *where);
+/**
+ * Returns true if the transaction has a single statement.
+ * Supposed to be used from a space on_replace trigger to
+ * detect transaction boundaries.
+ */
+static inline bool
+txn_is_first_statement(struct txn *txn)
+{
+ return stailq_last(&txn->stmts) == stailq_first(&txn->stmts);
+}
+
/** The current statement of the transaction. */
static inline struct txn_stmt *
txn_current_stmt(struct txn *txn)
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 24/25] Introduce _vinyl_deferred_delete system space
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (22 preceding siblings ...)
2018-07-27 11:30 ` [PATCH 23/25] txn: add helper to detect transaction boundaries Vladimir Davydov
@ 2018-07-27 11:30 ` Vladimir Davydov
2018-07-31 20:54 ` Konstantin Osipov
2018-07-27 11:30 ` [PATCH 25/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
24 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:30 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
The space is a blackhole. It will be used for writing deferred DELETE
statements generated by vinyl compaction tasks to WAL so that we can
recover deferred DELETEs that hadn't been dumped to disk before the
server was restarted.
Needed for #2129
---
src/box/bootstrap.snap | Bin 1540 -> 1605 bytes
src/box/lua/space.cc | 2 ++
src/box/lua/upgrade.lua | 21 +++++++++++
src/box/schema.cc | 72 ++++++++++++++++++++++++++-----------
src/box/schema_def.h | 2 ++
src/box/vinyl.c | 15 ++++++++
test/app-tap/tarantoolctl.test.lua | 2 +-
test/box-py/bootstrap.result | 5 ++-
test/box/access_misc.result | 3 ++
test/box/access_sysview.result | 2 +-
test/box/alter.result | 4 +--
test/wal_off/alter.result | 2 +-
test/xlog/upgrade.result | 5 ++-
13 files changed, 108 insertions(+), 27 deletions(-)
diff --git a/src/box/bootstrap.snap b/src/box/bootstrap.snap
index b610828c9c9ae9a22acdd8c150c16c6838b7a273..dd0766a1af6d2aa6577eb48ea3ccc876a5989b4b 100644
GIT binary patch
delta 1600
zcmV-G2EX})48;tP7=JM>GBh+TXEZc7Ic76rV+u)bb97;DV`VxZGBq_fF)}t}EipJS
zG%YkSFfc7*IWl7{Fg7z{V=-Z4Wi&7|3RXjGZ)0mZAbWiZ3e~y`y3Ga40M2jv`c0((
z00000D77#B08kY!0GdD~4oc8gQ2_v7Jir%5=(vCY{|RX2wtw1;q>_zSo~QIMph6SH
zKt!gJkdm06?)@c64FXTjX{eWg+ICbh5T%X;t23>bPxYamS}UcL0?h!!0N4Pe0k%$3
zye}H>VkiV&9D+45*7s|ku>4-PGS}nn%SL{JbeQ`E!4P{yltsYzp#AG=p_b0O^V*BC
zF7yKgWcKvezkka6dKc!Njjm9mAsi`yt&@!H`^sCixq3_g?)>JHu}4$^*g8odMQ)sE
zL8JeLIP1|ygE$sz>m;cGs>96p!)!uqo#YWU`m)z^Uo~p$Bx%08F{Mygbz*&`n$u0H
zWOM3HrHewHN~-SOm{2GbhAI0yA3<!BaVvCAC@Vh;sDBVu`4L4*&JzH2{sgeRMU?|v
zC;6=hGc0d$$m{rdtiq!Iu<N^9Hdx-GSJG62*?vBn7}z>VS0C?eKOeo4b}K@#9$7H2
zMr9V*I?1s=7lQT3@=6+`Ac#w_uUKYUP|trldo>D!fC$VZqAv$*o#Y5!SSjoG^N>&q
z25g<AzJCvf;R?jDcj1f6zHjGc&(--w+u!=GvrYWYg^1`MeLVMus7nE!&oCb2HkfCl
zEd=h~x4f?%vTCghvdXT<5PKi^nN_57OMh+1<12EnMguD)l+L)A6F0hJVkbujhvJmL
zA)y+f5}^tqtJM^UnNrz0Nh=GIN|{im8<S0U>VIlF8X1#J2PE?$W+tX&>m-3<Ja8Rk
z90wQ~8BEN}h8GPl8C?9jC}428;9|jrYk_I0(DG6%i2{`YHK8OI(gaE+*g8p;mMH^r
z@}rNe$C}Uo?m}~g8U=AMBysGe<{<4xzwC))xLY@$`C#bkfiJik`u^a5-uHVy>pu^S
z>3<CKme3{<zVCOx{+7q{k)j|;=ne#1Cn@U}>v4B)i0VABb&{jZyHOmLHQnj)HQ`mw
zO;^G+s#U4$OsCSROPNaF4UGw9V?%>s3YBp<Ix6I#IB@KZs<WfAVX7fRgPZXpgSN!5
zb&{2Z8kH3W)nV>fkE~mzGcNVw=;#Ovb$?;&BuVr6epuGyO~=+r0;rSDxKxY%;>b%{
zfd$T>_`8VO6%U462xo9LiZxniT&F_k*LO=wQ`kDm5iOQWjBzl>g$mq(#*XKS#ps>r
zg6Cl_7!%WV@sfC5c*tizAB8&qu~pJJ<2n?!PBQ1l3mOesI?wFq<IGG?7=b6K(SKNE
zkP?^y0003{01*a9A;*da5`e%kilZ<JU>F8O2rvo{7y*F5AS56Hs09NAqYq$Ri@=2Q
zn^SU0?7$tk19zDLPcXS|7A2K651YnC&+)Pt0-b=I4#Uq63zL*AOnHplt&3F?)hC9u
zS6aPkLp`%!61t`BSfQt)M|s|%Wq(|89EFUB1<RmBh@S<D?d*lxN`*%B3Z9L=dLAyg
z(Sjx<dh8|Hdk&-+C+E2ZjTTVUF=W6GumCo|4zK_=z&_Yw!RCbY`5b_b-xEd^GUn%E
zh~%?k{yF(ibm;$ZlQ_hD@R=4s!5$i7!vqm@Anyrz23*E9XaEh8*DRgQVSjgV0N-pz
za&hZ|$u0>5?lYW3oL#x^uqy_e!tt1X^qlS+CUk?yYz4UsCFV{tM`8d#X9-Rux+_T|
zY3n`4W>nIM;)VgVnzK&QA0hB{Ee&Fnlyz_$s?9Em#!w3%4h-`|HWkXbH<*YM?}`3B
z1Qa6cFSK9@3C`BFbfb(Nn18h+gw1i0G6qMfzamFw9pn@Bzx-ybPrOYa1JkKbq~ol_
z`z$aE#&Oi`lH$x*&+!XpK<`ZAj!Y(o+?Ji|Nw0$|$=OT*WC`XGk@tiP^&`yZRFYv$
z(@gA82zVGqoiK}wd+$)U4E&OpbR!cyI%6J*-C~xLe#w()a@GEk>^V(*r&WPUO-m#O
y=K9b;Rxp^_zF*hsu14bDtvX3J(`rc2zai@m65FtDt0OJ=DvX{az(x+$5UuS!*y?2f
delta 1534
zcmV<a1p)fS41^4j7=JJ=G%zh^V>dT4F*z~{Np5p=VQyn(Iv_b=V_`LAF*YqTV`gS8
zG-hTvEnzrjFfCy?W;Qf4VP!dGGGq!?Lu_wjYdRo%F*+bOeF_TIx(m9^1&9F7A|rpb
zr2qf`001bpFZ}>e{VM<lJ=RLl7I6XqUp$Ch%_7lYJ_Lj_@PFoXP~+we(A76EO7~<&
z)F3;RA|;uhw(`4WgTRw>I&<{eU?~Ai$VOI~v629dx2JaIm@kx4N&&zCxB$xlBK}6v
zM#}nreKVfmYggvF>;bcooE<&pjzw|&ydgCd;B)Xi4C|Up=iRyONm;Mu=<J|49rmyj
z!~XSg?^aVP(0>n31mId~$~K1OP2OJJrGIzc!y|v*5Q}mxHFpG|K|%#jeQ^A;E^l>*
z3*Bp}NefgxX1+gm^KmUTgDMc1Jsb?P)oZC4!mL!?OgGcvU!7uNP_0u4V%4c@$$Tde
z<JJuBKo=+bJ1@oWHsew%rA$(CR4!bhCq$aiEJ3j}Cx2ttc)M<~yh-n@9Ru2aUZxgY
zOU+j=F>OCDy|bcHe0;hLW1g*=6kJQqvOo9n=`!@rT1v3PA=sEaGtUmrgF1V*+HwF$
z%o|ck2Ck)MDdN~U>lpNk)Rh9)Qd8p>$FU`TvVVQ}JNw3+pEy`&7|(v!cb#wKcU}Xc
zztr*E^M9!-1a3aZxQyRo-mQ)X;JxvQVS5a1E-Z#;*X8(mp!hlb(b=Wn_TxWHa?e&p
z0<NVdiD5jvlVdHR&M*6U`SOA@wvN<F<4PlXstt_|t42g+g<yeLeOP%|b(q;~x1wEl
zEj90~M>ms_NyVg8C=MtXRLX=hZO}2jlujiL*MCw|r<8%iFyl-urCG}bk_A!)lFN(Z
z*9rxd2`ZdQ%ZOz}8m(9>QmHeYBvm9q6rtA9wbTrW=uAh+mpZa8Z9M*WuhEnWw8OPP
z0Ken?`duE+M1Fc0sWuO;rRJ>TPnWxUKDEZdwbU$UUQcC(VtJFR2G>&aT`%TX-lUJ$
zzkkQk8h`!+Ak42Gb5O~I9X_GL4xhR(RGq@`<8YEmeGLBa%Ed{YaV#4wH7X6(wXrob
zRx(mCOfa|?Bo}qm;#z9jPz5?Ii>k-mKV61Bw$iw6MNm)>lUj2vHAVFK{#e#!kBV!l
ziDGG`ab1f2WXaFUg5t{}`TM8d6EBXt_J3uuRg$)9X&lOg&M*+4wU)S+njl=^i<EJZ
zX2ppwLyQ}LBXiMF<cgb*y<$jG_{C2oetkYX_VcpV`IoM<QW}RMaV<4zXt3a^V}T&3
z(OhIq3CsWhfB+}}AqGb&W<?JYfWSD4qc93!7zRWLI0_9I0RX{)ke~>l77Rj$ynhqj
zxeG%W18Ui|uoin^FOJH&6BE><l!R4e54%@5i^3tJ2=RjuhoRy}$qP!PWxP;CvBe2+
zB`AQ$m7GU4)NS@llAFhZ>1-;h#Us!zr2@yPWOP}$xk=>TS*Ej6Q8ZgBx&}sXh}Wym
z;Yvs=Xm6s&oTIkS5VXc28)t^2+kc4qR4r1#;*gdpU~^0s$pG8pm?G)gVGer7p5*j@
zrQSOro_cGm&fR}<@b8h^=iXW!%4reE=RqJFW<g443Z&4$8`^`Azzc12c=p(Su(w)T
z1)jmCmtK6UOn@l~=HxoFllhK1;>sqt`1BcS0HC@As3ba#z^Ko9Pi8avYk$Oa!!S#&
zSx3_!A8xyrpfT#3Y0zk_Hi}Y>p%$pI61F+5M3N&%2xKJAC;$CDD}<6?=$#5uoTZ5Z
zCNj}EgB|fKv=$L@ag>MEJ_76DnN*W1Z`O`MZ9ptwx|K1yxMV1V1(Ltx%(4y7obk4E
zW?=^8GIQ=kv4A0`wS7N1*>`Z!opukPx!_g@c{8|RJi<`!QS!z#<&+X#nnV8V6uo^%
zFL^|hfwSZ-+|LAZ&d5i?Zt>-$tMH_1UnT!Yb{@SussOv@WfJ50JyfLCQd9Bw_N-dh
k2=V>Mm|$S01}gnuSl5<E4okR>OyKJnJxL~`9Mur5?efFfv;Y7A
diff --git a/src/box/lua/space.cc b/src/box/lua/space.cc
index 580e0ea2..6e180ad9 100644
--- a/src/box/lua/space.cc
+++ b/src/box/lua/space.cc
@@ -554,6 +554,8 @@ box_lua_space_init(struct lua_State *L)
lua_setfield(L, -2, "VSEQUENCE_ID");
lua_pushnumber(L, BOX_SPACE_SEQUENCE_ID);
lua_setfield(L, -2, "SPACE_SEQUENCE_ID");
+ lua_pushnumber(L, BOX_VINYL_DEFERRED_DELETE_ID);
+ lua_setfield(L, -2, "VINYL_DEFERRED_DELETE_ID");
lua_pushnumber(L, BOX_SYSTEM_ID_MIN);
lua_setfield(L, -2, "SYSTEM_ID_MIN");
lua_pushnumber(L, BOX_SYSTEM_ID_MAX);
diff --git a/src/box/lua/upgrade.lua b/src/box/lua/upgrade.lua
index 0293f6ef..bdbd7d4a 100644
--- a/src/box/lua/upgrade.lua
+++ b/src/box/lua/upgrade.lua
@@ -964,6 +964,26 @@ local function upgrade_to_1_10_0()
create_vsequence_space()
end
+--------------------------------------------------------------------------------
+--- Tarantool 1.10.1
+--------------------------------------------------------------------------------
+local function create_vinyl_deferred_delete_space()
+ local _space = box.space[box.schema.SPACE_ID]
+ local _vinyl_deferred_delete = box.space[box.schema.VINYL_DEFERRED_DELETE_ID]
+
+ local format = {}
+ format[1] = {name = 'space_id', type = 'unsigned'}
+ format[2] = {name = 'lsn', type = 'unsigned'}
+ format[3] = {name = 'tuple', type = 'array'}
+
+ log.info("create space _vinyl_deferred_delete")
+ _space:insert{_vinyl_deferred_delete.id, ADMIN, '_vinyl_deferred_delete',
+ 'blackhole', 0, {group_id = 1}, format}
+end
+
+local function upgrade_to_1_10_1()
+ create_vinyl_deferred_delete_space()
+end
local function get_version()
local version = box.space._schema:get{'version'}
@@ -991,6 +1011,7 @@ local function upgrade(options)
{version = mkversion(1, 7, 6), func = upgrade_to_1_7_6, auto = false},
{version = mkversion(1, 7, 7), func = upgrade_to_1_7_7, auto = true},
{version = mkversion(1, 10, 0), func = upgrade_to_1_10_0, auto = true},
+ {version = mkversion(1, 10, 1), func = upgrade_to_1_10_1, auto = true},
}
for _, handler in ipairs(handlers) do
diff --git a/src/box/schema.cc b/src/box/schema.cc
index 433f52c0..48644bbe 100644
--- a/src/box/schema.cc
+++ b/src/box/schema.cc
@@ -37,6 +37,7 @@
#include "scoped_guard.h"
#include "version.h"
#include "user.h"
+#include "vclock.h"
#include <stdio.h>
/**
* @module Data Dictionary
@@ -191,29 +192,16 @@ space_cache_replace(struct space *space)
/** A wrapper around space_new() for data dictionary spaces. */
static void
-sc_space_new(uint32_t id, const char *name, struct key_def *key_def,
- struct trigger *replace_trigger,
- struct trigger *stmt_begin_trigger)
+do_sc_space_new(uint32_t id, const char *name, const char *engine,
+ const struct space_opts *opts, struct rlist *key_list,
+ struct trigger *replace_trigger,
+ struct trigger *stmt_begin_trigger)
{
- struct index_def *index_def = index_def_new(id, /* space id */
- 0 /* index id */,
- "primary", /* name */
- strlen("primary"),
- TREE /* index type */,
- &index_opts_default,
- key_def, NULL);
- if (index_def == NULL)
- diag_raise();
- auto index_def_guard =
- make_scoped_guard([=] { index_def_delete(index_def); });
struct space_def *def =
- space_def_new_xc(id, ADMIN, 0, name, strlen(name), "memtx",
- strlen("memtx"), &space_opts_default, NULL, 0);
+ space_def_new_xc(id, ADMIN, 0, name, strlen(name), engine,
+ strlen(engine), opts, NULL, 0);
auto def_guard = make_scoped_guard([=] { space_def_delete(def); });
- struct rlist key_list;
- rlist_create(&key_list);
- rlist_add_entry(&key_list, index_def, link);
- struct space *space = space_new_xc(def, &key_list);
+ struct space *space = space_new_xc(def, key_list);
(void) space_cache_replace(space);
if (replace_trigger)
trigger_add(&space->on_replace, replace_trigger);
@@ -234,6 +222,40 @@ sc_space_new(uint32_t id, const char *name, struct key_def *key_def,
trigger_run_xc(&on_alter_space, space);
}
+static void
+sc_space_new(uint32_t id, const char *name, struct key_def *key_def,
+ struct trigger *replace_trigger,
+ struct trigger *stmt_begin_trigger)
+{
+ struct index_def *index_def = index_def_new(id, /* space id */
+ 0 /* index id */,
+ "primary", /* name */
+ strlen("primary"),
+ TREE /* index type */,
+ &index_opts_default,
+ key_def, NULL);
+ if (index_def == NULL)
+ diag_raise();
+ auto index_def_guard =
+ make_scoped_guard([=] { index_def_delete(index_def); });
+ struct rlist key_list;
+ rlist_create(&key_list);
+ rlist_add_entry(&key_list, index_def, link);
+ do_sc_space_new(id, name, "memtx", &space_opts_default, &key_list,
+ replace_trigger, stmt_begin_trigger);
+}
+
+static void
+sc_space_new_blackhole(uint32_t id, const char *name, uint32_t group_id,
+ struct trigger *replace_trigger)
+{
+ struct space_opts opts = space_opts_default;
+ opts.group_id = group_id;
+ RLIST_HEAD(key_list);
+ do_sc_space_new(id, name, "blackhole", &opts, &key_list,
+ replace_trigger, NULL);
+}
+
uint32_t
schema_find_id(uint32_t system_space_id, uint32_t index_id,
const char *name, uint32_t len)
@@ -351,6 +373,16 @@ schema_init()
FIELD_TYPE_UNSIGNED, false, NULL, COLL_NONE);
sc_space_new(BOX_INDEX_ID, "_index", key_def,
&alter_space_on_replace_index, &on_stmt_begin_index);
+
+ /*
+ * _vinyl_deferred_delete - blackhole that is needed
+ * for writing deferred DELETE statements generated by
+ * vinyl compaction tasks to WAL.
+ */
+ extern struct trigger on_replace_vinyl_deferred_delete;
+ sc_space_new_blackhole(BOX_VINYL_DEFERRED_DELETE_ID,
+ "_vinyl_deferred_delete", GROUP_LOCAL,
+ &on_replace_vinyl_deferred_delete);
}
void
diff --git a/src/box/schema_def.h b/src/box/schema_def.h
index 2edb8d37..f6704bf8 100644
--- a/src/box/schema_def.h
+++ b/src/box/schema_def.h
@@ -102,6 +102,8 @@ enum {
BOX_TRUNCATE_ID = 330,
/** Space id of _space_sequence. */
BOX_SPACE_SEQUENCE_ID = 340,
+ /** Space if of _vinyl_deferred_delete. */
+ BOX_VINYL_DEFERRED_DELETE_ID = 450,
/** End of the reserved range of system spaces. */
BOX_SYSTEM_ID_MAX = 511,
BOX_ID_NIL = 2147483647
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index ddaa22bb..340da35b 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -4266,6 +4266,21 @@ vinyl_space_build_index(struct space *src_space, struct index *new_index,
/* }}} Index build */
+/* {{{ Deferred DELETE handling */
+
+static void
+vy_deferred_delete_on_replace(struct trigger *trigger, void *event)
+{
+ (void)trigger;
+ (void)event;
+}
+
+struct trigger on_replace_vinyl_deferred_delete = {
+ RLIST_LINK_INITIALIZER, vy_deferred_delete_on_replace, NULL, NULL
+};
+
+/* }}} Deferred DELETE handling */
+
static const struct engine_vtab vinyl_engine_vtab = {
/* .shutdown = */ vinyl_engine_shutdown,
/* .create_space = */ vinyl_engine_create_space,
diff --git a/test/app-tap/tarantoolctl.test.lua b/test/app-tap/tarantoolctl.test.lua
index 6946c831..607dbf77 100755
--- a/test/app-tap/tarantoolctl.test.lua
+++ b/test/app-tap/tarantoolctl.test.lua
@@ -338,7 +338,7 @@ do
check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json --show-system --replica 1", "\n", 3)
check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json --show-system --replica 1 --replica 2", "\n", 3)
check_ctlcat_xlog(test_i, dir, "--from=3 --to=6 --format=json --show-system --replica 2", "\n", 0)
- check_ctlcat_snap(test_i, dir, "--space=280", "---\n", 18)
+ check_ctlcat_snap(test_i, dir, "--space=280", "---\n", 19)
check_ctlcat_snap(test_i, dir, "--space=288", "---\n", 43)
end)
end)
diff --git a/test/box-py/bootstrap.result b/test/box-py/bootstrap.result
index 16c2027c..a83f5d06 100644
--- a/test/box-py/bootstrap.result
+++ b/test/box-py/bootstrap.result
@@ -5,7 +5,7 @@ box.space._schema:select{}
---
- - ['cluster', '<cluster uuid>']
- ['max_id', 511]
- - ['version', 1, 10, 0]
+ - ['version', 1, 10, 1]
...
box.space._cluster:select{}
---
@@ -68,6 +68,9 @@ box.space._space:select{}
'type': 'unsigned'}]]
- [340, 1, '_space_sequence', 'memtx', 0, {}, [{'name': 'id', 'type': 'unsigned'},
{'name': 'sequence_id', 'type': 'unsigned'}, {'name': 'is_generated', 'type': 'boolean'}]]
+ - [450, 1, '_vinyl_deferred_delete', 'blackhole', 0, {'group_id': 1}, [{'name': 'space_id',
+ 'type': 'unsigned'}, {'name': 'lsn', 'type': 'unsigned'}, {'name': 'tuple',
+ 'type': 'array'}]]
...
box.space._index:select{}
---
diff --git a/test/box/access_misc.result b/test/box/access_misc.result
index 2d87fa2d..b2850504 100644
--- a/test/box/access_misc.result
+++ b/test/box/access_misc.result
@@ -807,6 +807,9 @@ box.space._space:select()
'type': 'unsigned'}]]
- [340, 1, '_space_sequence', 'memtx', 0, {}, [{'name': 'id', 'type': 'unsigned'},
{'name': 'sequence_id', 'type': 'unsigned'}, {'name': 'is_generated', 'type': 'boolean'}]]
+ - [450, 1, '_vinyl_deferred_delete', 'blackhole', 0, {'group_id': 1}, [{'name': 'space_id',
+ 'type': 'unsigned'}, {'name': 'lsn', 'type': 'unsigned'}, {'name': 'tuple',
+ 'type': 'array'}]]
...
box.space._func:select()
---
diff --git a/test/box/access_sysview.result b/test/box/access_sysview.result
index 20efd2bb..bc5e069f 100644
--- a/test/box/access_sysview.result
+++ b/test/box/access_sysview.result
@@ -230,7 +230,7 @@ box.session.su('guest')
...
#box.space._vspace:select{}
---
-- 19
+- 20
...
#box.space._vindex:select{}
---
diff --git a/test/box/alter.result b/test/box/alter.result
index eb7014d8..5eb8229e 100644
--- a/test/box/alter.result
+++ b/test/box/alter.result
@@ -107,7 +107,7 @@ space = box.space[t[1]]
...
space.id
---
-- 341
+- 451
...
space.field_count
---
@@ -152,7 +152,7 @@ space_deleted
...
space:replace{0}
---
-- error: Space '341' does not exist
+- error: Space '451' does not exist
...
_index:insert{_space.id, 0, 'primary', 'tree', 1, 1, 0, 'unsigned'}
---
diff --git a/test/wal_off/alter.result b/test/wal_off/alter.result
index afac1e55..f4703395 100644
--- a/test/wal_off/alter.result
+++ b/test/wal_off/alter.result
@@ -28,7 +28,7 @@ end;
...
#spaces;
---
-- 65515
+- 65513
...
-- cleanup
for k, v in pairs(spaces) do
diff --git a/test/xlog/upgrade.result b/test/xlog/upgrade.result
index f02996bb..9862eb43 100644
--- a/test/xlog/upgrade.result
+++ b/test/xlog/upgrade.result
@@ -36,7 +36,7 @@ box.space._schema:select()
---
- - ['cluster', '<server_uuid>']
- ['max_id', 513]
- - ['version', 1, 10, 0]
+ - ['version', 1, 10, 1]
...
box.space._space:select()
---
@@ -95,6 +95,9 @@ box.space._space:select()
'type': 'unsigned'}]]
- [340, 1, '_space_sequence', 'memtx', 0, {}, [{'name': 'id', 'type': 'unsigned'},
{'name': 'sequence_id', 'type': 'unsigned'}, {'name': 'is_generated', 'type': 'boolean'}]]
+ - [450, 1, '_vinyl_deferred_delete', 'blackhole', 0, {'group_id': 1}, [{'name': 'space_id',
+ 'type': 'unsigned'}, {'name': 'lsn', 'type': 'unsigned'}, {'name': 'tuple',
+ 'type': 'array'}]]
- [512, 1, 'distro', 'memtx', 0, {}, [{'name': 'os', 'type': 'str'}, {'name': 'dist',
'type': 'str'}, {'name': 'version', 'type': 'num'}, {'name': 'time', 'type': 'num'}]]
- [513, 1, 'temporary', 'memtx', 0, {'temporary': true}, []]
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [PATCH 25/25] vinyl: eliminate disk read on REPLACE/DELETE
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
` (23 preceding siblings ...)
2018-07-27 11:30 ` [PATCH 24/25] Introduce _vinyl_deferred_delete system space Vladimir Davydov
@ 2018-07-27 11:30 ` Vladimir Davydov
2018-07-31 20:55 ` Konstantin Osipov
24 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-07-27 11:30 UTC (permalink / raw)
To: kostja; +Cc: tarantool-patches
When executing a REPLACE or DELETE request for a vinyl space, we need to
delete the old tuple from secondary indexes if any, e.g. if there's a
space with the primary index over field 1 and a secondary index over
field 2 and there's REPLACE{1, 10} in the space, then REPLACE{1, 20} has
to generate DELETE{10, 1} in order to overwrite REPLACE{10, 1} before
inserting REPLACE{20, 1} into the secondary index. Currently, we
generate DELETEs for secondary indexes immediately on request execution,
which makes REPLACE/DELETE operations disk-bound in case the space has
secondary indexes, because in order to delete the old tuple we have to
look it up in the primary index first.
Actually, we can postpone DELETE generation and still yield correct
results. All we have to do is compare each tuple read from a secondary
index with the full tuple corresponding to it in the primary index: if
they match, then the tuple is OK to return to the user; if the don't,
then the tuple was overwritten in the primary index and we have to skip
it. This doesn't introduce any overhead, because we have to look up full
tuples in the primary index while reading a secondary index anyways.
For instance, consider the example given in the previous paragraph: if
we don't insert DELETE{10, 1} into the secondary index, then we will
encounter REPLACE{10, 1} when reading it, but the tuple corresponding to
it in the primary index is REPLACE{1, 20} != REPLACE{10, 1} so we skip
it. This is the first thing that this patch does.
However, skipping garbage tuples isn't enough. We have to purge them
sooner or later, otherwise we risk iterating over thousands of stale
tuples before encountering a fresh one, which would adversely affect
latency of SELECT requests over a secondary index. So we mark each and
every REPLACE and DELETE statement that was inserted into the primary
index without generating DELETEs for secondary index with a special per
statement flag VY_STMT_DEFERRED_DELETE and generate DELETEs for these
statements when the time comes.
The time comes when the primary index finally gets compacted. When
writing a compacted run, we iterate over all tuples in the order set by
the primary key from newer to older tuples, so each statement marked
with VY_STMT_DEFERRED_DELETE will be followed by the tuple it overwrote,
provided there's enough runs compacted. We take these tuples and send
them to the tx thread over cbus (compaction is done in a worker thread,
remember), where deferred DELETEs are generated and inserted into
secondary indexes. Well, it isn't that simple actually, but you should
have got the basic idea by now.
The first problem here is by the time we generate a deferred DELETE,
newer statements for the same key could have been inserted into the
index and dumped to disk, while the read iterator assumes that the newer
the source the newer statements it stores for the same key. In order not
to break the read iterator assumptions by inserting deferred DELETEs, we
mark them with another special per-statement flag, VY_STMT_SKIP_READ,
which renders them invisible to the read iterator. The flag doesn't
affect the write iterator though so deferred DELETEs will purge garbage
statements when the secondary index eventually gets compacted.
The second problem concerns the recovery procedure. Since we write
deferred DELETEs to the in-memory level, we need to recover them after
restart somehow in case they didn't get dumped. To do that, we write
them to WAL (along with LSN and space id) with the aid of a special
system blackhole space, _vinyl_deferred_delete. The insertion of
deferred DELETEs into in-memory trees is actually done by on_replace
trigger installed on the space so deferred DELETEs are generated and
recovered by the same code. In order not to recover statements that have
been dumped, we account LSNs of WAL rows that generates deferred DELETEs
to vy_lsm::dump_lsn and filter dumped statements with vy_is_committed(),
just like normal statements.
Finally, we may run out of memory while generating deferred DELETEs.
This is manageable if happens during compaction - we simply throttle the
compaction task until the memory level is dumped. However, we can't do
that while generating deferred DELETEs during index dump. Solution:
don't generate deferred DELETEs during dump. The thing is we can
generate a deferred DELETE during dump only if the overwritten tuple is
stored in memory, but if it is, the lookup is nearly free and so we can
generate a DELETE when the transaction gets committed. So we introduce a
special version of point lookup, vy_point_lookup_mem(), which look ups a
tuple by the full key in cache and in memory. When a transaction is
committed, we use this function to generate DELETEs.
This should outline the pivotal points of the algorithm. More details,
as usual, in the code.
Closes #2129
---
src/box/vinyl.c | 225 ++++++++++--
src/box/vy_lsm.h | 5 +
src/box/vy_mem.c | 1 +
src/box/vy_mem.h | 5 +
src/box/vy_point_lookup.c | 32 ++
src/box/vy_point_lookup.h | 18 +
src/box/vy_scheduler.c | 322 ++++++++++++++++-
src/box/vy_tx.c | 133 +++++++
test/unit/vy_point_lookup.c | 2 +
test/vinyl/deferred_delete.result | 677 ++++++++++++++++++++++++++++++++++++
test/vinyl/deferred_delete.test.lua | 261 ++++++++++++++
test/vinyl/info.result | 18 +-
test/vinyl/info.test.lua | 9 +-
test/vinyl/layout.result | 46 ++-
test/vinyl/quota.result | 2 +-
test/vinyl/tx_gap_lock.result | 16 +-
test/vinyl/tx_gap_lock.test.lua | 10 +-
test/vinyl/write_iterator.result | 5 +
test/vinyl/write_iterator.test.lua | 3 +
19 files changed, 1716 insertions(+), 74 deletions(-)
create mode 100644 test/vinyl/deferred_delete.result
create mode 100644 test/vinyl/deferred_delete.test.lua
diff --git a/src/box/vinyl.c b/src/box/vinyl.c
index 340da35b..a6f2aca4 100644
--- a/src/box/vinyl.c
+++ b/src/box/vinyl.c
@@ -65,6 +65,7 @@
#include "engine.h"
#include "space.h"
#include "index.h"
+#include "schema.h"
#include "xstream.h"
#include "info.h"
#include "column_mask.h"
@@ -1270,25 +1271,43 @@ vy_get_by_secondary_tuple(struct vy_lsm *lsm, struct vy_tx *tx,
struct tuple *tuple, struct tuple **result)
{
assert(lsm->index_id > 0);
- /*
- * No need in vy_tx_track() as the tuple must already be
- * tracked in the secondary index LSM tree.
- */
+
if (vy_point_lookup(lsm->pk, tx, rv, tuple, result) != 0)
return -1;
- if (*result == NULL) {
+ if (*result == NULL ||
+ vy_tuple_compare(*result, tuple, lsm->key_def) != 0) {
+ /*
+ * If a tuple read from a secondary index doesn't
+ * match the tuple corresponding to it in the
+ * primary index, it must have been overwritten or
+ * deleted, but the DELETE statement hasn't been
+ * propagated to the secondary index yet. In this
+ * case silently skip this tuple.
+ */
+ if (*result != NULL) {
+ tuple_unref(*result);
+ *result = NULL;
+ }
/*
- * All indexes of a space must be consistent, i.e.
- * if a tuple is present in one index, it must be
- * present in all other indexes as well, so we can
- * get here only if there's a bug somewhere in vinyl.
- * Don't abort as core dump won't really help us in
- * this case. Just warn the user and proceed to the
- * next tuple.
+ * Invalidate the cache entry so that we won't read
+ * the overwritten tuple again from the cache.
*/
- say_warn("%s: key %s missing in primary index",
- vy_lsm_name(lsm), vy_stmt_str(tuple));
+ vy_cache_on_write(&lsm->cache, tuple, NULL);
+ return 0;
+ }
+
+ /*
+ * Even though the tuple is tracked in the secondary index
+ * read set, we still must track the full tuple read from
+ * the primary index, otherwise the transaction won't be
+ * aborted if this tuple is overwritten or deleted, because
+ * the DELETE statement is not written to secondary indexes
+ * immediately.
+ */
+ if (tx != NULL && vy_tx_track_point(tx, lsm->pk, *result) != 0) {
+ tuple_unref(*result);
+ return -1;
}
if ((*rv)->vlsn == INT64_MAX)
@@ -1601,7 +1620,6 @@ vy_delete(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
struct vy_lsm *lsm = vy_lsm_find_unique(space, request->index_id);
if (lsm == NULL)
return -1;
- bool has_secondary = space->index_count > 1;
const char *key = request->key;
uint32_t part_count = mp_decode_array(&key);
if (vy_unique_key_validate(lsm, key, part_count))
@@ -1611,12 +1629,9 @@ vy_delete(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
* before deletion.
* - if the space has on_replace triggers and need to pass
* to them the old tuple.
- *
- * - if the space has one or more secondary indexes, then
- * we need to extract secondary keys from the old tuple
- * and pass them to indexes for deletion.
+ * - if deletion is done by a secondary index.
*/
- if (has_secondary || !rlist_empty(&space->on_replace)) {
+ if (lsm->index_id > 0 || !rlist_empty(&space->on_replace)) {
if (vy_get_by_raw_key(lsm, tx, vy_tx_read_view(tx),
key, part_count, &stmt->old_tuple) != 0)
return -1;
@@ -1625,8 +1640,7 @@ vy_delete(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
}
int rc = 0;
struct tuple *delete;
- if (has_secondary) {
- assert(stmt->old_tuple != NULL);
+ if (stmt->old_tuple != NULL) {
delete = vy_stmt_new_surrogate_delete(pk->mem_format,
stmt->old_tuple);
if (delete == NULL)
@@ -1639,12 +1653,14 @@ vy_delete(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
if (rc != 0)
break;
}
- } else { /* Primary is the single index in the space. */
+ } else {
assert(lsm->index_id == 0);
delete = vy_stmt_new_surrogate_delete_from_key(request->key,
pk->key_def, pk->mem_format);
if (delete == NULL)
return -1;
+ if (space->index_count > 1)
+ vy_stmt_set_flags(delete, VY_STMT_DEFERRED_DELETE);
rc = vy_tx_set(tx, pk, delete);
}
tuple_unref(delete);
@@ -2163,11 +2179,9 @@ vy_replace(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
/*
* Get the overwritten tuple from the primary index if
* the space has on_replace triggers, in which case we
- * need to pass the old tuple to trigger callbacks, or
- * if the space has secondary indexes and so we need
- * the old tuple to delete it from them.
+ * need to pass the old tuple to trigger callbacks.
*/
- if (space->index_count > 1 || !rlist_empty(&space->on_replace)) {
+ if (!rlist_empty(&space->on_replace)) {
if (vy_get(pk, tx, vy_tx_read_view(tx),
stmt->new_tuple, &stmt->old_tuple) != 0)
return -1;
@@ -2178,6 +2192,8 @@ vy_replace(struct vy_env *env, struct vy_tx *tx, struct txn_stmt *stmt,
*/
vy_stmt_set_type(stmt->new_tuple, IPROTO_INSERT);
}
+ } else if (space->index_count > 1) {
+ vy_stmt_set_flags(stmt->new_tuple, VY_STMT_DEFERRED_DELETE);
}
/*
* Replace in the primary index without explicit deletion
@@ -4268,11 +4284,164 @@ vinyl_space_build_index(struct space *src_space, struct index *new_index,
/* {{{ Deferred DELETE handling */
+/**
+ * Callback invoked after a deferred DELETE statement has been
+ * committed to _vinyl_deferred_delete system space.
+ */
+static void
+vy_deferred_delete_on_commit(struct trigger *trigger, void *event)
+{
+ struct txn *txn = event;
+ struct vy_mem *mem = trigger->data;
+ assert(mem->max_deferred_delete_wal_lsn <= txn->signature);
+ mem->max_deferred_delete_wal_lsn = txn->signature;
+ vy_mem_unpin(mem);
+}
+
+/**
+ * Callback invoked when a deferred DELETE statement is written
+ * to _vinyl_deferred_delete system space. It extracts the
+ * deleted tuple, its LSN, and the target space id from the
+ * system space row, then generates a deferred DELETE statement
+ * and inserts it into secondary indexes of the target space.
+ *
+ * Note, this callback is also invoked during local WAL recovery
+ * to restore deferred DELETE statements that haven't been dumped
+ * to disk. To skip deferred DELETEs that have been dumped, we
+ * use the same technique we employ for normal WAL statements,
+ * i.e. we filter them by LSN, see vy_is_committed_one(). To do
+ * that, we need to account the LSN of a WAL row that generated
+ * a deferred DELETE to vy_lsm::dump_lsn, so we install an
+ * on_commit trigger that propagates the LSN of the WAL row to
+ * vy_mem::max_deferred_delete_wal_lsn, which in turn will
+ * contribute to vy_lsm::dump_lsn when the in-memory tree is
+ * dumped, see vy_task_dump_new().
+ *
+ * This implies that we don't yield between statements of the
+ * same transaction, because if we did, two deferred DELETEs with
+ * the same WAL LSN could land in different in-memory trees: if
+ * one of the trees got dumped while the other didn't, we would
+ * mistakenly skip both statements on recovery.
+ */
static void
vy_deferred_delete_on_replace(struct trigger *trigger, void *event)
{
(void)trigger;
- (void)event;
+
+ struct txn *txn = event;
+ struct txn_stmt *stmt = txn_current_stmt(txn);
+ bool is_first_statement = txn_is_first_statement(txn);
+
+ if (stmt->new_tuple == NULL)
+ return;
+ /*
+ * Extract space id, LSN of the deferred DELETE statement,
+ * and the deleted tuple from the system space row.
+ */
+ uint32_t space_id;
+ if (tuple_field_u32(stmt->new_tuple, 0, &space_id) != 0)
+ diag_raise();
+ int64_t lsn;
+ if (tuple_field_i64(stmt->new_tuple, 1, &lsn) != 0)
+ diag_raise();
+ const char *delete_data = tuple_field(stmt->new_tuple, 2);
+ if (delete_data == NULL) {
+ diag_set(ClientError, ER_NO_SUCH_FIELD, 2);
+ diag_raise();
+ }
+ const char *delete_data_end = delete_data;
+ mp_next(&delete_data_end);
+
+ /* Look up the space. */
+ struct space *space = space_cache_find(space_id);
+ if (space == NULL)
+ diag_raise();
+ if (space->index_count <= 1)
+ return;
+ /*
+ * Wait for memory quota if necessary before starting to
+ * process the batch (we can't yield between statements).
+ */
+ struct vy_env *env = vy_env(space->engine);
+ if (is_first_statement)
+ vy_quota_wait(&env->quota);
+
+ /* Create the deferred DELETE statement. */
+ struct vy_lsm *pk = vy_lsm(space->index[0]);
+ struct tuple *delete = vy_stmt_new_surrogate_delete_raw(pk->mem_format,
+ delete_data, delete_data_end);
+ if (delete == NULL)
+ diag_raise();
+ vy_stmt_set_lsn(delete, lsn);
+ /*
+ * A deferred DELETE may be generated after new statements
+ * were committed for the deleted key while the read iterator
+ * assumes that newer sources always store newer statements.
+ * Mark deferred DELETEs with the VY_STMT_SKIP_READ flag so
+ * as not to break the read iterator assumptions.
+ */
+ vy_stmt_set_flags(delete, VY_STMT_SKIP_READ);
+
+ /* Insert the deferred DELETE into secondary indexes. */
+ int rc = 0;
+ size_t mem_used_before = lsregion_used(&env->mem_env.allocator);
+ const struct tuple *region_stmt = NULL;
+ for (uint32_t i = 1; i < space->index_count; i++) {
+ struct vy_lsm *lsm = vy_lsm(space->index[i]);
+ if (vy_is_committed_one(env, lsm))
+ continue;
+ /*
+ * As usual, rotate the active in-memory index if
+ * schema was changed or dump was triggered. Do it
+ * only if processing the first statement, because
+ * dump may be triggered by one of the statements
+ * of this transaction (see vy_quota_force_use()
+ * below), in which case we must not do rotation
+ * as we want all statements to land in the same
+ * in-memory index. This is safe, as long as we
+ * don't yield between statements.
+ */
+ struct vy_mem *mem = lsm->mem;
+ if (is_first_statement &&
+ (mem->space_cache_version != space_cache_version ||
+ mem->generation != *lsm->env->p_generation)) {
+ rc = vy_lsm_rotate_mem(lsm);
+ if (rc != 0)
+ break;
+ mem = lsm->mem;
+ }
+ rc = vy_lsm_set(lsm, mem, delete, ®ion_stmt);
+ if (rc != 0)
+ break;
+ vy_lsm_commit_stmt(lsm, mem, region_stmt);
+
+ if (!is_first_statement)
+ continue;
+ /*
+ * If this is the first statement of this
+ * transaction, install on_commit trigger
+ * which will propagate the WAL row LSN to
+ * the LSM tree.
+ */
+ struct trigger *on_commit = region_alloc(&fiber()->gc,
+ sizeof(*on_commit));
+ if (on_commit == NULL) {
+ diag_set(OutOfMemory, sizeof(*on_commit),
+ "region", "struct trigger");
+ rc = -1;
+ break;
+ }
+ vy_mem_pin(mem);
+ trigger_create(on_commit, vy_deferred_delete_on_commit, mem, NULL);
+ txn_on_commit(txn, on_commit);
+ }
+ size_t mem_used_after = lsregion_used(&env->mem_env.allocator);
+ assert(mem_used_after >= mem_used_before);
+ vy_quota_force_use(&env->quota, mem_used_after - mem_used_before);
+
+ tuple_unref(delete);
+ if (rc != 0)
+ diag_raise();
}
struct trigger on_replace_vinyl_deferred_delete = {
diff --git a/src/box/vy_lsm.h b/src/box/vy_lsm.h
index f0b7ec9c..d2aa0c43 100644
--- a/src/box/vy_lsm.h
+++ b/src/box/vy_lsm.h
@@ -50,6 +50,7 @@ extern "C" {
#endif /* defined(__cplusplus) */
struct histogram;
+struct index;
struct tuple;
struct tuple_format;
struct vy_lsm;
@@ -292,6 +293,10 @@ struct vy_lsm {
vy_lsm_read_set_t read_set;
};
+/** Extract vy_lsm from an index object. */
+struct vy_lsm *
+vy_lsm(struct index *index);
+
/** Return LSM tree name. Used for logging. */
const char *
vy_lsm_name(struct vy_lsm *lsm);
diff --git a/src/box/vy_mem.c b/src/box/vy_mem.c
index 0c46b93c..3926d9f9 100644
--- a/src/box/vy_mem.c
+++ b/src/box/vy_mem.c
@@ -110,6 +110,7 @@ vy_mem_new(struct vy_mem_env *env, int64_t generation,
index->env = env;
index->min_lsn = INT64_MAX;
index->max_lsn = -1;
+ index->max_deferred_delete_wal_lsn = -1;
index->cmp_def = cmp_def;
index->generation = generation;
index->space_cache_version = space_cache_version;
diff --git a/src/box/vy_mem.h b/src/box/vy_mem.h
index 52caa316..1742bcd5 100644
--- a/src/box/vy_mem.h
+++ b/src/box/vy_mem.h
@@ -170,6 +170,11 @@ struct vy_mem {
int64_t min_lsn;
int64_t max_lsn;
/**
+ * Max LSN of a WAL statement that generated a deferred
+ * DELETE inserted into this tree.
+ */
+ int64_t max_deferred_delete_wal_lsn;
+ /**
* Key definition for this index, extended with primary
* key parts.
*/
diff --git a/src/box/vy_point_lookup.c b/src/box/vy_point_lookup.c
index 5e43340b..7b704b84 100644
--- a/src/box/vy_point_lookup.c
+++ b/src/box/vy_point_lookup.c
@@ -293,3 +293,35 @@ done:
}
return 0;
}
+
+int
+vy_point_lookup_mem(struct vy_lsm *lsm, const struct vy_read_view **rv,
+ struct tuple *key, struct tuple **ret)
+{
+ assert(tuple_field_count(key) >= lsm->cmp_def->part_count);
+
+ int rc;
+ struct vy_history history;
+ vy_history_create(&history, &lsm->env->history_node_pool);
+
+ rc = vy_point_lookup_scan_cache(lsm, rv, key, &history);
+ if (rc != 0 || vy_history_is_terminal(&history))
+ goto done;
+
+ rc = vy_point_lookup_scan_mems(lsm, rv, key, &history);
+ if (rc != 0 || vy_history_is_terminal(&history))
+ goto done;
+
+ *ret = NULL;
+ goto out;
+done:
+ if (rc == 0) {
+ int upserts_applied;
+ rc = vy_history_apply(&history, lsm->cmp_def, lsm->mem_format,
+ true, &upserts_applied, ret);
+ lsm->stat.upsert.applied += upserts_applied;
+ }
+out:
+ vy_history_cleanup(&history);
+ return rc;
+}
diff --git a/src/box/vy_point_lookup.h b/src/box/vy_point_lookup.h
index 3b7c5a04..6d77ce9c 100644
--- a/src/box/vy_point_lookup.h
+++ b/src/box/vy_point_lookup.h
@@ -71,6 +71,24 @@ vy_point_lookup(struct vy_lsm *lsm, struct vy_tx *tx,
const struct vy_read_view **rv,
struct tuple *key, struct tuple **ret);
+/**
+ * Look up a tuple by key in memory.
+ *
+ * This function works just like vy_point_lookup() except:
+ *
+ * - It only scans in-memory level and cache and hence doesn't yield.
+ * - It doesn't turn DELETE into NULL so it returns NULL if and only
+ * if no terminal statement matching the key is present in memory
+ * (there still may be statements stored on disk though).
+ * - It doesn't account the lookup to LSM tree stats (as it never
+ * descends to lower levels).
+ *
+ * The function returns 0 on success, -1 on memory allocation error.
+ */
+int
+vy_point_lookup_mem(struct vy_lsm *lsm, const struct vy_read_view **rv,
+ struct tuple *key, struct tuple **ret);
+
#if defined(__cplusplus)
} /* extern "C" */
#endif /* defined(__cplusplus) */
diff --git a/src/box/vy_scheduler.c b/src/box/vy_scheduler.c
index 06dbb1f8..69f67924 100644
--- a/src/box/vy_scheduler.c
+++ b/src/box/vy_scheduler.c
@@ -49,6 +49,10 @@
#include "cbus.h"
#include "salad/stailq.h"
#include "say.h"
+#include "txn.h"
+#include "space.h"
+#include "schema.h"
+#include "xrow.h"
#include "vy_lsm.h"
#include "vy_log.h"
#include "vy_mem.h"
@@ -65,6 +69,8 @@ static int vy_worker_f(va_list);
static int vy_scheduler_f(va_list);
static void vy_task_execute_f(struct cmsg *);
static void vy_task_complete_f(struct cmsg *);
+static void vy_deferred_delete_batch_process_f(struct cmsg *);
+static void vy_deferred_delete_batch_free_f(struct cmsg *);
static const struct cmsg_hop vy_task_execute_route[] = {
{ vy_task_execute_f, NULL },
@@ -83,10 +89,42 @@ struct vy_worker {
struct cpipe tx_pipe;
/** Link in vy_scheduler::idle_workers. */
struct stailq_entry in_idle;
+ /** Route for sending deferred DELETEs back to tx. */
+ struct cmsg_hop deferred_delete_route[2];
};
struct vy_task;
+/** Max number of statements in a batch of deferred DELETEs. */
+enum { VY_DEFERRED_DELETE_BATCH_MAX = 100 };
+
+/** Deferred DELETE statement. */
+struct vy_deferred_delete_stmt {
+ /** Overwritten tuple. */
+ struct tuple *old_stmt;
+ /** Statement that overwrote @old_stmt. */
+ struct tuple *new_stmt;
+};
+
+/**
+ * Batch of deferred DELETE statements generated during
+ * a primary index compaction.
+ */
+struct vy_deferred_delete_batch {
+ /** CBus messages for sending the batch to tx. */
+ struct cmsg cmsg;
+ /** Task that generated this batch. */
+ struct vy_task *task;
+ /** Set if the tx thread failed to process the batch. */
+ bool is_failed;
+ /** In case of failure the error is stored here. */
+ struct diag diag;
+ /** Number of elements actually stored in @stmt array. */
+ int count;
+ /** Array of deferred DELETE statements. */
+ struct vy_deferred_delete_stmt stmt[VY_DEFERRED_DELETE_BATCH_MAX];
+};
+
struct vy_task_ops {
/**
* This function is called from a worker. It is supposed to do work
@@ -159,6 +197,13 @@ struct vy_task {
*/
double bloom_fpr;
int64_t page_size;
+ /** Batch of deferred deletes generated by this task. */
+ struct vy_deferred_delete_batch *deferred_delete_batch;
+ /**
+ * Number of batches of deferred DELETEs sent to tx
+ * and not yet processed.
+ */
+ int deferred_delete_in_progress;
/** Link in vy_scheduler::processed_tasks. */
struct stailq_entry in_processed;
};
@@ -204,6 +249,8 @@ vy_task_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
static void
vy_task_delete(struct vy_task *task)
{
+ assert(task->deferred_delete_batch == NULL);
+ assert(task->deferred_delete_in_progress == 0);
key_def_delete(task->cmp_def);
key_def_delete(task->key_def);
vy_lsm_unref(task->lsm);
@@ -293,6 +340,12 @@ vy_scheduler_start_workers(struct vy_scheduler *scheduler)
cpipe_create(&worker->worker_pipe, name);
stailq_add_tail_entry(&scheduler->idle_workers,
worker, in_idle);
+
+ struct cmsg_hop *route = worker->deferred_delete_route;
+ route[0].f = vy_deferred_delete_batch_process_f;
+ route[0].pipe = &worker->worker_pipe;
+ route[1].f = vy_deferred_delete_batch_free_f;
+ route[1].pipe = NULL;
}
}
@@ -652,6 +705,215 @@ vy_run_discard(struct vy_run *run)
vy_log_tx_try_commit();
}
+/**
+ * Encode and write a single deferred DELETE statement to
+ * _vinyl_deferred_delete system space. The rest will be
+ * done by the space trigger.
+ */
+static int
+vy_deferred_delete_process_one(struct space *deferred_delete_space,
+ uint32_t space_id, struct tuple_format *format,
+ struct vy_deferred_delete_stmt *stmt)
+{
+ int64_t lsn = vy_stmt_lsn(stmt->new_stmt);
+
+ struct tuple *delete;
+ delete = vy_stmt_new_surrogate_delete(format, stmt->old_stmt);
+ if (delete == NULL)
+ return -1;
+
+ uint32_t delete_data_size;
+ const char *delete_data = tuple_data_range(delete, &delete_data_size);
+
+ size_t buf_size = (mp_sizeof_array(3) + mp_sizeof_uint(space_id) +
+ mp_sizeof_uint(lsn) + delete_data_size);
+ char *data = region_alloc(&fiber()->gc, buf_size);
+ if (data == NULL) {
+ diag_set(OutOfMemory, buf_size, "region", "buf");
+ tuple_unref(delete);
+ return -1;
+ }
+
+ char *data_end = data;
+ data_end = mp_encode_array(data_end, 3);
+ data_end = mp_encode_uint(data_end, space_id);
+ data_end = mp_encode_uint(data_end, lsn);
+ memcpy(data_end, delete_data, delete_data_size);
+ data_end += delete_data_size;
+ assert(data_end <= data + buf_size);
+
+ struct request request;
+ memset(&request, 0, sizeof(request));
+ request.type = IPROTO_REPLACE;
+ request.space_id = BOX_VINYL_DEFERRED_DELETE_ID;
+ request.tuple = data;
+ request.tuple_end = data_end;
+
+ tuple_unref(delete);
+
+ struct txn *txn = txn_begin_stmt(deferred_delete_space);
+ if (txn == NULL)
+ return -1;
+
+ struct tuple *unused;
+ if (space_execute_dml(deferred_delete_space, txn,
+ &request, &unused) != 0) {
+ txn_rollback_stmt();
+ return -1;
+ }
+ return txn_commit_stmt(txn, &request);
+}
+
+/**
+ * Callback invoked by the tx thread to process deferred DELETE
+ * statements generated during compaction. It writes deferred
+ * DELETEs to a special system space, _vinyl_deferred_delete.
+ * The system space has an on_replace trigger installed which
+ * propagates the DELETEs to secondary indexes. This way, even
+ * if a deferred DELETE isn't dumped to disk by vinyl, it still
+ * can be recovered from WAL.
+ */
+static void
+vy_deferred_delete_batch_process_f(struct cmsg *cmsg)
+{
+ struct vy_deferred_delete_batch *batch = container_of(cmsg,
+ struct vy_deferred_delete_batch, cmsg);
+ struct vy_task *task = batch->task;
+ struct vy_lsm *pk = task->lsm;
+
+ assert(pk->index_id == 0);
+ /*
+ * A space can be dropped while a compaction task
+ * is in progress.
+ */
+ if (pk->is_dropped)
+ return;
+
+ struct space *deferred_delete_space;
+ deferred_delete_space = space_by_id(BOX_VINYL_DEFERRED_DELETE_ID);
+ assert(deferred_delete_space != NULL);
+
+ struct txn *txn = txn_begin(false);
+ if (txn == NULL)
+ goto fail;
+
+ for (int i = 0; i < batch->count; i++) {
+ if (vy_deferred_delete_process_one(deferred_delete_space,
+ pk->space_id, pk->mem_format,
+ &batch->stmt[i]) != 0)
+ goto fail;
+ }
+
+ if (txn_commit(txn) != 0)
+ goto fail;
+
+ return;
+fail:
+ batch->is_failed = true;
+ diag_move(diag_get(), &batch->diag);
+ txn_rollback();
+}
+
+/**
+ * Callback invoked by a worker thread to free processed deferred
+ * DELETE statements. It must be done on behalf the worker thread
+ * that generated those DELETEs, because a vinyl statement cannot
+ * be allocated and freed in different threads.
+ */
+static void
+vy_deferred_delete_batch_free_f(struct cmsg *cmsg)
+{
+ struct vy_deferred_delete_batch *batch = container_of(cmsg,
+ struct vy_deferred_delete_batch, cmsg);
+ struct vy_task *task = batch->task;
+ for (int i = 0; i < batch->count; i++) {
+ struct vy_deferred_delete_stmt *stmt = &batch->stmt[i];
+ vy_stmt_unref_if_possible(stmt->old_stmt);
+ vy_stmt_unref_if_possible(stmt->new_stmt);
+ }
+ /*
+ * Abort the task if the tx thread failed to process
+ * the batch unless it has already been aborted.
+ */
+ if (batch->is_failed && !task->is_failed) {
+ assert(!diag_is_empty(&batch->diag));
+ diag_move(&batch->diag, &task->diag);
+ task->is_failed = true;
+ fiber_cancel(task->fiber);
+ }
+ diag_destroy(&batch->diag);
+ free(batch);
+ /* Notify the caller if this is the last batch. */
+ assert(task->deferred_delete_in_progress > 0);
+ if (--task->deferred_delete_in_progress == 0)
+ fiber_wakeup(task->fiber);
+}
+
+/**
+ * Send all deferred DELETEs accumulated by a vinyl task to
+ * the tx thread where they will be processed.
+ */
+static void
+vy_task_deferred_delete_flush(struct vy_task *task)
+{
+ struct vy_worker *worker = task->worker;
+ struct vy_deferred_delete_batch *batch = task->deferred_delete_batch;
+
+ if (batch == NULL)
+ return;
+
+ task->deferred_delete_batch = NULL;
+ task->deferred_delete_in_progress++;
+
+ cmsg_init(&batch->cmsg, worker->deferred_delete_route);
+ cpipe_push(&worker->tx_pipe, &batch->cmsg);
+}
+
+/**
+ * Add a deferred DELETE to a batch. Once the batch gets full,
+ * submit it to tx where it will get processed.
+ */
+static int
+vy_task_deferred_delete(struct vy_task *task, struct tuple *old_stmt,
+ struct tuple *new_stmt)
+{
+ enum { MAX_IN_PROGRESS = 5 };
+
+ struct vy_deferred_delete_batch *batch = task->deferred_delete_batch;
+
+ /*
+ * Throttle compaction task if there are too many batches
+ * being processed so as to limit memory consumption.
+ */
+ while (task->deferred_delete_in_progress >= MAX_IN_PROGRESS)
+ fiber_sleep(TIMEOUT_INFINITY);
+
+ /* Allocate a new batch on demand. */
+ if (batch == NULL) {
+ batch = malloc(sizeof(*batch));
+ if (batch == NULL) {
+ diag_set(OutOfMemory, sizeof(*batch), "malloc",
+ "struct vy_deferred_delete_batch");
+ return -1;
+ }
+ memset(batch, 0, sizeof(*batch));
+ batch->task = task;
+ diag_create(&batch->diag);
+ task->deferred_delete_batch = batch;
+ }
+
+ assert(batch->count < VY_DEFERRED_DELETE_BATCH_MAX);
+ struct vy_deferred_delete_stmt *stmt = &batch->stmt[batch->count++];
+ stmt->old_stmt = old_stmt;
+ vy_stmt_ref_if_possible(old_stmt);
+ stmt->new_stmt = new_stmt;
+ vy_stmt_ref_if_possible(new_stmt);
+
+ if (batch->count == VY_DEFERRED_DELETE_BATCH_MAX)
+ vy_task_deferred_delete_flush(task);
+ return 0;
+}
+
static int
vy_task_write_run(struct vy_task *task)
{
@@ -932,6 +1194,27 @@ vy_task_dump_abort(struct vy_task *task)
}
/**
+ * Callback passed to the write iterator to process deferred
+ * DELETEs generated by a dump task.
+ *
+ * Note, since deferred DELETE are generated on tx commit in case
+ * the overwritten tuple is found in-memory, this function should
+ * never be called. We pass it in order to make the write iterator
+ * keep statements marked as VY_STMT_DEFERRED_DELETE so that they
+ * can be used for generating deferred DELETEs on compaction.
+ */
+static int
+vy_task_dump_deferred_delete(struct tuple *old_stmt,
+ struct tuple *new_stmt, void *arg)
+{
+ (void)old_stmt;
+ (void)new_stmt;
+ (void)arg;
+ unreachable();
+ return 0;
+}
+
+/**
* Create a task to dump an LSM tree.
*
* On success the task is supposed to dump all in-memory
@@ -983,6 +1266,13 @@ vy_task_dump_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
continue;
}
dump_lsn = MAX(dump_lsn, mem->max_lsn);
+ /*
+ * Take into account WAL statements that generated
+ * deferred DELETEs so as to skip dumped deferred
+ * DELETEs on local recovery. For more details, see
+ * vy_deferred_delete_on_replace().
+ */
+ dump_lsn = MAX(dump_lsn, mem->max_deferred_delete_wal_lsn);
}
if (dump_lsn < 0) {
@@ -1006,7 +1296,9 @@ vy_task_dump_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
bool is_last_level = (lsm->run_count == 0);
wi = vy_write_iterator_new(task->cmp_def, lsm->disk_format,
lsm->index_id == 0, is_last_level,
- scheduler->read_views, NULL, NULL);
+ scheduler->read_views,
+ lsm->index_id > 0 ? NULL :
+ vy_task_dump_deferred_delete, task);
if (wi == NULL)
goto err_wi;
rlist_foreach_entry(mem, &lsm->sealed, in_sealed) {
@@ -1235,6 +1527,18 @@ vy_task_compact_abort(struct vy_task *task)
vy_scheduler_update_lsm(scheduler, lsm);
}
+/**
+ * Callback passed to the write iterator to process deferred
+ * DELETEs generated by a compaction task.
+ */
+static int
+vy_task_compact_deferred_delete(struct tuple *old_stmt,
+ struct tuple *new_stmt, void *arg)
+{
+ struct vy_task *task = arg;
+ return vy_task_deferred_delete(task, old_stmt, new_stmt);
+}
+
static int
vy_task_compact_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
struct vy_task **p_task)
@@ -1273,7 +1577,9 @@ vy_task_compact_new(struct vy_scheduler *scheduler, struct vy_lsm *lsm,
bool is_last_level = (range->compact_priority == range->slice_count);
wi = vy_write_iterator_new(task->cmp_def, lsm->disk_format,
lsm->index_id == 0, is_last_level,
- scheduler->read_views, NULL, NULL);
+ scheduler->read_views,
+ lsm->index_id > 0 ? NULL :
+ vy_task_compact_deferred_delete, task);
if (wi == NULL)
goto err_wi;
@@ -1336,12 +1642,22 @@ static int
vy_task_f(va_list va)
{
struct vy_task *task = va_arg(va, struct vy_task *);
- if (task->ops->execute(task) != 0) {
+ if (task->ops->execute(task) != 0 && !task->is_failed) {
struct diag *diag = diag_get();
assert(!diag_is_empty(diag));
task->is_failed = true;
diag_move(diag, &task->diag);
}
+
+ /*
+ * We must not complete the task until we make sure that
+ * all deferred DELETEs generated during task execution
+ * have been successfully processed.
+ */
+ vy_task_deferred_delete_flush(task);
+ while (task->deferred_delete_in_progress > 0)
+ fiber_sleep(TIMEOUT_INFINITY);
+
cmsg_init(&task->cmsg, vy_task_complete_route);
cpipe_push(&task->worker->tx_pipe, &task->cmsg);
task->fiber = NULL;
diff --git a/src/box/vy_tx.c b/src/box/vy_tx.c
index e5fdaed1..a0f9389d 100644
--- a/src/box/vy_tx.c
+++ b/src/box/vy_tx.c
@@ -46,6 +46,7 @@
#include "iterator_type.h"
#include "salad/stailq.h"
#include "schema.h" /* space_cache_version */
+#include "space.h"
#include "trigger.h"
#include "trivia/util.h"
#include "tuple.h"
@@ -58,6 +59,7 @@
#include "vy_history.h"
#include "vy_read_set.h"
#include "vy_read_view.h"
+#include "vy_point_lookup.h"
int
write_set_cmp(struct txv *a, struct txv *b)
@@ -483,6 +485,106 @@ vy_tx_write(struct vy_lsm *lsm, struct vy_mem *mem,
return vy_lsm_set(lsm, mem, stmt, region_stmt);
}
+/**
+ * Try to generate a deferred DELETE statement on tx commit.
+ *
+ * This function is supposed to be called for a primary index
+ * statement which was executed without deletion of the overwritten
+ * tuple from secondary indexes. It looks up the overwritten tuple
+ * in memory and, if found, produces the deferred DELETEs and
+ * inserts them into the transaction log.
+ *
+ * Affects @tx->log, @v->stmt.
+ *
+ * Returns 0 on success, -1 on memory allocation error.
+ */
+static int
+vy_tx_handle_deferred_delete(struct vy_tx *tx, struct txv *v)
+{
+ struct vy_lsm *pk = v->lsm;
+ struct tuple *stmt = v->stmt;
+ uint8_t flags = vy_stmt_flags(stmt);
+
+ assert(pk->index_id == 0);
+ assert(flags & VY_STMT_DEFERRED_DELETE);
+
+ struct space *space = space_cache_find(pk->space_id);
+ if (space == NULL) {
+ /*
+ * Space was dropped while transaction was
+ * in progress. Nothing to do.
+ */
+ return 0;
+ }
+
+ /* Look up the tuple overwritten by this statement. */
+ struct tuple *tuple;
+ if (vy_point_lookup_mem(pk, &tx->xm->p_global_read_view,
+ stmt, &tuple) != 0)
+ return -1;
+
+ if (tuple == NULL) {
+ /*
+ * Nothing's found, but there still may be
+ * matching statements stored on disk so we
+ * have to defer generation of DELETE until
+ * compaction.
+ */
+ return 0;
+ }
+
+ /*
+ * If a terminal statement is found, we can produce
+ * DELETE right away so clear the flag now.
+ */
+ vy_stmt_set_flags(stmt, flags & ~VY_STMT_DEFERRED_DELETE);
+
+ if (vy_stmt_type(tuple) == IPROTO_DELETE) {
+ /* The tuple's already deleted, nothing to do. */
+ tuple_unref(tuple);
+ return 0;
+ }
+
+ struct tuple *delete_stmt;
+ delete_stmt = vy_stmt_new_surrogate_delete(pk->mem_format, tuple);
+ tuple_unref(tuple);
+ if (delete_stmt == NULL)
+ return -1;
+
+ if (vy_stmt_type(stmt) == IPROTO_DELETE) {
+ /*
+ * Since primary and secondary indexes of the
+ * same space share in-memory statements, we
+ * need to use the new DELETE in the primary
+ * index, because the original DELETE doesn't
+ * contain secondary key parts.
+ */
+ vy_stmt_counter_acct_tuple(&pk->stat.txw.count, delete_stmt);
+ vy_stmt_counter_unacct_tuple(&pk->stat.txw.count, stmt);
+ v->stmt = delete_stmt;
+ tuple_ref(delete_stmt);
+ tuple_unref(stmt);
+ }
+
+ /*
+ * Make DELETE statements for secondary indexes and
+ * insert them into the transaction log.
+ */
+ int rc = 0;
+ for (uint32_t i = 1; i < space->index_count; i++) {
+ struct vy_lsm *lsm = vy_lsm(space->index[i]);
+ struct txv *delete_txv = txv_new(tx, lsm, delete_stmt);
+ if (delete_txv == NULL) {
+ rc = -1;
+ break;
+ }
+ stailq_insert_entry(&tx->log, delete_txv, v, next_in_log);
+ vy_stmt_counter_acct_tuple(&lsm->stat.txw.count, delete_stmt);
+ }
+ tuple_unref(delete_stmt);
+ return rc;
+}
+
int
vy_tx_prepare(struct vy_tx *tx)
{
@@ -536,6 +638,22 @@ vy_tx_prepare(struct vy_tx *tx)
if (v->is_overwritten)
continue;
+ if (lsm->index_id > 0 && repsert == NULL && delete == NULL) {
+ /*
+ * This statement is for a secondary index,
+ * and the statement corresponding to it in
+ * the primary index was overwritten. This
+ * can only happen if insertion of DELETE
+ * into secondary indexes was postponed until
+ * primary index compaction. In this case
+ * the DELETE will not be generated, because
+ * the corresponding statement never made it
+ * to the primary index LSM tree. So we must
+ * skip it for secondary indexes as well.
+ */
+ continue;
+ }
+
enum iproto_type type = vy_stmt_type(v->stmt);
/* Optimize out INSERT + DELETE for the same key. */
@@ -550,6 +668,16 @@ vy_tx_prepare(struct vy_tx *tx)
*/
type = IPROTO_INSERT;
vy_stmt_set_type(v->stmt, type);
+ /*
+ * In case of INSERT, no statement was actually
+ * overwritten so no need to generate a deferred
+ * DELETE for secondary indexes.
+ */
+ uint8_t flags = vy_stmt_flags(v->stmt);
+ if (flags & VY_STMT_DEFERRED_DELETE) {
+ vy_stmt_set_flags(v->stmt, flags &
+ ~VY_STMT_DEFERRED_DELETE);
+ }
}
if (!v->is_first_insert && type == IPROTO_INSERT) {
@@ -565,6 +693,11 @@ vy_tx_prepare(struct vy_tx *tx)
return -1;
assert(v->mem != NULL);
+ if (lsm->index_id == 0 &&
+ vy_stmt_flags(v->stmt) & VY_STMT_DEFERRED_DELETE &&
+ vy_tx_handle_deferred_delete(tx, v) != 0)
+ return -1;
+
/* In secondary indexes only REPLACE/DELETE can be written. */
vy_stmt_set_lsn(v->stmt, MAX_LSN + tx->psn);
const struct tuple **region_stmt =
diff --git a/test/unit/vy_point_lookup.c b/test/unit/vy_point_lookup.c
index 54a34e98..093893bd 100644
--- a/test/unit/vy_point_lookup.c
+++ b/test/unit/vy_point_lookup.c
@@ -13,6 +13,8 @@
uint32_t schema_version;
uint32_t space_cache_version;
+struct space *space_by_id(uint32_t id) { return NULL; }
+struct vy_lsm *vy_lsm(struct index *index) { return NULL; }
static int
write_run(struct vy_run *run, const char *dir_name,
diff --git a/test/vinyl/deferred_delete.result b/test/vinyl/deferred_delete.result
new file mode 100644
index 00000000..9811b6bc
--- /dev/null
+++ b/test/vinyl/deferred_delete.result
@@ -0,0 +1,677 @@
+test_run = require('test_run').new()
+---
+...
+fiber = require('fiber')
+---
+...
+--
+-- Create a space with secondary indexes and check that REPLACE and
+-- DELETE requests do not look up the old tuple in the primary index
+-- to generate the DELETE statements for secondary indexes. Instead
+-- DELETEs are generated when the primary index is compacted (gh-2129).
+-- The optimization should work for both non-unique and unique indexes
+-- so mark one of the indexes unique.
+--
+s = box.schema.space.create('test', {engine = 'vinyl'})
+---
+...
+pk = s:create_index('pk', {run_count_per_level = 10})
+---
+...
+i1 = s:create_index('i1', {run_count_per_level = 10, parts = {2, 'unsigned'}, unique = false})
+---
+...
+i2 = s:create_index('i2', {run_count_per_level = 10, parts = {3, 'unsigned'}, unique = true})
+---
+...
+for i = 1, 10 do s:replace{i, i, i} end
+---
+...
+box.snapshot()
+---
+- ok
+...
+for i = 1, 10, 2 do s:delete{i} end
+---
+...
+for i = 2, 10, 2 do s:replace{i, i * 10, i * 100} end
+---
+...
+-- DELETE/REPLACE does not look up the old tuple in the primary index.
+pk:stat().lookup -- 0
+---
+- 0
+...
+-- DELETEs are not written to secondary indexes.
+pk:stat().rows -- 10 old REPLACEs + 5 new REPLACEs + 5 DELETEs
+---
+- 20
+...
+i1:stat().rows -- 10 old REPLACEs + 5 new REPLACEs
+---
+- 15
+...
+i2:stat().rows -- ditto
+---
+- 15
+...
+-- Although there are only 5 tuples in the space, we have to look up
+-- overwritten tuples in the primary index hence 15 lookups per SELECT
+-- in a secondary index.
+i1:select()
+---
+- - [2, 20, 200]
+ - [4, 40, 400]
+ - [6, 60, 600]
+ - [8, 80, 800]
+ - [10, 100, 1000]
+...
+i1:stat().get.rows -- 15
+---
+- 15
+...
+pk:stat().lookup -- 15
+---
+- 15
+...
+i2:select()
+---
+- - [2, 20, 200]
+ - [4, 40, 400]
+ - [6, 60, 600]
+ - [8, 80, 800]
+ - [10, 100, 1000]
+...
+i2:stat().get.rows -- 15
+---
+- 15
+...
+pk:stat().lookup -- 30
+---
+- 30
+...
+-- Overwritten/deleted tuples are not stored in the cache so calling
+-- SELECT for a second time does only 5 lookups.
+box.stat.reset()
+---
+...
+i1:select()
+---
+- - [2, 20, 200]
+ - [4, 40, 400]
+ - [6, 60, 600]
+ - [8, 80, 800]
+ - [10, 100, 1000]
+...
+i1:stat().get.rows -- 5
+---
+- 5
+...
+pk:stat().lookup -- 5
+---
+- 5
+...
+i2:select()
+---
+- - [2, 20, 200]
+ - [4, 40, 400]
+ - [6, 60, 600]
+ - [8, 80, 800]
+ - [10, 100, 1000]
+...
+i2:stat().get.rows -- 5
+---
+- 5
+...
+pk:stat().lookup -- 10
+---
+- 10
+...
+-- Cleanup the cache.
+vinyl_cache = box.cfg.vinyl_cache
+---
+...
+box.cfg{vinyl_cache = 0}
+---
+...
+box.cfg{vinyl_cache = vinyl_cache}
+---
+...
+-- Compact the primary index to generate deferred DELETEs.
+box.snapshot()
+---
+- ok
+...
+pk:compact()
+---
+...
+while pk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+---
+...
+pk:stat().rows -- 5 new REPLACEs
+---
+- 5
+...
+i1:stat().rows -- 10 old REPLACE + 5 new REPLACEs + 10 deferred DELETEs
+---
+- 25
+...
+i2:stat().rows -- ditto
+---
+- 25
+...
+-- Deferred DELETEs must be ignored by the read iterator, because
+-- they may break the read iterator invariant, so they don't reduce
+-- the number of lookups.
+box.stat.reset()
+---
+...
+i1:select()
+---
+- - [2, 20, 200]
+ - [4, 40, 400]
+ - [6, 60, 600]
+ - [8, 80, 800]
+ - [10, 100, 1000]
+...
+i1:stat().get.rows -- 15
+---
+- 15
+...
+pk:stat().lookup -- 15
+---
+- 15
+...
+i2:select()
+---
+- - [2, 20, 200]
+ - [4, 40, 400]
+ - [6, 60, 600]
+ - [8, 80, 800]
+ - [10, 100, 1000]
+...
+i2:stat().get.rows -- 15
+---
+- 15
+...
+pk:stat().lookup -- 30
+---
+- 30
+...
+-- Check that deferred DELETEs are not lost after restart.
+test_run:cmd("restart server default")
+fiber = require('fiber')
+---
+...
+s = box.space.test
+---
+...
+pk = s.index.pk
+---
+...
+i1 = s.index.i1
+---
+...
+i2 = s.index.i2
+---
+...
+i1:stat().rows -- 10 old REPLACEs + 5 new REPLACEs + 10 deferred DELETEs
+---
+- 25
+...
+i2:stat().rows -- ditto
+---
+- 25
+...
+-- Dump deferred DELETEs to disk and compact them.
+-- Check that they cleanup garbage statements.
+box.snapshot()
+---
+- ok
+...
+i1:compact()
+---
+...
+while i1:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+---
+...
+i2:compact()
+---
+...
+while i2:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+---
+...
+i1:stat().rows -- 5 new REPLACEs
+---
+- 5
+...
+i2:stat().rows -- ditto
+---
+- 5
+...
+box.stat.reset()
+---
+...
+i1:select()
+---
+- - [2, 20, 200]
+ - [4, 40, 400]
+ - [6, 60, 600]
+ - [8, 80, 800]
+ - [10, 100, 1000]
+...
+i1:stat().get.rows -- 5
+---
+- 5
+...
+pk:stat().lookup -- 5
+---
+- 5
+...
+i2:select()
+---
+- - [2, 20, 200]
+ - [4, 40, 400]
+ - [6, 60, 600]
+ - [8, 80, 800]
+ - [10, 100, 1000]
+...
+i2:stat().get.rows -- 5
+---
+- 5
+...
+pk:stat().lookup -- 10
+---
+- 10
+...
+s:drop()
+---
+...
+--
+-- Check that if the old tuple is found in cache or in memory, then
+-- the DELETE for secondary indexes is generated when the statement
+-- is committed.
+--
+s = box.schema.space.create('test', {engine = 'vinyl'})
+---
+...
+pk = s:create_index('pk', {run_count_per_level = 10})
+---
+...
+sk = s:create_index('sk', {run_count_per_level = 10, parts = {2, 'unsigned'}, unique = false})
+---
+...
+for i = 1, 10 do s:replace{i, i} end
+---
+...
+box.snapshot()
+---
+- ok
+...
+s:count() -- add tuples to the cache
+---
+- 10
+...
+box.stat.reset()
+---
+...
+for i = 1, 10, 2 do s:delete{i} end
+---
+...
+for i = 2, 10, 2 do s:replace{i, i * 10} end
+---
+...
+pk:stat().lookup -- 0
+---
+- 0
+...
+pk:stat().cache.lookup -- 10
+---
+- 10
+...
+pk:stat().cache.get.rows -- 10
+---
+- 10
+...
+pk:stat().memory.iterator.lookup -- 0
+---
+- 0
+...
+sk:stat().rows -- 10 old REPLACEs + 10 DELETEs + 5 new REPLACEs
+---
+- 25
+...
+box.stat.reset()
+---
+...
+for i = 1, 10 do s:replace{i, i * 100} end
+---
+...
+pk:stat().lookup -- 0
+---
+- 0
+...
+pk:stat().cache.lookup -- 10
+---
+- 10
+...
+pk:stat().cache.get.rows -- 0
+---
+- 0
+...
+pk:stat().memory.iterator.lookup -- 10
+---
+- 10
+...
+pk:stat().memory.iterator.get.rows -- 10
+---
+- 10
+...
+sk:stat().rows -- 15 old REPLACEs + 15 DELETEs + 10 new REPLACEs
+---
+- 40
+...
+box.stat.reset()
+---
+...
+for i = 1, 10 do s:delete{i} end
+---
+...
+pk:stat().lookup -- 0
+---
+- 0
+...
+pk:stat().cache.lookup -- 10
+---
+- 10
+...
+pk:stat().cache.get.rows -- 0
+---
+- 0
+...
+pk:stat().memory.iterator.lookup -- 10
+---
+- 10
+...
+pk:stat().memory.iterator.get.rows -- 10
+---
+- 10
+...
+sk:stat().rows -- 25 old REPLACEs + 25 DELETEs
+---
+- 50
+...
+sk:select()
+---
+- []
+...
+pk:stat().lookup -- 0
+---
+- 0
+...
+box.snapshot()
+---
+- ok
+...
+sk:compact()
+---
+...
+while sk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+---
+...
+sk:stat().run_count -- 0
+---
+- 0
+...
+s:drop()
+---
+...
+--
+-- Check that a transaction is aborted if it read a tuple from
+-- a secondary index that was overwritten in the primary index.
+--
+s = box.schema.space.create('test', {engine = 'vinyl'})
+---
+...
+pk = s:create_index('pk')
+---
+...
+sk = s:create_index('sk', {parts = {2, 'unsigned'}, unique = false})
+---
+...
+s:replace{1, 1}
+---
+- [1, 1]
+...
+box.snapshot()
+---
+- ok
+...
+box.begin()
+---
+...
+sk:select{1}
+---
+- - [1, 1]
+...
+c = fiber.channel(1)
+---
+...
+_ = fiber.create(function() s:replace{1, 10} c:put(true) end)
+---
+...
+c:get()
+---
+- true
+...
+sk:select{1}
+---
+- - [1, 1]
+...
+s:replace{10, 10}
+---
+- [10, 10]
+...
+box.commit() -- error
+---
+- error: Transaction has been aborted by conflict
+...
+s:drop()
+---
+...
+--
+-- Check that if a tuple was overwritten in the transaction write set,
+-- it won't be committed to secondary indexes.
+--
+s = box.schema.space.create('test', {engine = 'vinyl'})
+---
+...
+pk = s:create_index('pk', {run_count_per_level = 10})
+---
+...
+sk = s:create_index('sk', {run_count_per_level = 10, parts = {2, 'unsigned'}, unique = false})
+---
+...
+for i = 1, 10 do s:replace{i, i} end
+---
+...
+box.snapshot()
+---
+- ok
+...
+box.begin()
+---
+...
+for i = 1, 10 do s:replace{i, i * 10} end
+---
+...
+for i = 1, 10, 2 do s:delete{i} end
+---
+...
+for i = 2, 10, 2 do s:replace{i, i * 100} end
+---
+...
+box.commit()
+---
+...
+sk:select()
+---
+- - [2, 200]
+ - [4, 400]
+ - [6, 600]
+ - [8, 800]
+ - [10, 1000]
+...
+pk:stat().rows -- 10 old REPLACEs + 5 DELETEs + 5 new REPLACEs
+---
+- 20
+...
+sk:stat().rows -- 10 old REPLACEs + 5 new REPLACEs
+---
+- 15
+...
+-- Compact the primary index to generate deferred DELETEs.
+box.snapshot()
+---
+- ok
+...
+pk:compact()
+---
+...
+while pk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+---
+...
+-- Compact the secondary index to cleanup garbage.
+box.snapshot()
+---
+- ok
+...
+sk:compact()
+---
+...
+while sk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+---
+...
+sk:select()
+---
+- - [2, 200]
+ - [4, 400]
+ - [6, 600]
+ - [8, 800]
+ - [10, 1000]
+...
+pk:stat().rows -- 5 new REPLACEs
+---
+- 5
+...
+sk:stat().rows -- ditto
+---
+- 5
+...
+s:drop()
+---
+...
+--
+-- Check that on recovery we do not apply deferred DELETEs that
+-- have been dumped to disk.
+--
+test_run:cmd("create server test with script='vinyl/low_quota.lua'")
+---
+- true
+...
+test_run:cmd("start server test with args='1048576'")
+---
+- true
+...
+test_run:cmd("switch test")
+---
+- true
+...
+fiber = require('fiber')
+---
+...
+s = box.schema.space.create('test', {engine = 'vinyl'})
+---
+...
+pk = s:create_index('pk', {run_count_per_level = 10})
+---
+...
+sk = s:create_index('sk', {run_count_per_level = 10, parts = {2, 'unsigned', 3, 'string'}, unique = false})
+---
+...
+pad = string.rep('x', 10 * 1024)
+---
+...
+for i = 1, 120 do s:replace{i, i, pad} end
+---
+...
+box.snapshot()
+---
+- ok
+...
+pad = string.rep('y', 10 * 1024)
+---
+...
+for i = 1, 120 do s:replace{i, i, pad} end
+---
+...
+box.snapshot()
+---
+- ok
+...
+sk:stat().rows -- 120 old REPLACEs + 120 new REPLACEs
+---
+- 240
+...
+box.stat.reset()
+---
+...
+-- Compact the primary index to generate deferred DELETEs.
+-- Deferred DELETEs won't fit in memory and trigger dump
+-- of the secondary index.
+pk:compact()
+---
+...
+while pk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+---
+...
+sk:stat().disk.dump.count -- 1
+---
+- 1
+...
+sk:stat().rows -- 120 old REPLACEs + 120 new REPLACEs + 120 deferred DELETEs
+---
+- 360
+...
+test_run:cmd("restart server test with args='1048576'")
+s = box.space.test
+---
+...
+pk = s.index.pk
+---
+...
+sk = s.index.sk
+---
+...
+-- Should be 360, the same amount of statements as before restart.
+-- If we applied all deferred DELETEs, including the dumped ones,
+-- then there would be more.
+sk:stat().rows
+---
+- 360
+...
+s:drop()
+---
+...
+test_run:cmd("switch default")
+---
+- true
+...
+test_run:cmd("stop server test")
+---
+- true
+...
+test_run:cmd("cleanup server test")
+---
+- true
+...
diff --git a/test/vinyl/deferred_delete.test.lua b/test/vinyl/deferred_delete.test.lua
new file mode 100644
index 00000000..d18361a0
--- /dev/null
+++ b/test/vinyl/deferred_delete.test.lua
@@ -0,0 +1,261 @@
+test_run = require('test_run').new()
+fiber = require('fiber')
+
+--
+-- Create a space with secondary indexes and check that REPLACE and
+-- DELETE requests do not look up the old tuple in the primary index
+-- to generate the DELETE statements for secondary indexes. Instead
+-- DELETEs are generated when the primary index is compacted (gh-2129).
+-- The optimization should work for both non-unique and unique indexes
+-- so mark one of the indexes unique.
+--
+s = box.schema.space.create('test', {engine = 'vinyl'})
+pk = s:create_index('pk', {run_count_per_level = 10})
+i1 = s:create_index('i1', {run_count_per_level = 10, parts = {2, 'unsigned'}, unique = false})
+i2 = s:create_index('i2', {run_count_per_level = 10, parts = {3, 'unsigned'}, unique = true})
+for i = 1, 10 do s:replace{i, i, i} end
+box.snapshot()
+for i = 1, 10, 2 do s:delete{i} end
+for i = 2, 10, 2 do s:replace{i, i * 10, i * 100} end
+
+-- DELETE/REPLACE does not look up the old tuple in the primary index.
+pk:stat().lookup -- 0
+
+-- DELETEs are not written to secondary indexes.
+pk:stat().rows -- 10 old REPLACEs + 5 new REPLACEs + 5 DELETEs
+i1:stat().rows -- 10 old REPLACEs + 5 new REPLACEs
+i2:stat().rows -- ditto
+
+-- Although there are only 5 tuples in the space, we have to look up
+-- overwritten tuples in the primary index hence 15 lookups per SELECT
+-- in a secondary index.
+i1:select()
+i1:stat().get.rows -- 15
+pk:stat().lookup -- 15
+i2:select()
+i2:stat().get.rows -- 15
+pk:stat().lookup -- 30
+
+-- Overwritten/deleted tuples are not stored in the cache so calling
+-- SELECT for a second time does only 5 lookups.
+box.stat.reset()
+i1:select()
+i1:stat().get.rows -- 5
+pk:stat().lookup -- 5
+i2:select()
+i2:stat().get.rows -- 5
+pk:stat().lookup -- 10
+
+-- Cleanup the cache.
+vinyl_cache = box.cfg.vinyl_cache
+box.cfg{vinyl_cache = 0}
+box.cfg{vinyl_cache = vinyl_cache}
+
+-- Compact the primary index to generate deferred DELETEs.
+box.snapshot()
+pk:compact()
+while pk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+pk:stat().rows -- 5 new REPLACEs
+i1:stat().rows -- 10 old REPLACE + 5 new REPLACEs + 10 deferred DELETEs
+i2:stat().rows -- ditto
+
+-- Deferred DELETEs must be ignored by the read iterator, because
+-- they may break the read iterator invariant, so they don't reduce
+-- the number of lookups.
+box.stat.reset()
+i1:select()
+i1:stat().get.rows -- 15
+pk:stat().lookup -- 15
+i2:select()
+i2:stat().get.rows -- 15
+pk:stat().lookup -- 30
+
+-- Check that deferred DELETEs are not lost after restart.
+test_run:cmd("restart server default")
+fiber = require('fiber')
+s = box.space.test
+pk = s.index.pk
+i1 = s.index.i1
+i2 = s.index.i2
+i1:stat().rows -- 10 old REPLACEs + 5 new REPLACEs + 10 deferred DELETEs
+i2:stat().rows -- ditto
+
+-- Dump deferred DELETEs to disk and compact them.
+-- Check that they cleanup garbage statements.
+box.snapshot()
+i1:compact()
+while i1:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+i2:compact()
+while i2:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+i1:stat().rows -- 5 new REPLACEs
+i2:stat().rows -- ditto
+box.stat.reset()
+i1:select()
+i1:stat().get.rows -- 5
+pk:stat().lookup -- 5
+i2:select()
+i2:stat().get.rows -- 5
+pk:stat().lookup -- 10
+
+s:drop()
+
+--
+-- Check that if the old tuple is found in cache or in memory, then
+-- the DELETE for secondary indexes is generated when the statement
+-- is committed.
+--
+s = box.schema.space.create('test', {engine = 'vinyl'})
+pk = s:create_index('pk', {run_count_per_level = 10})
+sk = s:create_index('sk', {run_count_per_level = 10, parts = {2, 'unsigned'}, unique = false})
+
+for i = 1, 10 do s:replace{i, i} end
+box.snapshot()
+s:count() -- add tuples to the cache
+
+box.stat.reset()
+for i = 1, 10, 2 do s:delete{i} end
+for i = 2, 10, 2 do s:replace{i, i * 10} end
+pk:stat().lookup -- 0
+pk:stat().cache.lookup -- 10
+pk:stat().cache.get.rows -- 10
+pk:stat().memory.iterator.lookup -- 0
+sk:stat().rows -- 10 old REPLACEs + 10 DELETEs + 5 new REPLACEs
+
+box.stat.reset()
+for i = 1, 10 do s:replace{i, i * 100} end
+pk:stat().lookup -- 0
+pk:stat().cache.lookup -- 10
+pk:stat().cache.get.rows -- 0
+pk:stat().memory.iterator.lookup -- 10
+pk:stat().memory.iterator.get.rows -- 10
+sk:stat().rows -- 15 old REPLACEs + 15 DELETEs + 10 new REPLACEs
+
+box.stat.reset()
+for i = 1, 10 do s:delete{i} end
+pk:stat().lookup -- 0
+pk:stat().cache.lookup -- 10
+pk:stat().cache.get.rows -- 0
+pk:stat().memory.iterator.lookup -- 10
+pk:stat().memory.iterator.get.rows -- 10
+sk:stat().rows -- 25 old REPLACEs + 25 DELETEs
+
+sk:select()
+pk:stat().lookup -- 0
+
+box.snapshot()
+sk:compact()
+while sk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+sk:stat().run_count -- 0
+
+s:drop()
+
+--
+-- Check that a transaction is aborted if it read a tuple from
+-- a secondary index that was overwritten in the primary index.
+--
+s = box.schema.space.create('test', {engine = 'vinyl'})
+pk = s:create_index('pk')
+sk = s:create_index('sk', {parts = {2, 'unsigned'}, unique = false})
+s:replace{1, 1}
+box.snapshot()
+
+box.begin()
+sk:select{1}
+c = fiber.channel(1)
+_ = fiber.create(function() s:replace{1, 10} c:put(true) end)
+c:get()
+sk:select{1}
+s:replace{10, 10}
+box.commit() -- error
+
+s:drop()
+
+--
+-- Check that if a tuple was overwritten in the transaction write set,
+-- it won't be committed to secondary indexes.
+--
+s = box.schema.space.create('test', {engine = 'vinyl'})
+pk = s:create_index('pk', {run_count_per_level = 10})
+sk = s:create_index('sk', {run_count_per_level = 10, parts = {2, 'unsigned'}, unique = false})
+for i = 1, 10 do s:replace{i, i} end
+box.snapshot()
+
+box.begin()
+for i = 1, 10 do s:replace{i, i * 10} end
+for i = 1, 10, 2 do s:delete{i} end
+for i = 2, 10, 2 do s:replace{i, i * 100} end
+box.commit()
+
+sk:select()
+
+pk:stat().rows -- 10 old REPLACEs + 5 DELETEs + 5 new REPLACEs
+sk:stat().rows -- 10 old REPLACEs + 5 new REPLACEs
+
+-- Compact the primary index to generate deferred DELETEs.
+box.snapshot()
+pk:compact()
+while pk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+
+-- Compact the secondary index to cleanup garbage.
+box.snapshot()
+sk:compact()
+while sk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+
+sk:select()
+
+pk:stat().rows -- 5 new REPLACEs
+sk:stat().rows -- ditto
+
+s:drop()
+
+--
+-- Check that on recovery we do not apply deferred DELETEs that
+-- have been dumped to disk.
+--
+test_run:cmd("create server test with script='vinyl/low_quota.lua'")
+test_run:cmd("start server test with args='1048576'")
+test_run:cmd("switch test")
+
+fiber = require('fiber')
+
+s = box.schema.space.create('test', {engine = 'vinyl'})
+pk = s:create_index('pk', {run_count_per_level = 10})
+sk = s:create_index('sk', {run_count_per_level = 10, parts = {2, 'unsigned', 3, 'string'}, unique = false})
+
+pad = string.rep('x', 10 * 1024)
+for i = 1, 120 do s:replace{i, i, pad} end
+box.snapshot()
+
+pad = string.rep('y', 10 * 1024)
+for i = 1, 120 do s:replace{i, i, pad} end
+box.snapshot()
+
+sk:stat().rows -- 120 old REPLACEs + 120 new REPLACEs
+
+box.stat.reset()
+
+-- Compact the primary index to generate deferred DELETEs.
+-- Deferred DELETEs won't fit in memory and trigger dump
+-- of the secondary index.
+pk:compact()
+while pk:stat().disk.compact.count == 0 do fiber.sleep(0.001) end
+
+sk:stat().disk.dump.count -- 1
+
+sk:stat().rows -- 120 old REPLACEs + 120 new REPLACEs + 120 deferred DELETEs
+
+test_run:cmd("restart server test with args='1048576'")
+s = box.space.test
+pk = s.index.pk
+sk = s.index.sk
+
+-- Should be 360, the same amount of statements as before restart.
+-- If we applied all deferred DELETEs, including the dumped ones,
+-- then there would be more.
+sk:stat().rows
+
+s:drop()
+
+test_run:cmd("switch default")
+test_run:cmd("stop server test")
+test_run:cmd("cleanup server test")
diff --git a/test/vinyl/info.result b/test/vinyl/info.result
index 112ba85e..95e8cc60 100644
--- a/test/vinyl/info.result
+++ b/test/vinyl/info.result
@@ -1036,10 +1036,10 @@ s:bsize()
---
- 0
...
-i1 = s:create_index('i1', {parts = {1, 'unsigned'}, run_count_per_level = 1})
+i1 = s:create_index('i1', {parts = {1, 'unsigned'}, run_count_per_level = 10})
---
...
-i2 = s:create_index('i2', {parts = {2, 'unsigned'}, run_count_per_level = 1})
+i2 = s:create_index('i2', {parts = {2, 'unsigned'}, run_count_per_level = 10})
---
...
s:bsize()
@@ -1162,7 +1162,7 @@ s:bsize()
i1:len(), i2:len()
---
- 150
-- 150
+- 100
...
i1:bsize(), i2:bsize()
---
@@ -1189,13 +1189,25 @@ i2:bsize() == st2.memory.index_size + st2.disk.index_size + st2.disk.bloom_size
---
- true
...
+-- Compact the primary index first to generate deferred DELETEs.
+-- Then dump them and compact the secondary index.
box.snapshot()
---
- ok
...
+i1:compact()
+---
+...
wait(function() return i1:stat() end, st1, 'disk.compact.count', 1)
---
...
+box.snapshot()
+---
+- ok
+...
+i2:compact()
+---
+...
wait(function() return i2:stat() end, st2, 'disk.compact.count', 1)
---
...
diff --git a/test/vinyl/info.test.lua b/test/vinyl/info.test.lua
index 863a8793..5aebd0a8 100644
--- a/test/vinyl/info.test.lua
+++ b/test/vinyl/info.test.lua
@@ -322,8 +322,8 @@ s:drop()
s = box.schema.space.create('test', {engine = 'vinyl'})
s:bsize()
-i1 = s:create_index('i1', {parts = {1, 'unsigned'}, run_count_per_level = 1})
-i2 = s:create_index('i2', {parts = {2, 'unsigned'}, run_count_per_level = 1})
+i1 = s:create_index('i1', {parts = {1, 'unsigned'}, run_count_per_level = 10})
+i2 = s:create_index('i2', {parts = {2, 'unsigned'}, run_count_per_level = 10})
s:bsize()
i1:len(), i2:len()
i1:bsize(), i2:bsize()
@@ -365,8 +365,13 @@ i2:len() == st2.memory.rows + st2.disk.rows
i1:bsize() == st1.memory.index_size + st1.disk.index_size + st1.disk.bloom_size
i2:bsize() == st2.memory.index_size + st2.disk.index_size + st2.disk.bloom_size + st2.disk.bytes
+-- Compact the primary index first to generate deferred DELETEs.
+-- Then dump them and compact the secondary index.
box.snapshot()
+i1:compact()
wait(function() return i1:stat() end, st1, 'disk.compact.count', 1)
+box.snapshot()
+i2:compact()
wait(function() return i2:stat() end, st2, 'disk.compact.count', 1)
st1 = i1:stat()
st2 = i2:stat()
diff --git a/test/vinyl/layout.result b/test/vinyl/layout.result
index 49826302..45a8a87a 100644
--- a/test/vinyl/layout.result
+++ b/test/vinyl/layout.result
@@ -253,17 +253,17 @@ result
- - 00000000000000000008.run
- - HEADER:
lsn: 10
- type: INSERT
+ type: REPLACE
BODY:
tuple: ['ёёё', null]
- HEADER:
lsn: 9
- type: INSERT
+ type: REPLACE
BODY:
tuple: ['эээ', null]
- HEADER:
lsn: 8
- type: INSERT
+ type: REPLACE
BODY:
tuple: ['ЭЭЭ', null]
- HEADER:
@@ -285,8 +285,8 @@ result
BODY:
row_index_offset: <offset>
offset: <offset>
- size: 90
- unpacked_size: 71
+ size: 96
+ unpacked_size: 77
row_count: 3
min_key: ['ёёё']
- - 00000000000000000012.run
@@ -295,20 +295,23 @@ result
type: REPLACE
BODY:
tuple: ['ёёё', 123]
+ flags: 1
- HEADER:
lsn: 13
- type: INSERT
+ type: REPLACE
BODY:
tuple: ['ююю', 789]
+ flags: 1
- HEADER:
lsn: 12
- type: INSERT
+ type: REPLACE
BODY:
tuple: ['ЮЮЮ', 456]
+ flags: 1
- HEADER:
type: ROWINDEX
BODY:
- row_index: "\0\0\0\0\0\0\0\x10\0\0\0\""
+ row_index: "\0\0\0\0\0\0\0\x12\0\0\0&"
- - 00000000000000000006.index
- - HEADER:
type: RUNINFO
@@ -331,17 +334,17 @@ result
- - 00000000000000000006.run
- - HEADER:
lsn: 10
- type: INSERT
+ type: REPLACE
BODY:
tuple: [null, 'ёёё']
- HEADER:
lsn: 9
- type: INSERT
+ type: REPLACE
BODY:
tuple: [null, 'эээ']
- HEADER:
lsn: 8
- type: INSERT
+ type: REPLACE
BODY:
tuple: [null, 'ЭЭЭ']
- HEADER:
@@ -357,41 +360,36 @@ result
page_count: 1
bloom_filter: <bloom_filter>
max_lsn: 13
- min_key: [null, 'ёёё']
+ min_key: [123, 'ёёё']
- HEADER:
type: PAGEINFO
BODY:
row_index_offset: <offset>
offset: <offset>
- size: 110
- unpacked_size: 91
- row_count: 4
- min_key: [null, 'ёёё']
+ size: 90
+ unpacked_size: 71
+ row_count: 3
+ min_key: [123, 'ёёё']
- - 00000000000000000010.run
- - HEADER:
lsn: 11
- type: DELETE
- BODY:
- key: [null, 'ёёё']
- - HEADER:
- lsn: 11
type: REPLACE
BODY:
tuple: [123, 'ёёё']
- HEADER:
lsn: 12
- type: INSERT
+ type: REPLACE
BODY:
tuple: [456, 'ЮЮЮ']
- HEADER:
lsn: 13
- type: INSERT
+ type: REPLACE
BODY:
tuple: [789, 'ююю']
- HEADER:
type: ROWINDEX
BODY:
- row_index: "\0\0\0\0\0\0\0\x10\0\0\0 \0\0\02"
+ row_index: "\0\0\0\0\0\0\0\x10\0\0\0\""
...
test_run:cmd("clear filter")
---
diff --git a/test/vinyl/quota.result b/test/vinyl/quota.result
index e323bc4e..48042185 100644
--- a/test/vinyl/quota.result
+++ b/test/vinyl/quota.result
@@ -89,7 +89,7 @@ _ = space:replace{1, 1, string.rep('a', 1024 * 1024 * 5)}
...
box.stat.vinyl().quota.used
---
-- 5341228
+- 5341267
...
space:drop()
---
diff --git a/test/vinyl/tx_gap_lock.result b/test/vinyl/tx_gap_lock.result
index 150826cb..a456c017 100644
--- a/test/vinyl/tx_gap_lock.result
+++ b/test/vinyl/tx_gap_lock.result
@@ -1194,8 +1194,8 @@ s:drop()
---
...
----------------------------------------------------------------
--- gh-2534: Iterator over a secondary index doesn't double track
--- results in the primary index.
+-- Iterator over a secondary index tracks all results in the
+-- primary index. Needed for gh-2129.
----------------------------------------------------------------
s = box.schema.space.create('test', {engine = 'vinyl'})
---
@@ -1219,23 +1219,23 @@ gap_lock_count() -- 0
_ = s.index.sk:select({}, {limit = 50})
---
...
-gap_lock_count() -- 1
+gap_lock_count() -- 51
---
-- 1
+- 51
...
for i = 1, 100 do s.index.sk:get(i) end
---
...
-gap_lock_count() -- 51
+gap_lock_count() -- 151
---
-- 51
+- 151
...
_ = s.index.sk:select()
---
...
-gap_lock_count() -- 1
+gap_lock_count() -- 101
---
-- 1
+- 101
...
box.commit()
---
diff --git a/test/vinyl/tx_gap_lock.test.lua b/test/vinyl/tx_gap_lock.test.lua
index 4d8d21d8..4ad55860 100644
--- a/test/vinyl/tx_gap_lock.test.lua
+++ b/test/vinyl/tx_gap_lock.test.lua
@@ -380,8 +380,8 @@ c4:commit()
s:drop()
----------------------------------------------------------------
--- gh-2534: Iterator over a secondary index doesn't double track
--- results in the primary index.
+-- Iterator over a secondary index tracks all results in the
+-- primary index. Needed for gh-2129.
----------------------------------------------------------------
s = box.schema.space.create('test', {engine = 'vinyl'})
_ = s:create_index('pk', {parts = {1, 'unsigned'}})
@@ -390,11 +390,11 @@ for i = 1, 100 do s:insert{i, i} end
box.begin()
gap_lock_count() -- 0
_ = s.index.sk:select({}, {limit = 50})
-gap_lock_count() -- 1
-for i = 1, 100 do s.index.sk:get(i) end
gap_lock_count() -- 51
+for i = 1, 100 do s.index.sk:get(i) end
+gap_lock_count() -- 151
_ = s.index.sk:select()
-gap_lock_count() -- 1
+gap_lock_count() -- 101
box.commit()
gap_lock_count() -- 0
s:drop()
diff --git a/test/vinyl/write_iterator.result b/test/vinyl/write_iterator.result
index c38de5d3..cf1e426c 100644
--- a/test/vinyl/write_iterator.result
+++ b/test/vinyl/write_iterator.result
@@ -741,6 +741,11 @@ space:drop()
s = box.schema.space.create('test', {engine = 'vinyl'})
---
...
+-- Install on_replace trigger to disable DELETE optimization
+-- in the secondary index (gh-2129).
+_ = s:on_replace(function() end)
+---
+...
pk = s:create_index('primary', {run_count_per_level = 1})
---
...
diff --git a/test/vinyl/write_iterator.test.lua b/test/vinyl/write_iterator.test.lua
index 73c90c42..a1de240f 100644
--- a/test/vinyl/write_iterator.test.lua
+++ b/test/vinyl/write_iterator.test.lua
@@ -317,6 +317,9 @@ space:drop()
-- gh-2875 INSERT+DELETE pairs are annihilated on compaction
s = box.schema.space.create('test', {engine = 'vinyl'})
+-- Install on_replace trigger to disable DELETE optimization
+-- in the secondary index (gh-2129).
+_ = s:on_replace(function() end)
pk = s:create_index('primary', {run_count_per_level = 1})
sk = s:create_index('secondary', {run_count_per_level = 1, parts = {2, 'unsigned'}})
PAD1 = 100
--
2.11.0
^ permalink raw reply [flat|nested] 39+ messages in thread
* [tarantool-patches] Re: [PATCH 08/25] vinyl: remove env argument of vy_check_is_unique_{primary,secondary}
2018-07-27 11:29 ` [PATCH 08/25] vinyl: remove env argument of vy_check_is_unique_{primary,secondary} Vladimir Davydov
@ 2018-07-31 20:45 ` Konstantin Osipov
0 siblings, 0 replies; 39+ messages in thread
From: Konstantin Osipov @ 2018-07-31 20:45 UTC (permalink / raw)
To: tarantool-patches
* Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> Besides vy_check_is_unique, other callers of vy_check_is_unique_primary
> and vy_check_is_unique_secondary are only called when vinyl engine is
> online. So let's move the optimization that skips uniqueness check on
> recovery to vy_check_is_unique and remove the env argument.
OK to push.
--
Konstantin Osipov, Moscow, Russia, +7 903 626 22 32
http://tarantool.io - www.twitter.com/kostja_osipov
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [PATCH 09/25] vinyl: store full tuples in secondary index cache
2018-07-27 11:29 ` [PATCH 09/25] vinyl: store full tuples in secondary index cache Vladimir Davydov
@ 2018-07-31 20:47 ` Konstantin Osipov
0 siblings, 0 replies; 39+ messages in thread
From: Konstantin Osipov @ 2018-07-31 20:47 UTC (permalink / raw)
To: Vladimir Davydov; +Cc: tarantool-patches
* Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> Currently, both vy_read_iterator_next() and vy_point_lookup() add the
> returned tuple to the tuple cache. As a result, we store partial tuples
> in a secondary index tuple cache although we could store full tuples
> (we have to retrieve them anyway when reading a secondary index). This
> means wasting memory. Besides, when the #2129 gets implemented, there
> will be tuples in a secondary index that have to be skipped as they have
> been overwritten in the primary index. Caching them would be inefficient
> and error prone. So let's call vy_cache_add() from the upper level and
> add only full tuples to the cache.
OK to push.
--
Konstantin Osipov, Moscow, Russia, +7 903 626 22 32
http://tarantool.io - www.twitter.com/kostja_osipov
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [PATCH 10/25] vinyl: do not free pending tasks on shutdown
2018-07-27 11:29 ` [PATCH 10/25] vinyl: do not free pending tasks on shutdown Vladimir Davydov
@ 2018-07-31 20:48 ` Konstantin Osipov
0 siblings, 0 replies; 39+ messages in thread
From: Konstantin Osipov @ 2018-07-31 20:48 UTC (permalink / raw)
To: Vladimir Davydov; +Cc: tarantool-patches
* Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> This is a prerequisite for switching scheduler-worker communication from
> pthread mutex/cond to cbus, which in turn is needed to generate and send
> deferred DELETEs from workers back to tx (#2129).
>
> After this patch, pending tasks will be leaked on shutdown. This is OK,
> as we leak a lot of objects on shutdown anyway. The proper way of fixing
> this leak would be to rework shutdown without atexit() so that we can
> use cbus till the very end.
>
> Needed for #2129
> ---
OK to push.
--
Konstantin Osipov, Moscow, Russia, +7 903 626 22 32
http://tarantool.io - www.twitter.com/kostja_osipov
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [PATCH 11/25] vinyl: store pointer to scheduler in struct vy_task
2018-07-27 11:29 ` [PATCH 11/25] vinyl: store pointer to scheduler in struct vy_task Vladimir Davydov
@ 2018-07-31 20:49 ` Konstantin Osipov
0 siblings, 0 replies; 39+ messages in thread
From: Konstantin Osipov @ 2018-07-31 20:49 UTC (permalink / raw)
To: Vladimir Davydov; +Cc: tarantool-patches
* Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> Currently, we don't really need it, but once we switch communication
> channel between the scheduler and workers from pthread mutex/cond to
> cbus (needed for #2129), tasks won't be completed on behalf of the
> scheduler fiber and hence we will need a back pointer from vy_task to
> vy_scheduler.
>
I have already approved it in the earlier version of this patch
set.
--
Konstantin Osipov, Moscow, Russia, +7 903 626 22 32
http://tarantool.io - www.twitter.com/kostja_osipov
^ permalink raw reply [flat|nested] 39+ messages in thread
* [tarantool-patches] Re: [PATCH 23/25] txn: add helper to detect transaction boundaries
2018-07-27 11:30 ` [PATCH 23/25] txn: add helper to detect transaction boundaries Vladimir Davydov
@ 2018-07-31 20:52 ` Konstantin Osipov
0 siblings, 0 replies; 39+ messages in thread
From: Konstantin Osipov @ 2018-07-31 20:52 UTC (permalink / raw)
To: tarantool-patches
* Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> Add txn_is_first_statement() function, which returns true if this is the
> first statement of the transaction. The function is supposed to be used
> from on_replace trigger to detect transaction boundaries.
OK to push.
--
Konstantin Osipov, Moscow, Russia, +7 903 626 22 32
http://tarantool.io - www.twitter.com/kostja_osipov
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [PATCH 24/25] Introduce _vinyl_deferred_delete system space
2018-07-27 11:30 ` [PATCH 24/25] Introduce _vinyl_deferred_delete system space Vladimir Davydov
@ 2018-07-31 20:54 ` Konstantin Osipov
2018-08-01 14:00 ` Vladimir Davydov
0 siblings, 1 reply; 39+ messages in thread
From: Konstantin Osipov @ 2018-07-31 20:54 UTC (permalink / raw)
To: Vladimir Davydov; +Cc: tarantool-patches
* Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> The space is a blackhole. It will be used for writing deferred DELETE
> statements generated by vinyl compaction tasks to WAL so that we can
> recover deferred DELETEs that hadn't been dumped to disk before the
> server was restarted.
Why do you actually need a proto space for it? Please feel free to
put blackhole engine first in the list of engines.
--
Konstantin Osipov, Moscow, Russia, +7 903 626 22 32
http://tarantool.io - www.twitter.com/kostja_osipov
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [PATCH 25/25] vinyl: eliminate disk read on REPLACE/DELETE
2018-07-27 11:30 ` [PATCH 25/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
@ 2018-07-31 20:55 ` Konstantin Osipov
2018-08-01 16:03 ` Vladimir Davydov
2018-08-01 16:51 ` Vladimir Davydov
0 siblings, 2 replies; 39+ messages in thread
From: Konstantin Osipov @ 2018-07-31 20:55 UTC (permalink / raw)
To: Vladimir Davydov; +Cc: tarantool-patches
* Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> When executing a REPLACE or DELETE request for a vinyl space, we need to
> delete the old tuple from secondary indexes if any, e.g. if there's a
> space with the primary index over field 1 and a secondary index over
> field 2 and there's REPLACE{1, 10} in the space, then REPLACE{1, 20} has
> to generate DELETE{10, 1} in order to overwrite REPLACE{10, 1} before
> inserting REPLACE{20, 1} into the secondary index. Currently, we
> generate DELETEs for secondary indexes immediately on request execution,
> which makes REPLACE/DELETE operations disk-bound in case the space has
> secondary indexes, because in order to delete the old tuple we have to
> look it up in the primary index first.
Please salvage Vlad's test coverage.
Please rebase the patch set after pushing the approved patches.
--
Konstantin Osipov, Moscow, Russia, +7 903 626 22 32
http://tarantool.io - www.twitter.com/kostja_osipov
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [PATCH 24/25] Introduce _vinyl_deferred_delete system space
2018-07-31 20:54 ` Konstantin Osipov
@ 2018-08-01 14:00 ` Vladimir Davydov
2018-08-01 20:25 ` [tarantool-patches] " Konstantin Osipov
0 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-08-01 14:00 UTC (permalink / raw)
To: Konstantin Osipov; +Cc: tarantool-patches
On Tue, Jul 31, 2018 at 11:54:27PM +0300, Konstantin Osipov wrote:
> * Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> > The space is a blackhole. It will be used for writing deferred DELETE
> > statements generated by vinyl compaction tasks to WAL so that we can
> > recover deferred DELETEs that hadn't been dumped to disk before the
> > server was restarted.
>
> Why do you actually need a proto space for it? Please feel free to
> put blackhole engine first in the list of engines.
You mean making this space purely virtual? No record in _space, no info
in snap file, just struct space in the cache?
Hmm, I guess we can do that. However, I think that it's kinda weird
to have a space that can be written to, but which doesn't exist in the
data dictionary. This may also result in some anomalies, like the user
can grant access to it although it doesn't actually exist in _space
(yeah, this is pointless, I know, but still).
Is there any reason to do that? Or do you mean something different by
'proto space'?
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [PATCH 25/25] vinyl: eliminate disk read on REPLACE/DELETE
2018-07-31 20:55 ` Konstantin Osipov
@ 2018-08-01 16:03 ` Vladimir Davydov
2018-08-01 16:51 ` Vladimir Davydov
1 sibling, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-08-01 16:03 UTC (permalink / raw)
To: Konstantin Osipov; +Cc: tarantool-patches
On Tue, Jul 31, 2018 at 11:55:52PM +0300, Konstantin Osipov wrote:
> * Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> > When executing a REPLACE or DELETE request for a vinyl space, we need to
> > delete the old tuple from secondary indexes if any, e.g. if there's a
> > space with the primary index over field 1 and a secondary index over
> > field 2 and there's REPLACE{1, 10} in the space, then REPLACE{1, 20} has
> > to generate DELETE{10, 1} in order to overwrite REPLACE{10, 1} before
> > inserting REPLACE{20, 1} into the secondary index. Currently, we
> > generate DELETEs for secondary indexes immediately on request execution,
> > which makes REPLACE/DELETE operations disk-bound in case the space has
> > secondary indexes, because in order to delete the old tuple we have to
> > look it up in the primary index first.
>
> Please salvage Vlad's test coverage.
There's nothing to salvage. Vlad's tests check that
1. REPLACE/DELETE don't do lookups in the primary index.
2. SELECT from a secondary index skips stale tuples.
3. Stale tuples are not stored in the tuple cache.
4. Tuples that were overwritten in a transaction write set
are not committed.
5. Purging stale tuples on compaction.
6. Generation of "deleruns".
The tests added by this patch cover all those cases except 6, which is
irrelevant, and even more: recovery, purging stale tuples on commit,
out of memory handling while generating deferred DELETEs on compaction.
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [PATCH 25/25] vinyl: eliminate disk read on REPLACE/DELETE
2018-07-31 20:55 ` Konstantin Osipov
2018-08-01 16:03 ` Vladimir Davydov
@ 2018-08-01 16:51 ` Vladimir Davydov
1 sibling, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-08-01 16:51 UTC (permalink / raw)
To: Konstantin Osipov; +Cc: tarantool-patches
On Tue, Jul 31, 2018 at 11:55:52PM +0300, Konstantin Osipov wrote:
> Please rebase the patch set after pushing the approved patches.
Done.
^ permalink raw reply [flat|nested] 39+ messages in thread
* [tarantool-patches] Re: [PATCH 24/25] Introduce _vinyl_deferred_delete system space
2018-08-01 14:00 ` Vladimir Davydov
@ 2018-08-01 20:25 ` Konstantin Osipov
2018-08-02 9:43 ` Vladimir Davydov
0 siblings, 1 reply; 39+ messages in thread
From: Konstantin Osipov @ 2018-08-01 20:25 UTC (permalink / raw)
To: tarantool-patches
* Vladimir Davydov <vdavydov.dev@gmail.com> [18/08/01 18:14]:
> On Tue, Jul 31, 2018 at 11:54:27PM +0300, Konstantin Osipov wrote:
> > * Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> > > The space is a blackhole. It will be used for writing deferred DELETE
> > > statements generated by vinyl compaction tasks to WAL so that we can
> > > recover deferred DELETEs that hadn't been dumped to disk before the
> > > server was restarted.
> >
> > Why do you actually need a proto space for it? Please feel free to
> > put blackhole engine first in the list of engines.
>
> You mean making this space purely virtual? No record in _space, no info
> in snap file, just struct space in the cache?
No, I mean no changes in sc_* methods, only a record in _space?
> Hmm, I guess we can do that. However, I think that it's kinda weird
> to have a space that can be written to, but which doesn't exist in the
> data dictionary. This may also result in some anomalies, like the user
> can grant access to it although it doesn't actually exist in _space
> (yeah, this is pointless, I know, but still).
>
> Is there any reason to do that? Or do you mean something different by
> 'proto space'?
--
Konstantin Osipov, Moscow, Russia, +7 903 626 22 32
http://tarantool.io - www.twitter.com/kostja_osipov
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [tarantool-patches] Re: [PATCH 24/25] Introduce _vinyl_deferred_delete system space
2018-08-01 20:25 ` [tarantool-patches] " Konstantin Osipov
@ 2018-08-02 9:43 ` Vladimir Davydov
2018-08-06 8:42 ` Vladimir Davydov
0 siblings, 1 reply; 39+ messages in thread
From: Vladimir Davydov @ 2018-08-02 9:43 UTC (permalink / raw)
To: Konstantin Osipov; +Cc: tarantool-patches
On Wed, Aug 01, 2018 at 11:25:01PM +0300, Konstantin Osipov wrote:
> * Vladimir Davydov <vdavydov.dev@gmail.com> [18/08/01 18:14]:
> > On Tue, Jul 31, 2018 at 11:54:27PM +0300, Konstantin Osipov wrote:
> > > * Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> > > > The space is a blackhole. It will be used for writing deferred DELETE
> > > > statements generated by vinyl compaction tasks to WAL so that we can
> > > > recover deferred DELETEs that hadn't been dumped to disk before the
> > > > server was restarted.
> > >
> > > Why do you actually need a proto space for it? Please feel free to
> > > put blackhole engine first in the list of engines.
> >
> > You mean making this space purely virtual? No record in _space, no info
> > in snap file, just struct space in the cache?
>
> No, I mean no changes in sc_* methods, only a record in _space?
But we need to install on_replace trigger on that space. OTOH we can do
that in vinyl_engine_bootstrap or vinyl_engine_begin_initial_recovery -
obviously the space isn't used during recovery from memtx snapshot, we
only need the trigger on final recovery. Looks reasonable. Is this what
you mean?
^ permalink raw reply [flat|nested] 39+ messages in thread
* Re: [tarantool-patches] Re: [PATCH 24/25] Introduce _vinyl_deferred_delete system space
2018-08-02 9:43 ` Vladimir Davydov
@ 2018-08-06 8:42 ` Vladimir Davydov
0 siblings, 0 replies; 39+ messages in thread
From: Vladimir Davydov @ 2018-08-06 8:42 UTC (permalink / raw)
To: Konstantin Osipov; +Cc: tarantool-patches
On Thu, Aug 02, 2018 at 12:43:56PM +0300, Vladimir Davydov wrote:
> On Wed, Aug 01, 2018 at 11:25:01PM +0300, Konstantin Osipov wrote:
> > * Vladimir Davydov <vdavydov.dev@gmail.com> [18/08/01 18:14]:
> > > On Tue, Jul 31, 2018 at 11:54:27PM +0300, Konstantin Osipov wrote:
> > > > * Vladimir Davydov <vdavydov.dev@gmail.com> [18/07/27 16:55]:
> > > > > The space is a blackhole. It will be used for writing deferred DELETE
> > > > > statements generated by vinyl compaction tasks to WAL so that we can
> > > > > recover deferred DELETEs that hadn't been dumped to disk before the
> > > > > server was restarted.
> > > >
> > > > Why do you actually need a proto space for it? Please feel free to
> > > > put blackhole engine first in the list of engines.
> > >
> > > You mean making this space purely virtual? No record in _space, no info
> > > in snap file, just struct space in the cache?
> >
> > No, I mean no changes in sc_* methods, only a record in _space?
>
> But we need to install on_replace trigger on that space. OTOH we can do
> that in vinyl_engine_bootstrap or vinyl_engine_begin_initial_recovery -
> obviously the space isn't used during recovery from memtx snapshot, we
> only need the trigger on final recovery. Looks reasonable. Is this what
> you mean?
I'm afraid we can't do that. The problem is _vinyl_deferred_delete space
may not exist on recovery (when engine_begin_final_recovery is called),
because box.schema.upgrade() hasn't been called yet. In this case we
wouldn't be able to install the trigger as there's no proto space. We
could probably disable the optimization until box.schema.upgrade() is
called, but we wouldn't be able to enable it until restart... unless we
detected creation of _vinyl_deferred_delete space in alter.cc, but that
would be cumbersome IMO.
That said, I guess, we have to create a proto space after all. In order
not to clutter schema.cc we can probably do that in vinyl.c ...
^ permalink raw reply [flat|nested] 39+ messages in thread
end of thread, other threads:[~2018-08-06 8:42 UTC | newest]
Thread overview: 39+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-07-27 11:29 [PATCH 00/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
2018-07-27 11:29 ` [PATCH 01/25] vinyl: make point lookup always return the latest tuple version Vladimir Davydov
2018-07-27 11:29 ` [PATCH 02/25] vinyl: simplify vy_squash_process Vladimir Davydov
2018-07-27 11:29 ` [PATCH 03/25] vinyl: always get full tuple from pk after reading from secondary index Vladimir Davydov
2018-07-27 11:29 ` [PATCH 04/25] vinyl: fold vy_replace_one and vy_replace_impl Vladimir Davydov
2018-07-27 11:29 ` [PATCH 05/25] vinyl: fold vy_delete_impl Vladimir Davydov
2018-07-27 11:29 ` [PATCH 06/25] vinyl: refactor unique check Vladimir Davydov
2018-07-27 11:29 ` [PATCH 07/25] vinyl: check key uniqueness before modifying tx write set Vladimir Davydov
2018-07-27 11:29 ` [PATCH 08/25] vinyl: remove env argument of vy_check_is_unique_{primary,secondary} Vladimir Davydov
2018-07-31 20:45 ` [tarantool-patches] " Konstantin Osipov
2018-07-27 11:29 ` [PATCH 09/25] vinyl: store full tuples in secondary index cache Vladimir Davydov
2018-07-31 20:47 ` Konstantin Osipov
2018-07-27 11:29 ` [PATCH 10/25] vinyl: do not free pending tasks on shutdown Vladimir Davydov
2018-07-31 20:48 ` Konstantin Osipov
2018-07-27 11:29 ` [PATCH 11/25] vinyl: store pointer to scheduler in struct vy_task Vladimir Davydov
2018-07-31 20:49 ` Konstantin Osipov
2018-07-27 11:29 ` [PATCH 12/25] vinyl: rename some members of vy_scheduler and vy_task struct Vladimir Davydov
2018-07-27 11:29 ` [PATCH 13/25] vinyl: use cbus for communication between scheduler and worker threads Vladimir Davydov
2018-07-27 11:29 ` [PATCH 14/25] vinyl: zap vy_scheduler::is_worker_pool_running Vladimir Davydov
2018-07-27 11:29 ` [PATCH 15/25] vinyl: rename vy_task::status to is_failed Vladimir Davydov
2018-07-27 11:29 ` [PATCH 16/25] xrow: allow to store flags in DML requests Vladimir Davydov
2018-07-27 11:29 ` [PATCH 17/25] vinyl: pin last statement returned by write iterator explicitly Vladimir Davydov
2018-07-27 11:29 ` [PATCH 18/25] vinyl: teach write iterator to return overwritten tuples Vladimir Davydov
2018-07-27 11:29 ` [PATCH 19/25] vinyl: prepare write iterator heap comparator for deferred DELETEs Vladimir Davydov
2018-07-27 11:30 ` [PATCH 20/25] vinyl: allow to skip certain statements on read Vladimir Davydov
2018-07-27 11:30 ` [PATCH 21/25] vinyl: add function to create surrogate deletes from raw msgpack Vladimir Davydov
2018-07-27 11:30 ` [PATCH 22/25] vinyl: remove pointless assertion from vy_stmt_new_surrogate_delete Vladimir Davydov
2018-07-27 11:30 ` [PATCH 23/25] txn: add helper to detect transaction boundaries Vladimir Davydov
2018-07-31 20:52 ` [tarantool-patches] " Konstantin Osipov
2018-07-27 11:30 ` [PATCH 24/25] Introduce _vinyl_deferred_delete system space Vladimir Davydov
2018-07-31 20:54 ` Konstantin Osipov
2018-08-01 14:00 ` Vladimir Davydov
2018-08-01 20:25 ` [tarantool-patches] " Konstantin Osipov
2018-08-02 9:43 ` Vladimir Davydov
2018-08-06 8:42 ` Vladimir Davydov
2018-07-27 11:30 ` [PATCH 25/25] vinyl: eliminate disk read on REPLACE/DELETE Vladimir Davydov
2018-07-31 20:55 ` Konstantin Osipov
2018-08-01 16:03 ` Vladimir Davydov
2018-08-01 16:51 ` Vladimir Davydov
This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox