← Back to team overview

maria-developers team mailing list archive

Progress (by Knielsen): Efficient group commit for binary log (116)

 

-----------------------------------------------------------------------
                              WORKLOG TASK
-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-
TASK...........: Efficient group commit for binary log
CREATION DATE..: Mon, 26 Apr 2010, 13:28
SUPERVISOR.....: Knielsen
IMPLEMENTOR....: 
COPIES TO......: Serg
CATEGORY.......: Server-RawIdeaBin
TASK ID........: 116 (http://askmonty.org/worklog/?tid=116)
VERSION........: Server-9.x
STATUS.........: Un-Assigned
PRIORITY.......: 60
WORKED HOURS...: 60
ESTIMATE.......: 0 (hours remain)
ORIG. ESTIMATE.: 0

PROGRESS NOTES:

-=-=(Knielsen - Mon, 31 May 2010, 06:48)=-=-
Finish first architecture draft (changed my mind a number of times before I was satisfied).
Write up architecture in worklog.
Fix remaining test failures in proof-of-concept patch + implement xtradb part.
Run some benchmarks on proof-of-concept implementation.

Worked 11 hours and estimate 0 hours remain (original estimate increased by 11 hours).

-=-=(Knielsen - Tue, 25 May 2010, 13:19)=-=-
Low Level Design modified.
--- /tmp/wklog.116.old.14255    2010-05-25 13:19:00.000000000 +0000
+++ /tmp/wklog.116.new.14255    2010-05-25 13:19:00.000000000 +0000
@@ -1 +1,363 @@
+1. Changes for ha_commit_trans()
+
+The gut of the code for commit is in the function ha_commit_trans() (and in
+commit_one_phase() which is called from it). This must be extended to use the
+new prepare_ordered(), group_log_xid(), and commit_ordered() calls.
+
+1.1 Atomic queue of committing transactions
+
+To keep the right commit order among participants, we put transactions into a
+queue. The operations on the queue are non-locking:
+
+ - Insert THD at the head of the queue, and return old queue.
+
+    THD *enqueue_atomic(THD *thd)
+
+ - Fetch (and delete) the whole queue.
+
+    THD *atomic_grab_reverse_queue()
+
+These are simple to implement with atomic compare-and-set. Note that there is
+no ABA problem [2], as we do not delete individual elements from the queue, we
+grab the whole queue and replace it with NULL.
+
+A transaction enters the queue when it does prepare_ordered(). This way, the
+scheduling order for prepare_ordered() calls is what determines the sequence
+in the queue and effectively the commit order.
+
+The queue is grabbed by the code doing group_log_xid() and commit_ordered()
+calls. The queue is passed directly to group_log_xid(), and afterwards
+iterated to do individual commit_ordered() calls.
+
+Using a lock-free queue allows prepare_ordered() (for one transaction) to run
+in parallel with commit_ordered (in another transaction), increasing potential
+parallelism.
+
+The queue is simply a linked list of THD objects, linked through a
+THD::next_commit_ordered field. Since we add at the head of the queue, the
+list is actually in reverse order, so must be reversed when we grab and delete
+it.
+
+The reason that enqueue_atomic() returns the old queue is so that we can check
+if an insert goes to the head of the queue. The thread at the head of the
+queue will do the sequential part of group commit for everyone.
+
+
+1.2 Locks
+
+1.2.1 Global LOCK_prepare_ordered
+
+This lock is taken to serialise calls to prepare_ordered(). Note that
+effectively, the commit order is decided by the order in which threads obtain
+this lock.
+
+
+1.2.2 Global LOCK_group_commit and COND_group_commit
+
+This lock is used to protect the serial part of group commit. It is taken
+around the code where we grab the queue, call group_log_xid() on the queue,
+and call commit_ordered() on each element of the queue, to make sure they
+happen serialised and in consistent order. It also protects the variable
+group_commit_queue_busy, which is used when not using group_log_xid() to delay
+running over a new queue until the first queue is completely done.
+
+
+1.2.3 Global LOCK_commit_ordered
+
+This lock is taken around calls to commit_ordered(), to ensure they happen
+serialised.
+
+
+1.2.4 Per-thread thd->LOCK_commit_ordered and thd->COND_commit_ordered
+
+This lock protects the thd->group_commit_ready variable, as well as the
+condition variable used to wake up threads after log_xid() and
+commit_ordered() finishes.
+
+
+1.2.5 Global LOCK_group_commit_queue
+
+This is only used on platforms with no native compare-and-set operations, to
+make the queue operations atomic.
+
+
+1.3 Commit algorithm.
+
+This is the basic algorithm, simplified by
+
+ - omitting some error handling
+
+ - omitting looping over all handlers when invoking handler methods
+
+ - omitting some possible optimisations when not all calls needed (see next
+   section).
+
+ - Omitting the case where no group_log_xid() is used, see below.
+
+---- BEGIN ALGORITHM ----
+    ht->prepare()
+
+    // Call prepare_ordered() and enqueue in correct commit order
+    lock(LOCK_prepare_ordered)
+    ht->prepare_ordered()
+    old_queue= enqueue_atomic(thd)
+    thd->group_commit_ready= FALSE
+    is_group_commit_leader= (old_queue == NULL)
+    unlock(LOCK_prepare_ordered)
+
+    if (is_group_commit_leader)
+
+        // The first in queue handles group commit for everyone
+
+        lock(LOCK_group_commit)
+        // Wait while queue is busy, see below for when this occurs
+        while (group_commit_queue_busy)
+            cond_wait(COND_group_commit)
+
+        // Grab and reverse the queue to get correct order of transactions
+        queue= atomic_grab_reverse_queue()
+
+        // This call will set individual error codes in thd->xid_error
+        // It also sets the cookie for unlog() in thd->xid_cookie
+        group_log_xid(queue)
+
+        lock(LOCK_commit_ordered)
+        for (other IN queue)
+            if (!other->xid_error)
+                ht->commit_ordered()
+        unlock(LOCK_commit_ordered)
+
+        unlock(LOCK_group_commit)
+
+        // Now we are done, so wake up all the others.
+        for (other IN TAIL(queue))
+            lock(other->LOCK_commit_ordered)
+            other->group_commit_ready= TRUE
+            cond_signal(other->COND_commit_ordered)
+            unlock(other->LOCK_commit_ordered)
+    else
+        // If not the leader, just wait until leader did the work for us.
+        lock(thd->LOCK_commit_ordered)
+        while (!thd->group_commit_ready)
+            cond_wait(thd->LOCK_commit_ordered, thd->COND_commit_ordered)
+        unlock(other->LOCK_commit_ordered)
+
+    // Finally do any error reporting now that we're back in own thread.
+    if (thd->xid_error)
+        xid_delayed_error(thd)
+    else
+        ht->commit(thd)
+        unlog(thd->xid_cookie, thd->xid)
+---- END ALGORITHM ----
+
+If the transaction coordinator does not support group_log_xid(), we have to do
+things differently. In this case after the serialisation point at
+prepare_ordered(), we have to parallelise again when running log_xid()
+(otherwise we would loose group commit). But then when log_xid() is done, we
+have to serialise again to check for any error and call commit_ordered() in
+correct sequence for any transaction where log_xid() did not return error.
+
+The central part of the algorithm in this case (when using log_xid()) is:
+
+---- BEGIN ALGORITHM ----
+    cookie= log_xid(thd)
+    error= (cookie == 0)
+
+    if (is_group_commit_leader)
+
+        // The first to enqueue grabs the queue and runs first.
+        // But we must wait until a previous queue run is fully done.
+
+        lock(LOCK_group_commit)
+        while (group_commit_queue_busy)
+            cond_wait(COND_group_commit)
+        queue= atomic_grab_reverse_queue()
+        // The queue will be busy until last thread in it is done.
+        group_commit_queue_busy= TRUE
+        unlock(LOCK_group_commit)
+    else
+        // Not first in queue -> wait for previous one to wake us up.
+        lock(thd->LOCK_commit_ordered)
+        while (!thd->group_commit_ready)
+            cond_wait(thd->LOCK_commit_ordered, thd->COND_commit_ordered)
+        unlock(other->LOCK_commit_ordered)
+
+    if (!error)      // Only if log_xid() was successful
+        lock(LOCK_commit_ordered)
+        ht->commit_ordered()
+        unlock(LOCK_commit_ordered)
+
+    // Wake up the next thread, and release queue in last.
+    next= thd->next_commit_ordered
+
+    if (next)
+        lock(next->LOCK_commit_ordered)
+        next->group_commit_ready= TRUE
+        cond_signal(next->COND_commit_ordered)
+        unlock(next->LOCK_commit_ordered)
+    else
+        lock(LOCK_group_commit)
+        group_commit_queue_busy= FALSE
+        unlock(LOCK_group_commit)
+---- END ALGORITHM ----
+
+There are a number of locks taken in the algorithm, but in the group_log_xid()
+case most of them should be uncontended most of the time. The
+LOCK_group_commit of course will be contended, as new threads queue up waiting
+for the previous group commit (and binlog fsync()) to finish so they can do
+the next group commit. This is the whole point of implementing group commit.
+
+The LOCK_prepare_ordered and LOCK_commit_ordered mutexes should be not much
+contended as long as handlers follow the intension of having the corresponding
+handler calls execute quickly.
+
+The per-thread LOCK_commit_ordered mutexes should not be contended; they are
+only used to wake up a sleeping thread.
+
+
+1.4 Optimisations when not using all three new calls
+
+
+The prepare_ordered(), group_log_xid(), and commit_ordered() methods are
+optional, and if not implemented by a particular handler/transaction
+coordinator, we can optimise the algorithm to take advantage of not having to
+keep ordering for the missing parts.
+
+If there is no prepare_ordered(), then we need not take the
+LOCK_prepare_ordered mutex.
+
+If there is no commit_ordered(), then we need not take the LOCK_commit_ordered
+mutex.
+
+If there is no group_log_xid(), then we only need the queue to ensure same
+ordering of transactions for commit_ordered() as for prepare_ordered(). Thus,
+if either of these (or both) are also not present, we do not need to use the
+queue at all.
+
+
+2. Binlog code changes (log.cc)
+
+
+The bulk of the work needed for the binary log is to extend the code to allow
+group commit to the log. Unlike InnoDB/XtraDB, there is no existing support
+inside the binlog code for group commit.
+
+The existing code runs most of the write + fsync to the binary lock under the
+global LOCK_log mutex, preventing any group commit.
+
+To enable group commit, this code must be split into two parts:
+
+ - one part that runs per transaction, re-writing the embedded event positions
+   for the correct offset, and writing this into the in-memory log cache.
+
+ - another part that writes a set of transactions to the disk, and runs
+   fsync().
+
+Then in group_log_xid(), we can run the first part in a loop over all the
+transactions in the passed-in queue, and run the second part only once.
+
+The binlog code also has other code paths that write into the binlog,
+eg. non-transactional statements. These have to be adapted also to work with
+the new code.
+
+In order to get some group commit facility for these also, we change that part
+of the code in a similar way to ha_commit_trans. We keep another,
+binlog-internal queue of such non-transactional binlog writes, and such writes
+queue up here before sleeping on the LOCK_log mutex. Once a thread obtains the
+LOCK_log, it loops over the queue for the fast part, and does the slow part
+once, then finally wakes up the others in the queue.
+
+In the transactional case in group_log_xid(), before we run the passed-in
+queue, we add any members found in the binlog-internal queue. This allows
+these non-transactional writes to share the group commit.
+
+However, in the case where it is a non-transactional write that gets the
+LOCK_log, the transactional transactions from the ha_commit_trans() queue will
+not be able to take part (they will have to wait for their turn to do another
+fsync). It seems difficult to cleanly let the binlog code grab the queue from
+out of the ha_commit_trans() algorithm. I think the group commit is mostly
+useful in transactional workloads anyway (non-transactional engines will loose
+data anyway in case of crash, so why fsync() after each transaction?)
+
+
+3. XtraDB changes (ha_innodb.cc)
+
+The changes needed in XtraDB are comparatively simple, as XtraDB already
+implements group commit, it just needs to be enabled with the new
+commit_ordered() call.
+
+The existing commit() method already is logically in two parts. The first part
+runs under the prepare_commit_mutex() and must be run in same order as binlog
+commit. This part needs to be moved to commit_ordered(). The second part runs
+after releasing prepare_commit_mutex and does transaction log write+fsync; it
+can remain.
+
+Then the prepare_commit_mutex is removed (and the enable_unsafe_group_commit
+XtraDB option to disable it).
+
+There are two asserts that check that the thread running the first part of
+XtraDB commit is the same as the thread running the other operations for the
+transaction. These have to be removed (as commit_ordered() can run in a
+different thread). Also an error reporting with sql_print_error() has to be
+delayed until commit() time.
+
+
+4. Proof-of-concept implementation
+
+There is a proof-of-concept implementation of this architecture, in the form
+of a quilt patch series [3].
+
+A quick benchmark was done, with sync_binlog=1 and
+innodb_flush_log_at_trx_commit=1. 64 parallel threads doing single-row
+transactions against one table.
+
+Without the patch, we get only 25 queries per second.
+
+With the patch, we get 650 queries per second.
+
+
+5. Open issues/tasks
+
+5.1 XA / other prepare() and commit() call sites.
+
+Check that user-level XA is handled correctly and working. And covered
+sufficiently with tests. Also check that any other calls of ha->prepare() and
+ha->commit() outside of ha_commit_trans() are handled correctly.
+
+5.2 Testing
+
+This worklog needs additions to the test suite, including error inserts to
+check error handling, and synchronisation points to check thread parallelism
+correctness.
+
+
+6. Alternative implementations
+
+ - The binlog code maintains its own extra atomic transaction queue to handle
+   non-transactional commits in a good way together with transactional (with
+   respect to group commit). Alternatively, we could ignore this issue and
+   just give up on group commit for non-transactional statements, for some
+   code simplifications.
+
+ - The binlog code has two ways to prepare end_event and similar, one that
+   uses stack-allocation, and another for when stack allocation is not
+   possible that uses thd->mem_root. Probably the overhead of thd->mem_root is
+   so small that it would make sense to use the same code for both cases.
+
+ - Instead of adding extra fields to THD, we could allocate a separate
+   structure on the thd->mem_root() with the required extra fields (including
+   the THD pointer). Would seem to require initialising mutexes at every
+   commit though.
+
+ - It would probably be a good idea to implement TC_LOG_MMAP::group_log_xid()
+   (should not be hard).
+
+
+-----------------------------------------------------------------------
+
+References:
+
+[2] https://secure.wikimedia.org/wikipedia/en/wiki/ABA_problem
+
+[3] https://knielsen-hq.org/maria/patches.mwl116/
 

-=-=(Knielsen - Tue, 25 May 2010, 13:18)=-=-
High-Level Specification modified.
--- /tmp/wklog.116.old.14249    2010-05-25 13:18:34.000000000 +0000
+++ /tmp/wklog.116.new.14249    2010-05-25 13:18:34.000000000 +0000
@@ -1 +1,157 @@
+The basic idea in group commit is that multiple threads, each handling one
+transaction, prepare for commit and then queue up together waiting to do an
+fsync() on the transaction log. Then once the log is available, a single
+thread does the fsync() + other necessary book-keeping for all of the threads
+at once. After this, the single thread signals the other threads that it's
+done and they can finish up and return success (or failure) from the commit
+operation.
+
+So group commit has a parallel part, and a sequential part. So we need a
+facility for engines/binlog to participate in both the parallel and the
+sequential part.
+
+To do this, we add two new handlerton methods:
+
+    int (*prepare_ordered)(handlerton *hton, THD *thd, bool all);
+    void (*commit_ordered)(handlerton *hton, THD *thd, bool all);
+
+The idea is that the existing prepare() and commit() methods run in the
+parallel part of group commit, and the new prepare_ordered() and
+commit_ordered() run in the sequential part.
+
+The prepare_ordered() method is called after prepare(). The order of
+tranctions that call into prepare_ordered() is guaranteed to be the same among
+all storage engines and binlog, and it is serialised so no two calls can be
+running inside the same engine at the same time.
+
+The commit_ordered() method is called before commit(), and similarly is
+guaranteed to have same transaction order in all participants, and to be
+serialised within one engine.
+
+As the prepare_ordered() and commit_ordered() calls are serialised, the idea
+is that handlers should do the minimum amount of work needed in these calls,
+relaying most of the work (eg. fsync() ...) to prepare() and commit().
+
+As a concrete example, for InnoDB the commit_ordered() method will do the
+first part of commit that fixed the commit order in the transaction log
+buffer, and the commit() method will write the log to disk and fsync()
+it. This split already exists inside the InnoDB code, running before
+respectively after releasing the prepare_commit_mutex.
+
+In addition, the XA transaction coordinator (TC_LOG) is special, since it is
+the one responsible for deciding whether to commit or rollback the
+transaction. For this we need an extra method, since this decision can be done
+only after we know that all prepare() and prepare_ordered() calls succeed, and
+must be done to know whether to call commit_ordered()/commit(), or do rollback.
+
+The existing method for this is TC_LOG::log_xid(). To make implementing group
+commit simpler to implement in a transaction coordinator and more efficient,
+we introduce a new method:
+
+    void group_log_xid(THD *first_thd);
+
+This method runs in the sequential part of group commit. It receives a list of
+transactions to perform log_xid() on, in the correct commit order. (Note that
+TC_LOG can do parallel parts of group commit in its own prepare() and commit()
+methods).
+
+This method can make it easier to implement the group commit in TC_LOG, as it
+gets directly the list of transactions in the right order. Without it, it
+might need to compute such order anyway in a prepare_ordered() method, and the
+server has to create this ordered list anyway to implement the order guarantee
+for prepare_ordered() and commit_ordered().
+
+This group_log_xid() method also is more efficient, as it avoids some
+inter-thread synchronisation. Since group_log_xid() is serialised, we can run
+it together with all the commit_ordered() method calls and need only a single
+sequential code section. With the log_xid() methods, we would need first a
+sequential part for the prepare_ordered() calls, then a parallel part with
+log_xid() calls (to not loose group commit ability for log_xid()), then again
+a sequential part for the commit_ordered() method calls.
+
+The extra synchronisation is needed, as each commit_ordered() call will have
+to wait for log_xid() in one thread (if log_xid() fails then commit_ordered()
+should not be called), and also wait for commit_ordered() to finish in all
+threads handling earlier commits. In effect we will need to bounce the
+execution from one thread to the other among all participants in the group
+commit.
+
+As a consequence of the group_log_xid() optimisation, handlers must be aware
+that the commit_ordered() call can happen in another thread than the one
+running commit() (so thread local storage is not available). This should not
+be a big issue as the THD is available for storing any needed information.
+
+Since group_log_xid() runs for multiple transactions in a single thread, it
+can not do error reporting (my_error()) as that relies on thread local
+storage. Instead it sets an error code in THD::xid_error, and if there is an
+error then later another method will be called (in correct thread context) to
+actually report the error:
+
+    int xid_delayed_error(THD *thd)
+
+The three new methods prepare_ordered(), group_log_xid(), and commit_ordered()
+are optional (as is xid_delayed_error). A storage engine or transaction
+coordinator is free to not implement them if they are not needed. In this case
+there will be no order guarantee for the corresponding stage of group commit
+for that engine. For example, InnoDB needs no ordering of the prepare phase,
+so can omit implementing prepare_ordered(); TC_LOG_MMAP needs no ordering at
+all, so does not need to implement any of them.
+
+Note in particular that all existing engines (/binlog implementations if they
+exist) will work unmodified (and also without any change in group commit
+facilities or commit order guaranteed).
+
+Using these new APIs, the work will be to
+
+ - In ha_commit_trans(), implement the correct semantics for the three new
+   calls.
+
+ - In XtraDB, use the new commit_ordered() call to remove the
+   prepare_commit_mutex (and resurrect group commit) without loosing the
+   consistency with binlog commit order.
+
+ - In log.cc (binlog module), implement group_log_xid() to do group commit of
+   multiple transactions to the binlog with a single shared fsync() call.
+
+-----------------------------------------------------------------------
+Some possible alternative for this worklog:
+
+ - We could eliminate the group_log_xid() method for a simpler API, at the
+   cost of extra synchronisation between threads to do in-order
+   commit_ordered() method calls. This would also allow to call
+   commit_ordered() in the correct thread context.
+
+ - Alternatively, we could eliminate log_xid() and require that all
+   transaction coordinators implement group_log_xid() instead, again for some
+   moderate simplification.
+
+ - At the moment there is no plugin actually using prepare_ordered(), so, it
+   could be removed from the design. But it fits in well, is efficient to
+   implement, and could be useful later (eg. for the requested feature of
+   releasing locks early in InnoDB).
+
+-----------------------------------------------------------------------
+Some possible follow-up projects after this is implemented:
+
+ - Add statistics about how efficient group commit is (#fsyncs/#commits in
+   each engine and binlog).
+
+ - Implement an XtraDB prepare_ordered() methods that can release row locks
+   early (Mark Callaghan from Facebook advocates this, but need to determine
+   exactly how to do this safely).
+
+ - Implement a new crash recovery algorithm that uses the consistent commit
+   ordering to need only fsync() for the binlog. At crash recovery, any
+   missing transactions in an engine is replayed from the correct point in the
+   binlog (this point must be stored transactionally inside the engine, as
+   XtraDB already does today).
+
+ - Implement that START TRANSACTION WITH CONSISTENT SNAPSHOT 1) really gets a
+   consistent snapshow, with same set of committed and not committed
+   transactions in all engines, 2) returns a corresponding consistent binlog
+   position. This should be easy by piggybacking on the synchronisation
+   implemented for ha_commit_trans().
+
+ - Use this in XtraBackup to get consistent binlog position without having to
+   block all updates with FLUSH TABLES WITH READ LOCK.
 

-=-=(Knielsen - Tue, 25 May 2010, 13:18)=-=-
High Level Description modified.
--- /tmp/wklog.116.old.14234    2010-05-25 13:18:07.000000000 +0000
+++ /tmp/wklog.116.new.14234    2010-05-25 13:18:07.000000000 +0000
@@ -21,3 +21,69 @@
     http://kristiannielsen.livejournal.com/12408.html
     http://kristiannielsen.livejournal.com/12553.html
 
+----
+
+Implementing group commit in MySQL faces some challenges from the handler
+plugin architecture:
+
+1. Because storage engine handlers have separate transaction log from the
+mysql binlog (and from each other), there are multiple fsync() calls per
+commit that need the group commit optimisation (2 per participating storage
+engine + 1 for binlog).
+
+2. The code handling commit is split in several places, in main server code
+and in storage engine code. With pluggable binlog it will be split even
+more. This requires a good abstract yet powerful API to be able to implement
+group commit simply and efficiently in plugins without the different parts
+having to rely on iternals of the others.
+
+3. We want the order of commits to be the same in all engines participating in
+multiple transactions. This requirement is the reason that InnoDB currently
+breaks group commit with the infamous prepare_commit_mutex.
+
+While currently there is no server guarantee to get same commit order in
+engines an binlog (except for the InnoDB prepare_commit_mutex hack), there are
+several reasons why this could be desirable:
+
+ - InnoDB hot backup needs to be able to extract a binlog position that is
+   consistent with the hot backup to be able to provision a new slave, and
+   this is impossible without imposing at least partial consistent ordering
+   between InnoDB and binlog.
+
+ - Other backup methods could have similar needs, eg. XtraBackup or
+   `mysqldump --single-transaction`, to have consistent commit order between
+   binlog and storage engines without having to do FLUSH TABLES WITH READ LOCK
+   or similar expensive blocking operation. (other backup methods, like LVM
+   snapshot, don't need consistent commit order, as they can restore
+   out-of-order commits during crash recovery using XA).
+
+ - If we have consistent commit order, we can think about optimising commit to
+   need only one fsync (for binlog); lost commits in storage engines can then
+   be recovered from the binlog at crash recovery by re-playing against the
+   engine from a particular point in the binlog.
+
+ - With consistent commit order, we can get better semantics for START
+   TRANSACTION WITH CONSISTENT SNAPSHOT with multi-engine transactions (and we
+   could even get it to return also a matching binlog position). Currently,
+   this "CONSISTENT SNAPSHOT" can be inconsistent among multiple storage
+   engines.
+
+ - In InnoDB, the performance in the presense of hotspots can be improved if
+   we can release row locks early in the commit phase, but this requires that we
+release them in
+   the same order as commits in the binlog to ensure consistency between
+   master and slaves.
+
+ - There was some discussions around Galera [1] synchroneous replication and
+   global transaction ID that it needed consistent commit order among
+   participating engines.
+
+ - I believe there could be other applications for guaranteed consistent
+   commit order, and that the architecture described in this worklog can
+   implement such guarantee with reasonable overhead.
+
+
+References:
+
+[1] Galera: http://www.codership.com/products/galera_replication
+

-=-=(Knielsen - Tue, 25 May 2010, 08:28)=-=-
More thoughts on and changes to the archtecture. Got to something now that I am satisfied with and
that seems to be able to handle all issues.

Implement new prepare_ordered and commit_ordered handler methods and the logic in ha_commit_trans().
Implement TC_LOG::group_log_xid() method and logic in ha_commit_trans().
Implement XtraDB part, using commit_ordered() rather than prepare_commit_mutex.
Fix test suite failures.
Proof-of-concept patch series complete now.
Do initial benchmark, getting good results. With 64 threads, see 26x improvement in queries-per-sec.

Next step: write up the architecture description.

Worked 21 hours and estimate 0 hours remain (original estimate increased by 21 hours).

-=-=(Knielsen - Wed, 12 May 2010, 06:41)=-=-
Started work on a Quilt patch series, refactoring the binlog code to prepare for implementing the
group commit, and working on the design of group commit in parallel.

Found and fixed several problems in error handling when writing to binlog.

Removed redundant table map version locking.

Split binlog writing into two parts in preparations for group commit. When ready to write to the
binlog, threads enter a queue, and the first thread in the queue handles the binlog writing for
everyone. When it obtains the LOCK_log, it first loops over all threads, executing the first part of
binlog writing (the write(2) syscall essentially). It then runs the second part (fsync(2)
essentially) only once, and then wakes up the remaining threads in the queue.

Still to be done:

Finish the proof-of-concept group commit patch, by 1) implementing the prepare_fast() and
commit_fast() callbacks in handler.cc 2) move the binlog thread enqueue from log_xid() to
binlog_prepare_fast(), 3) move fast part of InnoDB commit to innobase_commit_fast(), removing the
prepare_commit_mutex().

Write up the final design in this worklog.

Evaluate the design to see if we can do better/different.

Think about possible next steps, such as releasing innodb row locks early (in
innobase_prepare_fast), and doing crash recovery by replaying transactions from the binlog (removing
the need for engine durability and 2 of 3 fsync() in commit).
Worked 28 hours and estimate 0 hours remain (original estimate increased by 28 hours).

-=-=(Serg - Mon, 26 Apr 2010, 14:10)=-=-
Observers changed: Serg



DESCRIPTION:

Currently, in order to ensure that the server can recover after a crash to a
state in which storage engines and binary log are consistent with each other,
it is necessary to use XA with durable commits for both storage engines
(innodb_flush_log_at_trx_commit=1) and binary log (sync_binlog=1).

This is _very_ expensive, since the server needs to do three fsync() operations
for every commit, as there is no working group commit when the binary log is
enabled.

The idea is to

 - Implement/fix group commit to work properly with the binary log enabled.

 - (Optionally) avoid the need to fsync() in the engine, and instead rely on
   replaying any lost transactions from the binary log against the engine
   during crash recovery.

For background see these articles:

    http://kristiannielsen.livejournal.com/12254.html
    http://kristiannielsen.livejournal.com/12408.html
    http://kristiannielsen.livejournal.com/12553.html

----

Implementing group commit in MySQL faces some challenges from the handler
plugin architecture:

1. Because storage engine handlers have separate transaction log from the
mysql binlog (and from each other), there are multiple fsync() calls per
commit that need the group commit optimisation (2 per participating storage
engine + 1 for binlog).

2. The code handling commit is split in several places, in main server code
and in storage engine code. With pluggable binlog it will be split even
more. This requires a good abstract yet powerful API to be able to implement
group commit simply and efficiently in plugins without the different parts
having to rely on iternals of the others.

3. We want the order of commits to be the same in all engines participating in
multiple transactions. This requirement is the reason that InnoDB currently
breaks group commit with the infamous prepare_commit_mutex.

While currently there is no server guarantee to get same commit order in
engines an binlog (except for the InnoDB prepare_commit_mutex hack), there are
several reasons why this could be desirable:

 - InnoDB hot backup needs to be able to extract a binlog position that is
   consistent with the hot backup to be able to provision a new slave, and
   this is impossible without imposing at least partial consistent ordering
   between InnoDB and binlog.

 - Other backup methods could have similar needs, eg. XtraBackup or
   `mysqldump --single-transaction`, to have consistent commit order between
   binlog and storage engines without having to do FLUSH TABLES WITH READ LOCK
   or similar expensive blocking operation. (other backup methods, like LVM
   snapshot, don't need consistent commit order, as they can restore
   out-of-order commits during crash recovery using XA).

 - If we have consistent commit order, we can think about optimising commit to
   need only one fsync (for binlog); lost commits in storage engines can then
   be recovered from the binlog at crash recovery by re-playing against the
   engine from a particular point in the binlog.

 - With consistent commit order, we can get better semantics for START
   TRANSACTION WITH CONSISTENT SNAPSHOT with multi-engine transactions (and we
   could even get it to return also a matching binlog position). Currently,
   this "CONSISTENT SNAPSHOT" can be inconsistent among multiple storage
   engines.

 - In InnoDB, the performance in the presense of hotspots can be improved if
   we can release row locks early in the commit phase, but this requires that we
release them in
   the same order as commits in the binlog to ensure consistency between
   master and slaves.

 - There was some discussions around Galera [1] synchroneous replication and
   global transaction ID that it needed consistent commit order among
   participating engines.

 - I believe there could be other applications for guaranteed consistent
   commit order, and that the architecture described in this worklog can
   implement such guarantee with reasonable overhead.


References:

[1] Galera: http://www.codership.com/products/galera_replication


HIGH-LEVEL SPECIFICATION:



The basic idea in group commit is that multiple threads, each handling one
transaction, prepare for commit and then queue up together waiting to do an
fsync() on the transaction log. Then once the log is available, a single
thread does the fsync() + other necessary book-keeping for all of the threads
at once. After this, the single thread signals the other threads that it's
done and they can finish up and return success (or failure) from the commit
operation.

So group commit has a parallel part, and a sequential part. So we need a
facility for engines/binlog to participate in both the parallel and the
sequential part.

To do this, we add two new handlerton methods:

    int (*prepare_ordered)(handlerton *hton, THD *thd, bool all);
    void (*commit_ordered)(handlerton *hton, THD *thd, bool all);

The idea is that the existing prepare() and commit() methods run in the
parallel part of group commit, and the new prepare_ordered() and
commit_ordered() run in the sequential part.

The prepare_ordered() method is called after prepare(). The order of
tranctions that call into prepare_ordered() is guaranteed to be the same among
all storage engines and binlog, and it is serialised so no two calls can be
running inside the same engine at the same time.

The commit_ordered() method is called before commit(), and similarly is
guaranteed to have same transaction order in all participants, and to be
serialised within one engine.

As the prepare_ordered() and commit_ordered() calls are serialised, the idea
is that handlers should do the minimum amount of work needed in these calls,
relaying most of the work (eg. fsync() ...) to prepare() and commit().

As a concrete example, for InnoDB the commit_ordered() method will do the
first part of commit that fixed the commit order in the transaction log
buffer, and the commit() method will write the log to disk and fsync()
it. This split already exists inside the InnoDB code, running before
respectively after releasing the prepare_commit_mutex.

In addition, the XA transaction coordinator (TC_LOG) is special, since it is
the one responsible for deciding whether to commit or rollback the
transaction. For this we need an extra method, since this decision can be done
only after we know that all prepare() and prepare_ordered() calls succeed, and
must be done to know whether to call commit_ordered()/commit(), or do rollback.

The existing method for this is TC_LOG::log_xid(). To make implementing group
commit simpler to implement in a transaction coordinator and more efficient,
we introduce a new method:

    void group_log_xid(THD *first_thd);

This method runs in the sequential part of group commit. It receives a list of
transactions to perform log_xid() on, in the correct commit order. (Note that
TC_LOG can do parallel parts of group commit in its own prepare() and commit()
methods).

This method can make it easier to implement the group commit in TC_LOG, as it
gets directly the list of transactions in the right order. Without it, it
might need to compute such order anyway in a prepare_ordered() method, and the
server has to create this ordered list anyway to implement the order guarantee
for prepare_ordered() and commit_ordered().

This group_log_xid() method also is more efficient, as it avoids some
inter-thread synchronisation. Since group_log_xid() is serialised, we can run
it together with all the commit_ordered() method calls and need only a single
sequential code section. With the log_xid() methods, we would need first a
sequential part for the prepare_ordered() calls, then a parallel part with
log_xid() calls (to not loose group commit ability for log_xid()), then again
a sequential part for the commit_ordered() method calls.

The extra synchronisation is needed, as each commit_ordered() call will have
to wait for log_xid() in one thread (if log_xid() fails then commit_ordered()
should not be called), and also wait for commit_ordered() to finish in all
threads handling earlier commits. In effect we will need to bounce the
execution from one thread to the other among all participants in the group
commit.

As a consequence of the group_log_xid() optimisation, handlers must be aware
that the commit_ordered() call can happen in another thread than the one
running commit() (so thread local storage is not available). This should not
be a big issue as the THD is available for storing any needed information.

Since group_log_xid() runs for multiple transactions in a single thread, it
can not do error reporting (my_error()) as that relies on thread local
storage. Instead it sets an error code in THD::xid_error, and if there is an
error then later another method will be called (in correct thread context) to
actually report the error:

    int xid_delayed_error(THD *thd)

The three new methods prepare_ordered(), group_log_xid(), and commit_ordered()
are optional (as is xid_delayed_error). A storage engine or transaction
coordinator is free to not implement them if they are not needed. In this case
there will be no order guarantee for the corresponding stage of group commit
for that engine. For example, InnoDB needs no ordering of the prepare phase,
so can omit implementing prepare_ordered(); TC_LOG_MMAP needs no ordering at
all, so does not need to implement any of them.

Note in particular that all existing engines (/binlog implementations if they
exist) will work unmodified (and also without any change in group commit
facilities or commit order guaranteed).

Using these new APIs, the work will be to

 - In ha_commit_trans(), implement the correct semantics for the three new
   calls.

 - In XtraDB, use the new commit_ordered() call to remove the
   prepare_commit_mutex (and resurrect group commit) without loosing the
   consistency with binlog commit order.

 - In log.cc (binlog module), implement group_log_xid() to do group commit of
   multiple transactions to the binlog with a single shared fsync() call.

-----------------------------------------------------------------------
Some possible alternative for this worklog:

 - We could eliminate the group_log_xid() method for a simpler API, at the
   cost of extra synchronisation between threads to do in-order
   commit_ordered() method calls. This would also allow to call
   commit_ordered() in the correct thread context.

 - Alternatively, we could eliminate log_xid() and require that all
   transaction coordinators implement group_log_xid() instead, again for some
   moderate simplification.

 - At the moment there is no plugin actually using prepare_ordered(), so, it
   could be removed from the design. But it fits in well, is efficient to
   implement, and could be useful later (eg. for the requested feature of
   releasing locks early in InnoDB).

-----------------------------------------------------------------------
Some possible follow-up projects after this is implemented:

 - Add statistics about how efficient group commit is (#fsyncs/#commits in
   each engine and binlog).

 - Implement an XtraDB prepare_ordered() methods that can release row locks
   early (Mark Callaghan from Facebook advocates this, but need to determine
   exactly how to do this safely).

 - Implement a new crash recovery algorithm that uses the consistent commit
   ordering to need only fsync() for the binlog. At crash recovery, any
   missing transactions in an engine is replayed from the correct point in the
   binlog (this point must be stored transactionally inside the engine, as
   XtraDB already does today).

 - Implement that START TRANSACTION WITH CONSISTENT SNAPSHOT 1) really gets a
   consistent snapshow, with same set of committed and not committed
   transactions in all engines, 2) returns a corresponding consistent binlog
   position. This should be easy by piggybacking on the synchronisation
   implemented for ha_commit_trans().

 - Use this in XtraBackup to get consistent binlog position without having to
   block all updates with FLUSH TABLES WITH READ LOCK.


LOW-LEVEL DESIGN:



1. Changes for ha_commit_trans()

The gut of the code for commit is in the function ha_commit_trans() (and in
commit_one_phase() which is called from it). This must be extended to use the
new prepare_ordered(), group_log_xid(), and commit_ordered() calls.

1.1 Atomic queue of committing transactions

To keep the right commit order among participants, we put transactions into a
queue. The operations on the queue are non-locking:

 - Insert THD at the head of the queue, and return old queue.

    THD *enqueue_atomic(THD *thd)

 - Fetch (and delete) the whole queue.

    THD *atomic_grab_reverse_queue()

These are simple to implement with atomic compare-and-set. Note that there is
no ABA problem [2], as we do not delete individual elements from the queue, we
grab the whole queue and replace it with NULL.

A transaction enters the queue when it does prepare_ordered(). This way, the
scheduling order for prepare_ordered() calls is what determines the sequence
in the queue and effectively the commit order.

The queue is grabbed by the code doing group_log_xid() and commit_ordered()
calls. The queue is passed directly to group_log_xid(), and afterwards
iterated to do individual commit_ordered() calls.

Using a lock-free queue allows prepare_ordered() (for one transaction) to run
in parallel with commit_ordered (in another transaction), increasing potential
parallelism.

The queue is simply a linked list of THD objects, linked through a
THD::next_commit_ordered field. Since we add at the head of the queue, the
list is actually in reverse order, so must be reversed when we grab and delete
it.

The reason that enqueue_atomic() returns the old queue is so that we can check
if an insert goes to the head of the queue. The thread at the head of the
queue will do the sequential part of group commit for everyone.


1.2 Locks

1.2.1 Global LOCK_prepare_ordered

This lock is taken to serialise calls to prepare_ordered(). Note that
effectively, the commit order is decided by the order in which threads obtain
this lock.


1.2.2 Global LOCK_group_commit and COND_group_commit

This lock is used to protect the serial part of group commit. It is taken
around the code where we grab the queue, call group_log_xid() on the queue,
and call commit_ordered() on each element of the queue, to make sure they
happen serialised and in consistent order. It also protects the variable
group_commit_queue_busy, which is used when not using group_log_xid() to delay
running over a new queue until the first queue is completely done.


1.2.3 Global LOCK_commit_ordered

This lock is taken around calls to commit_ordered(), to ensure they happen
serialised.


1.2.4 Per-thread thd->LOCK_commit_ordered and thd->COND_commit_ordered

This lock protects the thd->group_commit_ready variable, as well as the
condition variable used to wake up threads after log_xid() and
commit_ordered() finishes.


1.2.5 Global LOCK_group_commit_queue

This is only used on platforms with no native compare-and-set operations, to
make the queue operations atomic.


1.3 Commit algorithm.

This is the basic algorithm, simplified by

 - omitting some error handling

 - omitting looping over all handlers when invoking handler methods

 - omitting some possible optimisations when not all calls needed (see next
   section).

 - Omitting the case where no group_log_xid() is used, see below.

---- BEGIN ALGORITHM ----
    ht->prepare()

    // Call prepare_ordered() and enqueue in correct commit order
    lock(LOCK_prepare_ordered)
    ht->prepare_ordered()
    old_queue= enqueue_atomic(thd)
    thd->group_commit_ready= FALSE
    is_group_commit_leader= (old_queue == NULL)
    unlock(LOCK_prepare_ordered)

    if (is_group_commit_leader)

        // The first in queue handles group commit for everyone

        lock(LOCK_group_commit)
        // Wait while queue is busy, see below for when this occurs
        while (group_commit_queue_busy)
            cond_wait(COND_group_commit)

        // Grab and reverse the queue to get correct order of transactions
        queue= atomic_grab_reverse_queue()

        // This call will set individual error codes in thd->xid_error
        // It also sets the cookie for unlog() in thd->xid_cookie
        group_log_xid(queue)

        lock(LOCK_commit_ordered)
        for (other IN queue)
            if (!other->xid_error)
                ht->commit_ordered()
        unlock(LOCK_commit_ordered)

        unlock(LOCK_group_commit)

        // Now we are done, so wake up all the others.
        for (other IN TAIL(queue))
            lock(other->LOCK_commit_ordered)
            other->group_commit_ready= TRUE
            cond_signal(other->COND_commit_ordered)
            unlock(other->LOCK_commit_ordered)
    else
        // If not the leader, just wait until leader did the work for us.
        lock(thd->LOCK_commit_ordered)
        while (!thd->group_commit_ready)
            cond_wait(thd->LOCK_commit_ordered, thd->COND_commit_ordered)
        unlock(other->LOCK_commit_ordered)

    // Finally do any error reporting now that we're back in own thread.
    if (thd->xid_error)
        xid_delayed_error(thd)
    else
        ht->commit(thd)
        unlog(thd->xid_cookie, thd->xid)
---- END ALGORITHM ----

If the transaction coordinator does not support group_log_xid(), we have to do
things differently. In this case after the serialisation point at
prepare_ordered(), we have to parallelise again when running log_xid()
(otherwise we would loose group commit). But then when log_xid() is done, we
have to serialise again to check for any error and call commit_ordered() in
correct sequence for any transaction where log_xid() did not return error.

The central part of the algorithm in this case (when using log_xid()) is:

---- BEGIN ALGORITHM ----
    cookie= log_xid(thd)
    error= (cookie == 0)

    if (is_group_commit_leader)

        // The first to enqueue grabs the queue and runs first.
        // But we must wait until a previous queue run is fully done.

        lock(LOCK_group_commit)
        while (group_commit_queue_busy)
            cond_wait(COND_group_commit)
        queue= atomic_grab_reverse_queue()
        // The queue will be busy until last thread in it is done.
        group_commit_queue_busy= TRUE
        unlock(LOCK_group_commit)
    else
        // Not first in queue -> wait for previous one to wake us up.
        lock(thd->LOCK_commit_ordered)
        while (!thd->group_commit_ready)
            cond_wait(thd->LOCK_commit_ordered, thd->COND_commit_ordered)
        unlock(other->LOCK_commit_ordered)

    if (!error)      // Only if log_xid() was successful
        lock(LOCK_commit_ordered)
        ht->commit_ordered()
        unlock(LOCK_commit_ordered)

    // Wake up the next thread, and release queue in last.
    next= thd->next_commit_ordered

    if (next)
        lock(next->LOCK_commit_ordered)
        next->group_commit_ready= TRUE
        cond_signal(next->COND_commit_ordered)
        unlock(next->LOCK_commit_ordered)
    else
        lock(LOCK_group_commit)
        group_commit_queue_busy= FALSE
        unlock(LOCK_group_commit)
---- END ALGORITHM ----

There are a number of locks taken in the algorithm, but in the group_log_xid()
case most of them should be uncontended most of the time. The
LOCK_group_commit of course will be contended, as new threads queue up waiting
for the previous group commit (and binlog fsync()) to finish so they can do
the next group commit. This is the whole point of implementing group commit.

The LOCK_prepare_ordered and LOCK_commit_ordered mutexes should be not much
contended as long as handlers follow the intension of having the corresponding
handler calls execute quickly.

The per-thread LOCK_commit_ordered mutexes should not be contended; they are
only used to wake up a sleeping thread.


1.4 Optimisations when not using all three new calls


The prepare_ordered(), group_log_xid(), and commit_ordered() methods are
optional, and if not implemented by a particular handler/transaction
coordinator, we can optimise the algorithm to take advantage of not having to
keep ordering for the missing parts.

If there is no prepare_ordered(), then we need not take the
LOCK_prepare_ordered mutex.

If there is no commit_ordered(), then we need not take the LOCK_commit_ordered
mutex.

If there is no group_log_xid(), then we only need the queue to ensure same
ordering of transactions for commit_ordered() as for prepare_ordered(). Thus,
if either of these (or both) are also not present, we do not need to use the
queue at all.


2. Binlog code changes (log.cc)


The bulk of the work needed for the binary log is to extend the code to allow
group commit to the log. Unlike InnoDB/XtraDB, there is no existing support
inside the binlog code for group commit.

The existing code runs most of the write + fsync to the binary lock under the
global LOCK_log mutex, preventing any group commit.

To enable group commit, this code must be split into two parts:

 - one part that runs per transaction, re-writing the embedded event positions
   for the correct offset, and writing this into the in-memory log cache.

 - another part that writes a set of transactions to the disk, and runs
   fsync().

Then in group_log_xid(), we can run the first part in a loop over all the
transactions in the passed-in queue, and run the second part only once.

The binlog code also has other code paths that write into the binlog,
eg. non-transactional statements. These have to be adapted also to work with
the new code.

In order to get some group commit facility for these also, we change that part
of the code in a similar way to ha_commit_trans. We keep another,
binlog-internal queue of such non-transactional binlog writes, and such writes
queue up here before sleeping on the LOCK_log mutex. Once a thread obtains the
LOCK_log, it loops over the queue for the fast part, and does the slow part
once, then finally wakes up the others in the queue.

In the transactional case in group_log_xid(), before we run the passed-in
queue, we add any members found in the binlog-internal queue. This allows
these non-transactional writes to share the group commit.

However, in the case where it is a non-transactional write that gets the
LOCK_log, the transactional transactions from the ha_commit_trans() queue will
not be able to take part (they will have to wait for their turn to do another
fsync). It seems difficult to cleanly let the binlog code grab the queue from
out of the ha_commit_trans() algorithm. I think the group commit is mostly
useful in transactional workloads anyway (non-transactional engines will loose
data anyway in case of crash, so why fsync() after each transaction?)


3. XtraDB changes (ha_innodb.cc)

The changes needed in XtraDB are comparatively simple, as XtraDB already
implements group commit, it just needs to be enabled with the new
commit_ordered() call.

The existing commit() method already is logically in two parts. The first part
runs under the prepare_commit_mutex() and must be run in same order as binlog
commit. This part needs to be moved to commit_ordered(). The second part runs
after releasing prepare_commit_mutex and does transaction log write+fsync; it
can remain.

Then the prepare_commit_mutex is removed (and the enable_unsafe_group_commit
XtraDB option to disable it).

There are two asserts that check that the thread running the first part of
XtraDB commit is the same as the thread running the other operations for the
transaction. These have to be removed (as commit_ordered() can run in a
different thread). Also an error reporting with sql_print_error() has to be
delayed until commit() time.


4. Proof-of-concept implementation

There is a proof-of-concept implementation of this architecture, in the form
of a quilt patch series [3].

A quick benchmark was done, with sync_binlog=1 and
innodb_flush_log_at_trx_commit=1. 64 parallel threads doing single-row
transactions against one table.

Without the patch, we get only 25 queries per second.

With the patch, we get 650 queries per second.


5. Open issues/tasks

5.1 XA / other prepare() and commit() call sites.

Check that user-level XA is handled correctly and working. And covered
sufficiently with tests. Also check that any other calls of ha->prepare() and
ha->commit() outside of ha_commit_trans() are handled correctly.

5.2 Testing

This worklog needs additions to the test suite, including error inserts to
check error handling, and synchronisation points to check thread parallelism
correctness.


6. Alternative implementations

 - The binlog code maintains its own extra atomic transaction queue to handle
   non-transactional commits in a good way together with transactional (with
   respect to group commit). Alternatively, we could ignore this issue and
   just give up on group commit for non-transactional statements, for some
   code simplifications.

 - The binlog code has two ways to prepare end_event and similar, one that
   uses stack-allocation, and another for when stack allocation is not
   possible that uses thd->mem_root. Probably the overhead of thd->mem_root is
   so small that it would make sense to use the same code for both cases.

 - Instead of adding extra fields to THD, we could allocate a separate
   structure on the thd->mem_root() with the required extra fields (including
   the THD pointer). Would seem to require initialising mutexes at every
   commit though.

 - It would probably be a good idea to implement TC_LOG_MMAP::group_log_xid()
   (should not be hard).


-----------------------------------------------------------------------

References:

[2] https://secure.wikimedia.org/wikipedia/en/wiki/ABA_problem

[3] https://knielsen-hq.org/maria/patches.mwl116/


ESTIMATED WORK TIME

ESTIMATED COMPLETION DATE
-----------------------------------------------------------------------
WorkLog (v3.5.9)