← Back to team overview

maria-developers team mailing list archive

Re: Architecture review of MWL#116 "Efficient group commit for binary log"

 

Sergei Golubchik <serg@xxxxxxxxxxxx> writes:

> Hi, Kristian!
>
> Let's start from the WL description:

Thanks for your detailed comments!

>> 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:
> ...
>>  - 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).
>
> I doubt mysqldump --single-transaction cares about physical commit
> ordering in logs.

What I mean is that consistent commit order between binlog and engine ensures
that for every snapshot of the engine state (eg. mysqldump
--single-transaction), there will be a corresponding binlog position.

Without consistent order, we can have in InnoDB B committed after A, but in
binlog A committed after B. If we then take a mysqldump --single-transaction
and happen to get the point where A committed but not B, then there is no
binlog position that corresponds to the snapshot.

>>  - 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.
>
> Why do you need consistent commit order for that?

Hm. The case I have in mind is this:

Suppose binlog commits transactions like this:

    A B C D E

But engine commits them like this:

    C E D A B

Now we crash, and engine happens to lose the last three transactions, leaving
this after crash:

    C E

Now what will we do to recover? I guess we will have to replay A, B, and D but
not C and E. So we need to get a _list_ of missing transactions from the
engine somehow (and so also need something similar to unlog() against the
engine). We also have to ensure that replaying them in different order will
work (I think it will, but not 100% sure).

In contrast, with consistent ordering, we only need to identify the last
transaction committed in the engine, and replay the binlog from that point.

So I'm not 100% sure that the optimised recovery cannot work without
consistent commit ordering, but it does seem to be simpler to do in the
consistent ordering case.

What do you think?

>
>>  - 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.
>
> Okay, although it's a corner case.
>
> It would be more important to have consistent reads working across
> multiple storage engines without explicit START TRANSACTION WITH
> CONSISTENT SNAPSHOT. But it's doable too.

Do you mean per statement, or per transaction? For per statement, I agree. For
transaction I do not see how, without having to in effect start a snapshot
across all engines for every transaction?

>>  - 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.
>
> Conceptually InnoDB can release the locks only after the commit was made
> persistent.

Well, that was my immediate thought as well. But the longer I have tried
coming up with a counter-example, the less sure I am.

Note that the Facebook patch implements this early release of locks.

The thing is, this is not about releasing locks on modified rows. In InnoDB,
modified rows are locked in effect by having a higher LSN in the row, so there
is no explicit lock in memory like for gap locks in SERIALISABLE mode etc. So
modified rows are still only unlocked when the transaction commits.

So what effects can releasing these locks have? It cannot affect conflicting
row modifications, as the locks preventing these are not released until
commit, as before. It cannot affect commit order relative to another
transaction, as we released locks only after deciding on commit order. It also
cannot affect visibility of our changes to other transactions, as the
visibility of those changes is not changed.

So I actually do not see any problems with releasing locks early as done in
the Facebook patch, once commit order is determined. I would be _most_
interested in a counter-example, or alternatively in a proof/argument that
such counter example does not exist (I'm not 100% convinced yet...)

This release of locks only affects higher isolation levels and things like
LOCK IN SHARED MODE. Probably the statement-based replication, which needs
higher isolation levels, is the main motivation. My understanding is that
Facebook hit some bottlenecks in this respect.

We should get some discussions going about this with Mark in Istanbul!

> So, I think it's the same as "optimising commit to need only one
> fsync" - if we only sync the binlog, InnoDB can release its locks right
> after binlog sync.  And the question is the same, why do you need
> consistent commit order for that? (I suppose, the answer will be the
> same too :)

Actually, with this optimisation you could release the locks even before the
fsync() (just after prepare()), assuming the early release is actually safe.

>>  - 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.
>
> Okay, this much I agree with - there could be applications (galera, may
> be) that need this consistent commit order. But because it's not needed
> in many (if not most) cases, it'd be good to be able to switch it off,
> if it'll help to gain performance.

Yes.

Consistent commit ordering will be disabled if the engine does not install
prepare_ordered() or commit_ordered() methods. It will also be switched off if
there is no binlog _and_ no prepare_ordered() methods are installed. But the
engine can only decide to install methods or not at startup, not
dynamically.

So we could add a separate flag for the engine to dynamically say that
prepare_ordered()/commit_ordered() should or should not be called.

The other side of this is if there is really any performance gain in switching
it off. My intension with the design was that there should not be any
performance penalty from it. The idea is that engines put in the _ordered()
methods only code that in any case needs to run serialised (such as allocating
slots in the transaction log), and similarly that binlog only put in
group_log_xid() code that anyway needs to run serialised.

For example, InnoDB in any case holds the kernel_mutex over the
commit_ordered() call, and binlog similarly in any case holds the LOCK_log
during log_xid(), so there is no additional serialisation.

Of course, compared to the current code everything will be faster, as we get
rid of the Innodb prepare_commit_mutex, and add binlog group commit. Maybe we
can try benchmark this against a version where InnoDB installs no
commit_ordered() hook, and see if we can measure any difference.

> ================  High-Level Specification ===================
> ...
>> 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.
>
> How could that work ? If you run log_xid() is parallel, then you don't
> know in what order xid's are logged - which means, you don't know the
> commit order in the binlog. And if you don't know it, you cannot have
> the same order in the engines.
>  
>> 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.
>
> I failed to understand the last paragraph :(

Yes, I need to clarify the issue here.

Suppose we use the log_xid() interface.

Then it is necessary for multiple threads to _enter_ log_xid() in parallel,
otherwise group commit is impossible (group commit will do a single fsync()
for all transactions that entered log_xid() while the previous fsync() was
running). But inside log_xid(), the threads will need to synchronise with one
another, one thread running fsync() and the others waiting (the ordering among
transactions/xids would have been determined in prepare_ordered() for the
binlog storage engine). Then after waiting, the log_xid() calls will return,
the threads running in parallel again.

Then after returning from log_xid(), each thread will again have to wait, this
time for the commit_ordered() call of the previously committed transaction to
complete before running its own commit_ordered() and signalling the next
thread in line.

The result is that we have a context switch from one thread to the next after
every commit_ordered() call; this is what I refered to as "bouncing the
execution".

In contrast, with the group_log_xid() call, there is less context
switches. Threads still have to wait as they queue up for group commit; this
is unavoidable. But the serial execution of group_log_xid() and
commit_ordered() can now happen without any context switches in a single
thread. After the serial part of the algorithm, all threads are again woken
up, but this wakeup can happen in parallel in whatever order the kernel finds
most efficient.

The result is less synchronisation overhead, though I never tried to measure
if the difference is significant.

Note that any parallel part of the algorithm that does not need to run
serialised between threads can still be done in parallel in prepare() and
commit() (and should be done so).

I hope this explanation is clearer.

>> 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.
>
> I do not see how this is a consequence of the optimization.

What I meant was, the optimisation is to run this in a single thread:

    group_log_xid(<list of transactions>)
    for trx in (<list of transactions>)
      trx->commit_ordered()

so as a consequence, commit_ordered() for one transaction can be called from
the thread context of another transaction.

It is somewhat controversial to do this, normally all code for a transaction
runs in the same thread. And it has the consequence that my_error() cannot be
used in commit_ordered(). I like to do this to avoid the extra context
switches, but it would be good to get a second opinion.

> Hm. handlerton::prepare_ordered() and handlerton::commit_ordered() are
> truly optional, but TC_LOG::group_log_xid() is a replacement for
> TC_LOG::log_xid().
>
> I wonder if it would be cleaner to remove TC_LOG::log_xid() (from
> TC_LOG_MMAP too) completely and not keep the redundant functionality.

Yes.

But in MWL#132 I evolved this further, and generalised the TC_LOG a bit to
something that would work for Galera and similar plugins that need to decide
on commit order in TC (as opposed to just follow a pre-determined commit order
like binlog does in this worklog).

Then the different interfaces log_xid() and group_log_xid() are just two
simple subclasses of the general TC_LOG. And the ha_commit_ordered() gets rid
of a lot of complexity around the queueing of transactions for
commit_ordered() etc. I think this way in MWL#132 is much cleaner.

>> -----------------------------------------------------------------------
>> Some possible alternative for this worklog:
>> 
>>  - Alternatively, we could eliminate log_xid() and require that all
>>  transaction coordinators implement group_log_xid() instead, again for
>>  some moderate simplification.
>
> indeed, that's what I mean

Yes. In MWL#132, log_xid() is in a separate TC_LOG_unordered() subclass used
by TC_LOG_MMAP. If we think it could be useful to other plugins we can keep
it, if not we can just remove it, as it is completely separate.

>>  - 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).
>
> every time when a new api is implemented there is no plugin actually
> using it :)

True ;-)

One example where it is needed is in the release locks early patch for InnoDB
in the facebook patch.

Another use may be for engines that want to know the commit order already in
the prepare phase; these will need prepare_ordered() to do this and still
participate in consistent cross-engine commit order.

>> -----------------------------------------------------------------------
>> Some possible follow-up projects after this is implemented:
>> 
>>  - Add statistics about how efficient group commit is
>>  (#fsyncs/#commits in each engine and binlog).
>
> this is so simple that can be done im this WL.

You are probably right.

(I think I may have been wanting to leave some low-hanging fruits for
potential contributers / plugin writers).

>>  - 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).

The actual implementation of this is trivial, using the prepare_ordered()
call.

>>  - 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).
>
> I'd say it's important optimization - can you turn it into a WL ?
> It deserves more than a passing mentioning at the end of another WL.
>
> Perhaps other follow-up projects may be also turned into WLs.

Yes, good point, will do.

>> 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.
>
> How can that work ? If you insert into a queue without a lock, you
> cannot know what order of commits you eend up with. So, you cannot have
> the same order as you've got fof prepare_ordered. Besides, if
> prepare_ordered is, precisely, *ordered*, it must be done under a lock.

Right, you raise a number of issues below, and I think just answering each one
individually will become hard to understand, so let me explain my thinking
more in-depth.

It would be possible to iterate over the queue to invoke prepare_ordered() in
sequence from a single thread, just like group_log_xid() and
commit_ordered(). But this would delay the calls until the previous group
commit is done and the next one starts, and would thus introduce more
processing in the critical path where the main contended LOCK_group_commit is
held. To avoid this, I instead run them early, and use LOCK_prepare_ordered to
make them ordered.

It is correct that enqueue has to be done under the same LOCK_prepare_ordered
to ensure the same order in queue and of prepare_ordered() calls. However, by
making the queue lock-free, the _dequeue_ does not need to take
LOCK_prepare_ordered, which is a global mutex that I wanted to reduce
contention of as much as possible.

The reason for the LOCK_commit_ordered is that there are two code paths that
call commit_ordered(). One is the group commit leader thread that invokes
commit_ordered() for all threads that have passed group_log_xid(). The other
is when no 2-phase commit is done, ha_commit_one_phase(). It does not actually
matter in which order the non-2-phase commit_ordered() calls happen, but I
still wanted to provide the storage engine the guarantee that commit_ordered()
would not be invoked by multiple threads in parallel. It just seemed cleaner
that way to me.

As you see, generally I have strived for fine-grained locking to avoid
contention as much as possible. I am open for suggestions that I have overdone
it, and I hope this explanation makes it easier to discuss where we could
simplify.

One possible simplification would be to just say that no prepare_ordered()
call can run in parallel with commit_ordered() in another thread. Maybe the
additional contention is not a problem in practise. It might even be that
storage engines need this guarantee anyway (eg. probably InnoDB will take
its global kernel mutex in both calls anyway). Then we can replace
LOCK_prepare_ordered, LOCK_commit_ordered, and the atomic queue with just a
single mutex for the queue.

Another possibility would be to allow multiple calls in parallel into
commit_ordered() (in the cases where no specific commit ordering between the
parallel invocations is needed). Then the storage engine will have to be
prepared for this situation, but LOCK_commit_ordered is no longer needed.

If I had to choose I would still pick the method I describe in the worklog,
but I do not have any strong opinions, so if you have a preference I would
like to know.

Ok, so with the above as background, I will respond to each of your points
briefly below. I believe I address all your concerns, and that your suggestion
for different algorithm is not necessary. But if I missed something or if you
have another reason to prefer your suggestion, please let me know, maybe I
misunderstood.

> I see two possibilities of doing prepare_ordered and enqueue:
>
> 1.
>
>   pthread_mutex_lock(queue_lock);
>   prepare_ordered(thd);
>   enqueue(thd);
>   pthread_mutex_unlock(queue_lock);
>
> in this case prepare_ordered calls are strictly serialized by a mutex.
> There is no need for non-locking enqueue_atomic, because the same mutex
> must be used to define the queue ordering.

Correct, see above for why non-locking queue was used anyway.

> 2.
>
>   enqueue_atomic();
>
>   ... some time later ...
>   for (thd=queue_head; thd; thd= thd->next_in_queue)
>     prepare_ordered(thd);
>
> in this case insert into a queue is non-locking, the order of elements
> in a queue is undefined. Later on, you iterate the queue and call
> prepare_ordered in the queue order. In this implementation
> all prepare_ordered and commit_ordered will likely be called from one
> thread, not from the connection thread of this transaction.

Yes, see above for why I prefer something like 1.

> ==========
> Note the detail: for START TRANSACTION WITH CONSISTENT SNAPSHOT and
> inter-engine consistency to work, all connection threads must use the
> same commit order across all engines (similar to deadlock avoidance that
> MySQL uses - where all threads lock tables in the same order). A simple
> way to do it would be to change trans_reqister_ha and instead of adding
> participating engines to a list, mark them in an array:
>
>   bool engines[MAX_HA];
>   trans_reqister_ha() {
>      ...
>      engines[hton->slot]=1;
>      ...
>   }
>
> and in ha_prepare_ordered() you call hton->prepare_ordered()
> for all engines that have non-zero value in engines[] array.

Ah, interesting point. I can vaguely see what you are getting at, but I do
think I really understand. Here are my thoughts:

With current MySQL/MariaDB, there is no imposed commit ordering. So even with
START TRANSACTION WITH CONSISTENT SNAPSHOT, you can end up with a snapshot
where in engine E, A is committed but not B; and in engine F, B is committed
but not A.

Just by introducing consistent commit order, this inconsistency is
impossible. A and B will be consistently ordered (say A commits before B), and
it is not possible to see <B committed but not A> in any engine.

But there is still another inconsistency to think about, where we get a
snapshot where both A and B are committed in one engine, but only A is
committed in another. This inconsistency is not about commit order, but about
visibility, something I now see I have been vague about so far.

So let us say that commit_ordered() makes the commit visible to other
transactions/threads. I think this is a reasonable specification. Then to
implement a fully consistent START TRANSACTION WITH CONSISTENT SNAPSHOT, what
is needed is to create the snapshot while holding the LOCK_commit_ordered.
Then also the second type of inconsistency is impossible: the short window
where B is visible in one engine and invisible in another is protected by
LOCK_commit_ordered, so START TRANSACTION WITH CONSISTENT SNAPSHOT will wait
for all engines to make the transaction visible before proceeding (and
subsequent commits will wait for START TRANSACTION WITH CONSISTENT SNAPSHOT
before making new commits visible).

So this seems to be another argument why LOCK_commit_ordered is needed! I
actually like the idea of having a general mechanism (=mutex) for
synchronising with commit order, seems it could be useful for more things
(getting binlog position?).

However, I still do not understand your point about inter-engine consistency
and order of engines in calling prepare_ordered() :-(. Maybe you could
elaborate?

>> 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.
>
> Nope, in the approach 1 (with a mutex) as above, you can have this too.
> Simply as
>
>   1.
>     ... on commit ...
>     pthread_mutex_lock(queue_lock);
>     commit_queue= queue;
>     queue=NULL;
>     pthread_mutex_unlock(queue_lock);
>     ... now you can iterate the commit_queue in parallel ...
>     ... with preparing other transactions ...
>
> actually you'll have to do it not on commit but before group_log_xid().

Yes, a simple mutex on the queue would work well. Of course, lock-free
algorithms are all hot and cool, but if you prefer a simple mutex-protected
queue instead I will change it. Well, this is just what happens anyway when
atomic operations are not available, my_atomic falls back to mutex.

>> 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.
>
> Better to have a list of transaction objects, if possible - as
> eventually we'll need to decouple st_transaction and THD to be able to
> put transactions on hold and continue them in another thread, as
> needed for external XA.
>
> We aren't doing it now, but if possible, it'd better to avoid creating
> new dependencies between THD and st_transaction.

So in the MWL#132 I managed to clean this up. Now there is nothing in THD,
instead a small queue node is allocated on the stack in each thread, much
nicer.

I do not know much about the transaction object, but I think just a queue node
on the stack of the thread should be fine, if not let me know.

I still have a per-thread mutex and condition in THD, since I did not know
where else to put them without having to re-initialise on each transaction
start. Do you have a better suggestion?

>> 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.
>
> okay, in this case it's approach 1, enqueue must happen under the same
> lock and there's no need for a lock-free implementation

Yes (but see above about wanting to keep LOCK_prepare_ordered and
LOCK_commit_ordered separate).

>> 1.2.3 Global LOCK_commit_ordered
>> 
>> This lock is taken around calls to commit_ordered(), to ensure they
>> happen serialised.
>
> if you have only one thread iterating the queue and calling
> commit_ordered(), they'll be always serialized. why do you need a
> mutex ?

Because there could be other threads needing to synchronise commit_ordered()
as discussed above, in particular ha_commit_one_phase() (and maybe START
TRANSACTION WITH CONSISTENT SNAPSHOT)/

>> ---- 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)
>
> so, indeed, you enqueue_atomic under a mutex. Why do you want lock-free
> queue implementation ?

The dequeue happens without holding that mutex. So I needed either a separate
mutex for the enqueue/dequeue, or a lock-free queue. The only reason for
choosing the latter is a general belief that lock-free is better than mutex
(which belief I may or may not still hold ;-)

> commit_ordered() is called from one thread, and it's protected
> by LOCK_group_commit. There's no need for LOCK_commit_ordered.

See above wrt. other calls from ha_commit_one_phase() (and maybe START
TRANSACTION WITH CONSISTENT SNAPSHOT).

>  
>>         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)
>
> why do you want one condition per thread ?
> You can use one condition per queue, such as:
>
>      else
>          // If not the leader, just wait until leader did the work for us.
>          lock(old_queue->LOCK_commit_ordered)
>          while (!old_queue->group_commit_ready)
>              cond_wait(old_queue->LOCK_commit_ordered, old_queue->COND_commit_ordered)
>          unlock(old_queue->LOCK_commit_ordered)
>
> this way you'll need just one broadcast to wake up everybody.

The reason is the following:

With one condition, there will be also just one associated mutex. When the
condition is broad-cast, every waiting thread will then contend for that one
mutex. So one will wake up and get the mutex, then immediately release it,
waking up the next, which will immediately release it, and so on.

The end result is that the last thread will have to wait for context switch of
every other thread before it can run. I did not like this unnecessary
contention.

In contrast, with one condition (and one mutex) per thread, the thread that
does the group commit and runs all the commit_ordered() calls can just wake up
all the others immediately, and each thread is free to run as soon or as late
as scheduling resources permit.

Again, I wanted to avoid forcing the kernel scheduler to bounce around from
one thread to the next before the last thread can run.

Do you see any advantage from using just one condition, instead of one per
thread?

>
>>     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)
>
> This doesn't work, I'm afraid. The leader can complete its job and
> signal all threads in a queue before some of the queue threads has
> locked their thd->LOCK_commit_ordered mutex. These threads will miss the
> wakeup signal.

No, it is not a problem. If the leader completes first, it will have set
thd->group_commit_ready, and the thread will never enter cond_wait().
The variable thd->group_commit_ready is protected by the
thd->LOCK_commit_ordered mutex for exactly this reason.

>
> To close all loopholes, I think, you can use my suggestion above - with
> one broadcast and waiting on old_queue->COND_commit_ordered - and every
> thread, including the leader, must lock the leader->LOCK_commit_ordered
> before unlocking LOCK_prepare_ordered. That is, it'll be

>From my comments above, I believe that there are actually no loopholes, and
that this suggestion will offer less parallelism for no additional
benefit. But please let me know if I misunderstood, or if you have some other
reason to prefer this suggestion.

>
>      // Call prepare_ordered() and enqueue in correct commit order
>      lock(LOCK_prepare_ordered)
>      ht->prepare_ordered()
>      old_queue= queue_head
>      enqueue(thd)
>      leader= queue_head
>      lock(leader->LOCK_commit_ordered)
>      unlock(LOCK_prepare_ordered)
>
>      if (leader != thd)
>        while (!leader->group_commit_done)
>          cond_wait(leader->LOCK_commit_ordered, leader->COND_commit_ordered)
>      else
>        leader->group_commit_done= false;
>        .... the rest of your group commit code
>        broadcast(leader->COND_commit_ordered);
>        unlock(leader->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 ----

> I wouldn't bother implementing any grouping for non-transactional
> engines. They aren't crash safe, so practically users of these engines
> don't need or use --sync-binlog anyway.

Agree.

I actually implemented this in my proof-of-concept patch, at least
partially. The group_log_xid() will also run any waiting non-transactional
"commits". One cannot set --sync-binlog on a per-thread basis, so without
this, a moderate amount of non-transactional statements can kill performance
for transactional group commit.

But I agree with you. So if the code is complex to complete/fix, or if we
think it's ugly (and it probably is), I am happy to be rid of it :-)

> Please do RQG transactional tests (Philip can help with that)
> to verify that ACID wasn't compromized

Yes, will do.

I also want to add some testing of various thread interactions using the new
deb synchonisation facilities
http://forge.mysql.com/wiki/MySQL_Internals_Test_Synchronization

> Right, that's what I suggested above.
> Seems like whatever I write, you have the same later in the WL :)

Which I guess is a good sign, except maybe I need to reverse my writing ;-)

Thanks!

 - Kristian.



Follow ups

References