Tarantool development patches archive
 help / color / mirror / Atom feed
* [Tarantool-patches] [RFC v30 0/3] qsync: implement packet filtering (part 1)
@ 2022-02-24 20:18 Cyrill Gorcunov via Tarantool-patches
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 1/3] latch: add latch_is_locked helper Cyrill Gorcunov via Tarantool-patches
                   ` (2 more replies)
  0 siblings, 3 replies; 7+ messages in thread
From: Cyrill Gorcunov via Tarantool-patches @ 2022-02-24 20:18 UTC (permalink / raw)
  To: tml; +Cc: Vladislav Shpilevoy

Guys, take a look please, once time permit. Comments are welcome!

v29:
 - rework test into luaform
 - drop fine-grained locks idea since it requires too much code churn,
   instead lets fence out a big code parts

v30:
 - addess commenst (redundant code modifications due to more general
   locking scheme)
 - leave `latched` status for transactions which are started to
   process (ie entered limbo processing) but not yet complete due
   to WAL stalls, since I need to test exactly such state
 - extend the test

branch gorcunov/gh-6036-rollback-confirm-30
issue https://github.com/tarantool/tarantool/issues/6036
previous series https://lists.tarantool.org/tarantool-patches/20220131215554.1367429-1-gorcunov@gmail.com/

Cyrill Gorcunov (3):
  latch: add latch_is_locked helper
  qsync: order access to the limbo terms
  test: add gh-6036-qsync-order test

 src/box/applier.cc                            |   6 +-
 src/box/box.cc                                |   8 +-
 src/box/lua/info.c                            |   4 +-
 src/box/txn_limbo.c                           |  16 +-
 src/box/txn_limbo.h                           |  51 +++++-
 src/lib/core/latch.h                          |  11 ++
 .../gh_6036_qsync_order_test.lua              | 157 ++++++++++++++++++
 test/replication-luatest/suite.ini            |   1 +
 8 files changed, 245 insertions(+), 9 deletions(-)
 create mode 100644 test/replication-luatest/gh_6036_qsync_order_test.lua


base-commit: 3a98e0794e06bee9ecccbe2bc875e87f612d5560
-- 
2.35.1


^ permalink raw reply	[flat|nested] 7+ messages in thread

* [Tarantool-patches] [PATCH v30 1/3] latch: add latch_is_locked helper
  2022-02-24 20:18 [Tarantool-patches] [RFC v30 0/3] qsync: implement packet filtering (part 1) Cyrill Gorcunov via Tarantool-patches
@ 2022-02-24 20:18 ` Cyrill Gorcunov via Tarantool-patches
  2022-02-28  8:13   ` Serge Petrenko via Tarantool-patches
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 2/3] qsync: order access to the limbo terms Cyrill Gorcunov via Tarantool-patches
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 3/3] test: add gh-6036-qsync-order test Cyrill Gorcunov via Tarantool-patches
  2 siblings, 1 reply; 7+ messages in thread
From: Cyrill Gorcunov via Tarantool-patches @ 2022-02-24 20:18 UTC (permalink / raw)
  To: tml; +Cc: Vladislav Shpilevoy

To test if latch is locked.

Part-of #6036

Signed-off-by: Cyrill Gorcunov <gorcunov@gmail.com>
---
 src/lib/core/latch.h | 11 +++++++++++
 1 file changed, 11 insertions(+)

diff --git a/src/lib/core/latch.h b/src/lib/core/latch.h
index 49c59cf63..0aaa8b634 100644
--- a/src/lib/core/latch.h
+++ b/src/lib/core/latch.h
@@ -95,6 +95,17 @@ latch_owner(struct latch *l)
 	return l->owner;
 }
 
+/**
+ * Return true if the latch is locked.
+ *
+ * @param l - latch to be tested.
+ */
+static inline bool
+latch_is_locked(const struct latch *l)
+{
+	return l->owner != NULL;
+}
+
 /**
  * Lock a latch. If the latch is already locked by another fiber,
  * waits for timeout.
-- 
2.35.1


^ permalink raw reply	[flat|nested] 7+ messages in thread

* [Tarantool-patches] [PATCH v30 2/3] qsync: order access to the limbo terms
  2022-02-24 20:18 [Tarantool-patches] [RFC v30 0/3] qsync: implement packet filtering (part 1) Cyrill Gorcunov via Tarantool-patches
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 1/3] latch: add latch_is_locked helper Cyrill Gorcunov via Tarantool-patches
@ 2022-02-24 20:18 ` Cyrill Gorcunov via Tarantool-patches
  2022-02-28  8:24   ` Serge Petrenko via Tarantool-patches
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 3/3] test: add gh-6036-qsync-order test Cyrill Gorcunov via Tarantool-patches
  2 siblings, 1 reply; 7+ messages in thread
From: Cyrill Gorcunov via Tarantool-patches @ 2022-02-24 20:18 UTC (permalink / raw)
  To: tml; +Cc: Vladislav Shpilevoy

Limbo terms tracking is shared between appliers and when
one of appliers is waiting for write to complete inside
journal_write() routine, an other may need to access read
term value to figure out if promote request is valid to
apply. Due to cooperative multitasking access to the terms
is not consistent so we need to be sure that other fibers
read up to date terms (ie written to the WAL).

For this sake we use a latching mechanism, when one fiber
takes a lock for updating other readers are waiting until
the operation is complete.

For example here is a call graph of two appliers

applier 1
---------
applier_apply_tx
  (promote term = 3
   current max term = 2)
  applier_synchro_filter_tx
  apply_synchro_row
    journal_write
      (sleeping)

at this moment another applier comes in with obsolete
data and term 2

                              applier 2
                              ---------
                              applier_apply_tx
                                (term 2)
                                applier_synchro_filter_tx
                                  txn_limbo_is_replica_outdated -> false
                                journal_write (sleep)

applier 1
---------
journal wakes up
  apply_synchro_row_cb
    set max term to 3

So the applier 2 didn't notice that term 3 is already seen
and wrote obsolete data. With locking the applier 2 will
wait until applier 1 has finished its write.

We introduce the following helpers:

1) txn_limbo_begin: which takes a lock
2) txn_limbo_commit and txn_limbo_rollback which simply release
   the lock but have different names for better semantics
3) txn_limbo_process is a general function which uses x_begin
   and x_commit helper internally
4) txn_limbo_apply to do a real job over processing the
   request, it implies that txn_limbo_begin been called

Testing such in-flight condition won't be easy so we introduce
"box.info.synchro.queue.latched" field to report if limbo is
currently latched and processing a sync request.

@TarantoolBot document
Title: synchronous replication changes

`box.info.synchro.queue` gets a new `latched` field.
It is set to `true` when there is a synchronous transaction is
processing but not yet complete. Thus any other incoming synchronous
transactions will be delayed until active one is finished.

Part-of #6036
---
 src/box/applier.cc  |  6 +++++-
 src/box/box.cc      |  8 +++++--
 src/box/lua/info.c  |  4 +++-
 src/box/txn_limbo.c | 16 +++++++++++++-
 src/box/txn_limbo.h | 51 +++++++++++++++++++++++++++++++++++++++++----
 5 files changed, 76 insertions(+), 9 deletions(-)

diff --git a/src/box/applier.cc b/src/box/applier.cc
index 0ac7ac3c7..99484eea2 100644
--- a/src/box/applier.cc
+++ b/src/box/applier.cc
@@ -959,7 +959,7 @@ apply_synchro_req_cb(struct journal_entry *entry)
 		applier_rollback_by_wal_io(entry->res);
 	} else {
 		replica_txn_wal_write_cb(synchro_entry->rcb);
-		txn_limbo_process(&txn_limbo, synchro_entry->req);
+		txn_limbo_apply(&txn_limbo, synchro_entry->req);
 		trigger_run(&replicaset.applier.on_wal_write, NULL);
 	}
 	fiber_wakeup(synchro_entry->owner);
@@ -1004,14 +1004,18 @@ apply_synchro_req(uint32_t replica_id, struct xrow_header *row, struct synchro_r
 	 * before trying to commit. But that requires extra steps from the
 	 * transactions side, including the async ones.
 	 */
+	txn_limbo_begin(&txn_limbo);
 	if (journal_write(&entry.base) != 0)
 		goto err;
 	if (entry.base.res < 0) {
 		diag_set_journal_res(entry.base.res);
 		goto err;
 	}
+	txn_limbo_commit(&txn_limbo);
 	return 0;
+
 err:
+	txn_limbo_rollback(&txn_limbo);
 	diag_log();
 	return -1;
 }
diff --git a/src/box/box.cc b/src/box/box.cc
index 6a33203df..fd34ddb0c 100644
--- a/src/box/box.cc
+++ b/src/box/box.cc
@@ -1780,6 +1780,7 @@ box_issue_promote(uint32_t prev_leader_id, int64_t promote_lsn)
 	struct raft *raft = box_raft();
 	assert(raft->volatile_term == raft->term);
 	assert(promote_lsn >= 0);
+	txn_limbo_begin(&txn_limbo);
 	txn_limbo_write_promote(&txn_limbo, promote_lsn,
 				raft->term);
 	struct synchro_request req = {
@@ -1789,7 +1790,8 @@ box_issue_promote(uint32_t prev_leader_id, int64_t promote_lsn)
 		.lsn = promote_lsn,
 		.term = raft->term,
 	};
-	txn_limbo_process(&txn_limbo, &req);
+	txn_limbo_apply(&txn_limbo, &req);
+	txn_limbo_commit(&txn_limbo);
 	assert(txn_limbo_is_empty(&txn_limbo));
 }
 
@@ -1802,6 +1804,7 @@ box_issue_demote(uint32_t prev_leader_id, int64_t promote_lsn)
 {
 	assert(box_raft()->volatile_term == box_raft()->term);
 	assert(promote_lsn >= 0);
+	txn_limbo_begin(&txn_limbo);
 	txn_limbo_write_demote(&txn_limbo, promote_lsn,
 				box_raft()->term);
 	struct synchro_request req = {
@@ -1811,7 +1814,8 @@ box_issue_demote(uint32_t prev_leader_id, int64_t promote_lsn)
 		.lsn = promote_lsn,
 		.term = box_raft()->term,
 	};
-	txn_limbo_process(&txn_limbo, &req);
+	txn_limbo_apply(&txn_limbo, &req);
+	txn_limbo_commit(&txn_limbo);
 	assert(txn_limbo_is_empty(&txn_limbo));
 }
 
diff --git a/src/box/lua/info.c b/src/box/lua/info.c
index b75a301eb..4b6df2e59 100644
--- a/src/box/lua/info.c
+++ b/src/box/lua/info.c
@@ -637,11 +637,13 @@ lbox_info_synchro(struct lua_State *L)
 
 	/* Queue information. */
 	struct txn_limbo *queue = &txn_limbo;
-	lua_createtable(L, 0, 2);
+	lua_createtable(L, 0, 3);
 	lua_pushnumber(L, queue->len);
 	lua_setfield(L, -2, "len");
 	lua_pushnumber(L, queue->owner_id);
 	lua_setfield(L, -2, "owner");
+	lua_pushboolean(L, queue->promote_is_latched);
+	lua_setfield(L, -2, "latched");
 	lua_setfield(L, -2, "queue");
 
 	return 1;
diff --git a/src/box/txn_limbo.c b/src/box/txn_limbo.c
index 70447caaf..7607e084b 100644
--- a/src/box/txn_limbo.c
+++ b/src/box/txn_limbo.c
@@ -47,6 +47,8 @@ txn_limbo_create(struct txn_limbo *limbo)
 	vclock_create(&limbo->vclock);
 	vclock_create(&limbo->promote_term_map);
 	limbo->promote_greatest_term = 0;
+	latch_create(&limbo->promote_latch);
+	limbo->promote_is_latched = false;
 	limbo->confirmed_lsn = 0;
 	limbo->rollback_count = 0;
 	limbo->is_in_rollback = false;
@@ -724,8 +726,11 @@ txn_limbo_wait_empty(struct txn_limbo *limbo, double timeout)
 }
 
 void
-txn_limbo_process(struct txn_limbo *limbo, const struct synchro_request *req)
+txn_limbo_apply(struct txn_limbo *limbo,
+		const struct synchro_request *req)
 {
+	assert(latch_is_locked(&limbo->promote_latch));
+
 	uint64_t term = req->term;
 	uint32_t origin = req->origin_id;
 	if (txn_limbo_replica_term(limbo, origin) < term) {
@@ -786,6 +791,15 @@ txn_limbo_process(struct txn_limbo *limbo, const struct synchro_request *req)
 	return;
 }
 
+void
+txn_limbo_process(struct txn_limbo *limbo,
+		  const struct synchro_request *req)
+{
+	txn_limbo_begin(limbo);
+	txn_limbo_apply(limbo, req);
+	txn_limbo_commit(limbo);
+}
+
 void
 txn_limbo_on_parameters_change(struct txn_limbo *limbo)
 {
diff --git a/src/box/txn_limbo.h b/src/box/txn_limbo.h
index 53e52f676..1f5da7abb 100644
--- a/src/box/txn_limbo.h
+++ b/src/box/txn_limbo.h
@@ -31,6 +31,7 @@
  */
 #include "small/rlist.h"
 #include "vclock/vclock.h"
+#include "latch.h"
 
 #include <stdint.h>
 
@@ -147,6 +148,14 @@ struct txn_limbo {
 	 * limbo and raft are in sync and the terms are the same.
 	 */
 	uint64_t promote_greatest_term;
+	/**
+	 * To order access to the promote data.
+	 */
+	struct latch promote_latch;
+	/**
+	 * A flag to inform if limbo is locked (for tests mostly).
+	 */
+	bool promote_is_latched;
 	/**
 	 * Maximal LSN gathered quorum and either already confirmed in WAL, or
 	 * whose confirmation is in progress right now. Any attempt to confirm
@@ -226,11 +235,14 @@ txn_limbo_replica_term(const struct txn_limbo *limbo, uint32_t replica_id)
  * data from it. The check is only valid when elections are enabled.
  */
 static inline bool
-txn_limbo_is_replica_outdated(const struct txn_limbo *limbo,
+txn_limbo_is_replica_outdated(struct txn_limbo *limbo,
 			      uint32_t replica_id)
 {
-	return txn_limbo_replica_term(limbo, replica_id) <
-	       limbo->promote_greatest_term;
+	latch_lock(&limbo->promote_latch);
+	uint64_t v = vclock_get(&limbo->promote_term_map, replica_id);
+	bool res = v < limbo->promote_greatest_term;
+	latch_unlock(&limbo->promote_latch);
+	return res;
 }
 
 /**
@@ -300,7 +312,38 @@ txn_limbo_ack(struct txn_limbo *limbo, uint32_t replica_id, int64_t lsn);
 int
 txn_limbo_wait_complete(struct txn_limbo *limbo, struct txn_limbo_entry *entry);
 
-/** Execute a synchronous replication request. */
+/**
+ * Initiate execution of a synchronous replication request.
+ */
+static inline void
+txn_limbo_begin(struct txn_limbo *limbo)
+{
+	latch_lock(&limbo->promote_latch);
+	limbo->promote_is_latched = true;
+}
+
+/** Commit a synchronous replication request. */
+static inline void
+txn_limbo_commit(struct txn_limbo *limbo)
+{
+	limbo->promote_is_latched = false;
+	latch_unlock(&limbo->promote_latch);
+}
+
+/** Rollback a synchronous replication request. */
+static inline void
+txn_limbo_rollback(struct txn_limbo *limbo)
+{
+	limbo->promote_is_latched = false;
+	latch_unlock(&limbo->promote_latch);
+}
+
+/** Apply a synchronous replication request after processing stage. */
+void
+txn_limbo_apply(struct txn_limbo *limbo,
+		const struct synchro_request *req);
+
+/** Process a synchronous replication request. */
 void
 txn_limbo_process(struct txn_limbo *limbo, const struct synchro_request *req);
 
-- 
2.35.1


^ permalink raw reply	[flat|nested] 7+ messages in thread

* [Tarantool-patches] [PATCH v30 3/3] test: add gh-6036-qsync-order test
  2022-02-24 20:18 [Tarantool-patches] [RFC v30 0/3] qsync: implement packet filtering (part 1) Cyrill Gorcunov via Tarantool-patches
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 1/3] latch: add latch_is_locked helper Cyrill Gorcunov via Tarantool-patches
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 2/3] qsync: order access to the limbo terms Cyrill Gorcunov via Tarantool-patches
@ 2022-02-24 20:18 ` Cyrill Gorcunov via Tarantool-patches
  2022-02-28  8:13   ` Serge Petrenko via Tarantool-patches
  2 siblings, 1 reply; 7+ messages in thread
From: Cyrill Gorcunov via Tarantool-patches @ 2022-02-24 20:18 UTC (permalink / raw)
  To: tml; +Cc: Vladislav Shpilevoy

To test that promotion requests are handled only when appropriate
write to WAL completes, because we update memory data before the
write finishes.

Part-of #6036

Signed-off-by: Cyrill Gorcunov <gorcunov@gmail.com>
---
 .../gh_6036_qsync_order_test.lua              | 157 ++++++++++++++++++
 test/replication-luatest/suite.ini            |   1 +
 2 files changed, 158 insertions(+)
 create mode 100644 test/replication-luatest/gh_6036_qsync_order_test.lua

diff --git a/test/replication-luatest/gh_6036_qsync_order_test.lua b/test/replication-luatest/gh_6036_qsync_order_test.lua
new file mode 100644
index 000000000..95ed3a517
--- /dev/null
+++ b/test/replication-luatest/gh_6036_qsync_order_test.lua
@@ -0,0 +1,157 @@
+local t = require('luatest')
+local cluster = require('test.luatest_helpers.cluster')
+local server = require('test.luatest_helpers.server')
+local fiber = require('fiber')
+
+local g = t.group('gh-6036')
+
+g.before_each(function(cg)
+    cg.cluster = cluster:new({})
+
+    local box_cfg = {
+        replication = {
+            server.build_instance_uri('r1'),
+            server.build_instance_uri('r2'),
+            server.build_instance_uri('r3'),
+        },
+        replication_timeout         = 0.1,
+        replication_connect_quorum  = 1,
+        election_mode               = 'manual',
+        election_timeout            = 0.1,
+        replication_synchro_quorum  = 1,
+        replication_synchro_timeout = 0.1,
+        log_level                   = 6,
+    }
+
+    cg.r1 = cg.cluster:build_server({ alias = 'r1', box_cfg = box_cfg })
+    cg.r2 = cg.cluster:build_server({ alias = 'r2', box_cfg = box_cfg })
+    cg.r3 = cg.cluster:build_server({ alias = 'r3', box_cfg = box_cfg })
+
+    cg.cluster:add_server(cg.r1)
+    cg.cluster:add_server(cg.r2)
+    cg.cluster:add_server(cg.r3)
+    cg.cluster:start()
+end)
+
+g.after_each(function(cg)
+    cg.cluster:drop()
+    cg.cluster.servers = nil
+end)
+
+g.test_qsync_order = function(cg)
+    cg.cluster:wait_fullmesh()
+
+    --
+    -- Create a synchro space on the r1 node and make
+    -- sure the write processed just fine.
+    cg.r1:exec(function()
+        box.ctl.promote()
+        box.ctl.wait_rw()
+        local s = box.schema.create_space('test', {is_sync = true})
+        s:create_index('pk')
+        s:insert{1}
+    end)
+
+    local vclock = cg.r1:get_vclock()
+    vclock[0] = nil
+    cg.r2:wait_vclock(vclock)
+    cg.r3:wait_vclock(vclock)
+
+    t.assert_equals(cg.r1:eval("return box.space.test:select()"), {{1}})
+    t.assert_equals(cg.r2:eval("return box.space.test:select()"), {{1}})
+    t.assert_equals(cg.r3:eval("return box.space.test:select()"), {{1}})
+
+    local function update_replication(...)
+        return (box.cfg{ replication = { ... } })
+    end
+
+    --
+    -- Drop connection between r1 and r2.
+    cg.r1:exec(update_replication, {
+            server.build_instance_uri("r1"),
+            server.build_instance_uri("r3"),
+        })
+
+    --
+    -- Drop connection between r2 and r1.
+    cg.r2:exec(update_replication, {
+        server.build_instance_uri("r2"),
+        server.build_instance_uri("r3"),
+    })
+
+    --
+    -- Here we have the following scheme
+    --
+    --      r3 (WAL delay)
+    --      /            \
+    --    r1              r2
+    --
+
+    --
+    -- Initiate disk delay in a bit tricky way: the next write will
+    -- fall into forever sleep.
+    cg.r3:eval("box.error.injection.set('ERRINJ_WAL_DELAY', true)")
+
+    --
+    -- Make r2 been a leader and start writting data, the PROMOTE
+    -- request get queued on r3 and not yet processed, same time
+    -- the INSERT won't complete either waiting for the PROMOTE
+    -- completion first. Note that we enter r3 as well just to be
+    -- sure the PROMOTE has reached it via queue state test.
+    cg.r2:exec(function()
+        box.ctl.promote()
+        box.ctl.wait_rw()
+    end)
+    t.helpers.retrying({}, function()
+        assert(cg.r3:exec(function()
+            return box.info.synchro.queue.latched == true
+        end))
+    end)
+    cg.r2:eval("box.space.test:insert{2}")
+
+    --
+    -- The r1 node has no clue that there is a new leader and continue
+    -- writing data with obsolete term. Since r3 is delayed now
+    -- the INSERT won't proceed yet but get queued.
+    cg.r1:eval("box.space.test:insert{3}")
+
+    --
+    -- Finally enable r3 back. Make sure the data from new r2 leader get
+    -- writing while old leader's data ignored.
+    cg.r3:eval("box.error.injection.set('ERRINJ_WAL_DELAY', false)")
+    t.helpers.retrying({}, function()
+        assert(cg.r3:exec(function()
+            return box.space.test:get{2} ~= nil
+        end))
+    end)
+
+    t.assert_equals(cg.r3:eval("return box.space.test:select()"), {{1},{2}})
+
+    --
+    -- Make sure that while we're processing PROMOTE no other records
+    -- get sneaked in via applier code from other replicas. For this
+    -- sake initiate voting and stop inside wal thread just before
+    -- PROMOTE get written. Another replica sends us new record and
+    -- it should be dropped.
+    cg.r1:exec(function()
+        box.ctl.promote()
+        box.ctl.wait_rw()
+    end)
+    vclock = cg.r1:get_vclock()
+    vclock[0] = nil
+    cg.r2:wait_vclock(vclock)
+    cg.r3:wait_vclock(vclock)
+
+    cg.r3:exec(function()
+        box.error.injection.set('ERRINJ_WAL_DELAY_COUNTDOWN', 2)
+        require('fiber').create(function() box.ctl.promote() end)
+    end)
+    cg.r1:eval("box.space.test:insert{4}")
+    cg.r3:exec(function()
+        assert(box.info.synchro.queue.latched == true)
+        box.error.injection.set('ERRINJ_WAL_DELAY', false)
+        box.ctl.wait_rw()
+    end)
+
+    t.assert_equals(cg.r3:eval("return box.space.test:select()"), {{1},{2}})
+end
diff --git a/test/replication-luatest/suite.ini b/test/replication-luatest/suite.ini
index 374f1b87a..07ec93a52 100644
--- a/test/replication-luatest/suite.ini
+++ b/test/replication-luatest/suite.ini
@@ -2,3 +2,4 @@
 core = luatest
 description = replication luatests
 is_parallel = True
+release_disabled = gh_6036_qsync_order_test.lua
-- 
2.35.1


^ permalink raw reply	[flat|nested] 7+ messages in thread

* Re: [Tarantool-patches] [PATCH v30 3/3] test: add gh-6036-qsync-order test
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 3/3] test: add gh-6036-qsync-order test Cyrill Gorcunov via Tarantool-patches
@ 2022-02-28  8:13   ` Serge Petrenko via Tarantool-patches
  0 siblings, 0 replies; 7+ messages in thread
From: Serge Petrenko via Tarantool-patches @ 2022-02-28  8:13 UTC (permalink / raw)
  To: Cyrill Gorcunov, tml; +Cc: Vladislav Shpilevoy



24.02.2022 23:18, Cyrill Gorcunov пишет:
> To test that promotion requests are handled only when appropriate
> write to WAL completes, because we update memory data before the
> write finishes.
>
> Part-of #6036
>
> Signed-off-by: Cyrill Gorcunov <gorcunov@gmail.com>

Thanks for the patch and the fixes overall!

The test finally works fine on my machine.
I've experienced some flakiness, but I was able to fix
that with the following diff. Please, consider:

======================

diff --git a/test/replication-luatest/gh_6036_qsync_order_test.lua 
b/test/replication-luatest/gh_6036_qsync_order_test.lua
index 95ed3a517..d71739dcc 100644
--- a/test/replication-luatest/gh_6036_qsync_order_test.lua
+++ b/test/replication-luatest/gh_6036_qsync_order_test.lua
@@ -142,10 +142,19 @@ g.test_qsync_order = function(cg)
      cg.r2:wait_vclock(vclock)
      cg.r3:wait_vclock(vclock)

+    -- Drop connection between r1 and the rest of the cluster.
+    -- Otherwise r1 might become Raft follower before attempting insert{4}.
+    cg.r1:exec(function() box.cfg{replication=""} end)
      cg.r3:exec(function()
          box.error.injection.set('ERRINJ_WAL_DELAY_COUNTDOWN', 2)
          require('fiber').create(function() box.ctl.promote() end)
      end)
+    t.helpers.retrying({}, function()
+        t.assert(cg.r3:exec(function()
+            return box.info.synchro.queue.latched
+        end))
+    end)
+    t.assert(cg.r1:exec(function() return box.info.ro == false end))
      cg.r1:eval("box.space.test:insert{4}")
      cg.r3:exec(function()
          assert(box.info.synchro.queue.latched == true)

=======================

Also please address a couple of style-related comments below:


> ---
>   .../gh_6036_qsync_order_test.lua              | 157 ++++++++++++++++++
>   test/replication-luatest/suite.ini            |   1 +
>   2 files changed, 158 insertions(+)
>   create mode 100644 test/replication-luatest/gh_6036_qsync_order_test.lua
>
> diff --git a/test/replication-luatest/gh_6036_qsync_order_test.lua b/test/replication-luatest/gh_6036_qsync_order_test.lua
> new file mode 100644
> index 000000000..95ed3a517
> --- /dev/null
> +++ b/test/replication-luatest/gh_6036_qsync_order_test.lua
> @@ -0,0 +1,157 @@
> +local t = require('luatest')
> +local cluster = require('test.luatest_helpers.cluster')
> +local server = require('test.luatest_helpers.server')
> +local fiber = require('fiber')
> +
> +local g = t.group('gh-6036')
> +
> +g.before_each(function(cg)
> +    cg.cluster = cluster:new({})
> +
> +    local box_cfg = {
> +        replication = {
> +            server.build_instance_uri('r1'),
> +            server.build_instance_uri('r2'),
> +            server.build_instance_uri('r3'),
> +        },
> +        replication_timeout         = 0.1,
> +        replication_connect_quorum  = 1,
> +        election_mode               = 'manual',
> +        election_timeout            = 0.1,
> +        replication_synchro_quorum  = 1,
> +        replication_synchro_timeout = 0.1,
> +        log_level                   = 6,
> +    }
> +
> +    cg.r1 = cg.cluster:build_server({ alias = 'r1', box_cfg = box_cfg })
> +    cg.r2 = cg.cluster:build_server({ alias = 'r2', box_cfg = box_cfg })
> +    cg.r3 = cg.cluster:build_server({ alias = 'r3', box_cfg = box_cfg })
> +
> +    cg.cluster:add_server(cg.r1)
> +    cg.cluster:add_server(cg.r2)
> +    cg.cluster:add_server(cg.r3)
> +    cg.cluster:start()
> +end)
> +
> +g.after_each(function(cg)
> +    cg.cluster:drop()
> +    cg.cluster.servers = nil
> +end)
> +
> +g.test_qsync_order = function(cg)
> +    cg.cluster:wait_fullmesh()
> +
> +    --
> +    -- Create a synchro space on the r1 node and make
> +    -- sure the write processed just fine.
> +    cg.r1:exec(function()
> +        box.ctl.promote()
> +        box.ctl.wait_rw()
> +        local s = box.schema.create_space('test', {is_sync = true})
> +        s:create_index('pk')
> +        s:insert{1}
> +    end)
> +
> +    local vclock = cg.r1:get_vclock()
> +    vclock[0] = nil
> +    cg.r2:wait_vclock(vclock)
> +    cg.r3:wait_vclock(vclock)
> +
> +    t.assert_equals(cg.r1:eval("return box.space.test:select()"), {{1}})
> +    t.assert_equals(cg.r2:eval("return box.space.test:select()"), {{1}})
> +    t.assert_equals(cg.r3:eval("return box.space.test:select()"), {{1}})
> +
> +    local function update_replication(...)
> +        return (box.cfg{ replication = { ... } })
> +    end
> +
> +    --
> +    -- Drop connection between r1 and r2.
> +    cg.r1:exec(update_replication, {
> +            server.build_instance_uri("r1"),
> +            server.build_instance_uri("r3"),
> +        })
> +
> +    --
> +    -- Drop connection between r2 and r1.
> +    cg.r2:exec(update_replication, {
> +        server.build_instance_uri("r2"),
> +        server.build_instance_uri("r3"),
> +    })
> +
> +    --
> +    -- Here we have the following scheme
> +    --
> +    --      r3 (WAL delay)
> +    --      /            \
> +    --    r1              r2
> +    --
> +
> +    --
> +    -- Initiate disk delay in a bit tricky way: the next write will
> +    -- fall into forever sleep.
> +    cg.r3:eval("box.error.injection.set('ERRINJ_WAL_DELAY', true)")

1. Sometimes you use 'eval' and sometimes you use 'exec', and I don't see
    a pattern behind that. Please check every case with 'eval' and 
replace it
    with 'exec' when possible.

> +
> +    --
> +    -- Make r2 been a leader and start writting data, the PROMOTE
> +    -- request get queued on r3 and not yet processed, same time
> +    -- the INSERT won't complete either waiting for the PROMOTE
> +    -- completion first. Note that we enter r3 as well just to be
> +    -- sure the PROMOTE has reached it via queue state test.
> +    cg.r2:exec(function()
> +        box.ctl.promote()
> +        box.ctl.wait_rw()
> +    end)
> +    t.helpers.retrying({}, function()
> +        assert(cg.r3:exec(function()
> +            return box.info.synchro.queue.latched == true
> +        end))

2. Here you use a plain 'assert' instead of 't.assert'. Please avoid
    plain assertions in luatest tests.

> +    end)
> +    cg.r2:eval("box.space.test:insert{2}")

3. Like I already mentioned above, could you wrap that into an 'exec' 
instead?

> +
> +    --
> +    -- The r1 node has no clue that there is a new leader and continue
> +    -- writing data with obsolete term. Since r3 is delayed now
> +    -- the INSERT won't proceed yet but get queued.
> +    cg.r1:eval("box.space.test:insert{3}")
> +
> +    --
> +    -- Finally enable r3 back. Make sure the data from new r2 leader get
> +    -- writing while old leader's data ignored.
> +    cg.r3:eval("box.error.injection.set('ERRINJ_WAL_DELAY', false)")
> +    t.helpers.retrying({}, function()
> +        assert(cg.r3:exec(function()
> +            return box.space.test:get{2} ~= nil
> +        end))
> +    end)
> +
> +    t.assert_equals(cg.r3:eval("return box.space.test:select()"), {{1},{2}})
> +

4. You group two tests in one function. Let's better extract the test 
below into
    a separate function. For example, g.test_promote_order, or something.

    First of all, you may get rid of the 3rd instance in this test (you 
only need 2 of them),
    secondly, now you enter the test with a dirty config from the 
previous test:
    r1 <-> r2 <-> r3 (no connection between r1 and r3).

> +    --
> +    -- Make sure that while we're processing PROMOTE no other records
> +    -- get sneaked in via applier code from other replicas. For this
> +    -- sake initiate voting and stop inside wal thread just before
> +    -- PROMOTE get written. Another replica sends us new record and
> +    -- it should be dropped.
> +    cg.r1:exec(function()
> +        box.ctl.promote()
> +        box.ctl.wait_rw()
> +    end)
> +    vclock = cg.r1:get_vclock()
> +    vclock[0] = nil
> +    cg.r2:wait_vclock(vclock)
> +    cg.r3:wait_vclock(vclock)
> +
> +    cg.r3:exec(function()
> +        box.error.injection.set('ERRINJ_WAL_DELAY_COUNTDOWN', 2)
> +        require('fiber').create(function() box.ctl.promote() end)
> +    end)
> +    cg.r1:eval("box.space.test:insert{4}")
> +    cg.r3:exec(function()
> +        assert(box.info.synchro.queue.latched == true)
> +        box.error.injection.set('ERRINJ_WAL_DELAY', false)
> +        box.ctl.wait_rw()
> +    end)
> +
> +    t.assert_equals(cg.r3:eval("return box.space.test:select()"), {{1},{2}})
> +end
> diff --git a/test/replication-luatest/suite.ini b/test/replication-luatest/suite.ini
> index 374f1b87a..07ec93a52 100644
> --- a/test/replication-luatest/suite.ini
> +++ b/test/replication-luatest/suite.ini
> @@ -2,3 +2,4 @@
>   core = luatest
>   description = replication luatests
>   is_parallel = True
> +release_disabled = gh_6036_qsync_order_test.lua

-- 
Serge Petrenko


^ permalink raw reply	[flat|nested] 7+ messages in thread

* Re: [Tarantool-patches] [PATCH v30 1/3] latch: add latch_is_locked helper
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 1/3] latch: add latch_is_locked helper Cyrill Gorcunov via Tarantool-patches
@ 2022-02-28  8:13   ` Serge Petrenko via Tarantool-patches
  0 siblings, 0 replies; 7+ messages in thread
From: Serge Petrenko via Tarantool-patches @ 2022-02-28  8:13 UTC (permalink / raw)
  To: Cyrill Gorcunov, tml; +Cc: Vladislav Shpilevoy



24.02.2022 23:18, Cyrill Gorcunov пишет:
> To test if latch is locked.
>
> Part-of #6036
>
> Signed-off-by: Cyrill Gorcunov <gorcunov@gmail.com>
> ---
>   src/lib/core/latch.h | 11 +++++++++++
>   1 file changed, 11 insertions(+)
>
> diff --git a/src/lib/core/latch.h b/src/lib/core/latch.h
> index 49c59cf63..0aaa8b634 100644
> --- a/src/lib/core/latch.h
> +++ b/src/lib/core/latch.h
> @@ -95,6 +95,17 @@ latch_owner(struct latch *l)
>   	return l->owner;
>   }
>   
> +/**
> + * Return true if the latch is locked.
> + *
> + * @param l - latch to be tested.
> + */
> +static inline bool
> +latch_is_locked(const struct latch *l)
> +{
> +	return l->owner != NULL;
> +}
> +
>   /**
>    * Lock a latch. If the latch is already locked by another fiber,
>    * waits for timeout.

Thanks! LGTM.

-- 
Serge Petrenko


^ permalink raw reply	[flat|nested] 7+ messages in thread

* Re: [Tarantool-patches] [PATCH v30 2/3] qsync: order access to the limbo terms
  2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 2/3] qsync: order access to the limbo terms Cyrill Gorcunov via Tarantool-patches
@ 2022-02-28  8:24   ` Serge Petrenko via Tarantool-patches
  0 siblings, 0 replies; 7+ messages in thread
From: Serge Petrenko via Tarantool-patches @ 2022-02-28  8:24 UTC (permalink / raw)
  To: Cyrill Gorcunov, tml; +Cc: Vladislav Shpilevoy



24.02.2022 23:18, Cyrill Gorcunov пишет:
> ---

Thanks for the fixes!
Only two minor comments left. Please find them below.

Also, please, consider this diff fixing code style:

==============================

diff --git a/src/box/txn_limbo.c b/src/box/txn_limbo.c
index 7607e084b..8634b63b4 100644
--- a/src/box/txn_limbo.c
+++ b/src/box/txn_limbo.c
@@ -726,8 +726,7 @@ txn_limbo_wait_empty(struct txn_limbo *limbo, double 
timeout)
  }

  void
-txn_limbo_apply(struct txn_limbo *limbo,
-               const struct synchro_request *req)
+txn_limbo_apply(struct txn_limbo *limbo, const struct synchro_request *req)
  {
         assert(latch_is_locked(&limbo->promote_latch));

@@ -792,8 +791,7 @@ txn_limbo_apply(struct txn_limbo *limbo,
  }

  void
-txn_limbo_process(struct txn_limbo *limbo,
-                 const struct synchro_request *req)
+txn_limbo_process(struct txn_limbo *limbo, const struct synchro_request 
*req)
  {
         txn_limbo_begin(limbo);
         txn_limbo_apply(limbo, req);
diff --git a/src/box/txn_limbo.h b/src/box/txn_limbo.h
index 1f5da7abb..8c9671ca5 100644
--- a/src/box/txn_limbo.h
+++ b/src/box/txn_limbo.h
@@ -235,8 +235,7 @@ txn_limbo_replica_term(const struct txn_limbo 
*limbo, uint32_t replica_id)
   * data from it. The check is only valid when elections are enabled.
   */
  static inline bool
-txn_limbo_is_replica_outdated(struct txn_limbo *limbo,
-                             uint32_t replica_id)
+txn_limbo_is_replica_outdated(struct txn_limbo *limbo, uint32_t replica_id)
  {
         latch_lock(&limbo->promote_latch);
         uint64_t v = vclock_get(&limbo->promote_term_map, replica_id);

==============================


...

> diff --git a/src/box/txn_limbo.h b/src/box/txn_limbo.h
> index 53e52f676..1f5da7abb 100644
> --- a/src/box/txn_limbo.h
> +++ b/src/box/txn_limbo.h
> @@ -31,6 +31,7 @@
>    */
>   #include "small/rlist.h"
>   #include "vclock/vclock.h"
> +#include "latch.h"
>   
>   #include <stdint.h>
>   
> @@ -147,6 +148,14 @@ struct txn_limbo {
>   	 * limbo and raft are in sync and the terms are the same.
>   	 */
>   	uint64_t promote_greatest_term;
> +	/**
> +	 * To order access to the promote data.
> +	 */
> +	struct latch promote_latch;
> +	/**
> +	 * A flag to inform if limbo is locked (for tests mostly).
> +	 */
> +	bool promote_is_latched;

1. As I can see, you only use `promote_is_latched` to
     push box.info.synchro.queue.latched.

    I suggest you simply use latch_is_locked instead.

2. Also, let's rename `box.info.synchro.queue.latched`
    to something like `box.info.synchro.queue.busy`.
    Feel free to suggest other variants.

    "Latched" just sounds too technical IMO.

>   	/**
>   	 * Maximal LSN gathered quorum and either already confirmed in WAL, or
>   	 * whose confirmation is in progress right now. Any attempt to confirm
> @@ -226,11 +235,14 @@ txn_limbo_replica_term(const struct txn_limbo *limbo, uint32_t replica_id)
>    * data from it. The check is only valid when elections are enabled.
>    */
>   static inline bool
> -txn_limbo_is_replica_outdated(const struct txn_limbo *limbo,
> +txn_limbo_is_replica_outdated(struct txn_limbo *limbo,
>   			      uint32_t replica_id)
>   {
> -	return txn_limbo_replica_term(limbo, replica_id) <
> -	       limbo->promote_greatest_term;
> +	latch_lock(&limbo->promote_latch);
> +	uint64_t v = vclock_get(&limbo->promote_term_map, replica_id);
> +	bool res = v < limbo->promote_greatest_term;
> +	latch_unlock(&limbo->promote_latch);
> +	return res;
>   }
>   
>
...

-- 
Serge Petrenko


^ permalink raw reply	[flat|nested] 7+ messages in thread

end of thread, other threads:[~2022-02-28  8:24 UTC | newest]

Thread overview: 7+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-02-24 20:18 [Tarantool-patches] [RFC v30 0/3] qsync: implement packet filtering (part 1) Cyrill Gorcunov via Tarantool-patches
2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 1/3] latch: add latch_is_locked helper Cyrill Gorcunov via Tarantool-patches
2022-02-28  8:13   ` Serge Petrenko via Tarantool-patches
2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 2/3] qsync: order access to the limbo terms Cyrill Gorcunov via Tarantool-patches
2022-02-28  8:24   ` Serge Petrenko via Tarantool-patches
2022-02-24 20:18 ` [Tarantool-patches] [PATCH v30 3/3] test: add gh-6036-qsync-order test Cyrill Gorcunov via Tarantool-patches
2022-02-28  8:13   ` Serge Petrenko via Tarantool-patches

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox