2017-06-18 05:42:16 +02:00
|
|
|
#include "mariadb.h"
|
2013-06-24 10:50:25 +02:00
|
|
|
#include "rpl_parallel.h"
|
|
|
|
#include "slave.h"
|
|
|
|
#include "rpl_mi.h"
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
#include "sql_parse.h"
|
2013-12-18 16:26:22 +01:00
|
|
|
#include "debug_sync.h"
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2013-07-03 13:46:33 +02:00
|
|
|
/*
|
|
|
|
Code for optional parallel execution of replicated events on the slave.
|
|
|
|
*/
|
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
|
|
|
|
/*
|
|
|
|
Maximum number of queued events to accumulate in a local free list, before
|
|
|
|
moving them to the global free list. There is additional a limit of how much
|
|
|
|
to accumulate based on opt_slave_parallel_max_queued.
|
|
|
|
*/
|
|
|
|
#define QEV_BATCH_FREE 200
|
|
|
|
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
struct rpl_parallel_thread_pool global_rpl_thread_pool;
|
|
|
|
|
2014-03-21 10:11:28 +01:00
|
|
|
static void signal_error_to_sql_driver_thread(THD *thd, rpl_group_info *rgi,
|
|
|
|
int err);
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2013-10-14 15:28:16 +02:00
|
|
|
static int
|
2013-06-24 10:50:25 +02:00
|
|
|
rpt_handle_event(rpl_parallel_thread::queued_event *qev,
|
|
|
|
struct rpl_parallel_thread *rpt)
|
|
|
|
{
|
2013-12-13 14:26:51 +01:00
|
|
|
int err;
|
2013-09-13 15:09:57 +02:00
|
|
|
rpl_group_info *rgi= qev->rgi;
|
2013-06-28 15:19:30 +02:00
|
|
|
Relay_log_info *rli= rgi->rli;
|
2013-07-08 16:47:07 +02:00
|
|
|
THD *thd= rgi->thd;
|
MDEV-6551: Some replication errors are ignored if slave_parallel_threads > 0
The problem occured when using parallel replication, and an error occured that
caused the SQL thread to stop when the IO thread had already reached a
following binlog file from the master (or otherwise performed a relay log
rotation).
In this case, the Rotate Event at the end of the relay log file could still be
executed, even though an earlier event in that relay log file had gotten an
error. This would cause the position to be incorrectly updated, so that upon
restart of the SQL thread, the event that had failed would be silently skipped
and ignored, causing replication corruption.
Fixed by checking before executing Rotate Event, whether an earlier event
has failed. If so, the Rotate Event is not executed, just dequeued, same as
for other normal events following a failing event.
2014-08-15 11:31:13 +02:00
|
|
|
Log_event *ev;
|
|
|
|
|
|
|
|
DBUG_ASSERT(qev->typ == rpl_parallel_thread::queued_event::QUEUED_EVENT);
|
|
|
|
ev= qev->ev;
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2014-04-25 12:58:31 +02:00
|
|
|
thd->system_thread_info.rpl_sql_info->rpl_filter = rli->mi->rpl_filter;
|
MDEV-6551: Some replication errors are ignored if slave_parallel_threads > 0
The problem occured when using parallel replication, and an error occured that
caused the SQL thread to stop when the IO thread had already reached a
following binlog file from the master (or otherwise performed a relay log
rotation).
In this case, the Rotate Event at the end of the relay log file could still be
executed, even though an earlier event in that relay log file had gotten an
error. This would cause the position to be incorrectly updated, so that upon
restart of the SQL thread, the event that had failed would be silently skipped
and ignored, causing replication corruption.
Fixed by checking before executing Rotate Event, whether an earlier event
has failed. If so, the Rotate Event is not executed, just dequeued, same as
for other normal events following a failing event.
2014-08-15 11:31:13 +02:00
|
|
|
ev->thd= thd;
|
2013-07-03 13:46:33 +02:00
|
|
|
|
2013-10-23 15:03:03 +02:00
|
|
|
strcpy(rgi->event_relay_log_name_buf, qev->event_relay_log_name);
|
|
|
|
rgi->event_relay_log_name= rgi->event_relay_log_name_buf;
|
|
|
|
rgi->event_relay_log_pos= qev->event_relay_log_pos;
|
2013-10-17 14:11:19 +02:00
|
|
|
rgi->future_event_relay_log_pos= qev->future_event_relay_log_pos;
|
2013-10-23 15:03:03 +02:00
|
|
|
strcpy(rgi->future_event_master_log_name, qev->future_event_master_log_name);
|
2015-10-22 10:28:51 +02:00
|
|
|
if (!(ev->is_artificial_event() || ev->is_relay_log_event() ||
|
|
|
|
(ev->when == 0)))
|
|
|
|
rgi->last_master_timestamp= ev->when + (time_t)ev->exec_time;
|
2016-09-20 15:30:57 +02:00
|
|
|
err= apply_event_and_update_pos_for_parallel(ev, thd, rgi);
|
2013-10-14 23:17:16 +02:00
|
|
|
|
2015-01-12 17:03:45 +01:00
|
|
|
thread_safe_increment64(&rli->executed_entries);
|
2013-06-24 10:50:25 +02:00
|
|
|
/* ToDo: error handling. */
|
2013-10-14 15:28:16 +02:00
|
|
|
return err;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-10-31 14:11:41 +01:00
|
|
|
static void
|
|
|
|
handle_queued_pos_update(THD *thd, rpl_parallel_thread::queued_event *qev)
|
|
|
|
{
|
|
|
|
int cmp;
|
|
|
|
Relay_log_info *rli;
|
MDEV-6551: Some replication errors are ignored if slave_parallel_threads > 0
The problem occured when using parallel replication, and an error occured that
caused the SQL thread to stop when the IO thread had already reached a
following binlog file from the master (or otherwise performed a relay log
rotation).
In this case, the Rotate Event at the end of the relay log file could still be
executed, even though an earlier event in that relay log file had gotten an
error. This would cause the position to be incorrectly updated, so that upon
restart of the SQL thread, the event that had failed would be silently skipped
and ignored, causing replication corruption.
Fixed by checking before executing Rotate Event, whether an earlier event
has failed. If so, the Rotate Event is not executed, just dequeued, same as
for other normal events following a failing event.
2014-08-15 11:31:13 +02:00
|
|
|
rpl_parallel_entry *e;
|
|
|
|
|
2013-10-31 14:11:41 +01:00
|
|
|
/*
|
|
|
|
Events that are not part of an event group, such as Format Description,
|
|
|
|
Stop, GTID List and such, are executed directly in the driver SQL thread,
|
|
|
|
to keep the relay log state up-to-date. But the associated position update
|
|
|
|
is done here, in sync with other normal events as they are queued to
|
|
|
|
worker threads.
|
|
|
|
*/
|
|
|
|
if ((thd->variables.option_bits & OPTION_BEGIN) &&
|
|
|
|
opt_using_transactions)
|
|
|
|
return;
|
MDEV-6551: Some replication errors are ignored if slave_parallel_threads > 0
The problem occured when using parallel replication, and an error occured that
caused the SQL thread to stop when the IO thread had already reached a
following binlog file from the master (or otherwise performed a relay log
rotation).
In this case, the Rotate Event at the end of the relay log file could still be
executed, even though an earlier event in that relay log file had gotten an
error. This would cause the position to be incorrectly updated, so that upon
restart of the SQL thread, the event that had failed would be silently skipped
and ignored, causing replication corruption.
Fixed by checking before executing Rotate Event, whether an earlier event
has failed. If so, the Rotate Event is not executed, just dequeued, same as
for other normal events following a failing event.
2014-08-15 11:31:13 +02:00
|
|
|
|
|
|
|
/* Do not update position if an earlier event group caused an error abort. */
|
|
|
|
DBUG_ASSERT(qev->typ == rpl_parallel_thread::queued_event::QUEUED_POS_UPDATE);
|
|
|
|
e= qev->entry_for_queued;
|
|
|
|
if (e->stop_on_error_sub_id < (uint64)ULONGLONG_MAX || e->force_abort)
|
|
|
|
return;
|
|
|
|
|
2013-10-31 14:11:41 +01:00
|
|
|
rli= qev->rgi->rli;
|
|
|
|
mysql_mutex_lock(&rli->data_lock);
|
|
|
|
cmp= strcmp(rli->group_relay_log_name, qev->event_relay_log_name);
|
|
|
|
if (cmp < 0)
|
|
|
|
{
|
|
|
|
rli->group_relay_log_pos= qev->future_event_relay_log_pos;
|
|
|
|
strmake_buf(rli->group_relay_log_name, qev->event_relay_log_name);
|
|
|
|
rli->notify_group_relay_log_name_update();
|
|
|
|
} else if (cmp == 0 &&
|
|
|
|
rli->group_relay_log_pos < qev->future_event_relay_log_pos)
|
|
|
|
rli->group_relay_log_pos= qev->future_event_relay_log_pos;
|
|
|
|
|
|
|
|
cmp= strcmp(rli->group_master_log_name, qev->future_event_master_log_name);
|
|
|
|
if (cmp < 0)
|
|
|
|
{
|
|
|
|
strcpy(rli->group_master_log_name, qev->future_event_master_log_name);
|
|
|
|
rli->group_master_log_pos= qev->future_event_master_log_pos;
|
|
|
|
}
|
|
|
|
else if (cmp == 0
|
|
|
|
&& rli->group_master_log_pos < qev->future_event_master_log_pos)
|
|
|
|
rli->group_master_log_pos= qev->future_event_master_log_pos;
|
|
|
|
mysql_mutex_unlock(&rli->data_lock);
|
|
|
|
mysql_cond_broadcast(&rli->data_cond);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-09-08 15:25:40 +02:00
|
|
|
/*
|
|
|
|
Wait for any pending deadlock kills. Since deadlock kills happen
|
|
|
|
asynchronously, we need to be sure they will be completed before starting a
|
|
|
|
new transaction. Otherwise the new transaction might suffer a spurious kill.
|
|
|
|
*/
|
|
|
|
static void
|
|
|
|
wait_for_pending_deadlock_kill(THD *thd, rpl_group_info *rgi)
|
|
|
|
{
|
|
|
|
PSI_stage_info old_stage;
|
|
|
|
|
|
|
|
mysql_mutex_lock(&thd->LOCK_wakeup_ready);
|
|
|
|
thd->ENTER_COND(&thd->COND_wakeup_ready, &thd->LOCK_wakeup_ready,
|
|
|
|
&stage_waiting_for_deadlock_kill, &old_stage);
|
|
|
|
while (rgi->killed_for_retry == rpl_group_info::RETRY_KILL_PENDING)
|
|
|
|
mysql_cond_wait(&thd->COND_wakeup_ready, &thd->LOCK_wakeup_ready);
|
|
|
|
thd->EXIT_COND(&old_stage);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-11-05 12:01:26 +01:00
|
|
|
static void
|
2015-01-07 14:45:39 +01:00
|
|
|
finish_event_group(rpl_parallel_thread *rpt, uint64 sub_id,
|
|
|
|
rpl_parallel_entry *entry, rpl_group_info *rgi)
|
2013-11-05 12:01:26 +01:00
|
|
|
{
|
2015-01-07 14:45:39 +01:00
|
|
|
THD *thd= rpt->thd;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
wait_for_commit *wfc= &rgi->commit_orderer;
|
2014-03-21 10:11:28 +01:00
|
|
|
int err;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
MDEV-7847: "Slave worker thread retried transaction 10 time(s) in vain, giving up", followed by replication hanging
This patch fixes a bug in the error handling in parallel replication, when one
worker thread gets a failure and other worker threads processing later
transactions have to rollback and abort.
The problem was with the lifetime of group_commit_orderer objects (GCOs).
A GCO is freed when we register that its last event group has committed. This
relies on register_wait_for_prior_commit() and wait_for_prior_commit() to
ensure that the fact that T2 has committed implies that any earlier T1 has
also committed, and can thus no longer execute mark_start_commit().
However, in the error case, the code was skipping the
register_wait_for_prior_commit() and wait_for_prior_commit() calls. Thus
commit ordering was not guaranteed, and a GCO could be freed too early. Then a
later mark_start_commit() would reference deallocated GCO, which could lead to
lost wakeup (causing slave threads to hang) or other corruption.
This patch makes also the error case respect commit order. This way, also the
error case gets the GCO lifetime correct, and the hang no longer occurs.
2015-03-30 14:33:44 +02:00
|
|
|
thd->get_stmt_da()->set_overwrite_status(true);
|
2013-11-05 12:01:26 +01:00
|
|
|
/*
|
|
|
|
Remove any left-over registration to wait for a prior commit to
|
|
|
|
complete. Normally, such wait would already have been removed at
|
MDEV-4506: Parallel replication
MDEV-5217: Incorrect MyISAM event execution order causing incorrect parallel replication
In parallel replication, if transactions A,B group-commit together on the
master, we can execute them in parallel on a replication slave. But then, if
transaction C follows on the master, on the slave, we need to be sure that
both A and B have completed before starting on C to be sure to avoid
conflicts.
The necessary wait is implemented such that B waits for A to commit before it
commits itself (thus preserving commit order). And C waits for B to commit
before it itself can start executing. This way C does not start until both A
and B have completed.
The wait for B's commit on A happens inside the commit processing. However, in
the case of MyISAM with no binlog enabled on the slave, it appears that no
commit processing takes place (since MyISAM is non-transactional), and thus
the wait of B for A was not done. This allowed C to start before A, which can
lead to conflicts and incorrect replication.
Fixed by doing an extra wait for A at the end of B before signalling C.
2013-11-06 14:51:06 +01:00
|
|
|
this point by wait_for_prior_commit() called from within COMMIT
|
|
|
|
processing. However, in case of MyISAM and no binlog, we might not
|
|
|
|
have any commit processing, and so we need to do the wait here,
|
|
|
|
before waking up any subsequent commits, to preserve correct
|
|
|
|
order of event execution. Also, in the error case we might have
|
|
|
|
skipped waiting and thus need to remove it explicitly.
|
|
|
|
|
|
|
|
It is important in the non-error case to do a wait, not just an
|
|
|
|
unregister. Because we might be last in a group-commit that is
|
|
|
|
replicated in parallel, and the following event will then wait
|
|
|
|
for us to complete and rely on this also ensuring that any other
|
|
|
|
event in the group has completed.
|
|
|
|
|
MDEV-7847: "Slave worker thread retried transaction 10 time(s) in vain, giving up", followed by replication hanging
This patch fixes a bug in the error handling in parallel replication, when one
worker thread gets a failure and other worker threads processing later
transactions have to rollback and abort.
The problem was with the lifetime of group_commit_orderer objects (GCOs).
A GCO is freed when we register that its last event group has committed. This
relies on register_wait_for_prior_commit() and wait_for_prior_commit() to
ensure that the fact that T2 has committed implies that any earlier T1 has
also committed, and can thus no longer execute mark_start_commit().
However, in the error case, the code was skipping the
register_wait_for_prior_commit() and wait_for_prior_commit() calls. Thus
commit ordering was not guaranteed, and a GCO could be freed too early. Then a
later mark_start_commit() would reference deallocated GCO, which could lead to
lost wakeup (causing slave threads to hang) or other corruption.
This patch makes also the error case respect commit order. This way, also the
error case gets the GCO lifetime correct, and the hang no longer occurs.
2015-03-30 14:33:44 +02:00
|
|
|
And in the error case, correct GCO lifetime relies on the fact that once
|
|
|
|
the last event group in the GCO has executed wait_for_prior_commit(),
|
|
|
|
all earlier event groups have also committed; this way no more
|
|
|
|
mark_start_commit() calls can be made and it is safe to de-allocate
|
|
|
|
the GCO.
|
2013-11-05 12:01:26 +01:00
|
|
|
*/
|
MDEV-7847: "Slave worker thread retried transaction 10 time(s) in vain, giving up", followed by replication hanging
This patch fixes a bug in the error handling in parallel replication, when one
worker thread gets a failure and other worker threads processing later
transactions have to rollback and abort.
The problem was with the lifetime of group_commit_orderer objects (GCOs).
A GCO is freed when we register that its last event group has committed. This
relies on register_wait_for_prior_commit() and wait_for_prior_commit() to
ensure that the fact that T2 has committed implies that any earlier T1 has
also committed, and can thus no longer execute mark_start_commit().
However, in the error case, the code was skipping the
register_wait_for_prior_commit() and wait_for_prior_commit() calls. Thus
commit ordering was not guaranteed, and a GCO could be freed too early. Then a
later mark_start_commit() would reference deallocated GCO, which could lead to
lost wakeup (causing slave threads to hang) or other corruption.
This patch makes also the error case respect commit order. This way, also the
error case gets the GCO lifetime correct, and the hang no longer occurs.
2015-03-30 14:33:44 +02:00
|
|
|
err= wfc->wait_for_prior_commit(thd);
|
|
|
|
if (unlikely(err) && !rgi->worker_error)
|
2014-03-21 10:11:28 +01:00
|
|
|
signal_error_to_sql_driver_thread(thd, rgi, err);
|
2013-11-05 12:01:26 +01:00
|
|
|
thd->wait_for_commit_ptr= NULL;
|
|
|
|
|
2015-01-07 14:45:39 +01:00
|
|
|
mysql_mutex_lock(&entry->LOCK_parallel_entry);
|
2013-11-05 12:01:26 +01:00
|
|
|
/*
|
2015-01-07 14:45:39 +01:00
|
|
|
We need to mark that this event group started its commit phase, in case we
|
|
|
|
missed it before (otherwise we would deadlock the next event group that is
|
|
|
|
waiting for this). In most cases (normal DML), it will be a no-op.
|
2013-11-05 12:01:26 +01:00
|
|
|
*/
|
2015-01-07 14:45:39 +01:00
|
|
|
rgi->mark_start_commit_no_lock();
|
|
|
|
|
2013-11-05 12:01:26 +01:00
|
|
|
if (entry->last_committed_sub_id < sub_id)
|
2015-01-07 14:45:39 +01:00
|
|
|
{
|
|
|
|
/*
|
|
|
|
Record that this event group has finished (eg. transaction is
|
|
|
|
committed, if transactional), so other event groups will no longer
|
|
|
|
attempt to wait for us to commit. Once we have increased
|
|
|
|
entry->last_committed_sub_id, no other threads will execute
|
|
|
|
register_wait_for_prior_commit() against us. Thus, by doing one
|
|
|
|
extra (usually redundant) wakeup_subsequent_commits() we can ensure
|
|
|
|
that no register_wait_for_prior_commit() can ever happen without a
|
|
|
|
subsequent wakeup_subsequent_commits() to wake it up.
|
|
|
|
|
|
|
|
We can race here with the next transactions, but that is fine, as
|
|
|
|
long as we check that we do not decrease last_committed_sub_id. If
|
|
|
|
this commit is done, then any prior commits will also have been
|
|
|
|
done and also no longer need waiting for.
|
|
|
|
*/
|
2013-11-05 12:01:26 +01:00
|
|
|
entry->last_committed_sub_id= sub_id;
|
2015-05-26 12:47:35 +02:00
|
|
|
if (entry->need_sub_id_signal)
|
|
|
|
mysql_cond_broadcast(&entry->COND_parallel_entry);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
2015-01-07 14:45:39 +01:00
|
|
|
/* Now free any GCOs in which all transactions have committed. */
|
|
|
|
group_commit_orderer *tmp_gco= rgi->gco;
|
|
|
|
while (tmp_gco &&
|
2015-04-08 11:01:18 +02:00
|
|
|
(!tmp_gco->next_gco || tmp_gco->last_sub_id > sub_id ||
|
|
|
|
tmp_gco->next_gco->wait_count > entry->count_committing_event_groups))
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
We must not free a GCO before the wait_count of the following GCO has
|
|
|
|
been reached and wakeup has been sent. Otherwise we will lose the
|
|
|
|
wakeup and hang (there were several such bugs in the past).
|
|
|
|
|
|
|
|
The intention is that this is ensured already since we only free when
|
|
|
|
the last event group in the GCO has committed
|
|
|
|
(tmp_gco->last_sub_id <= sub_id). However, if we have a bug, we have
|
|
|
|
extra check on next_gco->wait_count to hopefully avoid hanging; we
|
|
|
|
have here an assertion in debug builds that this check does not in
|
|
|
|
fact trigger.
|
|
|
|
*/
|
|
|
|
DBUG_ASSERT(!tmp_gco->next_gco || tmp_gco->last_sub_id > sub_id);
|
2015-01-07 14:45:39 +01:00
|
|
|
tmp_gco= tmp_gco->prev_gco;
|
2015-04-08 11:01:18 +02:00
|
|
|
}
|
2015-01-07 14:45:39 +01:00
|
|
|
while (tmp_gco)
|
|
|
|
{
|
|
|
|
group_commit_orderer *prev_gco= tmp_gco->prev_gco;
|
|
|
|
tmp_gco->next_gco->prev_gco= NULL;
|
|
|
|
rpt->loc_free_gco(tmp_gco);
|
|
|
|
tmp_gco= prev_gco;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
/*
|
|
|
|
If this event group got error, then any following event groups that have
|
|
|
|
not yet started should just skip their group, preparing for stop of the
|
|
|
|
SQL driver thread.
|
|
|
|
*/
|
2014-03-21 10:11:28 +01:00
|
|
|
if (unlikely(rgi->worker_error) &&
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
entry->stop_on_error_sub_id == (uint64)ULONGLONG_MAX)
|
|
|
|
entry->stop_on_error_sub_id= sub_id;
|
2013-11-05 12:01:26 +01:00
|
|
|
mysql_mutex_unlock(&entry->LOCK_parallel_entry);
|
|
|
|
|
2018-02-09 14:00:23 +01:00
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_simulate_wait_at_retry", {
|
|
|
|
if (rgi->current_gtid.seq_no == 1000) {
|
|
|
|
DBUG_ASSERT(entry->stop_on_error_sub_id == sub_id);
|
|
|
|
debug_sync_set_action(thd,
|
|
|
|
STRING_WITH_LEN("now WAIT_FOR proceed_by_1000"));
|
|
|
|
}
|
|
|
|
});
|
|
|
|
|
2016-09-08 15:25:40 +02:00
|
|
|
if (rgi->killed_for_retry == rpl_group_info::RETRY_KILL_PENDING)
|
|
|
|
wait_for_pending_deadlock_kill(thd, rgi);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
thd->clear_error();
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
thd->reset_killed();
|
MDEV-7847: "Slave worker thread retried transaction 10 time(s) in vain, giving up", followed by replication hanging
This patch fixes a bug in the error handling in parallel replication, when one
worker thread gets a failure and other worker threads processing later
transactions have to rollback and abort.
The problem was with the lifetime of group_commit_orderer objects (GCOs).
A GCO is freed when we register that its last event group has committed. This
relies on register_wait_for_prior_commit() and wait_for_prior_commit() to
ensure that the fact that T2 has committed implies that any earlier T1 has
also committed, and can thus no longer execute mark_start_commit().
However, in the error case, the code was skipping the
register_wait_for_prior_commit() and wait_for_prior_commit() calls. Thus
commit ordering was not guaranteed, and a GCO could be freed too early. Then a
later mark_start_commit() would reference deallocated GCO, which could lead to
lost wakeup (causing slave threads to hang) or other corruption.
This patch makes also the error case respect commit order. This way, also the
error case gets the GCO lifetime correct, and the hang no longer occurs.
2015-03-30 14:33:44 +02:00
|
|
|
/*
|
|
|
|
Would do thd->get_stmt_da()->set_overwrite_status(false) here, but
|
|
|
|
reset_diagnostics_area() already does that.
|
|
|
|
*/
|
2014-02-26 16:38:42 +01:00
|
|
|
thd->get_stmt_da()->reset_diagnostics_area();
|
2014-03-21 10:11:28 +01:00
|
|
|
wfc->wakeup_subsequent_commits(rgi->worker_error);
|
2013-11-05 12:01:26 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-12-13 14:26:51 +01:00
|
|
|
static void
|
2014-03-21 10:11:28 +01:00
|
|
|
signal_error_to_sql_driver_thread(THD *thd, rpl_group_info *rgi, int err)
|
2013-12-13 14:26:51 +01:00
|
|
|
{
|
2014-03-21 10:11:28 +01:00
|
|
|
rgi->worker_error= err;
|
2015-08-04 11:20:03 +02:00
|
|
|
/*
|
|
|
|
In case we get an error during commit, inform following transactions that
|
|
|
|
we aborted our commit.
|
|
|
|
*/
|
|
|
|
rgi->unmark_start_commit();
|
2013-12-13 14:26:51 +01:00
|
|
|
rgi->cleanup_context(thd, true);
|
|
|
|
rgi->rli->abort_slave= true;
|
2014-03-03 12:13:55 +01:00
|
|
|
rgi->rli->stop_for_until= false;
|
2013-12-13 14:26:51 +01:00
|
|
|
mysql_mutex_lock(rgi->rli->relay_log.get_log_lock());
|
2017-10-26 16:20:20 +02:00
|
|
|
rgi->rli->relay_log.signal_relay_log_update();
|
2013-12-13 14:26:51 +01:00
|
|
|
mysql_mutex_unlock(rgi->rli->relay_log.get_log_lock());
|
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
static void
|
|
|
|
unlock_or_exit_cond(THD *thd, mysql_mutex_t *lock, bool *did_enter_cond,
|
2014-02-26 16:38:42 +01:00
|
|
|
PSI_stage_info *old_stage)
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
|
|
|
if (*did_enter_cond)
|
|
|
|
{
|
2014-02-26 16:38:42 +01:00
|
|
|
thd->EXIT_COND(old_stage);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
*did_enter_cond= false;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
mysql_mutex_unlock(lock);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
static void
|
|
|
|
register_wait_for_prior_event_group_commit(rpl_group_info *rgi,
|
|
|
|
rpl_parallel_entry *entry)
|
|
|
|
{
|
|
|
|
mysql_mutex_assert_owner(&entry->LOCK_parallel_entry);
|
|
|
|
if (rgi->wait_commit_sub_id > entry->last_committed_sub_id)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
Register that the commit of this event group must wait for the
|
|
|
|
commit of the previous event group to complete before it may
|
|
|
|
complete itself, so that we preserve commit order.
|
|
|
|
*/
|
|
|
|
wait_for_commit *waitee=
|
|
|
|
&rgi->wait_commit_group_info->commit_orderer;
|
|
|
|
rgi->commit_orderer.register_wait_for_prior_commit(waitee);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-05-28 12:32:19 +02:00
|
|
|
/*
|
|
|
|
Do not start parallel execution of this event group until all prior groups
|
|
|
|
have reached the commit phase that are not safe to run in parallel with.
|
|
|
|
*/
|
|
|
|
static bool
|
|
|
|
do_gco_wait(rpl_group_info *rgi, group_commit_orderer *gco,
|
|
|
|
bool *did_enter_cond, PSI_stage_info *old_stage)
|
|
|
|
{
|
|
|
|
THD *thd= rgi->thd;
|
|
|
|
rpl_parallel_entry *entry= rgi->parallel_entry;
|
|
|
|
uint64 wait_count;
|
|
|
|
|
2015-10-22 11:18:34 +02:00
|
|
|
mysql_mutex_assert_owner(&entry->LOCK_parallel_entry);
|
|
|
|
|
2015-05-28 12:32:19 +02:00
|
|
|
if (!gco->installed)
|
|
|
|
{
|
|
|
|
group_commit_orderer *prev_gco= gco->prev_gco;
|
|
|
|
if (prev_gco)
|
|
|
|
{
|
|
|
|
prev_gco->last_sub_id= gco->prior_sub_id;
|
|
|
|
prev_gco->next_gco= gco;
|
|
|
|
}
|
|
|
|
gco->installed= true;
|
|
|
|
}
|
|
|
|
wait_count= gco->wait_count;
|
|
|
|
if (wait_count > entry->count_committing_event_groups)
|
|
|
|
{
|
|
|
|
DEBUG_SYNC(thd, "rpl_parallel_start_waiting_for_prior");
|
|
|
|
thd->ENTER_COND(&gco->COND_group_commit_orderer,
|
|
|
|
&entry->LOCK_parallel_entry,
|
|
|
|
&stage_waiting_for_prior_transaction_to_start_commit,
|
|
|
|
old_stage);
|
|
|
|
*did_enter_cond= true;
|
2016-08-29 12:10:17 +02:00
|
|
|
thd->set_time_for_next_stage();
|
2015-05-28 12:32:19 +02:00
|
|
|
do
|
|
|
|
{
|
2018-05-26 15:57:18 +02:00
|
|
|
if (!rgi->worker_error && unlikely(thd->check_killed(1)))
|
2015-05-28 12:32:19 +02:00
|
|
|
{
|
|
|
|
DEBUG_SYNC(thd, "rpl_parallel_start_waiting_for_prior_killed");
|
|
|
|
thd->clear_error();
|
|
|
|
thd->get_stmt_da()->reset_diagnostics_area();
|
|
|
|
thd->send_kill_message();
|
|
|
|
slave_output_error_info(rgi, thd);
|
|
|
|
signal_error_to_sql_driver_thread(thd, rgi, 1);
|
|
|
|
/*
|
|
|
|
Even though we were killed, we need to continue waiting for the
|
|
|
|
prior event groups to signal that we can continue. Otherwise we
|
|
|
|
mess up the accounting for ordering. However, now that we have
|
|
|
|
marked the error, events will just be skipped rather than
|
|
|
|
executed, and things will progress quickly towards stop.
|
|
|
|
*/
|
|
|
|
}
|
|
|
|
mysql_cond_wait(&gco->COND_group_commit_orderer,
|
|
|
|
&entry->LOCK_parallel_entry);
|
|
|
|
} while (wait_count > entry->count_committing_event_groups);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (entry->force_abort && wait_count > entry->stop_count)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
We are stopping (STOP SLAVE), and this event group is beyond the point
|
|
|
|
where we can safely stop. So return a flag that will cause us to skip,
|
|
|
|
rather than execute, the following events.
|
|
|
|
*/
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-10-22 11:18:34 +02:00
|
|
|
static void
|
|
|
|
do_ftwrl_wait(rpl_group_info *rgi,
|
|
|
|
bool *did_enter_cond, PSI_stage_info *old_stage)
|
|
|
|
{
|
|
|
|
THD *thd= rgi->thd;
|
|
|
|
rpl_parallel_entry *entry= rgi->parallel_entry;
|
|
|
|
uint64 sub_id= rgi->gtid_sub_id;
|
2015-11-23 18:58:30 +01:00
|
|
|
DBUG_ENTER("do_ftwrl_wait");
|
2015-10-22 11:18:34 +02:00
|
|
|
|
|
|
|
mysql_mutex_assert_owner(&entry->LOCK_parallel_entry);
|
|
|
|
|
|
|
|
/*
|
|
|
|
If a FLUSH TABLES WITH READ LOCK (FTWRL) is pending, check if this
|
|
|
|
transaction is later than transactions that have priority to complete
|
|
|
|
before FTWRL. If so, wait here so that FTWRL can proceed and complete
|
|
|
|
first.
|
|
|
|
|
|
|
|
(entry->pause_sub_id is ULONGLONG_MAX if no FTWRL is pending, which makes
|
|
|
|
this test false as required).
|
|
|
|
*/
|
|
|
|
if (unlikely(sub_id > entry->pause_sub_id))
|
|
|
|
{
|
|
|
|
thd->ENTER_COND(&entry->COND_parallel_entry, &entry->LOCK_parallel_entry,
|
|
|
|
&stage_waiting_for_ftwrl, old_stage);
|
|
|
|
*did_enter_cond= true;
|
2016-08-29 12:10:17 +02:00
|
|
|
thd->set_time_for_next_stage();
|
2015-10-22 11:18:34 +02:00
|
|
|
do
|
|
|
|
{
|
|
|
|
if (entry->force_abort || rgi->worker_error)
|
|
|
|
break;
|
2018-04-04 11:16:12 +02:00
|
|
|
if (unlikely(thd->check_killed()))
|
2015-10-22 11:18:34 +02:00
|
|
|
{
|
|
|
|
slave_output_error_info(rgi, thd);
|
|
|
|
signal_error_to_sql_driver_thread(thd, rgi, 1);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
mysql_cond_wait(&entry->COND_parallel_entry, &entry->LOCK_parallel_entry);
|
|
|
|
} while (sub_id > entry->pause_sub_id);
|
|
|
|
|
|
|
|
/*
|
|
|
|
We do not call EXIT_COND() here, as this will be done later by our
|
|
|
|
caller (since we set *did_enter_cond to true).
|
|
|
|
*/
|
|
|
|
}
|
|
|
|
|
|
|
|
if (sub_id > entry->largest_started_sub_id)
|
|
|
|
entry->largest_started_sub_id= sub_id;
|
2015-11-23 18:58:30 +01:00
|
|
|
|
|
|
|
DBUG_VOID_RETURN;
|
2015-10-22 11:18:34 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
static int
|
|
|
|
pool_mark_busy(rpl_parallel_thread_pool *pool, THD *thd)
|
|
|
|
{
|
|
|
|
PSI_stage_info old_stage;
|
|
|
|
int res= 0;
|
|
|
|
|
|
|
|
/*
|
|
|
|
Wait here while the queue is busy. This is done to make FLUSH TABLES WITH
|
|
|
|
READ LOCK work correctly, without incuring extra locking penalties in
|
|
|
|
normal operation. FLUSH TABLES WITH READ LOCK needs to lock threads in the
|
|
|
|
thread pool, and for this we need to make sure the pool will not go away
|
|
|
|
during the operation. The LOCK_rpl_thread_pool is not suitable for
|
|
|
|
this. It is taken by release_thread() while holding LOCK_rpl_thread; so it
|
|
|
|
must be released before locking any LOCK_rpl_thread lock, or a deadlock
|
|
|
|
can occur.
|
|
|
|
|
|
|
|
So we protect the infrequent operations of FLUSH TABLES WITH READ LOCK and
|
|
|
|
pool size changes with this condition wait.
|
|
|
|
*/
|
|
|
|
mysql_mutex_lock(&pool->LOCK_rpl_thread_pool);
|
|
|
|
if (thd)
|
2016-08-29 12:10:17 +02:00
|
|
|
{
|
2015-10-22 11:18:34 +02:00
|
|
|
thd->ENTER_COND(&pool->COND_rpl_thread_pool, &pool->LOCK_rpl_thread_pool,
|
|
|
|
&stage_waiting_for_rpl_thread_pool, &old_stage);
|
2016-08-29 12:10:17 +02:00
|
|
|
thd->set_time_for_next_stage();
|
|
|
|
}
|
2015-10-22 11:18:34 +02:00
|
|
|
while (pool->busy)
|
|
|
|
{
|
2018-04-04 11:16:12 +02:00
|
|
|
if (thd && unlikely(thd->check_killed()))
|
2015-10-22 11:18:34 +02:00
|
|
|
{
|
|
|
|
res= 1;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
mysql_cond_wait(&pool->COND_rpl_thread_pool, &pool->LOCK_rpl_thread_pool);
|
|
|
|
}
|
|
|
|
if (!res)
|
|
|
|
pool->busy= true;
|
|
|
|
if (thd)
|
|
|
|
thd->EXIT_COND(&old_stage);
|
|
|
|
else
|
|
|
|
mysql_mutex_unlock(&pool->LOCK_rpl_thread_pool);
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
static void
|
|
|
|
pool_mark_not_busy(rpl_parallel_thread_pool *pool)
|
|
|
|
{
|
|
|
|
mysql_mutex_lock(&pool->LOCK_rpl_thread_pool);
|
|
|
|
DBUG_ASSERT(pool->busy);
|
|
|
|
pool->busy= false;
|
|
|
|
mysql_cond_broadcast(&pool->COND_rpl_thread_pool);
|
|
|
|
mysql_mutex_unlock(&pool->LOCK_rpl_thread_pool);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void
|
|
|
|
rpl_unpause_after_ftwrl(THD *thd)
|
|
|
|
{
|
|
|
|
uint32 i;
|
|
|
|
rpl_parallel_thread_pool *pool= &global_rpl_thread_pool;
|
2015-11-23 18:58:30 +01:00
|
|
|
DBUG_ENTER("rpl_unpause_after_ftwrl");
|
2015-10-22 11:18:34 +02:00
|
|
|
|
|
|
|
DBUG_ASSERT(pool->busy);
|
|
|
|
|
|
|
|
for (i= 0; i < pool->count; ++i)
|
|
|
|
{
|
|
|
|
rpl_parallel_entry *e;
|
|
|
|
rpl_parallel_thread *rpt= pool->threads[i];
|
|
|
|
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
|
|
|
if (!rpt->current_owner)
|
|
|
|
{
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
e= rpt->current_entry;
|
|
|
|
mysql_mutex_lock(&e->LOCK_parallel_entry);
|
|
|
|
rpt->pause_for_ftwrl = false;
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
e->pause_sub_id= (uint64)ULONGLONG_MAX;
|
|
|
|
mysql_cond_broadcast(&e->COND_parallel_entry);
|
|
|
|
mysql_mutex_unlock(&e->LOCK_parallel_entry);
|
|
|
|
}
|
|
|
|
|
|
|
|
pool_mark_not_busy(pool);
|
2015-11-23 18:58:30 +01:00
|
|
|
DBUG_VOID_RETURN;
|
2015-10-22 11:18:34 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
/*
|
|
|
|
.
|
|
|
|
|
|
|
|
Note: in case of error return, rpl_unpause_after_ftwrl() must _not_ be called.
|
|
|
|
*/
|
|
|
|
int
|
|
|
|
rpl_pause_for_ftwrl(THD *thd)
|
|
|
|
{
|
|
|
|
uint32 i;
|
|
|
|
rpl_parallel_thread_pool *pool= &global_rpl_thread_pool;
|
|
|
|
int err;
|
2015-11-23 18:58:30 +01:00
|
|
|
DBUG_ENTER("rpl_pause_for_ftwrl");
|
2015-10-22 11:18:34 +02:00
|
|
|
|
|
|
|
/*
|
|
|
|
While the count_pending_pause_for_ftwrl counter is non-zero, the pool
|
|
|
|
cannot be shutdown/resized, so threads are guaranteed to not disappear.
|
|
|
|
|
|
|
|
This is required to safely be able to access the individual threads below.
|
|
|
|
(We cannot lock an individual thread while holding LOCK_rpl_thread_pool,
|
|
|
|
as this can deadlock against release_thread()).
|
|
|
|
*/
|
|
|
|
if ((err= pool_mark_busy(pool, thd)))
|
2015-11-23 18:58:30 +01:00
|
|
|
DBUG_RETURN(err);
|
2015-10-22 11:18:34 +02:00
|
|
|
|
|
|
|
for (i= 0; i < pool->count; ++i)
|
|
|
|
{
|
|
|
|
PSI_stage_info old_stage;
|
|
|
|
rpl_parallel_entry *e;
|
|
|
|
rpl_parallel_thread *rpt= pool->threads[i];
|
|
|
|
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
|
|
|
if (!rpt->current_owner)
|
|
|
|
{
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
e= rpt->current_entry;
|
|
|
|
mysql_mutex_lock(&e->LOCK_parallel_entry);
|
|
|
|
/*
|
|
|
|
Setting the rpt->pause_for_ftwrl flag makes sure that the thread will not
|
|
|
|
de-allocate itself until signalled to do so by rpl_unpause_after_ftwrl().
|
|
|
|
*/
|
|
|
|
rpt->pause_for_ftwrl = true;
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
++e->need_sub_id_signal;
|
|
|
|
if (e->pause_sub_id == (uint64)ULONGLONG_MAX)
|
|
|
|
e->pause_sub_id= e->largest_started_sub_id;
|
|
|
|
thd->ENTER_COND(&e->COND_parallel_entry, &e->LOCK_parallel_entry,
|
|
|
|
&stage_waiting_for_ftwrl_threads_to_pause, &old_stage);
|
2016-08-29 12:10:17 +02:00
|
|
|
thd->set_time_for_next_stage();
|
2015-10-22 11:18:34 +02:00
|
|
|
while (e->pause_sub_id < (uint64)ULONGLONG_MAX &&
|
|
|
|
e->last_committed_sub_id < e->pause_sub_id &&
|
|
|
|
!err)
|
|
|
|
{
|
2018-04-04 11:16:12 +02:00
|
|
|
if (unlikely(thd->check_killed()))
|
2015-10-22 11:18:34 +02:00
|
|
|
{
|
|
|
|
err= 1;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
mysql_cond_wait(&e->COND_parallel_entry, &e->LOCK_parallel_entry);
|
|
|
|
};
|
|
|
|
--e->need_sub_id_signal;
|
|
|
|
thd->EXIT_COND(&old_stage);
|
|
|
|
if (err)
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (err)
|
|
|
|
rpl_unpause_after_ftwrl(thd);
|
2015-11-23 18:58:30 +01:00
|
|
|
DBUG_RETURN(err);
|
2015-10-22 11:18:34 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-05-13 13:42:06 +02:00
|
|
|
#ifndef DBUG_OFF
|
|
|
|
static int
|
|
|
|
dbug_simulate_tmp_error(rpl_group_info *rgi, THD *thd)
|
|
|
|
{
|
|
|
|
if (rgi->current_gtid.domain_id == 0 && rgi->current_gtid.seq_no == 100 &&
|
|
|
|
rgi->retry_event_count == 4)
|
|
|
|
{
|
|
|
|
thd->clear_error();
|
|
|
|
thd->get_stmt_da()->reset_diagnostics_area();
|
|
|
|
my_error(ER_LOCK_DEADLOCK, MYF(0));
|
|
|
|
return 1;
|
|
|
|
}
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
|
2014-05-08 14:20:18 +02:00
|
|
|
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
/*
|
|
|
|
If we detect a deadlock due to eg. storage engine locks that conflict with
|
|
|
|
the fixed commit order, then the later transaction will be killed
|
|
|
|
asynchroneously to allow the former to complete its commit.
|
|
|
|
|
|
|
|
In this case, we convert the 'killed' error into a deadlock error, and retry
|
2014-12-05 16:09:48 +01:00
|
|
|
the later transaction.
|
|
|
|
|
|
|
|
If we are doing optimistic parallel apply of transactions not known to be
|
|
|
|
safe, we convert any error to a deadlock error, but then at retry we will
|
|
|
|
wait for prior transactions to commit first, so that the retries can be
|
|
|
|
done non-speculative.
|
|
|
|
*/
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
static void
|
|
|
|
convert_kill_to_deadlock_error(rpl_group_info *rgi)
|
|
|
|
{
|
|
|
|
THD *thd= rgi->thd;
|
2014-07-10 13:55:53 +02:00
|
|
|
int err_code;
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
|
2014-07-10 13:55:53 +02:00
|
|
|
if (!thd->get_stmt_da()->is_error())
|
|
|
|
return;
|
|
|
|
err_code= thd->get_stmt_da()->sql_errno();
|
2014-12-05 16:09:48 +01:00
|
|
|
if ((rgi->speculation == rpl_group_info::SPECULATE_OPTIMISTIC &&
|
|
|
|
err_code != ER_PRIOR_COMMIT_FAILED) ||
|
|
|
|
((err_code == ER_QUERY_INTERRUPTED || err_code == ER_CONNECTION_KILLED) &&
|
|
|
|
rgi->killed_for_retry))
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
{
|
|
|
|
thd->clear_error();
|
|
|
|
my_error(ER_LOCK_DEADLOCK, MYF(0));
|
|
|
|
thd->reset_killed();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-09-02 09:57:18 +02:00
|
|
|
/*
|
|
|
|
Check if an event marks the end of an event group. Returns non-zero if so,
|
|
|
|
zero otherwise.
|
|
|
|
|
|
|
|
In addition, returns 1 if the group is committing, 2 if it is rolling back.
|
|
|
|
*/
|
|
|
|
static int
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
is_group_ending(Log_event *ev, Log_event_type event_type)
|
|
|
|
{
|
2015-09-02 09:57:18 +02:00
|
|
|
if (event_type == XID_EVENT)
|
|
|
|
return 1;
|
2016-10-20 18:00:59 +02:00
|
|
|
if (event_type == QUERY_EVENT) // COMMIT/ROLLBACK are never compressed
|
2015-09-02 09:57:18 +02:00
|
|
|
{
|
|
|
|
Query_log_event *qev = (Query_log_event *)ev;
|
|
|
|
if (qev->is_commit())
|
|
|
|
return 1;
|
|
|
|
if (qev->is_rollback())
|
|
|
|
return 2;
|
|
|
|
}
|
|
|
|
return 0;
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-05-08 14:20:18 +02:00
|
|
|
static int
|
|
|
|
retry_event_group(rpl_group_info *rgi, rpl_parallel_thread *rpt,
|
|
|
|
rpl_parallel_thread::queued_event *orig_qev)
|
|
|
|
{
|
|
|
|
IO_CACHE rlog;
|
2014-05-15 15:52:08 +02:00
|
|
|
LOG_INFO linfo;
|
|
|
|
File fd= (File)-1;
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
const char *errmsg;
|
2014-05-08 14:20:18 +02:00
|
|
|
inuse_relaylog *ir= rgi->relay_log;
|
2014-05-13 13:42:06 +02:00
|
|
|
uint64 event_count;
|
2014-05-08 14:20:18 +02:00
|
|
|
uint64 events_to_execute= rgi->retry_event_count;
|
|
|
|
Relay_log_info *rli= rgi->rli;
|
2014-05-13 13:42:06 +02:00
|
|
|
int err;
|
2014-05-08 14:20:18 +02:00
|
|
|
ulonglong cur_offset, old_offset;
|
|
|
|
char log_name[FN_REFLEN];
|
|
|
|
THD *thd= rgi->thd;
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
rpl_parallel_entry *entry= rgi->parallel_entry;
|
2014-05-13 13:42:06 +02:00
|
|
|
ulong retries= 0;
|
2014-11-13 10:09:46 +01:00
|
|
|
Format_description_log_event *description_event= NULL;
|
2014-05-08 14:20:18 +02:00
|
|
|
|
|
|
|
do_retry:
|
2014-05-13 13:42:06 +02:00
|
|
|
event_count= 0;
|
|
|
|
err= 0;
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
errmsg= NULL;
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
|
|
|
|
/*
|
|
|
|
If we already started committing before getting the deadlock (or other
|
|
|
|
error) that caused us to need to retry, we have already signalled
|
|
|
|
subsequent transactions that we have started committing. This is
|
|
|
|
potentially a problem, as now we will rollback, and if subsequent
|
|
|
|
transactions would start to execute now, they could see an unexpected
|
|
|
|
state of the database and get eg. key not found or duplicate key error.
|
|
|
|
|
|
|
|
However, to get a deadlock in the first place, there must have been
|
|
|
|
another earlier transaction that is waiting for us. Thus that other
|
|
|
|
transaction has _not_ yet started to commit, and any subsequent
|
|
|
|
transactions will still be waiting at this point.
|
|
|
|
|
|
|
|
So here, we decrement back the count of transactions that started
|
|
|
|
committing (if we already incremented it), undoing the effect of an
|
|
|
|
earlier mark_start_commit(). Then later, when the retry succeeds and we
|
|
|
|
commit again, we can do a new mark_start_commit() and eventually wake up
|
|
|
|
subsequent transactions at the proper time.
|
|
|
|
|
|
|
|
We need to do the unmark before the rollback, to be sure that the
|
|
|
|
transaction we deadlocked with will not signal that it started to commit
|
|
|
|
until after the unmark.
|
|
|
|
*/
|
2015-08-04 11:20:03 +02:00
|
|
|
DBUG_EXECUTE_IF("inject_mdev8302", { my_sleep(20000);});
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
rgi->unmark_start_commit();
|
2015-01-07 14:45:39 +01:00
|
|
|
DEBUG_SYNC(thd, "rpl_parallel_retry_after_unmark");
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
|
|
|
|
/*
|
|
|
|
We might get the deadlock error that causes the retry during commit, while
|
|
|
|
sitting in wait_for_prior_commit(). If this happens, we will have a
|
|
|
|
pending error in the wait_for_commit object. So clear this by
|
|
|
|
unregistering (and later re-registering) the wait.
|
|
|
|
*/
|
|
|
|
if(thd->wait_for_commit_ptr)
|
|
|
|
thd->wait_for_commit_ptr->unregister_wait_for_prior_commit();
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
DBUG_EXECUTE_IF("inject_mdev8031", {
|
|
|
|
/* Simulate that we get deadlock killed at this exact point. */
|
2016-09-08 15:25:40 +02:00
|
|
|
rgi->killed_for_retry= rpl_group_info::RETRY_KILL_KILLED;
|
2017-08-05 18:26:10 +02:00
|
|
|
thd->set_killed(KILL_CONNECTION);
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
});
|
2018-02-09 14:00:23 +01:00
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_simulate_wait_at_retry", {
|
|
|
|
if (rgi->current_gtid.seq_no == 1001) {
|
|
|
|
debug_sync_set_action(thd,
|
|
|
|
STRING_WITH_LEN("rpl_parallel_simulate_wait_at_retry WAIT_FOR proceed_by_1001"));
|
|
|
|
}
|
|
|
|
DEBUG_SYNC(thd, "rpl_parallel_simulate_wait_at_retry");
|
|
|
|
});
|
|
|
|
|
2014-05-08 14:20:18 +02:00
|
|
|
rgi->cleanup_context(thd, 1);
|
2016-09-08 15:25:40 +02:00
|
|
|
wait_for_pending_deadlock_kill(thd, rgi);
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
thd->reset_killed();
|
|
|
|
thd->clear_error();
|
2016-09-08 15:25:40 +02:00
|
|
|
rgi->killed_for_retry = rpl_group_info::RETRY_KILL_NONE;
|
2014-05-08 14:20:18 +02:00
|
|
|
|
2014-08-13 13:34:28 +02:00
|
|
|
/*
|
2016-03-04 01:09:37 +01:00
|
|
|
If we retry due to a deadlock kill that occurred during the commit step, we
|
2014-08-13 13:34:28 +02:00
|
|
|
might have already updated (but not committed) an update of table
|
|
|
|
mysql.gtid_slave_pos, and cleared the gtid_pending flag. Now we have
|
|
|
|
rolled back any such update, so we must set the gtid_pending flag back to
|
|
|
|
true so that we will do a new update when/if we succeed with the retry.
|
|
|
|
*/
|
|
|
|
rgi->gtid_pending= true;
|
|
|
|
|
2014-05-08 14:20:18 +02:00
|
|
|
mysql_mutex_lock(&rli->data_lock);
|
|
|
|
++rli->retried_trans;
|
|
|
|
statistic_increment(slave_retried_transactions, LOCK_status);
|
|
|
|
mysql_mutex_unlock(&rli->data_lock);
|
|
|
|
|
2014-12-05 16:09:48 +01:00
|
|
|
for (;;)
|
|
|
|
{
|
|
|
|
mysql_mutex_lock(&entry->LOCK_parallel_entry);
|
2018-02-09 14:00:23 +01:00
|
|
|
if (entry->stop_on_error_sub_id == (uint64) ULONGLONG_MAX ||
|
|
|
|
#ifndef DBUG_OFF
|
|
|
|
(DBUG_EVALUATE_IF("simulate_mdev_12746", 1, 0)) ||
|
|
|
|
#endif
|
|
|
|
rgi->gtid_sub_id < entry->stop_on_error_sub_id)
|
|
|
|
{
|
|
|
|
register_wait_for_prior_event_group_commit(rgi, entry);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
A failure of a preceeding "parent" transaction may not be
|
|
|
|
seen by the current one through its own worker_error.
|
|
|
|
Such induced error gets set by ourselves now.
|
|
|
|
*/
|
|
|
|
err= rgi->worker_error= 1;
|
|
|
|
my_error(ER_PRIOR_COMMIT_FAILED, MYF(0));
|
|
|
|
mysql_mutex_unlock(&entry->LOCK_parallel_entry);
|
|
|
|
goto err;
|
|
|
|
}
|
2014-12-05 16:09:48 +01:00
|
|
|
mysql_mutex_unlock(&entry->LOCK_parallel_entry);
|
|
|
|
|
|
|
|
/*
|
2015-03-30 14:16:57 +02:00
|
|
|
Let us wait for all prior transactions to complete before trying again.
|
|
|
|
This way, we avoid repeatedly conflicting with and getting deadlock
|
|
|
|
killed by the same earlier transaction.
|
2014-12-05 16:09:48 +01:00
|
|
|
*/
|
|
|
|
if (!(err= thd->wait_for_prior_commit()))
|
|
|
|
{
|
|
|
|
rgi->speculation = rpl_group_info::SPECULATE_WAIT;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
convert_kill_to_deadlock_error(rgi);
|
|
|
|
if (!has_temporary_error(thd))
|
|
|
|
goto err;
|
|
|
|
/*
|
|
|
|
If we get a temporary error such as a deadlock kill, we can safely
|
|
|
|
ignore it, as we already rolled back.
|
|
|
|
|
|
|
|
But we still want to retry the wait for the prior transaction to
|
|
|
|
complete its commit.
|
|
|
|
*/
|
|
|
|
thd->clear_error();
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
thd->reset_killed();
|
2014-12-05 16:09:48 +01:00
|
|
|
if(thd->wait_for_commit_ptr)
|
|
|
|
thd->wait_for_commit_ptr->unregister_wait_for_prior_commit();
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
DBUG_EXECUTE_IF("inject_mdev8031", {
|
|
|
|
/* Inject a small sleep to give prior transaction a chance to commit. */
|
|
|
|
my_sleep(100000);
|
|
|
|
});
|
2014-12-05 16:09:48 +01:00
|
|
|
}
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
/*
|
|
|
|
Let us clear any lingering deadlock kill one more time, here after
|
|
|
|
wait_for_prior_commit() has completed. This should rule out any
|
|
|
|
possibility of an old deadlock kill lingering on beyond this point.
|
|
|
|
*/
|
|
|
|
thd->reset_killed();
|
|
|
|
|
2014-07-08 12:54:47 +02:00
|
|
|
strmake_buf(log_name, ir->name);
|
2014-05-08 14:20:18 +02:00
|
|
|
if ((fd= open_binlog(&rlog, log_name, &errmsg)) <0)
|
2014-05-15 15:52:08 +02:00
|
|
|
{
|
|
|
|
err= 1;
|
|
|
|
goto err;
|
|
|
|
}
|
2014-05-08 14:20:18 +02:00
|
|
|
cur_offset= rgi->retry_start_offset;
|
2014-11-13 10:09:46 +01:00
|
|
|
delete description_event;
|
|
|
|
description_event=
|
|
|
|
read_relay_log_description_event(&rlog, cur_offset, &errmsg);
|
|
|
|
if (!description_event)
|
|
|
|
{
|
|
|
|
err= 1;
|
|
|
|
goto err;
|
|
|
|
}
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
DBUG_EXECUTE_IF("inject_mdev8031", {
|
|
|
|
/* Simulate pending KILL caught in read_relay_log_description_event(). */
|
2018-05-26 15:57:18 +02:00
|
|
|
if (unlikely(thd->check_killed()))
|
|
|
|
{
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
err= 1;
|
|
|
|
goto err;
|
|
|
|
}
|
|
|
|
});
|
2014-05-08 14:20:18 +02:00
|
|
|
my_b_seek(&rlog, cur_offset);
|
|
|
|
|
|
|
|
do
|
|
|
|
{
|
|
|
|
Log_event_type event_type;
|
|
|
|
Log_event *ev;
|
2014-05-15 15:52:08 +02:00
|
|
|
rpl_parallel_thread::queued_event *qev;
|
2014-05-08 14:20:18 +02:00
|
|
|
|
2014-05-15 15:52:08 +02:00
|
|
|
/* The loop is here so we can try again the next relay log file on EOF. */
|
|
|
|
for (;;)
|
2014-05-08 14:20:18 +02:00
|
|
|
{
|
2014-05-15 15:52:08 +02:00
|
|
|
old_offset= cur_offset;
|
2017-10-26 11:46:45 +02:00
|
|
|
ev= Log_event::read_log_event(&rlog, description_event,
|
2014-05-15 15:52:08 +02:00
|
|
|
opt_slave_sql_verify_checksum);
|
|
|
|
cur_offset= my_b_tell(&rlog);
|
2014-05-08 14:20:18 +02:00
|
|
|
|
2014-05-15 15:52:08 +02:00
|
|
|
if (ev)
|
|
|
|
break;
|
2018-04-04 11:16:12 +02:00
|
|
|
if (unlikely(rlog.error < 0))
|
2014-05-08 14:20:18 +02:00
|
|
|
{
|
2014-05-15 15:52:08 +02:00
|
|
|
errmsg= "slave SQL thread aborted because of I/O error";
|
2014-05-08 14:20:18 +02:00
|
|
|
err= 1;
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
goto check_retry;
|
2014-05-08 14:20:18 +02:00
|
|
|
}
|
2018-04-04 11:16:12 +02:00
|
|
|
if (unlikely(rlog.error > 0))
|
2014-05-15 15:52:08 +02:00
|
|
|
{
|
|
|
|
sql_print_error("Slave SQL thread: I/O error reading "
|
|
|
|
"event(errno: %d cur_log->error: %d)",
|
|
|
|
my_errno, rlog.error);
|
|
|
|
errmsg= "Aborting slave SQL thread because of partial event read";
|
|
|
|
err= 1;
|
|
|
|
goto err;
|
|
|
|
}
|
|
|
|
/* EOF. Move to the next relay log. */
|
|
|
|
end_io_cache(&rlog);
|
|
|
|
mysql_file_close(fd, MYF(MY_WME));
|
|
|
|
fd= (File)-1;
|
|
|
|
|
|
|
|
/* Find the next relay log file. */
|
|
|
|
if((err= rli->relay_log.find_log_pos(&linfo, log_name, 1)) ||
|
|
|
|
(err= rli->relay_log.find_next_log(&linfo, 1)))
|
|
|
|
{
|
|
|
|
char buff[22];
|
|
|
|
sql_print_error("next log error: %d offset: %s log: %s",
|
|
|
|
err,
|
|
|
|
llstr(linfo.index_file_offset, buff),
|
|
|
|
log_name);
|
|
|
|
goto err;
|
|
|
|
}
|
|
|
|
strmake_buf(log_name ,linfo.log_file_name);
|
|
|
|
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
DBUG_EXECUTE_IF("inject_retry_event_group_open_binlog_kill", {
|
|
|
|
if (retries < 2)
|
|
|
|
{
|
|
|
|
/* Simulate that we get deadlock killed during open_binlog(). */
|
2015-05-13 13:28:34 +02:00
|
|
|
thd->reset_for_next_command();
|
2016-09-08 15:25:40 +02:00
|
|
|
rgi->killed_for_retry= rpl_group_info::RETRY_KILL_KILLED;
|
2017-08-05 18:26:10 +02:00
|
|
|
thd->set_killed(KILL_CONNECTION);
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
thd->send_kill_message();
|
|
|
|
fd= (File)-1;
|
|
|
|
err= 1;
|
|
|
|
goto check_retry;
|
|
|
|
}
|
|
|
|
});
|
2014-05-15 15:52:08 +02:00
|
|
|
if ((fd= open_binlog(&rlog, log_name, &errmsg)) <0)
|
|
|
|
{
|
|
|
|
err= 1;
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
goto check_retry;
|
2014-05-15 15:52:08 +02:00
|
|
|
}
|
2015-09-02 09:58:08 +02:00
|
|
|
description_event->reset_crypto();
|
2014-05-15 15:52:08 +02:00
|
|
|
/* Loop to try again on the new log file. */
|
2014-05-08 14:20:18 +02:00
|
|
|
}
|
2014-05-15 15:52:08 +02:00
|
|
|
|
|
|
|
event_type= ev->get_type_code();
|
2014-11-13 10:09:46 +01:00
|
|
|
if (event_type == FORMAT_DESCRIPTION_EVENT)
|
|
|
|
{
|
2015-09-02 09:58:08 +02:00
|
|
|
Format_description_log_event *newde= (Format_description_log_event*)ev;
|
|
|
|
newde->copy_crypto_data(description_event);
|
2014-11-13 10:09:46 +01:00
|
|
|
delete description_event;
|
2015-09-02 09:58:08 +02:00
|
|
|
description_event= newde;
|
2014-11-13 10:09:46 +01:00
|
|
|
continue;
|
2015-09-02 09:58:08 +02:00
|
|
|
}
|
|
|
|
else if (event_type == START_ENCRYPTION_EVENT)
|
|
|
|
{
|
|
|
|
description_event->start_decryption((Start_encryption_log_event*)ev);
|
|
|
|
delete ev;
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
else if (!Log_event::is_group_event(event_type))
|
2014-05-15 15:52:08 +02:00
|
|
|
{
|
|
|
|
delete ev;
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
ev->thd= thd;
|
|
|
|
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
2014-11-13 10:46:09 +01:00
|
|
|
qev= rpt->retry_get_qev(ev, orig_qev, log_name, old_offset,
|
2014-05-15 15:52:08 +02:00
|
|
|
cur_offset - old_offset);
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
if (!qev)
|
|
|
|
{
|
|
|
|
delete ev;
|
|
|
|
my_error(ER_OUT_OF_RESOURCES, MYF(0));
|
|
|
|
err= 1;
|
|
|
|
goto err;
|
|
|
|
}
|
2015-09-02 09:57:18 +02:00
|
|
|
if (is_group_ending(ev, event_type) == 1)
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
rgi->mark_start_commit();
|
|
|
|
|
2014-05-15 15:52:08 +02:00
|
|
|
err= rpt_handle_event(qev, rpt);
|
|
|
|
++event_count;
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
|
|
|
rpt->free_qev(qev);
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
|
2014-05-08 14:20:18 +02:00
|
|
|
delete_or_keep_event_post_apply(rgi, event_type, ev);
|
2014-05-13 13:42:06 +02:00
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_simulate_double_temp_err_gtid_0_x_100",
|
|
|
|
if (retries == 0) err= dbug_simulate_tmp_error(rgi, thd););
|
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_simulate_infinite_temp_err_gtid_0_x_100",
|
|
|
|
err= dbug_simulate_tmp_error(rgi, thd););
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
if (!err)
|
|
|
|
continue;
|
|
|
|
|
|
|
|
check_retry:
|
|
|
|
convert_kill_to_deadlock_error(rgi);
|
|
|
|
if (has_temporary_error(thd))
|
2014-05-08 14:20:18 +02:00
|
|
|
{
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
++retries;
|
|
|
|
if (retries < slave_trans_retries)
|
2014-05-13 13:42:06 +02:00
|
|
|
{
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
if (fd >= 0)
|
2014-05-13 13:42:06 +02:00
|
|
|
{
|
|
|
|
end_io_cache(&rlog);
|
|
|
|
mysql_file_close(fd, MYF(MY_WME));
|
2014-05-15 15:52:08 +02:00
|
|
|
fd= (File)-1;
|
2014-05-13 13:42:06 +02:00
|
|
|
}
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
goto do_retry;
|
2014-05-13 13:42:06 +02:00
|
|
|
}
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
sql_print_error("Slave worker thread retried transaction %lu time(s) "
|
|
|
|
"in vain, giving up. Consider raising the value of "
|
|
|
|
"the slave_transaction_retries variable.",
|
|
|
|
slave_trans_retries);
|
2014-05-08 14:20:18 +02:00
|
|
|
}
|
MDEV-8031: Parallel replication stops on "connection killed" error (probably incorrectly handled deadlock kill)
There was a rare race, where a deadlock error might not be correctly
handled, causing the slave to stop with something like this in the error
log:
150423 14:04:10 [ERROR] Slave SQL: Connection was killed, Gtid 0-1-2, Internal MariaDB error code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Deadlock found when trying to get lock; try restarting transaction Error_code: 1213
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [Warning] Slave: Connection was killed Error_code: 1927
150423 14:04:10 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'master-bin.000001 position 1234
The problem was incorrect error handling. When a deadlock is detected, it
causes a KILL CONNECTION on the offending thread. This error is then later
converted to a deadlock error, and the transaction is retried.
However, the deadlock error was not cleared at the start of the retry, nor
was the lingering kill signal. So it was possible to get another deadlock
kill early during retry. If this happened with particular thread
scheduling/timing, it was possible that the new KILL CONNECTION error was
masked by the earlier deadlock error, so that the second kill was not
properly converted into a deadlock error and retry.
This patch adds code that clears the old error and killed flag before
starting the retry. It also adds code to handle a deadlock kill caught in a
couple of places where it was not handled before.
2015-04-23 14:09:15 +02:00
|
|
|
goto err;
|
|
|
|
|
2014-05-08 14:20:18 +02:00
|
|
|
} while (event_count < events_to_execute);
|
|
|
|
|
|
|
|
err:
|
|
|
|
|
2014-11-13 10:09:46 +01:00
|
|
|
if (description_event)
|
|
|
|
delete description_event;
|
2014-05-15 15:52:08 +02:00
|
|
|
if (fd >= 0)
|
|
|
|
{
|
|
|
|
end_io_cache(&rlog);
|
|
|
|
mysql_file_close(fd, MYF(MY_WME));
|
|
|
|
}
|
|
|
|
if (errmsg)
|
|
|
|
sql_print_error("Error reading relay log event: %s", errmsg);
|
2014-05-08 14:20:18 +02:00
|
|
|
return err;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
pthread_handler_t
|
|
|
|
handle_rpl_parallel_thread(void *arg)
|
|
|
|
{
|
|
|
|
THD *thd;
|
2013-11-01 12:00:11 +01:00
|
|
|
PSI_stage_info old_stage;
|
2013-06-24 10:50:25 +02:00
|
|
|
struct rpl_parallel_thread::queued_event *events;
|
|
|
|
bool group_standalone= true;
|
|
|
|
bool in_event_group= false;
|
2014-03-09 10:27:38 +01:00
|
|
|
bool skip_event_group= false;
|
2013-11-05 12:01:26 +01:00
|
|
|
rpl_group_info *group_rgi= NULL;
|
2015-01-07 14:45:39 +01:00
|
|
|
group_commit_orderer *gco;
|
2013-07-04 13:17:01 +02:00
|
|
|
uint64 event_gtid_sub_id= 0;
|
2014-04-25 12:58:31 +02:00
|
|
|
rpl_sql_thread_info sql_info(NULL);
|
2013-10-14 15:28:16 +02:00
|
|
|
int err;
|
2013-06-24 10:50:25 +02:00
|
|
|
|
|
|
|
struct rpl_parallel_thread *rpt= (struct rpl_parallel_thread *)arg;
|
|
|
|
|
|
|
|
my_thread_init();
|
2016-04-07 18:51:40 +02:00
|
|
|
thd = new THD(next_thread_id());
|
2013-06-24 10:50:25 +02:00
|
|
|
thd->thread_stack = (char*)&thd;
|
2019-01-19 23:32:35 +01:00
|
|
|
server_threads.insert(thd);
|
2013-06-24 10:50:25 +02:00
|
|
|
set_current_thd(thd);
|
|
|
|
pthread_detach_this_thread();
|
|
|
|
thd->init_for_queries();
|
|
|
|
thd->variables.binlog_annotate_row_events= 0;
|
|
|
|
init_thr_lock();
|
|
|
|
thd->store_globals();
|
|
|
|
thd->system_thread= SYSTEM_THREAD_SLAVE_SQL;
|
|
|
|
thd->security_ctx->skip_grants();
|
|
|
|
thd->variables.max_allowed_packet= slave_max_allowed_packet;
|
2018-05-07 11:24:58 +02:00
|
|
|
/* Ensure that slave can exeute any alter table it gets from master */
|
|
|
|
thd->variables.alter_algorithm= (ulong) Alter_info::ALTER_TABLE_ALGORITHM_DEFAULT;
|
2013-06-24 10:50:25 +02:00
|
|
|
thd->slave_thread= 1;
|
2017-07-21 18:56:41 +02:00
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
set_slave_thread_options(thd);
|
|
|
|
thd->client_capabilities = CLIENT_LOCAL_FILES;
|
2013-12-18 16:26:22 +01:00
|
|
|
thd->net.reading_or_writing= 0;
|
2013-06-24 10:50:25 +02:00
|
|
|
thd_proc_info(thd, "Waiting for work from main SQL threads");
|
|
|
|
thd->variables.lock_wait_timeout= LONG_TIMEOUT;
|
2014-04-25 12:58:31 +02:00
|
|
|
thd->system_thread_info.rpl_sql_info= &sql_info;
|
2014-12-05 16:09:48 +01:00
|
|
|
/*
|
|
|
|
We need to use (at least) REPEATABLE READ isolation level. Otherwise
|
|
|
|
speculative parallel apply can run out-of-order and give wrong results
|
|
|
|
for statement-based replication.
|
|
|
|
*/
|
|
|
|
thd->variables.tx_isolation= ISO_REPEATABLE_READ;
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
|
|
|
rpt->thd= thd;
|
|
|
|
|
|
|
|
while (rpt->delay_start)
|
|
|
|
mysql_cond_wait(&rpt->COND_rpl_thread, &rpt->LOCK_rpl_thread);
|
|
|
|
|
|
|
|
rpt->running= true;
|
2013-10-14 15:28:16 +02:00
|
|
|
mysql_cond_signal(&rpt->COND_rpl_thread);
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2016-08-29 12:10:17 +02:00
|
|
|
thd->set_command(COM_SLAVE_WORKER);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
while (!rpt->stop)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
2016-08-29 12:10:17 +02:00
|
|
|
uint wait_count= 0;
|
2014-11-13 10:20:48 +01:00
|
|
|
rpl_parallel_thread::queued_event *qev, *next_qev;
|
|
|
|
|
2013-11-01 12:00:11 +01:00
|
|
|
thd->ENTER_COND(&rpt->COND_rpl_thread, &rpt->LOCK_rpl_thread,
|
|
|
|
&stage_waiting_for_work_from_sql_thread, &old_stage);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
/*
|
|
|
|
There are 4 cases that should cause us to wake up:
|
|
|
|
- Events have been queued for us to handle.
|
|
|
|
- We have an owner, but no events and not inside event group -> we need
|
|
|
|
to release ourself to the thread pool
|
|
|
|
- SQL thread is stopping, and we have an owner but no events, and we are
|
|
|
|
inside an event group; no more events will be queued to us, so we need
|
|
|
|
to abort the group (force_abort==1).
|
|
|
|
- Thread pool shutdown (rpt->stop==1).
|
|
|
|
*/
|
|
|
|
while (!( (events= rpt->event_queue) ||
|
|
|
|
(rpt->current_owner && !in_event_group) ||
|
|
|
|
(rpt->current_owner && group_rgi->parallel_entry->force_abort) ||
|
|
|
|
rpt->stop))
|
2016-08-29 12:10:17 +02:00
|
|
|
{
|
|
|
|
if (!wait_count++)
|
|
|
|
thd->set_time_for_next_stage();
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_cond_wait(&rpt->COND_rpl_thread, &rpt->LOCK_rpl_thread);
|
2016-08-29 12:10:17 +02:00
|
|
|
}
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpt->dequeue1(events);
|
2013-11-01 12:00:11 +01:00
|
|
|
thd->EXIT_COND(&old_stage);
|
2013-06-24 10:50:25 +02:00
|
|
|
|
|
|
|
more_events:
|
2014-11-13 10:20:48 +01:00
|
|
|
for (qev= events; qev; qev= next_qev)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
2013-10-31 14:11:41 +01:00
|
|
|
Log_event_type event_type;
|
2014-11-13 10:20:48 +01:00
|
|
|
rpl_group_info *rgi= qev->rgi;
|
2013-07-03 13:46:33 +02:00
|
|
|
rpl_parallel_entry *entry= rgi->parallel_entry;
|
2015-09-02 09:57:18 +02:00
|
|
|
bool end_of_group;
|
|
|
|
int group_ending;
|
2013-07-03 13:46:33 +02:00
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
next_qev= qev->next;
|
|
|
|
if (qev->typ == rpl_parallel_thread::queued_event::QUEUED_POS_UPDATE)
|
2013-10-31 14:11:41 +01:00
|
|
|
{
|
2014-11-13 10:20:48 +01:00
|
|
|
handle_queued_pos_update(thd, qev);
|
|
|
|
rpt->loc_free_qev(qev);
|
2013-10-31 14:11:41 +01:00
|
|
|
continue;
|
|
|
|
}
|
2014-11-13 10:20:48 +01:00
|
|
|
else if (qev->typ ==
|
2014-08-19 14:26:42 +02:00
|
|
|
rpl_parallel_thread::queued_event::QUEUED_MASTER_RESTART)
|
|
|
|
{
|
|
|
|
if (in_event_group)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
Master restarted (crashed) in the middle of an event group.
|
|
|
|
So we need to roll back and discard that event group.
|
|
|
|
*/
|
|
|
|
group_rgi->cleanup_context(thd, 1);
|
|
|
|
in_event_group= false;
|
2015-01-07 14:45:39 +01:00
|
|
|
finish_event_group(rpt, group_rgi->gtid_sub_id,
|
2014-11-13 10:20:48 +01:00
|
|
|
qev->entry_for_queued, group_rgi);
|
2014-08-19 14:26:42 +02:00
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
rpt->loc_free_rgi(group_rgi);
|
2014-08-19 14:26:42 +02:00
|
|
|
thd->rgi_slave= group_rgi= NULL;
|
|
|
|
}
|
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
rpt->loc_free_qev(qev);
|
2014-08-19 14:26:42 +02:00
|
|
|
continue;
|
|
|
|
}
|
2014-11-13 10:20:48 +01:00
|
|
|
DBUG_ASSERT(qev->typ==rpl_parallel_thread::queued_event::QUEUED_EVENT);
|
2013-10-31 14:11:41 +01:00
|
|
|
|
2015-02-24 14:03:14 +01:00
|
|
|
thd->rgi_slave= rgi;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
gco= rgi->gco;
|
2013-09-13 15:09:57 +02:00
|
|
|
/* Handle a new event group, which will be initiated by a GTID event. */
|
2014-11-13 10:20:48 +01:00
|
|
|
if ((event_type= qev->ev->get_type_code()) == GTID_EVENT)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
bool did_enter_cond= false;
|
2014-02-26 16:38:42 +01:00
|
|
|
PSI_stage_info old_stage;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
MDEV-7237: Parallel replication: incorrect relaylog position after stop/start the slave
The replication relay log position was sometimes updated incorrectly at the
end of a transaction in parallel replication. This happened because the relay
log file name was taken from the current Relay_log_info (SQL driver thread),
not the correct value for the transaction in question.
The result was that if a transaction was applied while the SQL driver thread
was at least one relay log file ahead, _and_ the SQL thread was subsequently
stopped before applying any events from the most recent relay log file, then
the relay log position would be incorrect - wrong relay log file name. Thus,
when the slave was started again, usually a relay log read error would result,
or in rare cases, if the position happened to be readable, the slave might
even skip arbitrary amounts of events.
In GTID mode, the relay log position is reset when both slave threads are
restarted, so this bug would only be seen in non-GTID mode, or in GTID mode
when only the SQL thread, not the IO thread, was stopped.
2014-12-01 13:53:57 +01:00
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_scheduled_gtid_0_x_100", {
|
|
|
|
if (rgi->current_gtid.domain_id == 0 &&
|
|
|
|
rgi->current_gtid.seq_no == 100) {
|
|
|
|
debug_sync_set_action(thd,
|
|
|
|
STRING_WITH_LEN("now SIGNAL scheduled_gtid_0_x_100"));
|
|
|
|
}
|
|
|
|
});
|
|
|
|
|
2015-02-24 14:03:14 +01:00
|
|
|
if(unlikely(thd->wait_for_commit_ptr) && group_rgi != NULL)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
This indicates that we get a new GTID event in the middle of
|
|
|
|
a not completed event group. This is corrupt binlog (the master
|
|
|
|
will never write such binlog), so it does not happen unless
|
|
|
|
someone tries to inject wrong crafted binlog, but let us still
|
|
|
|
try to handle it somewhat nicely.
|
|
|
|
*/
|
|
|
|
group_rgi->cleanup_context(thd, true);
|
|
|
|
finish_event_group(rpt, group_rgi->gtid_sub_id,
|
|
|
|
group_rgi->parallel_entry, group_rgi);
|
|
|
|
rpt->loc_free_rgi(group_rgi);
|
|
|
|
}
|
|
|
|
|
2014-12-05 16:09:48 +01:00
|
|
|
thd->tx_isolation= (enum_tx_isolation)thd->variables.tx_isolation;
|
2013-07-03 13:46:33 +02:00
|
|
|
in_event_group= true;
|
2013-09-13 15:09:57 +02:00
|
|
|
/*
|
|
|
|
If the standalone flag is set, then this event group consists of a
|
|
|
|
single statement (possibly preceeded by some Intvar_log_event and
|
|
|
|
similar), without any terminating COMMIT/ROLLBACK/XID.
|
|
|
|
*/
|
2013-06-24 10:50:25 +02:00
|
|
|
group_standalone=
|
2014-11-13 10:20:48 +01:00
|
|
|
(0 != (static_cast<Gtid_log_event *>(qev->ev)->flags2 &
|
2013-06-24 10:50:25 +02:00
|
|
|
Gtid_log_event::FL_STANDALONE));
|
2013-07-03 13:46:33 +02:00
|
|
|
|
2013-07-04 13:17:01 +02:00
|
|
|
event_gtid_sub_id= rgi->gtid_sub_id;
|
2013-07-08 16:47:07 +02:00
|
|
|
rgi->thd= thd;
|
|
|
|
|
2015-05-28 12:32:19 +02:00
|
|
|
mysql_mutex_lock(&entry->LOCK_parallel_entry);
|
|
|
|
skip_event_group= do_gco_wait(rgi, gco, &did_enter_cond, &old_stage);
|
|
|
|
|
|
|
|
if (unlikely(entry->stop_on_error_sub_id <= rgi->wait_commit_sub_id))
|
|
|
|
skip_event_group= true;
|
|
|
|
if (likely(!skip_event_group))
|
|
|
|
do_ftwrl_wait(rgi, &did_enter_cond, &old_stage);
|
|
|
|
|
2013-07-03 13:46:33 +02:00
|
|
|
/*
|
|
|
|
Register ourself to wait for the previous commit, if we need to do
|
|
|
|
such registration _and_ that previous commit has not already
|
2016-03-04 01:09:37 +01:00
|
|
|
occurred.
|
2013-07-03 13:46:33 +02:00
|
|
|
*/
|
MDEV-7847: "Slave worker thread retried transaction 10 time(s) in vain, giving up", followed by replication hanging
This patch fixes a bug in the error handling in parallel replication, when one
worker thread gets a failure and other worker threads processing later
transactions have to rollback and abort.
The problem was with the lifetime of group_commit_orderer objects (GCOs).
A GCO is freed when we register that its last event group has committed. This
relies on register_wait_for_prior_commit() and wait_for_prior_commit() to
ensure that the fact that T2 has committed implies that any earlier T1 has
also committed, and can thus no longer execute mark_start_commit().
However, in the error case, the code was skipping the
register_wait_for_prior_commit() and wait_for_prior_commit() calls. Thus
commit ordering was not guaranteed, and a GCO could be freed too early. Then a
later mark_start_commit() would reference deallocated GCO, which could lead to
lost wakeup (causing slave threads to hang) or other corruption.
This patch makes also the error case respect commit order. This way, also the
error case gets the GCO lifetime correct, and the hang no longer occurs.
2015-03-30 14:33:44 +02:00
|
|
|
register_wait_for_prior_event_group_commit(rgi, entry);
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
unlock_or_exit_cond(thd, &entry->LOCK_parallel_entry,
|
2014-02-26 16:38:42 +01:00
|
|
|
&did_enter_cond, &old_stage);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
2013-07-03 13:46:33 +02:00
|
|
|
thd->wait_for_commit_ptr= &rgi->commit_orderer;
|
2014-03-09 10:27:38 +01:00
|
|
|
|
2016-12-01 09:59:58 +01:00
|
|
|
if (opt_gtid_ignore_duplicates &&
|
|
|
|
rgi->rli->mi->using_gtid != Master_info::USE_GTID_NO)
|
2014-03-09 10:27:38 +01:00
|
|
|
{
|
|
|
|
int res=
|
2015-11-29 16:51:23 +01:00
|
|
|
rpl_global_gtid_slave_state->check_duplicate_gtid(&rgi->current_gtid,
|
2014-03-12 00:14:49 +01:00
|
|
|
rgi);
|
|
|
|
if (res < 0)
|
|
|
|
{
|
|
|
|
/* Error. */
|
2014-06-25 15:17:03 +02:00
|
|
|
slave_output_error_info(rgi, thd);
|
2014-03-21 10:11:28 +01:00
|
|
|
signal_error_to_sql_driver_thread(thd, rgi, 1);
|
2014-03-12 00:14:49 +01:00
|
|
|
}
|
|
|
|
else if (!res)
|
|
|
|
{
|
|
|
|
/* GTID already applied by another master connection, skip. */
|
2014-03-09 10:27:38 +01:00
|
|
|
skip_event_group= true;
|
2014-03-12 00:14:49 +01:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/* We have to apply the event. */
|
|
|
|
}
|
2014-03-09 10:27:38 +01:00
|
|
|
}
|
2014-12-05 16:09:48 +01:00
|
|
|
/*
|
|
|
|
If we are optimistically running transactions in parallel, but this
|
|
|
|
particular event group should not run in parallel with what came
|
|
|
|
before, then wait now for the prior transaction to complete its
|
|
|
|
commit.
|
|
|
|
*/
|
|
|
|
if (rgi->speculation == rpl_group_info::SPECULATE_WAIT &&
|
|
|
|
(err= thd->wait_for_prior_commit()))
|
|
|
|
{
|
|
|
|
slave_output_error_info(rgi, thd);
|
|
|
|
signal_error_to_sql_driver_thread(thd, rgi, 1);
|
|
|
|
}
|
2013-07-03 13:46:33 +02:00
|
|
|
}
|
|
|
|
|
2015-02-24 14:03:14 +01:00
|
|
|
group_rgi= rgi;
|
2014-11-13 10:20:48 +01:00
|
|
|
group_ending= is_group_ending(qev->ev, event_type);
|
2015-09-02 09:57:18 +02:00
|
|
|
/*
|
|
|
|
We do not unmark_start_commit() here in case of an explicit ROLLBACK
|
|
|
|
statement. Such events should be very rare, there is no real reason
|
|
|
|
to try to group commit them - on the contrary, it seems best to avoid
|
|
|
|
running them in parallel with following group commits, as with
|
|
|
|
ROLLBACK events we are already deep in dangerous corner cases with
|
|
|
|
mix of transactional and non-transactional tables or the like. And
|
|
|
|
avoiding the mark_start_commit() here allows us to keep an assertion
|
|
|
|
in ha_rollback_trans() that we do not rollback after doing
|
|
|
|
mark_start_commit().
|
|
|
|
*/
|
|
|
|
if (group_ending == 1 && likely(!rgi->worker_error))
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
2015-08-04 11:20:03 +02:00
|
|
|
/*
|
|
|
|
Do an extra check for (deadlock) kill here. This helps prevent a
|
2016-03-04 01:09:37 +01:00
|
|
|
lingering deadlock kill that occurred during normal DML processing to
|
2015-08-04 11:20:03 +02:00
|
|
|
propagate past the mark_start_commit(). If we detect a deadlock only
|
|
|
|
after mark_start_commit(), we have to unmark, which has at least a
|
|
|
|
theoretical possibility of leaving a window where it looks like all
|
|
|
|
transactions in a GCO have started committing, while in fact one
|
|
|
|
will need to rollback and retry. This is not supposed to be possible
|
|
|
|
(since there is a deadlock, at least one transaction should be
|
|
|
|
blocked from reaching commit), but this seems a fragile ensurance,
|
|
|
|
and there were historically a number of subtle bugs in this area.
|
|
|
|
*/
|
|
|
|
if (!thd->killed)
|
|
|
|
{
|
|
|
|
DEBUG_SYNC(thd, "rpl_parallel_before_mark_start_commit");
|
|
|
|
rgi->mark_start_commit();
|
|
|
|
DEBUG_SYNC(thd, "rpl_parallel_after_mark_start_commit");
|
|
|
|
}
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
}
|
|
|
|
|
2013-10-08 14:36:06 +02:00
|
|
|
/*
|
|
|
|
If the SQL thread is stopping, we just skip execution of all the
|
|
|
|
following event groups. We still do all the normal waiting and wakeup
|
|
|
|
processing between the event groups as a simple way to ensure that
|
|
|
|
everything is stopped and cleaned up correctly.
|
|
|
|
*/
|
2014-06-27 13:34:29 +02:00
|
|
|
if (likely(!rgi->worker_error) && !skip_event_group)
|
2014-05-08 14:20:18 +02:00
|
|
|
{
|
|
|
|
++rgi->retry_event_count;
|
2014-11-13 10:46:09 +01:00
|
|
|
#ifndef DBUG_OFF
|
|
|
|
err= 0;
|
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_simulate_temp_err_xid",
|
|
|
|
if (event_type == XID_EVENT)
|
|
|
|
{
|
|
|
|
thd->clear_error();
|
|
|
|
thd->get_stmt_da()->reset_diagnostics_area();
|
|
|
|
my_error(ER_LOCK_DEADLOCK, MYF(0));
|
|
|
|
err= 1;
|
2015-01-07 14:45:39 +01:00
|
|
|
DEBUG_SYNC(thd, "rpl_parallel_simulate_temp_err_xid");
|
2014-11-13 10:46:09 +01:00
|
|
|
});
|
|
|
|
if (!err)
|
|
|
|
#endif
|
2015-08-04 11:20:03 +02:00
|
|
|
{
|
2018-04-04 11:16:12 +02:00
|
|
|
if (unlikely(thd->check_killed()))
|
2015-08-04 11:20:03 +02:00
|
|
|
{
|
|
|
|
thd->clear_error();
|
|
|
|
thd->get_stmt_da()->reset_diagnostics_area();
|
|
|
|
thd->send_kill_message();
|
|
|
|
err= 1;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
err= rpt_handle_event(qev, rpt);
|
|
|
|
}
|
2014-11-13 10:20:48 +01:00
|
|
|
delete_or_keep_event_post_apply(rgi, event_type, qev->ev);
|
2014-05-13 13:42:06 +02:00
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_simulate_temp_err_gtid_0_x_100",
|
|
|
|
err= dbug_simulate_tmp_error(rgi, thd););
|
2018-04-04 11:16:12 +02:00
|
|
|
if (unlikely(err))
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
{
|
|
|
|
convert_kill_to_deadlock_error(rgi);
|
|
|
|
if (has_temporary_error(thd) && slave_trans_retries > 0)
|
2014-11-13 10:20:48 +01:00
|
|
|
err= retry_event_group(rgi, rpt, qev);
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
}
|
2014-05-08 14:20:18 +02:00
|
|
|
}
|
2013-10-08 14:36:06 +02:00
|
|
|
else
|
2014-05-13 13:42:06 +02:00
|
|
|
{
|
2014-11-13 10:20:48 +01:00
|
|
|
delete qev->ev;
|
MDEV-7847: "Slave worker thread retried transaction 10 time(s) in vain, giving up", followed by replication hanging
This patch fixes a bug in the error handling in parallel replication, when one
worker thread gets a failure and other worker threads processing later
transactions have to rollback and abort.
The problem was with the lifetime of group_commit_orderer objects (GCOs).
A GCO is freed when we register that its last event group has committed. This
relies on register_wait_for_prior_commit() and wait_for_prior_commit() to
ensure that the fact that T2 has committed implies that any earlier T1 has
also committed, and can thus no longer execute mark_start_commit().
However, in the error case, the code was skipping the
register_wait_for_prior_commit() and wait_for_prior_commit() calls. Thus
commit ordering was not guaranteed, and a GCO could be freed too early. Then a
later mark_start_commit() would reference deallocated GCO, which could lead to
lost wakeup (causing slave threads to hang) or other corruption.
This patch makes also the error case respect commit order. This way, also the
error case gets the GCO lifetime correct, and the hang no longer occurs.
2015-03-30 14:33:44 +02:00
|
|
|
thd->get_stmt_da()->set_overwrite_status(true);
|
2013-10-14 15:28:16 +02:00
|
|
|
err= thd->wait_for_prior_commit();
|
MDEV-7847: "Slave worker thread retried transaction 10 time(s) in vain, giving up", followed by replication hanging
This patch fixes a bug in the error handling in parallel replication, when one
worker thread gets a failure and other worker threads processing later
transactions have to rollback and abort.
The problem was with the lifetime of group_commit_orderer objects (GCOs).
A GCO is freed when we register that its last event group has committed. This
relies on register_wait_for_prior_commit() and wait_for_prior_commit() to
ensure that the fact that T2 has committed implies that any earlier T1 has
also committed, and can thus no longer execute mark_start_commit().
However, in the error case, the code was skipping the
register_wait_for_prior_commit() and wait_for_prior_commit() calls. Thus
commit ordering was not guaranteed, and a GCO could be freed too early. Then a
later mark_start_commit() would reference deallocated GCO, which could lead to
lost wakeup (causing slave threads to hang) or other corruption.
This patch makes also the error case respect commit order. This way, also the
error case gets the GCO lifetime correct, and the hang no longer occurs.
2015-03-30 14:33:44 +02:00
|
|
|
thd->get_stmt_da()->set_overwrite_status(false);
|
2014-05-13 13:42:06 +02:00
|
|
|
}
|
2013-07-03 13:46:33 +02:00
|
|
|
|
2013-07-04 13:17:01 +02:00
|
|
|
end_of_group=
|
|
|
|
in_event_group &&
|
|
|
|
((group_standalone && !Log_event::is_part_of_group(event_type)) ||
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
group_ending);
|
2013-07-03 13:46:33 +02:00
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
rpt->loc_free_qev(qev);
|
2013-07-03 13:46:33 +02:00
|
|
|
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
if (unlikely(err))
|
2013-10-28 13:24:56 +01:00
|
|
|
{
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
In parallel replication, we run transactions from the master in parallel, but
force them to commit in the same order they did on the master. If we force T1
to commit before T2, but T2 holds eg. a row lock that is needed by T1, we get
a deadlock when T2 waits until T1 has committed.
Usually, we do not run T1 and T2 in parallel if there is a chance that they
can have conflicting locks like this, but there are certain edge cases where
it can occasionally happen (eg. MDEV-5914, MDEV-5941, MDEV-6020). The bug was
that this would cause replication to hang, eventually getting a lock timeout
and causing the slave to stop with error.
With this patch, InnoDB will report back to the upper layer whenever a
transactions T1 is about to do a lock wait on T2. If T1 and T2 are parallel
replication transactions, and T2 needs to commit later than T1, we can thus
detect the deadlock; we then kill T2, setting a flag that causes it to catch
the kill and convert it to a deadlock error; this error will then cause T2 to
roll back and release its locks (so that T1 can commit), and later T2 will be
re-tried and eventually also committed.
The kill happens asynchroneously in a slave background thread; this is
necessary, as the reporting from InnoDB about lock waits happen deep inside
the locking code, at a point where it is not possible to directly call
THD::awake() due to mutexes held.
Deadlock is assumed to be (very) rarely occuring, so this patch tries to
minimise the performance impact on the normal case where no deadlocks occur,
rather than optimise the handling of the occasional deadlock.
Also fix transaction retry due to deadlock when it happens after a transaction
already signalled to later transactions that it started to commit. In this
case we need to undo this signalling (and later redo it when we commit again
during retry), so following transactions will not start too early.
Also add a missing thd->send_kill_message() that got triggered during testing
(this corrects an incorrect fix for MySQL Bug#58933).
2014-06-03 10:31:11 +02:00
|
|
|
if (!rgi->worker_error)
|
|
|
|
{
|
|
|
|
slave_output_error_info(rgi, thd);
|
|
|
|
signal_error_to_sql_driver_thread(thd, rgi, err);
|
|
|
|
}
|
|
|
|
thd->reset_killed();
|
2013-10-28 13:24:56 +01:00
|
|
|
}
|
2013-07-04 13:17:01 +02:00
|
|
|
if (end_of_group)
|
|
|
|
{
|
|
|
|
in_event_group= false;
|
2015-01-07 14:45:39 +01:00
|
|
|
finish_event_group(rpt, event_gtid_sub_id, entry, rgi);
|
2014-11-13 10:20:48 +01:00
|
|
|
rpt->loc_free_rgi(rgi);
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
Remove the temporary fix for MDEV-5914, which used READ COMMITTED for parallel
replication worker threads. Replace it with a better, more selective solution.
The issue is with certain edge cases of InnoDB gap locks, for example between
INSERT and ranged DELETE. It is possible for the gap lock set by the DELETE to
block the INSERT, if the DELETE runs first, while the record lock set by
INSERT does not block the DELETE, if the INSERT runs first. This can cause a
conflict between the two in parallel replication on the slave even though they
ran without conflicts on the master.
With this patch, InnoDB will ask the server layer about the two involved
transactions before blocking on a gap lock. If the server layer tells InnoDB
that the transactions are already fixed wrt. commit order, as they are in
parallel replication, InnoDB will ignore the gap lock and allow the two
transactions to proceed in parallel, avoiding the conflict.
Improve the fix for MDEV-6020. When InnoDB itself detects a deadlock, it now
asks the server layer for any preferences about which transaction to roll
back. In case of parallel replication with two transactions T1 and T2 fixed to
commit T1 before T2, the server layer will ask InnoDB to roll back T2 as the
deadlock victim, not T1. This helps in some cases to avoid excessive deadlock
rollback, as T2 will in any case need to wait for T1 to complete before it can
itself commit.
Also some misc. fixes found during development and testing:
- Remove thd_rpl_is_parallel(), it is not used or needed.
- Use KILL_CONNECTION instead of KILL_QUERY when a parallel replication
worker thread is killed to resolve a deadlock with fixed commit
ordering. There are some cases, eg. in sql/sql_parse.cc, where a KILL_QUERY
can be ignored if the query otherwise completed successfully, and this
could cause the deadlock kill to be lost, so that the deadlock was not
correctly resolved.
- Fix random test failure due to missing wait_for_binlog_checkpoint.inc.
- Make sure that deadlock or other temporary errors during parallel
replication are not printed to the the error log; there were some places
around the replication code with extra error logging. These conditions can
occur occasionally and are handled automatically without breaking
replication, so they should not pollute the error log.
- Fix handling of rgi->gtid_sub_id. We need to be able to access this also at
the end of a transaction, to be able to detect and resolve deadlocks due to
commit ordering. But this value was also used as a flag to mark whether
record_gtid() had been called, by being set to zero, losing the value. Now,
introduce a separate flag rgi->gtid_pending, so rgi->gtid_sub_id remains
valid for the entire duration of the transaction.
- Fix one place where the code to handle ignored errors called reset_killed()
unconditionally, even if no error was caught that should be ignored. This
could cause loss of a deadlock kill signal, breaking deadlock detection and
resolution.
- Fix a couple of missing mysql_reset_thd_for_next_command(). This could
cause a prior error condition to remain for the next event executed,
causing assertions about errors already being set and possibly giving
incorrect error handling for following event executions.
- Fix code that cleared thd->rgi_slave in the parallel replication worker
threads after each event execution; this caused the deadlock detection and
handling code to not be able to correctly process the associated
transactions as belonging to replication worker threads.
- Remove useless error code in slave_background_kill_request().
- Fix bug where wfc->wakeup_error was not cleared at
wait_for_commit::unregister_wait_for_prior_commit(). This could cause the
error condition to wrongly propagate to a later wait_for_prior_commit(),
causing spurious ER_PRIOR_COMMIT_FAILED errors.
- Do not put the binlog background thread into the processlist. It causes
too many result differences in mtr, but also it probably is not useful
for users to pollute the process list with a system thread that does not
really perform any user-visible tasks...
2014-06-10 10:13:15 +02:00
|
|
|
thd->rgi_slave= group_rgi= rgi= NULL;
|
2014-03-09 10:27:38 +01:00
|
|
|
skip_event_group= false;
|
2013-12-18 16:26:22 +01:00
|
|
|
DEBUG_SYNC(thd, "rpl_parallel_end_of_group");
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
2014-11-13 10:20:48 +01:00
|
|
|
/*
|
|
|
|
Now that we have the lock, we can move everything from our local free
|
|
|
|
lists to the real free lists that are also accessible from the SQL
|
|
|
|
driver thread.
|
|
|
|
*/
|
|
|
|
rpt->batch_free();
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
2017-02-10 21:35:04 +01:00
|
|
|
if ((events= rpt->event_queue) != NULL)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
2013-09-23 14:46:57 +02:00
|
|
|
/*
|
2017-02-10 21:35:04 +01:00
|
|
|
Take next group of events from the replication pool.
|
|
|
|
This is faster than having to wakeup the pool manager thread to give
|
|
|
|
us a new event.
|
2013-09-23 14:46:57 +02:00
|
|
|
*/
|
2017-02-10 21:35:04 +01:00
|
|
|
rpt->dequeue1(events);
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
2017-02-10 21:35:04 +01:00
|
|
|
goto more_events;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
2015-10-22 11:18:34 +02:00
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
rpt->inuse_relaylog_refcount_update();
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2013-11-05 12:01:26 +01:00
|
|
|
if (in_event_group && group_rgi->parallel_entry->force_abort)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
We are asked to abort, without getting the remaining events in the
|
|
|
|
current event group.
|
|
|
|
|
|
|
|
We have to rollback the current transaction and update the last
|
|
|
|
sub_id value so that SQL thread will know we are done with the
|
|
|
|
half-processed event group.
|
|
|
|
*/
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
2014-03-21 10:11:28 +01:00
|
|
|
signal_error_to_sql_driver_thread(thd, group_rgi, 1);
|
2015-01-07 14:45:39 +01:00
|
|
|
finish_event_group(rpt, group_rgi->gtid_sub_id,
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
group_rgi->parallel_entry, group_rgi);
|
2013-11-05 12:01:26 +01:00
|
|
|
in_event_group= false;
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpt->free_rgi(group_rgi);
|
MDEV-5262, MDEV-5914, MDEV-5941, MDEV-6020: Deadlocks during parallel
replication causing replication to fail.
Remove the temporary fix for MDEV-5914, which used READ COMMITTED for parallel
replication worker threads. Replace it with a better, more selective solution.
The issue is with certain edge cases of InnoDB gap locks, for example between
INSERT and ranged DELETE. It is possible for the gap lock set by the DELETE to
block the INSERT, if the DELETE runs first, while the record lock set by
INSERT does not block the DELETE, if the INSERT runs first. This can cause a
conflict between the two in parallel replication on the slave even though they
ran without conflicts on the master.
With this patch, InnoDB will ask the server layer about the two involved
transactions before blocking on a gap lock. If the server layer tells InnoDB
that the transactions are already fixed wrt. commit order, as they are in
parallel replication, InnoDB will ignore the gap lock and allow the two
transactions to proceed in parallel, avoiding the conflict.
Improve the fix for MDEV-6020. When InnoDB itself detects a deadlock, it now
asks the server layer for any preferences about which transaction to roll
back. In case of parallel replication with two transactions T1 and T2 fixed to
commit T1 before T2, the server layer will ask InnoDB to roll back T2 as the
deadlock victim, not T1. This helps in some cases to avoid excessive deadlock
rollback, as T2 will in any case need to wait for T1 to complete before it can
itself commit.
Also some misc. fixes found during development and testing:
- Remove thd_rpl_is_parallel(), it is not used or needed.
- Use KILL_CONNECTION instead of KILL_QUERY when a parallel replication
worker thread is killed to resolve a deadlock with fixed commit
ordering. There are some cases, eg. in sql/sql_parse.cc, where a KILL_QUERY
can be ignored if the query otherwise completed successfully, and this
could cause the deadlock kill to be lost, so that the deadlock was not
correctly resolved.
- Fix random test failure due to missing wait_for_binlog_checkpoint.inc.
- Make sure that deadlock or other temporary errors during parallel
replication are not printed to the the error log; there were some places
around the replication code with extra error logging. These conditions can
occur occasionally and are handled automatically without breaking
replication, so they should not pollute the error log.
- Fix handling of rgi->gtid_sub_id. We need to be able to access this also at
the end of a transaction, to be able to detect and resolve deadlocks due to
commit ordering. But this value was also used as a flag to mark whether
record_gtid() had been called, by being set to zero, losing the value. Now,
introduce a separate flag rgi->gtid_pending, so rgi->gtid_sub_id remains
valid for the entire duration of the transaction.
- Fix one place where the code to handle ignored errors called reset_killed()
unconditionally, even if no error was caught that should be ignored. This
could cause loss of a deadlock kill signal, breaking deadlock detection and
resolution.
- Fix a couple of missing mysql_reset_thd_for_next_command(). This could
cause a prior error condition to remain for the next event executed,
causing assertions about errors already being set and possibly giving
incorrect error handling for following event executions.
- Fix code that cleared thd->rgi_slave in the parallel replication worker
threads after each event execution; this caused the deadlock detection and
handling code to not be able to correctly process the associated
transactions as belonging to replication worker threads.
- Remove useless error code in slave_background_kill_request().
- Fix bug where wfc->wakeup_error was not cleared at
wait_for_commit::unregister_wait_for_prior_commit(). This could cause the
error condition to wrongly propagate to a later wait_for_prior_commit(),
causing spurious ER_PRIOR_COMMIT_FAILED errors.
- Do not put the binlog background thread into the processlist. It causes
too many result differences in mtr, but also it probably is not useful
for users to pollute the process list with a system thread that does not
really perform any user-visible tasks...
2014-06-10 10:13:15 +02:00
|
|
|
thd->rgi_slave= group_rgi= NULL;
|
2014-03-09 10:27:38 +01:00
|
|
|
skip_event_group= false;
|
2013-11-05 12:01:26 +01:00
|
|
|
}
|
2013-06-24 10:50:25 +02:00
|
|
|
if (!in_event_group)
|
|
|
|
{
|
2017-01-29 21:10:56 +01:00
|
|
|
/* If we are in a FLUSH TABLES FOR READ LOCK, wait for it */
|
|
|
|
while (rpt->current_entry && rpt->pause_for_ftwrl)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
We are currently in the delicate process of pausing parallel
|
|
|
|
replication while FLUSH TABLES WITH READ LOCK is starting. We must
|
|
|
|
not de-allocate the thread (setting rpt->current_owner= NULL) until
|
|
|
|
rpl_unpause_after_ftwrl() has woken us up.
|
|
|
|
*/
|
|
|
|
rpl_parallel_entry *e= rpt->current_entry;
|
|
|
|
/*
|
2017-03-06 15:02:50 +01:00
|
|
|
Wait for rpl_unpause_after_ftwrl() to wake us up.
|
|
|
|
Note that rpl_pause_for_ftwrl() may wait for 'e->pause_sub_id'
|
|
|
|
to change. This should happen eventually in finish_event_group()
|
2017-01-29 21:10:56 +01:00
|
|
|
*/
|
|
|
|
mysql_mutex_lock(&e->LOCK_parallel_entry);
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
if (rpt->pause_for_ftwrl)
|
|
|
|
mysql_cond_wait(&e->COND_parallel_entry, &e->LOCK_parallel_entry);
|
|
|
|
mysql_mutex_unlock(&e->LOCK_parallel_entry);
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
|
|
|
}
|
2017-02-10 21:35:04 +01:00
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpt->current_owner= NULL;
|
|
|
|
/* Tell wait_for_done() that we are done, if it is waiting. */
|
|
|
|
if (likely(rpt->current_entry) &&
|
|
|
|
unlikely(rpt->current_entry->force_abort))
|
2015-11-23 18:58:30 +01:00
|
|
|
mysql_cond_broadcast(&rpt->COND_rpl_thread_stop);
|
2017-02-10 21:35:04 +01:00
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
rpt->current_entry= NULL;
|
2013-09-23 14:46:57 +02:00
|
|
|
if (!rpt->stop)
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpt->pool->release_thread(rpt);
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2013-07-09 13:15:53 +02:00
|
|
|
rpt->thd= NULL;
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
|
|
|
|
thd->clear_error();
|
|
|
|
thd->catalog= 0;
|
|
|
|
thd->reset_query();
|
2018-01-07 17:03:44 +01:00
|
|
|
thd->reset_db(&null_clex_str);
|
2013-07-09 13:15:53 +02:00
|
|
|
thd_proc_info(thd, "Slave worker thread exiting");
|
|
|
|
thd->temporary_tables= 0;
|
2016-02-01 11:45:39 +01:00
|
|
|
|
2013-07-09 13:15:53 +02:00
|
|
|
THD_CHECK_SENTRY(thd);
|
2019-01-19 23:32:35 +01:00
|
|
|
server_threads.erase(thd);
|
2013-07-09 13:15:53 +02:00
|
|
|
delete thd;
|
|
|
|
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
2013-06-24 10:50:25 +02:00
|
|
|
rpt->running= false;
|
2013-07-09 13:15:53 +02:00
|
|
|
mysql_cond_signal(&rpt->COND_rpl_thread);
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
|
2013-07-09 13:15:53 +02:00
|
|
|
my_thread_end();
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
return NULL;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
static void
|
|
|
|
dealloc_gco(group_commit_orderer *gco)
|
|
|
|
{
|
|
|
|
mysql_cond_destroy(&gco->COND_group_commit_orderer);
|
|
|
|
my_free(gco);
|
|
|
|
}
|
|
|
|
|
2017-01-29 22:44:24 +01:00
|
|
|
/**
|
|
|
|
Change thread count for global parallel worker threads
|
|
|
|
|
|
|
|
@param pool parallel thread pool
|
|
|
|
@param new_count Number of threads to be in pool. 0 in shutdown
|
|
|
|
@param force Force thread count to new_count even if slave
|
|
|
|
threads are running
|
|
|
|
|
|
|
|
By default we don't resize pool of there are running threads.
|
|
|
|
However during shutdown we will always do it.
|
|
|
|
This is needed as any_slave_sql_running() returns 1 during shutdown
|
|
|
|
as we don't want to access master_info while
|
|
|
|
Master_info_index::free_connections are running.
|
|
|
|
*/
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
2015-03-11 09:18:16 +01:00
|
|
|
static int
|
2013-06-24 10:50:25 +02:00
|
|
|
rpl_parallel_change_thread_count(rpl_parallel_thread_pool *pool,
|
2017-01-29 22:44:24 +01:00
|
|
|
uint32 new_count, bool force)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
|
|
|
uint32 i;
|
2015-02-04 13:57:09 +01:00
|
|
|
rpl_parallel_thread **old_list= NULL;
|
2013-06-24 10:50:25 +02:00
|
|
|
rpl_parallel_thread **new_list= NULL;
|
|
|
|
rpl_parallel_thread *new_free_list= NULL;
|
2013-09-23 14:46:57 +02:00
|
|
|
rpl_parallel_thread *rpt_array= NULL;
|
2015-10-22 11:18:34 +02:00
|
|
|
int res;
|
|
|
|
|
|
|
|
if ((res= pool_mark_busy(pool, current_thd)))
|
|
|
|
return res;
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2017-01-29 21:10:56 +01:00
|
|
|
/* Protect against parallel pool resizes */
|
|
|
|
if (pool->count == new_count)
|
|
|
|
{
|
|
|
|
pool_mark_not_busy(pool);
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
If we are about to delete pool, do an extra check that there are no new
|
|
|
|
slave threads running since we marked pool busy
|
|
|
|
*/
|
2017-01-29 22:44:24 +01:00
|
|
|
if (!new_count && !force)
|
2017-01-29 21:10:56 +01:00
|
|
|
{
|
2017-04-23 10:49:58 +02:00
|
|
|
if (any_slave_sql_running(false))
|
2017-01-29 21:10:56 +01:00
|
|
|
{
|
|
|
|
DBUG_PRINT("warning",
|
|
|
|
("SQL threads running while trying to reset parallel pool"));
|
|
|
|
pool_mark_not_busy(pool);
|
|
|
|
return 0; // Ok to not resize pool
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
/*
|
|
|
|
Allocate the new list of threads up-front.
|
|
|
|
That way, if we fail half-way, we only need to free whatever we managed
|
|
|
|
to allocate, and will not be left with a half-functional thread pool.
|
|
|
|
*/
|
|
|
|
if (new_count &&
|
2013-09-23 14:46:57 +02:00
|
|
|
!my_multi_malloc(MYF(MY_WME|MY_ZEROFILL),
|
|
|
|
&new_list, new_count*sizeof(*new_list),
|
|
|
|
&rpt_array, new_count*sizeof(*rpt_array),
|
|
|
|
NULL))
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
2013-09-23 14:46:57 +02:00
|
|
|
my_error(ER_OUTOFMEMORY, MYF(0), (int(new_count*sizeof(*new_list) +
|
|
|
|
new_count*sizeof(*rpt_array))));
|
2017-01-29 21:10:56 +01:00
|
|
|
goto err;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
for (i= 0; i < new_count; ++i)
|
|
|
|
{
|
|
|
|
pthread_t th;
|
|
|
|
|
2013-09-23 14:46:57 +02:00
|
|
|
new_list[i]= &rpt_array[i];
|
2013-06-24 10:50:25 +02:00
|
|
|
new_list[i]->delay_start= true;
|
|
|
|
mysql_mutex_init(key_LOCK_rpl_thread, &new_list[i]->LOCK_rpl_thread,
|
|
|
|
MY_MUTEX_INIT_SLOW);
|
|
|
|
mysql_cond_init(key_COND_rpl_thread, &new_list[i]->COND_rpl_thread, NULL);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
mysql_cond_init(key_COND_rpl_thread_queue,
|
|
|
|
&new_list[i]->COND_rpl_thread_queue, NULL);
|
2015-11-23 18:58:30 +01:00
|
|
|
mysql_cond_init(key_COND_rpl_thread_stop,
|
|
|
|
&new_list[i]->COND_rpl_thread_stop, NULL);
|
2013-06-24 10:50:25 +02:00
|
|
|
new_list[i]->pool= pool;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
if (mysql_thread_create(key_rpl_parallel_thread, &th, &connection_attrib,
|
2013-06-24 10:50:25 +02:00
|
|
|
handle_rpl_parallel_thread, new_list[i]))
|
|
|
|
{
|
|
|
|
my_error(ER_OUT_OF_RESOURCES, MYF(0));
|
|
|
|
goto err;
|
|
|
|
}
|
|
|
|
new_list[i]->next= new_free_list;
|
|
|
|
new_free_list= new_list[i];
|
|
|
|
}
|
|
|
|
|
2013-09-23 14:46:57 +02:00
|
|
|
/*
|
|
|
|
Grab each old thread in turn, and signal it to stop.
|
|
|
|
|
|
|
|
Note that since we require all replication threads to be stopped before
|
|
|
|
changing the parallel replication worker thread pool, all the threads will
|
|
|
|
be already idle and will terminate immediately.
|
|
|
|
*/
|
2013-06-24 10:50:25 +02:00
|
|
|
for (i= 0; i < pool->count; ++i)
|
|
|
|
{
|
2015-10-22 11:18:34 +02:00
|
|
|
rpl_parallel_thread *rpt;
|
|
|
|
|
|
|
|
mysql_mutex_lock(&pool->LOCK_rpl_thread_pool);
|
|
|
|
while ((rpt= pool->free_list) == NULL)
|
|
|
|
mysql_cond_wait(&pool->COND_rpl_thread_pool, &pool->LOCK_rpl_thread_pool);
|
|
|
|
pool->free_list= rpt->next;
|
|
|
|
mysql_mutex_unlock(&pool->LOCK_rpl_thread_pool);
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
2013-06-24 10:50:25 +02:00
|
|
|
rpt->stop= true;
|
2013-07-09 13:15:53 +02:00
|
|
|
mysql_cond_signal(&rpt->COND_rpl_thread);
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
}
|
|
|
|
|
|
|
|
for (i= 0; i < pool->count; ++i)
|
|
|
|
{
|
|
|
|
rpl_parallel_thread *rpt= pool->threads[i];
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
|
|
|
while (rpt->running)
|
|
|
|
mysql_cond_wait(&rpt->COND_rpl_thread, &rpt->LOCK_rpl_thread);
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
2013-07-09 13:15:53 +02:00
|
|
|
mysql_mutex_destroy(&rpt->LOCK_rpl_thread);
|
|
|
|
mysql_cond_destroy(&rpt->COND_rpl_thread);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
while (rpt->qev_free_list)
|
|
|
|
{
|
|
|
|
rpl_parallel_thread::queued_event *next= rpt->qev_free_list->next;
|
|
|
|
my_free(rpt->qev_free_list);
|
|
|
|
rpt->qev_free_list= next;
|
|
|
|
}
|
|
|
|
while (rpt->rgi_free_list)
|
|
|
|
{
|
|
|
|
rpl_group_info *next= rpt->rgi_free_list->next;
|
|
|
|
delete rpt->rgi_free_list;
|
|
|
|
rpt->rgi_free_list= next;
|
|
|
|
}
|
|
|
|
while (rpt->gco_free_list)
|
|
|
|
{
|
|
|
|
group_commit_orderer *next= rpt->gco_free_list->next_gco;
|
|
|
|
dealloc_gco(rpt->gco_free_list);
|
|
|
|
rpt->gco_free_list= next;
|
|
|
|
}
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
|
2015-02-04 13:57:09 +01:00
|
|
|
old_list= pool->threads;
|
|
|
|
if (new_count < pool->count)
|
|
|
|
pool->count= new_count;
|
2013-06-24 10:50:25 +02:00
|
|
|
pool->threads= new_list;
|
2015-02-04 13:57:09 +01:00
|
|
|
if (new_count > pool->count)
|
|
|
|
pool->count= new_count;
|
|
|
|
my_free(old_list);
|
2013-06-24 10:50:25 +02:00
|
|
|
pool->free_list= new_free_list;
|
|
|
|
for (i= 0; i < pool->count; ++i)
|
|
|
|
{
|
|
|
|
mysql_mutex_lock(&pool->threads[i]->LOCK_rpl_thread);
|
|
|
|
pool->threads[i]->delay_start= false;
|
|
|
|
mysql_cond_signal(&pool->threads[i]->COND_rpl_thread);
|
2013-10-14 15:28:16 +02:00
|
|
|
while (!pool->threads[i]->running)
|
|
|
|
mysql_cond_wait(&pool->threads[i]->COND_rpl_thread,
|
|
|
|
&pool->threads[i]->LOCK_rpl_thread);
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_mutex_unlock(&pool->threads[i]->LOCK_rpl_thread);
|
|
|
|
}
|
|
|
|
|
2015-10-22 11:18:34 +02:00
|
|
|
pool_mark_not_busy(pool);
|
2014-03-07 12:08:38 +01:00
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
return 0;
|
|
|
|
|
|
|
|
err:
|
|
|
|
if (new_list)
|
|
|
|
{
|
|
|
|
while (new_free_list)
|
|
|
|
{
|
|
|
|
mysql_mutex_lock(&new_free_list->LOCK_rpl_thread);
|
|
|
|
new_free_list->delay_start= false;
|
|
|
|
new_free_list->stop= true;
|
2013-07-09 13:15:53 +02:00
|
|
|
mysql_cond_signal(&new_free_list->COND_rpl_thread);
|
2013-06-24 10:50:25 +02:00
|
|
|
while (!new_free_list->running)
|
|
|
|
mysql_cond_wait(&new_free_list->COND_rpl_thread,
|
|
|
|
&new_free_list->LOCK_rpl_thread);
|
|
|
|
while (new_free_list->running)
|
|
|
|
mysql_cond_wait(&new_free_list->COND_rpl_thread,
|
|
|
|
&new_free_list->LOCK_rpl_thread);
|
2013-07-08 16:47:07 +02:00
|
|
|
mysql_mutex_unlock(&new_free_list->LOCK_rpl_thread);
|
2013-09-23 14:46:57 +02:00
|
|
|
new_free_list= new_free_list->next;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
my_free(new_list);
|
|
|
|
}
|
2015-10-22 11:18:34 +02:00
|
|
|
pool_mark_not_busy(pool);
|
2013-06-24 10:50:25 +02:00
|
|
|
return 1;
|
|
|
|
}
|
|
|
|
|
2017-01-29 21:10:56 +01:00
|
|
|
/*
|
|
|
|
Deactivate the parallel replication thread pool, if there are now no more
|
|
|
|
SQL threads running.
|
|
|
|
*/
|
|
|
|
|
|
|
|
int rpl_parallel_resize_pool_if_no_slaves(void)
|
|
|
|
{
|
|
|
|
/* master_info_index is set to NULL on shutdown */
|
2017-04-23 10:49:58 +02:00
|
|
|
if (opt_slave_parallel_threads > 0 && !any_slave_sql_running(false))
|
2017-01-29 21:10:56 +01:00
|
|
|
return rpl_parallel_inactivate_pool(&global_rpl_thread_pool);
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2018-01-24 22:33:21 +01:00
|
|
|
/**
|
2018-10-02 13:30:44 +02:00
|
|
|
Pool activation is preceeded by taking a "lock" of pool_mark_busy
|
|
|
|
which guarantees the number of running slaves drops to zero atomicly
|
|
|
|
with the number of pool workers.
|
|
|
|
This resolves race between the function caller thread and one
|
|
|
|
that may be attempting to deactivate the pool.
|
2018-01-24 22:33:21 +01:00
|
|
|
*/
|
2015-03-11 09:18:16 +01:00
|
|
|
int
|
|
|
|
rpl_parallel_activate_pool(rpl_parallel_thread_pool *pool)
|
|
|
|
{
|
2018-10-02 13:30:44 +02:00
|
|
|
int rc= 0;
|
|
|
|
|
|
|
|
if ((rc= pool_mark_busy(pool, current_thd)))
|
|
|
|
return rc; // killed
|
|
|
|
|
2015-03-11 09:18:16 +01:00
|
|
|
if (!pool->count)
|
2018-10-02 13:30:44 +02:00
|
|
|
{
|
|
|
|
pool_mark_not_busy(pool);
|
|
|
|
rc= rpl_parallel_change_thread_count(pool, opt_slave_parallel_threads,
|
|
|
|
0);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
pool_mark_not_busy(pool);
|
|
|
|
}
|
|
|
|
return rc;
|
2015-03-11 09:18:16 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
int
|
|
|
|
rpl_parallel_inactivate_pool(rpl_parallel_thread_pool *pool)
|
|
|
|
{
|
2017-01-29 22:44:24 +01:00
|
|
|
return rpl_parallel_change_thread_count(pool, 0, 0);
|
2015-03-11 09:18:16 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
void
|
|
|
|
rpl_parallel_thread::batch_free()
|
|
|
|
{
|
|
|
|
mysql_mutex_assert_owner(&LOCK_rpl_thread);
|
|
|
|
if (loc_qev_list)
|
|
|
|
{
|
|
|
|
*loc_qev_last_ptr_ptr= qev_free_list;
|
|
|
|
qev_free_list= loc_qev_list;
|
|
|
|
loc_qev_list= NULL;
|
|
|
|
dequeue2(loc_qev_size);
|
|
|
|
/* Signal that our queue can now accept more events. */
|
|
|
|
mysql_cond_signal(&COND_rpl_thread_queue);
|
|
|
|
loc_qev_size= 0;
|
|
|
|
qev_free_pending= 0;
|
|
|
|
}
|
|
|
|
if (loc_rgi_list)
|
|
|
|
{
|
|
|
|
*loc_rgi_last_ptr_ptr= rgi_free_list;
|
|
|
|
rgi_free_list= loc_rgi_list;
|
|
|
|
loc_rgi_list= NULL;
|
|
|
|
}
|
|
|
|
if (loc_gco_list)
|
|
|
|
{
|
|
|
|
*loc_gco_last_ptr_ptr= gco_free_list;
|
|
|
|
gco_free_list= loc_gco_list;
|
|
|
|
loc_gco_list= NULL;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void
|
|
|
|
rpl_parallel_thread::inuse_relaylog_refcount_update()
|
|
|
|
{
|
|
|
|
inuse_relaylog *ir= accumulated_ir_last;
|
|
|
|
if (ir)
|
|
|
|
{
|
|
|
|
my_atomic_add64(&ir->dequeued_count, accumulated_ir_count);
|
|
|
|
accumulated_ir_count= 0;
|
|
|
|
accumulated_ir_last= NULL;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpl_parallel_thread::queued_event *
|
2014-05-08 14:20:18 +02:00
|
|
|
rpl_parallel_thread::get_qev_common(Log_event *ev, ulonglong event_size)
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
|
|
|
queued_event *qev;
|
|
|
|
mysql_mutex_assert_owner(&LOCK_rpl_thread);
|
|
|
|
if ((qev= qev_free_list))
|
|
|
|
qev_free_list= qev->next;
|
|
|
|
else if(!(qev= (queued_event *)my_malloc(sizeof(*qev), MYF(0))))
|
|
|
|
{
|
|
|
|
my_error(ER_OUTOFMEMORY, MYF(0), (int)sizeof(*qev));
|
|
|
|
return NULL;
|
|
|
|
}
|
MDEV-6551: Some replication errors are ignored if slave_parallel_threads > 0
The problem occured when using parallel replication, and an error occured that
caused the SQL thread to stop when the IO thread had already reached a
following binlog file from the master (or otherwise performed a relay log
rotation).
In this case, the Rotate Event at the end of the relay log file could still be
executed, even though an earlier event in that relay log file had gotten an
error. This would cause the position to be incorrectly updated, so that upon
restart of the SQL thread, the event that had failed would be silently skipped
and ignored, causing replication corruption.
Fixed by checking before executing Rotate Event, whether an earlier event
has failed. If so, the Rotate Event is not executed, just dequeued, same as
for other normal events following a failing event.
2014-08-15 11:31:13 +02:00
|
|
|
qev->typ= rpl_parallel_thread::queued_event::QUEUED_EVENT;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
qev->ev= ev;
|
2017-09-28 12:38:02 +02:00
|
|
|
qev->event_size= (size_t)event_size;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
qev->next= NULL;
|
2014-05-08 14:20:18 +02:00
|
|
|
return qev;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
rpl_parallel_thread::queued_event *
|
|
|
|
rpl_parallel_thread::get_qev(Log_event *ev, ulonglong event_size,
|
|
|
|
Relay_log_info *rli)
|
|
|
|
{
|
|
|
|
queued_event *qev= get_qev_common(ev, event_size);
|
|
|
|
if (!qev)
|
|
|
|
return NULL;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
strcpy(qev->event_relay_log_name, rli->event_relay_log_name);
|
|
|
|
qev->event_relay_log_pos= rli->event_relay_log_pos;
|
|
|
|
qev->future_event_relay_log_pos= rli->future_event_relay_log_pos;
|
|
|
|
strcpy(qev->future_event_master_log_name, rli->future_event_master_log_name);
|
|
|
|
return qev;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-05-08 14:20:18 +02:00
|
|
|
rpl_parallel_thread::queued_event *
|
|
|
|
rpl_parallel_thread::retry_get_qev(Log_event *ev, queued_event *orig_qev,
|
|
|
|
const char *relay_log_name,
|
|
|
|
ulonglong event_pos, ulonglong event_size)
|
|
|
|
{
|
|
|
|
queued_event *qev= get_qev_common(ev, event_size);
|
|
|
|
if (!qev)
|
|
|
|
return NULL;
|
|
|
|
qev->rgi= orig_qev->rgi;
|
|
|
|
strcpy(qev->event_relay_log_name, relay_log_name);
|
|
|
|
qev->event_relay_log_pos= event_pos;
|
|
|
|
qev->future_event_relay_log_pos= event_pos+event_size;
|
|
|
|
strcpy(qev->future_event_master_log_name,
|
|
|
|
orig_qev->future_event_master_log_name);
|
|
|
|
return qev;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
void
|
|
|
|
rpl_parallel_thread::loc_free_qev(rpl_parallel_thread::queued_event *qev)
|
|
|
|
{
|
|
|
|
inuse_relaylog *ir= qev->ir;
|
|
|
|
inuse_relaylog *last_ir= accumulated_ir_last;
|
|
|
|
if (ir != last_ir)
|
|
|
|
{
|
|
|
|
if (last_ir)
|
|
|
|
inuse_relaylog_refcount_update();
|
|
|
|
accumulated_ir_last= ir;
|
|
|
|
}
|
|
|
|
++accumulated_ir_count;
|
|
|
|
if (!loc_qev_list)
|
|
|
|
loc_qev_last_ptr_ptr= &qev->next;
|
|
|
|
else
|
|
|
|
qev->next= loc_qev_list;
|
|
|
|
loc_qev_list= qev;
|
|
|
|
loc_qev_size+= qev->event_size;
|
|
|
|
/*
|
|
|
|
We want to release to the global free list only occasionally, to avoid
|
|
|
|
having to take the LOCK_rpl_thread muted too many times.
|
|
|
|
|
|
|
|
However, we do need to release regularly. If we let the unreleased part
|
|
|
|
grow too large, then the SQL driver thread may go to sleep waiting for
|
|
|
|
the queue to drop below opt_slave_parallel_max_queued, and this in turn
|
|
|
|
can stall all other worker threads for more stuff to do.
|
|
|
|
*/
|
|
|
|
if (++qev_free_pending >= QEV_BATCH_FREE ||
|
|
|
|
loc_qev_size >= opt_slave_parallel_max_queued/3)
|
|
|
|
{
|
|
|
|
mysql_mutex_lock(&LOCK_rpl_thread);
|
|
|
|
batch_free();
|
|
|
|
mysql_mutex_unlock(&LOCK_rpl_thread);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
void
|
|
|
|
rpl_parallel_thread::free_qev(rpl_parallel_thread::queued_event *qev)
|
|
|
|
{
|
|
|
|
mysql_mutex_assert_owner(&LOCK_rpl_thread);
|
|
|
|
qev->next= qev_free_list;
|
|
|
|
qev_free_list= qev;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
rpl_group_info*
|
|
|
|
rpl_parallel_thread::get_rgi(Relay_log_info *rli, Gtid_log_event *gtid_ev,
|
2014-05-08 14:20:18 +02:00
|
|
|
rpl_parallel_entry *e, ulonglong event_size)
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
|
|
|
rpl_group_info *rgi;
|
|
|
|
mysql_mutex_assert_owner(&LOCK_rpl_thread);
|
|
|
|
if ((rgi= rgi_free_list))
|
|
|
|
{
|
|
|
|
rgi_free_list= rgi->next;
|
|
|
|
rgi->reinit(rli);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
if(!(rgi= new rpl_group_info(rli)))
|
|
|
|
{
|
|
|
|
my_error(ER_OUTOFMEMORY, MYF(0), (int)sizeof(*rgi));
|
|
|
|
return NULL;
|
|
|
|
}
|
|
|
|
rgi->is_parallel_exec = true;
|
|
|
|
}
|
2014-03-07 12:02:09 +01:00
|
|
|
if ((rgi->deferred_events_collecting= rli->mi->rpl_filter->is_on()) &&
|
|
|
|
!rgi->deferred_events)
|
|
|
|
rgi->deferred_events= new Deferred_log_events(rli);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
if (event_group_new_gtid(rgi, gtid_ev))
|
|
|
|
{
|
|
|
|
free_rgi(rgi);
|
|
|
|
my_error(ER_OUT_OF_RESOURCES, MYF(MY_WME));
|
|
|
|
return NULL;
|
|
|
|
}
|
|
|
|
rgi->parallel_entry= e;
|
2014-05-08 14:20:18 +02:00
|
|
|
rgi->relay_log= rli->last_inuse_relaylog;
|
|
|
|
rgi->retry_start_offset= rli->future_event_relay_log_pos-event_size;
|
|
|
|
rgi->retry_event_count= 0;
|
2016-09-08 15:25:40 +02:00
|
|
|
rgi->killed_for_retry= rpl_group_info::RETRY_KILL_NONE;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
|
|
|
return rgi;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-11-13 10:20:48 +01:00
|
|
|
void
|
|
|
|
rpl_parallel_thread::loc_free_rgi(rpl_group_info *rgi)
|
|
|
|
{
|
|
|
|
DBUG_ASSERT(rgi->commit_orderer.waitee == NULL);
|
|
|
|
rgi->free_annotate_event();
|
|
|
|
if (!loc_rgi_list)
|
|
|
|
loc_rgi_last_ptr_ptr= &rgi->next;
|
|
|
|
else
|
|
|
|
rgi->next= loc_rgi_list;
|
|
|
|
loc_rgi_list= rgi;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
void
|
|
|
|
rpl_parallel_thread::free_rgi(rpl_group_info *rgi)
|
|
|
|
{
|
|
|
|
mysql_mutex_assert_owner(&LOCK_rpl_thread);
|
|
|
|
DBUG_ASSERT(rgi->commit_orderer.waitee == NULL);
|
|
|
|
rgi->free_annotate_event();
|
|
|
|
rgi->next= rgi_free_list;
|
|
|
|
rgi_free_list= rgi;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
group_commit_orderer *
|
2015-01-07 14:45:39 +01:00
|
|
|
rpl_parallel_thread::get_gco(uint64 wait_count, group_commit_orderer *prev,
|
|
|
|
uint64 prior_sub_id)
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
|
|
|
group_commit_orderer *gco;
|
|
|
|
mysql_mutex_assert_owner(&LOCK_rpl_thread);
|
|
|
|
if ((gco= gco_free_list))
|
|
|
|
gco_free_list= gco->next_gco;
|
|
|
|
else if(!(gco= (group_commit_orderer *)my_malloc(sizeof(*gco), MYF(0))))
|
|
|
|
{
|
|
|
|
my_error(ER_OUTOFMEMORY, MYF(0), (int)sizeof(*gco));
|
|
|
|
return NULL;
|
|
|
|
}
|
|
|
|
mysql_cond_init(key_COND_group_commit_orderer,
|
|
|
|
&gco->COND_group_commit_orderer, NULL);
|
|
|
|
gco->wait_count= wait_count;
|
|
|
|
gco->prev_gco= prev;
|
|
|
|
gco->next_gco= NULL;
|
2015-01-07 14:45:39 +01:00
|
|
|
gco->prior_sub_id= prior_sub_id;
|
2015-03-24 16:33:51 +01:00
|
|
|
gco->installed= false;
|
2014-12-05 16:09:48 +01:00
|
|
|
gco->flags= 0;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
return gco;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void
|
2014-11-13 10:20:48 +01:00
|
|
|
rpl_parallel_thread::loc_free_gco(group_commit_orderer *gco)
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
2014-11-13 10:20:48 +01:00
|
|
|
if (!loc_gco_list)
|
|
|
|
loc_gco_last_ptr_ptr= &gco->next_gco;
|
|
|
|
else
|
|
|
|
gco->next_gco= loc_gco_list;
|
|
|
|
loc_gco_list= gco;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
rpl_parallel_thread_pool::rpl_parallel_thread_pool()
|
2015-10-22 11:18:34 +02:00
|
|
|
: threads(0), free_list(0), count(0), inited(false), busy(false)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
int
|
|
|
|
rpl_parallel_thread_pool::init(uint32 size)
|
|
|
|
{
|
|
|
|
threads= NULL;
|
|
|
|
free_list= NULL;
|
2015-10-22 11:18:34 +02:00
|
|
|
count= 0;
|
|
|
|
busy= false;
|
2013-06-24 10:50:25 +02:00
|
|
|
|
|
|
|
mysql_mutex_init(key_LOCK_rpl_thread_pool, &LOCK_rpl_thread_pool,
|
|
|
|
MY_MUTEX_INIT_SLOW);
|
|
|
|
mysql_cond_init(key_COND_rpl_thread_pool, &COND_rpl_thread_pool, NULL);
|
|
|
|
inited= true;
|
|
|
|
|
2015-03-11 09:18:16 +01:00
|
|
|
/*
|
|
|
|
The pool is initially empty. Threads will be spawned when a slave SQL
|
|
|
|
thread is started.
|
|
|
|
*/
|
|
|
|
|
|
|
|
return 0;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void
|
|
|
|
rpl_parallel_thread_pool::destroy()
|
|
|
|
{
|
|
|
|
if (!inited)
|
|
|
|
return;
|
2017-01-29 22:44:24 +01:00
|
|
|
rpl_parallel_change_thread_count(this, 0, 1);
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_mutex_destroy(&LOCK_rpl_thread_pool);
|
|
|
|
mysql_cond_destroy(&COND_rpl_thread_pool);
|
2013-06-25 09:30:19 +02:00
|
|
|
inited= false;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-09-23 14:46:57 +02:00
|
|
|
/*
|
|
|
|
Wait for a worker thread to become idle. When one does, grab the thread for
|
|
|
|
our use and return it.
|
|
|
|
|
|
|
|
Note that we return with the worker threads's LOCK_rpl_thread mutex locked.
|
|
|
|
*/
|
2013-06-24 10:50:25 +02:00
|
|
|
struct rpl_parallel_thread *
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpl_parallel_thread_pool::get_thread(rpl_parallel_thread **owner,
|
|
|
|
rpl_parallel_entry *entry)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
|
|
|
rpl_parallel_thread *rpt;
|
|
|
|
|
2017-01-29 21:10:56 +01:00
|
|
|
DBUG_ASSERT(count > 0);
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_mutex_lock(&LOCK_rpl_thread_pool);
|
2015-10-22 11:18:34 +02:00
|
|
|
while (unlikely(busy) || !(rpt= free_list))
|
2013-06-24 10:50:25 +02:00
|
|
|
mysql_cond_wait(&COND_rpl_thread_pool, &LOCK_rpl_thread_pool);
|
|
|
|
free_list= rpt->next;
|
|
|
|
mysql_mutex_unlock(&LOCK_rpl_thread_pool);
|
2013-06-25 09:30:19 +02:00
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpt->current_owner= owner;
|
2013-06-24 10:50:25 +02:00
|
|
|
rpt->current_entry= entry;
|
|
|
|
|
|
|
|
return rpt;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
/*
|
|
|
|
Release a thread to the thread pool.
|
|
|
|
The thread should be locked, and should not have any work queued for it.
|
|
|
|
*/
|
|
|
|
void
|
|
|
|
rpl_parallel_thread_pool::release_thread(rpl_parallel_thread *rpt)
|
|
|
|
{
|
|
|
|
rpl_parallel_thread *list;
|
|
|
|
|
|
|
|
mysql_mutex_assert_owner(&rpt->LOCK_rpl_thread);
|
|
|
|
DBUG_ASSERT(rpt->current_owner == NULL);
|
|
|
|
mysql_mutex_lock(&LOCK_rpl_thread_pool);
|
|
|
|
list= free_list;
|
|
|
|
rpt->next= list;
|
|
|
|
free_list= rpt;
|
|
|
|
if (!list)
|
|
|
|
mysql_cond_broadcast(&COND_rpl_thread_pool);
|
|
|
|
mysql_mutex_unlock(&LOCK_rpl_thread_pool);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
/*
|
|
|
|
Obtain a worker thread that we can queue an event to.
|
|
|
|
|
|
|
|
Each invocation allocates a new worker thread, to maximise
|
|
|
|
parallelism. However, only up to a maximum of
|
|
|
|
--slave-domain-parallel-threads workers can be occupied by a single
|
|
|
|
replication domain; after that point, we start re-using worker threads that
|
|
|
|
are still executing events that were queued earlier for this thread.
|
|
|
|
|
|
|
|
We never queue more than --rpl-parallel-wait-queue_max amount of events
|
|
|
|
for one worker, to avoid the SQL driver thread using up all memory with
|
|
|
|
queued events while worker threads are stalling.
|
|
|
|
|
|
|
|
Note that this function returns with rpl_parallel_thread::LOCK_rpl_thread
|
|
|
|
locked. Exception is if we were killed, in which case NULL is returned.
|
|
|
|
|
|
|
|
The *did_enter_cond flag is set true if we had to wait for a worker thread
|
2014-02-26 16:38:42 +01:00
|
|
|
to become free (with mysql_cond_wait()). If so, old_stage will also be set,
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
and the LOCK_rpl_thread must be released with THD::EXIT_COND() instead
|
|
|
|
of mysql_mutex_unlock.
|
|
|
|
|
|
|
|
If the flag `reuse' is set, the last worker thread will be returned again,
|
|
|
|
if it is still available. Otherwise a new worker thread is allocated.
|
|
|
|
*/
|
|
|
|
rpl_parallel_thread *
|
2014-06-25 15:17:03 +02:00
|
|
|
rpl_parallel_entry::choose_thread(rpl_group_info *rgi, bool *did_enter_cond,
|
2014-02-26 16:38:42 +01:00
|
|
|
PSI_stage_info *old_stage, bool reuse)
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
|
|
|
uint32 idx;
|
2014-06-25 15:17:03 +02:00
|
|
|
Relay_log_info *rli= rgi->rli;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpl_parallel_thread *thr;
|
|
|
|
|
|
|
|
idx= rpl_thread_idx;
|
|
|
|
if (!reuse)
|
|
|
|
{
|
|
|
|
++idx;
|
|
|
|
if (idx >= rpl_thread_max)
|
|
|
|
idx= 0;
|
|
|
|
rpl_thread_idx= idx;
|
|
|
|
}
|
|
|
|
thr= rpl_threads[idx];
|
|
|
|
if (thr)
|
|
|
|
{
|
|
|
|
*did_enter_cond= false;
|
|
|
|
mysql_mutex_lock(&thr->LOCK_rpl_thread);
|
|
|
|
for (;;)
|
|
|
|
{
|
|
|
|
if (thr->current_owner != &rpl_threads[idx])
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
The worker thread became idle, and returned to the free list and
|
|
|
|
possibly was allocated to a different request. So we should allocate
|
|
|
|
a new worker thread.
|
|
|
|
*/
|
|
|
|
unlock_or_exit_cond(rli->sql_driver_thd, &thr->LOCK_rpl_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
did_enter_cond, old_stage);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
thr= NULL;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
else if (thr->queued_size <= opt_slave_parallel_max_queued)
|
|
|
|
{
|
|
|
|
/* The thread is ready to queue into. */
|
|
|
|
break;
|
|
|
|
}
|
2018-05-26 15:57:18 +02:00
|
|
|
else if (unlikely(rli->sql_driver_thd->check_killed(1)))
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
|
|
|
unlock_or_exit_cond(rli->sql_driver_thd, &thr->LOCK_rpl_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
did_enter_cond, old_stage);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
my_error(ER_CONNECTION_KILLED, MYF(0));
|
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_wait_queue_max",
|
|
|
|
{
|
|
|
|
debug_sync_set_action(rli->sql_driver_thd,
|
|
|
|
STRING_WITH_LEN("now SIGNAL wait_queue_killed"));
|
|
|
|
};);
|
2014-06-25 15:17:03 +02:00
|
|
|
slave_output_error_info(rgi, rli->sql_driver_thd);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
return NULL;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
We have reached the limit of how much memory we are allowed to use
|
|
|
|
for queuing events, so wait for the thread to consume some of its
|
|
|
|
queue.
|
|
|
|
*/
|
|
|
|
if (!*did_enter_cond)
|
|
|
|
{
|
|
|
|
/*
|
2014-02-26 16:38:42 +01:00
|
|
|
We need to do the debug_sync before ENTER_COND().
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
Because debug_sync changes the thd->mysys_var->current_mutex,
|
|
|
|
and this can cause THD::awake to use the wrong mutex.
|
|
|
|
*/
|
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_wait_queue_max",
|
|
|
|
{
|
|
|
|
debug_sync_set_action(rli->sql_driver_thd,
|
|
|
|
STRING_WITH_LEN("now SIGNAL wait_queue_ready"));
|
|
|
|
};);
|
2014-02-26 16:38:42 +01:00
|
|
|
rli->sql_driver_thd->ENTER_COND(&thr->COND_rpl_thread_queue,
|
|
|
|
&thr->LOCK_rpl_thread,
|
|
|
|
&stage_waiting_for_room_in_worker_thread,
|
|
|
|
old_stage);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
*did_enter_cond= true;
|
|
|
|
}
|
|
|
|
mysql_cond_wait(&thr->COND_rpl_thread_queue, &thr->LOCK_rpl_thread);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (!thr)
|
|
|
|
rpl_threads[idx]= thr= global_rpl_thread_pool.get_thread(&rpl_threads[idx],
|
|
|
|
this);
|
|
|
|
|
|
|
|
return thr;
|
|
|
|
}
|
|
|
|
|
2013-09-16 14:33:49 +02:00
|
|
|
static void
|
|
|
|
free_rpl_parallel_entry(void *element)
|
|
|
|
{
|
|
|
|
rpl_parallel_entry *e= (rpl_parallel_entry *)element;
|
2015-01-07 14:45:39 +01:00
|
|
|
while (e->current_gco)
|
|
|
|
{
|
|
|
|
group_commit_orderer *prev_gco= e->current_gco->prev_gco;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
dealloc_gco(e->current_gco);
|
2015-01-07 14:45:39 +01:00
|
|
|
e->current_gco= prev_gco;
|
|
|
|
}
|
2013-09-16 14:33:49 +02:00
|
|
|
mysql_cond_destroy(&e->COND_parallel_entry);
|
|
|
|
mysql_mutex_destroy(&e->LOCK_parallel_entry);
|
|
|
|
my_free(e);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
rpl_parallel::rpl_parallel() :
|
2013-10-08 14:36:06 +02:00
|
|
|
current(NULL), sql_thread_stopping(false)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
|
|
|
my_hash_init(&domain_hash, &my_charset_bin, 32,
|
|
|
|
offsetof(rpl_parallel_entry, domain_id), sizeof(uint32),
|
2013-09-16 14:33:49 +02:00
|
|
|
NULL, free_rpl_parallel_entry, HASH_UNIQUE);
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-09-17 14:07:21 +02:00
|
|
|
void
|
|
|
|
rpl_parallel::reset()
|
|
|
|
{
|
|
|
|
my_hash_reset(&domain_hash);
|
|
|
|
current= NULL;
|
2013-10-08 14:36:06 +02:00
|
|
|
sql_thread_stopping= false;
|
2013-09-17 14:07:21 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
rpl_parallel::~rpl_parallel()
|
|
|
|
{
|
|
|
|
my_hash_free(&domain_hash);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
rpl_parallel_entry *
|
|
|
|
rpl_parallel::find(uint32 domain_id)
|
|
|
|
{
|
|
|
|
struct rpl_parallel_entry *e;
|
|
|
|
|
|
|
|
if (!(e= (rpl_parallel_entry *)my_hash_search(&domain_hash,
|
|
|
|
(const uchar *)&domain_id, 0)))
|
|
|
|
{
|
|
|
|
/* Allocate a new, empty one. */
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
ulong count= opt_slave_domain_parallel_threads;
|
|
|
|
if (count == 0 || count > opt_slave_parallel_threads)
|
|
|
|
count= opt_slave_parallel_threads;
|
|
|
|
rpl_parallel_thread **p;
|
|
|
|
if (!my_multi_malloc(MYF(MY_WME|MY_ZEROFILL),
|
|
|
|
&e, sizeof(*e),
|
|
|
|
&p, count*sizeof(*p),
|
|
|
|
NULL))
|
|
|
|
{
|
|
|
|
my_error(ER_OUTOFMEMORY, MYF(0), (int)(sizeof(*e)+count*sizeof(*p)));
|
2013-06-24 10:50:25 +02:00
|
|
|
return NULL;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
}
|
|
|
|
e->rpl_threads= p;
|
|
|
|
e->rpl_thread_max= count;
|
2013-06-24 10:50:25 +02:00
|
|
|
e->domain_id= domain_id;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
e->stop_on_error_sub_id= (uint64)ULONGLONG_MAX;
|
2015-10-22 11:18:34 +02:00
|
|
|
e->pause_sub_id= (uint64)ULONGLONG_MAX;
|
2013-06-24 10:50:25 +02:00
|
|
|
if (my_hash_insert(&domain_hash, (uchar *)e))
|
|
|
|
{
|
|
|
|
my_free(e);
|
|
|
|
return NULL;
|
|
|
|
}
|
2013-07-03 13:46:33 +02:00
|
|
|
mysql_mutex_init(key_LOCK_parallel_entry, &e->LOCK_parallel_entry,
|
|
|
|
MY_MUTEX_INIT_FAST);
|
2013-07-04 09:20:56 +02:00
|
|
|
mysql_cond_init(key_COND_parallel_entry, &e->COND_parallel_entry, NULL);
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
2013-11-05 12:01:26 +01:00
|
|
|
else
|
|
|
|
e->force_abort= false;
|
2013-06-24 10:50:25 +02:00
|
|
|
|
|
|
|
return e;
|
|
|
|
}
|
|
|
|
|
2017-02-10 21:35:04 +01:00
|
|
|
/**
|
|
|
|
Wait until all sql worker threads has stopped processing
|
|
|
|
|
|
|
|
This is called when sql thread has been killed/stopped
|
|
|
|
*/
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2013-07-04 09:20:56 +02:00
|
|
|
void
|
2014-03-03 12:13:55 +01:00
|
|
|
rpl_parallel::wait_for_done(THD *thd, Relay_log_info *rli)
|
2013-07-04 09:20:56 +02:00
|
|
|
{
|
|
|
|
struct rpl_parallel_entry *e;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
rpl_parallel_thread *rpt;
|
|
|
|
uint32 i, j;
|
2013-07-04 09:20:56 +02:00
|
|
|
|
2013-11-05 12:01:26 +01:00
|
|
|
/*
|
|
|
|
First signal all workers that they must force quit; no more events will
|
|
|
|
be queued to complete any partial event groups executed.
|
|
|
|
*/
|
|
|
|
for (i= 0; i < domain_hash.records; ++i)
|
|
|
|
{
|
|
|
|
e= (struct rpl_parallel_entry *)my_hash_element(&domain_hash, i);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
mysql_mutex_lock(&e->LOCK_parallel_entry);
|
|
|
|
/*
|
|
|
|
We want the worker threads to stop as quickly as is safe. If the slave
|
|
|
|
SQL threads are behind, we could have significant amount of events
|
|
|
|
queued for the workers, and we want to stop without waiting for them
|
|
|
|
all to be applied first. But if any event group has already started
|
|
|
|
executing in a worker, we want to be sure that all prior event groups
|
|
|
|
are also executed, so that we stop at a consistent point in the binlog
|
|
|
|
stream (per replication domain).
|
|
|
|
|
|
|
|
All event groups wait for e->count_committing_event_groups to reach
|
|
|
|
the value of group_commit_orderer::wait_count before starting to
|
|
|
|
execute. Thus, at this point we know that any event group with a
|
|
|
|
strictly larger wait_count are safe to skip, none of them can have
|
|
|
|
started executing yet. So we set e->stop_count here and use it to
|
|
|
|
decide in the worker threads whether to continue executing an event
|
|
|
|
group or whether to skip it, when force_abort is set.
|
2014-03-03 12:13:55 +01:00
|
|
|
|
|
|
|
If we stop due to reaching the START SLAVE UNTIL condition, then we
|
|
|
|
need to continue executing any queued events up to that point.
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
*/
|
2013-11-05 12:01:26 +01:00
|
|
|
e->force_abort= true;
|
2014-03-03 12:13:55 +01:00
|
|
|
e->stop_count= rli->stop_for_until ?
|
|
|
|
e->count_queued_event_groups : e->count_committing_event_groups;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
mysql_mutex_unlock(&e->LOCK_parallel_entry);
|
|
|
|
for (j= 0; j < e->rpl_thread_max; ++j)
|
2013-11-05 12:01:26 +01:00
|
|
|
{
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
if ((rpt= e->rpl_threads[j]))
|
|
|
|
{
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
|
|
|
if (rpt->current_owner == &e->rpl_threads[j])
|
|
|
|
mysql_cond_signal(&rpt->COND_rpl_thread);
|
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
}
|
2013-11-05 12:01:26 +01:00
|
|
|
}
|
|
|
|
}
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
DBUG_EXECUTE_IF("rpl_parallel_wait_for_done_trigger",
|
|
|
|
{
|
|
|
|
debug_sync_set_action(thd,
|
|
|
|
STRING_WITH_LEN("now SIGNAL wait_for_done_waiting"));
|
|
|
|
};);
|
2013-11-05 12:01:26 +01:00
|
|
|
|
2013-07-04 09:20:56 +02:00
|
|
|
for (i= 0; i < domain_hash.records; ++i)
|
|
|
|
{
|
|
|
|
e= (struct rpl_parallel_entry *)my_hash_element(&domain_hash, i);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
for (j= 0; j < e->rpl_thread_max; ++j)
|
|
|
|
{
|
|
|
|
if ((rpt= e->rpl_threads[j]))
|
|
|
|
{
|
|
|
|
mysql_mutex_lock(&rpt->LOCK_rpl_thread);
|
|
|
|
while (rpt->current_owner == &e->rpl_threads[j])
|
2015-11-23 18:58:30 +01:00
|
|
|
mysql_cond_wait(&rpt->COND_rpl_thread_stop, &rpt->LOCK_rpl_thread);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
mysql_mutex_unlock(&rpt->LOCK_rpl_thread);
|
|
|
|
}
|
|
|
|
}
|
2013-07-04 09:20:56 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-03-03 12:13:55 +01:00
|
|
|
/*
|
|
|
|
This function handles the case where the SQL driver thread reached the
|
|
|
|
START SLAVE UNTIL position; we stop queueing more events but continue
|
|
|
|
processing remaining, already queued events; then use executes manual
|
|
|
|
STOP SLAVE; then this function signals to worker threads that they
|
|
|
|
should stop the processing of any remaining queued events.
|
|
|
|
*/
|
|
|
|
void
|
|
|
|
rpl_parallel::stop_during_until()
|
|
|
|
{
|
|
|
|
struct rpl_parallel_entry *e;
|
|
|
|
uint32 i;
|
|
|
|
|
|
|
|
for (i= 0; i < domain_hash.records; ++i)
|
|
|
|
{
|
|
|
|
e= (struct rpl_parallel_entry *)my_hash_element(&domain_hash, i);
|
|
|
|
mysql_mutex_lock(&e->LOCK_parallel_entry);
|
|
|
|
if (e->force_abort)
|
|
|
|
e->stop_count= e->count_committing_event_groups;
|
|
|
|
mysql_mutex_unlock(&e->LOCK_parallel_entry);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-01-08 11:00:44 +01:00
|
|
|
bool
|
|
|
|
rpl_parallel::workers_idle()
|
|
|
|
{
|
|
|
|
struct rpl_parallel_entry *e;
|
|
|
|
uint32 i, max_i;
|
|
|
|
|
|
|
|
max_i= domain_hash.records;
|
|
|
|
for (i= 0; i < max_i; ++i)
|
|
|
|
{
|
|
|
|
bool active;
|
|
|
|
e= (struct rpl_parallel_entry *)my_hash_element(&domain_hash, i);
|
|
|
|
mysql_mutex_lock(&e->LOCK_parallel_entry);
|
|
|
|
active= e->current_sub_id > e->last_committed_sub_id;
|
|
|
|
mysql_mutex_unlock(&e->LOCK_parallel_entry);
|
|
|
|
if (active)
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
return (i == max_i);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-08-19 14:26:42 +02:00
|
|
|
int
|
|
|
|
rpl_parallel_entry::queue_master_restart(rpl_group_info *rgi,
|
|
|
|
Format_description_log_event *fdev)
|
|
|
|
{
|
|
|
|
uint32 idx;
|
|
|
|
rpl_parallel_thread *thr;
|
|
|
|
rpl_parallel_thread::queued_event *qev;
|
|
|
|
Relay_log_info *rli= rgi->rli;
|
|
|
|
|
|
|
|
/*
|
|
|
|
We only need to queue the server restart if we still have a thread working
|
|
|
|
on a (potentially partial) event group.
|
|
|
|
|
|
|
|
If the last thread we queued for has finished, then it cannot have any
|
|
|
|
partial event group that needs aborting.
|
|
|
|
|
|
|
|
Thus there is no need for the full complexity of choose_thread(). We only
|
|
|
|
need to check if we have a current worker thread, and queue for it if so.
|
|
|
|
*/
|
|
|
|
idx= rpl_thread_idx;
|
|
|
|
thr= rpl_threads[idx];
|
|
|
|
if (!thr)
|
|
|
|
return 0;
|
|
|
|
mysql_mutex_lock(&thr->LOCK_rpl_thread);
|
|
|
|
if (thr->current_owner != &rpl_threads[idx])
|
|
|
|
{
|
|
|
|
/* No active worker thread, so no need to queue the master restart. */
|
|
|
|
mysql_mutex_unlock(&thr->LOCK_rpl_thread);
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!(qev= thr->get_qev(fdev, 0, rli)))
|
|
|
|
{
|
|
|
|
mysql_mutex_unlock(&thr->LOCK_rpl_thread);
|
|
|
|
return 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
qev->rgi= rgi;
|
|
|
|
qev->typ= rpl_parallel_thread::queued_event::QUEUED_MASTER_RESTART;
|
|
|
|
qev->entry_for_queued= this;
|
|
|
|
qev->ir= rli->last_inuse_relaylog;
|
|
|
|
++qev->ir->queued_count;
|
|
|
|
thr->enqueue(qev);
|
2014-11-17 12:42:02 +01:00
|
|
|
mysql_cond_signal(&thr->COND_rpl_thread);
|
2014-08-19 14:26:42 +02:00
|
|
|
mysql_mutex_unlock(&thr->LOCK_rpl_thread);
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-08-20 10:59:39 +02:00
|
|
|
int
|
2014-07-02 12:51:45 +02:00
|
|
|
rpl_parallel::wait_for_workers_idle(THD *thd)
|
|
|
|
{
|
|
|
|
uint32 i, max_i;
|
|
|
|
|
2014-08-20 10:59:39 +02:00
|
|
|
/*
|
|
|
|
The domain_hash is only accessed by the SQL driver thread, so it is safe
|
|
|
|
to iterate over without a lock.
|
|
|
|
*/
|
2014-07-02 12:51:45 +02:00
|
|
|
max_i= domain_hash.records;
|
|
|
|
for (i= 0; i < max_i; ++i)
|
|
|
|
{
|
2015-05-26 12:47:35 +02:00
|
|
|
PSI_stage_info old_stage;
|
2014-07-02 12:51:45 +02:00
|
|
|
struct rpl_parallel_entry *e;
|
2015-05-26 12:47:35 +02:00
|
|
|
int err= 0;
|
2014-07-02 12:51:45 +02:00
|
|
|
|
|
|
|
e= (struct rpl_parallel_entry *)my_hash_element(&domain_hash, i);
|
|
|
|
mysql_mutex_lock(&e->LOCK_parallel_entry);
|
2015-10-22 11:18:34 +02:00
|
|
|
++e->need_sub_id_signal;
|
2015-05-26 12:47:35 +02:00
|
|
|
thd->ENTER_COND(&e->COND_parallel_entry, &e->LOCK_parallel_entry,
|
|
|
|
&stage_waiting_for_workers_idle, &old_stage);
|
|
|
|
while (e->current_sub_id > e->last_committed_sub_id)
|
2014-07-02 12:51:45 +02:00
|
|
|
{
|
2018-04-04 11:16:12 +02:00
|
|
|
if (unlikely(thd->check_killed()))
|
2015-05-26 12:47:35 +02:00
|
|
|
{
|
|
|
|
err= 1;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
mysql_cond_wait(&e->COND_parallel_entry, &e->LOCK_parallel_entry);
|
2014-07-02 12:51:45 +02:00
|
|
|
}
|
2015-10-22 11:18:34 +02:00
|
|
|
--e->need_sub_id_signal;
|
2015-05-26 12:47:35 +02:00
|
|
|
thd->EXIT_COND(&old_stage);
|
|
|
|
if (err)
|
|
|
|
return err;
|
2014-07-02 12:51:45 +02:00
|
|
|
}
|
2014-08-20 10:59:39 +02:00
|
|
|
return 0;
|
2014-07-02 12:51:45 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-6589: Incorrect relay log start position when restarting SQL thread after error in parallel replication
The problem occurs in parallel replication in GTID mode, when we are using
multiple replication domains. In this case, if the SQL thread stops, the
slave GTID position may refer to a different point in the relay log for each
domain.
The bug was that when the SQL thread was stopped and restarted (but the IO
thread was kept running), the SQL thread would resume applying the relay log
from the point of the most advanced replication domain, silently skipping all
earlier events within other domains. This caused replication corruption.
This patch solves the problem by storing, when the SQL thread stops with
multiple parallel replication domains active, the current GTID
position. Additionally, the current position in the relay logs is moved back
to a point known to be earlier than the current position of any replication
domain. Then when the SQL thread restarts from the earlier position, GTIDs
encountered are compared against the stored GTID position. Any GTID that was
already applied before the stop is skipped to avoid duplicate apply.
This patch should have no effect if multi-domain GTID parallel replication is
not used. Similarly, if both SQL and IO thread are stopped and restarted, the
patch has no effect, as in this case the existing relay logs are removed and
re-fetched from the master at the current global @@gtid_slave_pos.
2015-02-18 12:22:50 +01:00
|
|
|
/*
|
|
|
|
Handle seeing a GTID during slave restart in GTID mode. If we stopped with
|
|
|
|
different replication domains having reached different positions in the relay
|
|
|
|
log, we need to skip event groups in domains that are further progressed.
|
|
|
|
|
|
|
|
Updates the state with the seen GTID, and returns true if this GTID should
|
|
|
|
be skipped, false otherwise.
|
|
|
|
*/
|
|
|
|
bool
|
|
|
|
process_gtid_for_restart_pos(Relay_log_info *rli, rpl_gtid *gtid)
|
|
|
|
{
|
|
|
|
slave_connection_state::entry *gtid_entry;
|
|
|
|
slave_connection_state *state= &rli->restart_gtid_pos;
|
|
|
|
|
|
|
|
if (likely(state->count() == 0) ||
|
|
|
|
!(gtid_entry= state->find_entry(gtid->domain_id)))
|
|
|
|
return false;
|
|
|
|
if (gtid->server_id == gtid_entry->gtid.server_id)
|
|
|
|
{
|
|
|
|
uint64 seq_no= gtid_entry->gtid.seq_no;
|
|
|
|
if (gtid->seq_no >= seq_no)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
This domain has reached its start position. So remove it, so that
|
|
|
|
further events will be processed normally.
|
|
|
|
*/
|
|
|
|
state->remove(>id_entry->gtid);
|
|
|
|
}
|
|
|
|
return gtid->seq_no <= seq_no;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
/*
|
|
|
|
This is used when we get an error during processing in do_event();
|
|
|
|
We will not queue any event to the thread, but we still need to wake it up
|
|
|
|
to be sure that it will be returned to the pool.
|
|
|
|
*/
|
|
|
|
static void
|
|
|
|
abandon_worker_thread(THD *thd, rpl_parallel_thread *cur_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
bool *did_enter_cond, PSI_stage_info *old_stage)
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
{
|
|
|
|
unlock_or_exit_cond(thd, &cur_thread->LOCK_rpl_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
did_enter_cond, old_stage);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
mysql_cond_signal(&cur_thread->COND_rpl_thread);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
Fixes for parallel slave:
- Made slaves temporary table multi-thread slave safe by adding mutex around save_temporary_table usage.
- rli->save_temporary_tables is the active list of all used temporary tables
- This is copied to THD->temporary_tables when temporary tables are opened and updated when temporary tables are closed
- Added THD->lock_temporary_tables() and THD->unlock_temporary_tables() to simplify this.
- Relay_log_info->sql_thd renamed to Relay_log_info->sql_driver_thd to avoid wrong usage for merged code.
- Added is_part_of_group() to mark functions that are part of the next function. This replaces setting IN_STMT when events are executed.
- Added is_begin(), is_commit() and is_rollback() functions to Query_log_event to simplify code.
- If slave_skip_counter is set run things in single threaded mode. This simplifies code for skipping events.
- Updating state of relay log (IN_STMT and IN_TRANSACTION) is moved to one single function: update_state_of_relay_log()
We can't use OPTION_BEGIN to check for the state anymore as the sql_driver and sql execution threads may be different.
Clear IN_STMT and IN_TRANSACTION in init_relay_log_pos() and Relay_log_info::cleanup_context() to ensure the flags doesn't survive slave restarts
is_in_group() is now independent of state of executed transaction.
- Reset thd->transaction.all.modified_non_trans_table() if we did set it for single table row events.
This was mainly for keeping the flag as documented.
- Changed slave_open_temp_tables to uint32 to be able to use atomic operators on it.
- Relay_log_info::sleep_lock -> rpl_group_info::sleep_lock
- Relay_log_info::sleep_cond -> rpl_group_info::sleep_cond
- Changed some functions to take rpl_group_info instead of Relay_log_info to make them multi-slave safe and to simplify usage
- do_shall_skip()
- continue_group()
- sql_slave_killed()
- next_event()
- Simplifed arguments to io_salve_killed(), check_io_slave_killed() and sql_slave_killed(); No reason to supply THD as this is part of the given structure.
- set_thd_in_use_temporary_tables() removed as in_use is set on usage
- Added information to thd_proc_info() which thread is waiting for slave mutex to exit.
- In open_table() reuse code from find_temporary_table()
Other things:
- More DBUG statements
- Fixed the rpl_incident.test can be run with --debug
- More comments
- Disabled not used function rpl_connect_master()
mysql-test/suite/perfschema/r/all_instances.result:
Moved sleep_lock and sleep_cond to rpl_group_info
mysql-test/suite/rpl/r/rpl_incident.result:
Updated result
mysql-test/suite/rpl/t/rpl_incident-master.opt:
Not needed anymore
mysql-test/suite/rpl/t/rpl_incident.test:
Fixed that test can be run with --debug
sql/handler.cc:
More DBUG_PRINT
sql/log.cc:
More comments
sql/log_event.cc:
Added DBUG statements
do_shall_skip(), continue_group() now takes rpl_group_info param
Use is_begin(), is_commit() and is_rollback() functions instead of inspecting query string
We don't have set slaves temporary tables 'in_use' as this is now done when tables are opened.
Removed IN_STMT flag setting. This is now done in update_state_of_relay_log()
Use IN_TRANSACTION flag to test state of relay log.
In rows_event_stmt_cleanup() reset thd->transaction.all.modified_non_trans_table if we had set this before.
sql/log_event.h:
do_shall_skip(), continue_group() now takes rpl_group_info param
Added is_part_of_group() to mark events that are part of the next event. This replaces setting IN_STMT when events are executed.
Added is_begin(), is_commit() and is_rollback() functions to Query_log_event to simplify code.
sql/log_event_old.cc:
Removed IN_STMT flag setting. This is now done in update_state_of_relay_log()
do_shall_skip(), continue_group() now takes rpl_group_info param
sql/log_event_old.h:
Added is_part_of_group() to mark events that are part of the next event.
do_shall_skip(), continue_group() now takes rpl_group_info param
sql/mysqld.cc:
Changed slave_open_temp_tables to uint32 to be able to use atomic operators on it.
Relay_log_info::sleep_lock -> Rpl_group_info::sleep_lock
Relay_log_info::sleep_cond -> Rpl_group_info::sleep_cond
sql/mysqld.h:
Updated types and names
sql/rpl_gtid.cc:
More DBUG
sql/rpl_parallel.cc:
Updated TODO section
Set thd for event that is execution
Use new is_begin(), is_commit() and is_rollback() functions.
More comments
sql/rpl_rli.cc:
sql_thd -> sql_driver_thd
Relay_log_info::sleep_lock -> rpl_group_info::sleep_lock
Relay_log_info::sleep_cond -> rpl_group_info::sleep_cond
Clear IN_STMT and IN_TRANSACTION in init_relay_log_pos() and Relay_log_info::cleanup_context() to ensure the flags doesn't survive slave restarts.
Reset table->in_use for temporary tables as the table may have been used by another THD.
Use IN_TRANSACTION instead of OPTION_BEGIN to check state of relay log.
Removed IN_STMT flag setting. This is now done in update_state_of_relay_log()
sql/rpl_rli.h:
Changed relay log state flags to bit masks instead of bit positions (most other code we have uses bit masks)
Added IN_TRANSACTION to mark if we are in a BEGIN ... COMMIT section.
save_temporary_tables is now thread safe
Relay_log_info::sleep_lock -> rpl_group_info::sleep_lock
Relay_log_info::sleep_cond -> rpl_group_info::sleep_cond
Relay_log_info->sql_thd renamed to Relay_log_info->sql_driver_thd to avoid wrong usage for merged code
is_in_group() is now independent of state of executed transaction.
sql/slave.cc:
Simplifed arguments to io_salve_killed(), sql_slave_killed() and check_io_slave_killed(); No reason to supply THD as this is part of the given structure.
set_thd_in_use_temporary_tables() removed as in_use is set on usage in sql_base.cc
sql_thd -> sql_driver_thd
More DBUG
Added update_state_of_relay_log() which will calculate the IN_STMT and IN_TRANSACTION state of the relay log after the current element is executed.
If slave_skip_counter is set run things in single threaded mode.
Simplifed arguments to io_salve_killed(), check_io_slave_killed() and sql_slave_killed(); No reason to supply THD as this is part of the given structure.
Added information to thd_proc_info() which thread is waiting for slave mutex to exit.
Disabled not used function rpl_connect_master()
Updated argument to next_event()
sql/sql_base.cc:
Added mutex around usage of slave's temporary tables. The active list is always kept up to date in sql->rgi_slave->save_temporary_tables.
Clear thd->temporary_tables after query (safety)
More DBUG
When using temporary table, set table->in_use to current thd as the THD may be different for slave threads.
Some code is ifdef:ed with REMOVE_AFTER_MERGE_WITH_10 as the given code in 10.0 is not yet in this tree.
In open_table() reuse code from find_temporary_table()
sql/sql_binlog.cc:
rli->sql_thd -> rli->sql_driver_thd
Remove duplicate setting of rgi->rli
sql/sql_class.cc:
Added helper functions rgi_lock_temporary_tables() and rgi_unlock_temporary_tables()
Would have been nicer to have these inline, but there was no easy way to do that
sql/sql_class.h:
Added functions to protect slaves temporary tables
sql/sql_parse.cc:
Added DBUG_PRINT
sql/transaction.cc:
Added comment
2013-10-13 23:24:05 +02:00
|
|
|
/*
|
|
|
|
do_event() is executed by the sql_driver_thd thread.
|
2013-10-14 23:17:16 +02:00
|
|
|
It's main purpose is to find a thread that can execute the query.
|
|
|
|
|
2014-03-04 08:48:32 +01:00
|
|
|
@retval 0 ok, event was accepted
|
|
|
|
@retval 1 error
|
|
|
|
@retval -1 event should be executed serially, in the sql driver thread
|
Fixes for parallel slave:
- Made slaves temporary table multi-thread slave safe by adding mutex around save_temporary_table usage.
- rli->save_temporary_tables is the active list of all used temporary tables
- This is copied to THD->temporary_tables when temporary tables are opened and updated when temporary tables are closed
- Added THD->lock_temporary_tables() and THD->unlock_temporary_tables() to simplify this.
- Relay_log_info->sql_thd renamed to Relay_log_info->sql_driver_thd to avoid wrong usage for merged code.
- Added is_part_of_group() to mark functions that are part of the next function. This replaces setting IN_STMT when events are executed.
- Added is_begin(), is_commit() and is_rollback() functions to Query_log_event to simplify code.
- If slave_skip_counter is set run things in single threaded mode. This simplifies code for skipping events.
- Updating state of relay log (IN_STMT and IN_TRANSACTION) is moved to one single function: update_state_of_relay_log()
We can't use OPTION_BEGIN to check for the state anymore as the sql_driver and sql execution threads may be different.
Clear IN_STMT and IN_TRANSACTION in init_relay_log_pos() and Relay_log_info::cleanup_context() to ensure the flags doesn't survive slave restarts
is_in_group() is now independent of state of executed transaction.
- Reset thd->transaction.all.modified_non_trans_table() if we did set it for single table row events.
This was mainly for keeping the flag as documented.
- Changed slave_open_temp_tables to uint32 to be able to use atomic operators on it.
- Relay_log_info::sleep_lock -> rpl_group_info::sleep_lock
- Relay_log_info::sleep_cond -> rpl_group_info::sleep_cond
- Changed some functions to take rpl_group_info instead of Relay_log_info to make them multi-slave safe and to simplify usage
- do_shall_skip()
- continue_group()
- sql_slave_killed()
- next_event()
- Simplifed arguments to io_salve_killed(), check_io_slave_killed() and sql_slave_killed(); No reason to supply THD as this is part of the given structure.
- set_thd_in_use_temporary_tables() removed as in_use is set on usage
- Added information to thd_proc_info() which thread is waiting for slave mutex to exit.
- In open_table() reuse code from find_temporary_table()
Other things:
- More DBUG statements
- Fixed the rpl_incident.test can be run with --debug
- More comments
- Disabled not used function rpl_connect_master()
mysql-test/suite/perfschema/r/all_instances.result:
Moved sleep_lock and sleep_cond to rpl_group_info
mysql-test/suite/rpl/r/rpl_incident.result:
Updated result
mysql-test/suite/rpl/t/rpl_incident-master.opt:
Not needed anymore
mysql-test/suite/rpl/t/rpl_incident.test:
Fixed that test can be run with --debug
sql/handler.cc:
More DBUG_PRINT
sql/log.cc:
More comments
sql/log_event.cc:
Added DBUG statements
do_shall_skip(), continue_group() now takes rpl_group_info param
Use is_begin(), is_commit() and is_rollback() functions instead of inspecting query string
We don't have set slaves temporary tables 'in_use' as this is now done when tables are opened.
Removed IN_STMT flag setting. This is now done in update_state_of_relay_log()
Use IN_TRANSACTION flag to test state of relay log.
In rows_event_stmt_cleanup() reset thd->transaction.all.modified_non_trans_table if we had set this before.
sql/log_event.h:
do_shall_skip(), continue_group() now takes rpl_group_info param
Added is_part_of_group() to mark events that are part of the next event. This replaces setting IN_STMT when events are executed.
Added is_begin(), is_commit() and is_rollback() functions to Query_log_event to simplify code.
sql/log_event_old.cc:
Removed IN_STMT flag setting. This is now done in update_state_of_relay_log()
do_shall_skip(), continue_group() now takes rpl_group_info param
sql/log_event_old.h:
Added is_part_of_group() to mark events that are part of the next event.
do_shall_skip(), continue_group() now takes rpl_group_info param
sql/mysqld.cc:
Changed slave_open_temp_tables to uint32 to be able to use atomic operators on it.
Relay_log_info::sleep_lock -> Rpl_group_info::sleep_lock
Relay_log_info::sleep_cond -> Rpl_group_info::sleep_cond
sql/mysqld.h:
Updated types and names
sql/rpl_gtid.cc:
More DBUG
sql/rpl_parallel.cc:
Updated TODO section
Set thd for event that is execution
Use new is_begin(), is_commit() and is_rollback() functions.
More comments
sql/rpl_rli.cc:
sql_thd -> sql_driver_thd
Relay_log_info::sleep_lock -> rpl_group_info::sleep_lock
Relay_log_info::sleep_cond -> rpl_group_info::sleep_cond
Clear IN_STMT and IN_TRANSACTION in init_relay_log_pos() and Relay_log_info::cleanup_context() to ensure the flags doesn't survive slave restarts.
Reset table->in_use for temporary tables as the table may have been used by another THD.
Use IN_TRANSACTION instead of OPTION_BEGIN to check state of relay log.
Removed IN_STMT flag setting. This is now done in update_state_of_relay_log()
sql/rpl_rli.h:
Changed relay log state flags to bit masks instead of bit positions (most other code we have uses bit masks)
Added IN_TRANSACTION to mark if we are in a BEGIN ... COMMIT section.
save_temporary_tables is now thread safe
Relay_log_info::sleep_lock -> rpl_group_info::sleep_lock
Relay_log_info::sleep_cond -> rpl_group_info::sleep_cond
Relay_log_info->sql_thd renamed to Relay_log_info->sql_driver_thd to avoid wrong usage for merged code
is_in_group() is now independent of state of executed transaction.
sql/slave.cc:
Simplifed arguments to io_salve_killed(), sql_slave_killed() and check_io_slave_killed(); No reason to supply THD as this is part of the given structure.
set_thd_in_use_temporary_tables() removed as in_use is set on usage in sql_base.cc
sql_thd -> sql_driver_thd
More DBUG
Added update_state_of_relay_log() which will calculate the IN_STMT and IN_TRANSACTION state of the relay log after the current element is executed.
If slave_skip_counter is set run things in single threaded mode.
Simplifed arguments to io_salve_killed(), check_io_slave_killed() and sql_slave_killed(); No reason to supply THD as this is part of the given structure.
Added information to thd_proc_info() which thread is waiting for slave mutex to exit.
Disabled not used function rpl_connect_master()
Updated argument to next_event()
sql/sql_base.cc:
Added mutex around usage of slave's temporary tables. The active list is always kept up to date in sql->rgi_slave->save_temporary_tables.
Clear thd->temporary_tables after query (safety)
More DBUG
When using temporary table, set table->in_use to current thd as the THD may be different for slave threads.
Some code is ifdef:ed with REMOVE_AFTER_MERGE_WITH_10 as the given code in 10.0 is not yet in this tree.
In open_table() reuse code from find_temporary_table()
sql/sql_binlog.cc:
rli->sql_thd -> rli->sql_driver_thd
Remove duplicate setting of rgi->rli
sql/sql_class.cc:
Added helper functions rgi_lock_temporary_tables() and rgi_unlock_temporary_tables()
Would have been nicer to have these inline, but there was no easy way to do that
sql/sql_class.h:
Added functions to protect slaves temporary tables
sql/sql_parse.cc:
Added DBUG_PRINT
sql/transaction.cc:
Added comment
2013-10-13 23:24:05 +02:00
|
|
|
*/
|
|
|
|
|
2014-03-04 08:48:32 +01:00
|
|
|
int
|
2013-10-24 12:44:21 +02:00
|
|
|
rpl_parallel::do_event(rpl_group_info *serial_rgi, Log_event *ev,
|
|
|
|
ulonglong event_size)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
|
|
|
rpl_parallel_entry *e;
|
|
|
|
rpl_parallel_thread *cur_thread;
|
|
|
|
rpl_parallel_thread::queued_event *qev;
|
2013-09-23 14:46:57 +02:00
|
|
|
rpl_group_info *rgi= NULL;
|
2013-07-03 13:46:33 +02:00
|
|
|
Relay_log_info *rli= serial_rgi->rli;
|
|
|
|
enum Log_event_type typ;
|
2013-10-08 14:36:06 +02:00
|
|
|
bool is_group_event;
|
2013-12-06 13:28:23 +01:00
|
|
|
bool did_enter_cond= false;
|
2014-02-03 15:22:39 +01:00
|
|
|
PSI_stage_info old_stage;
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2014-12-05 16:09:48 +01:00
|
|
|
DBUG_EXECUTE_IF("slave_crash_if_parallel_apply", DBUG_SUICIDE(););
|
2014-03-04 08:48:32 +01:00
|
|
|
/* Handle master log name change, seen in Rotate_log_event. */
|
|
|
|
typ= ev->get_type_code();
|
|
|
|
if (unlikely(typ == ROTATE_EVENT))
|
|
|
|
{
|
|
|
|
Rotate_log_event *rev= static_cast<Rotate_log_event *>(ev);
|
|
|
|
if ((rev->server_id != global_system_variables.server_id ||
|
|
|
|
rli->replicate_same_server_id) &&
|
|
|
|
!rev->is_relay_log_event() &&
|
|
|
|
!rli->is_in_group())
|
|
|
|
{
|
|
|
|
memcpy(rli->future_event_master_log_name,
|
|
|
|
rev->new_log_ident, rev->ident_len+1);
|
2015-09-11 10:51:56 +02:00
|
|
|
rli->notify_group_master_log_name_update();
|
2014-03-04 08:48:32 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/*
|
|
|
|
Execute queries non-parallel if slave_skip_counter is set, as it's is
|
|
|
|
easier to skip queries in single threaded mode.
|
|
|
|
*/
|
|
|
|
if (rli->slave_skip_counter)
|
|
|
|
return -1;
|
|
|
|
|
|
|
|
/* Execute pre-10.0 event, which have no GTID, in single-threaded mode. */
|
MDEV-6589: Incorrect relay log start position when restarting SQL thread after error in parallel replication
The problem occurs in parallel replication in GTID mode, when we are using
multiple replication domains. In this case, if the SQL thread stops, the
slave GTID position may refer to a different point in the relay log for each
domain.
The bug was that when the SQL thread was stopped and restarted (but the IO
thread was kept running), the SQL thread would resume applying the relay log
from the point of the most advanced replication domain, silently skipping all
earlier events within other domains. This caused replication corruption.
This patch solves the problem by storing, when the SQL thread stops with
multiple parallel replication domains active, the current GTID
position. Additionally, the current position in the relay logs is moved back
to a point known to be earlier than the current position of any replication
domain. Then when the SQL thread restarts from the earlier position, GTIDs
encountered are compared against the stored GTID position. Any GTID that was
already applied before the stop is skipped to avoid duplicate apply.
This patch should have no effect if multi-domain GTID parallel replication is
not used. Similarly, if both SQL and IO thread are stopped and restarted, the
patch has no effect, as in this case the existing relay logs are removed and
re-fetched from the master at the current global @@gtid_slave_pos.
2015-02-18 12:22:50 +01:00
|
|
|
is_group_event= Log_event::is_group_event(typ);
|
|
|
|
if (unlikely(!current) && typ != GTID_EVENT &&
|
|
|
|
!(unlikely(rli->gtid_skip_flag != GTID_SKIP_NOT) && is_group_event))
|
2014-03-04 08:48:32 +01:00
|
|
|
return -1;
|
|
|
|
|
2016-10-14 12:22:00 +02:00
|
|
|
/* Note: rli->data_lock is released by sql_delay_event(). */
|
|
|
|
if (sql_delay_event(ev, rli->sql_driver_thd, serial_rgi))
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
If sql_delay_event() returns non-zero, it means that the wait timed out
|
|
|
|
due to slave stop. We should not queue the event in this case, it must
|
|
|
|
not be applied yet.
|
|
|
|
*/
|
|
|
|
delete ev;
|
|
|
|
return 1;
|
|
|
|
}
|
2013-06-25 09:30:19 +02:00
|
|
|
|
MDEV-6589: Incorrect relay log start position when restarting SQL thread after error in parallel replication
The problem occurs in parallel replication in GTID mode, when we are using
multiple replication domains. In this case, if the SQL thread stops, the
slave GTID position may refer to a different point in the relay log for each
domain.
The bug was that when the SQL thread was stopped and restarted (but the IO
thread was kept running), the SQL thread would resume applying the relay log
from the point of the most advanced replication domain, silently skipping all
earlier events within other domains. This caused replication corruption.
This patch solves the problem by storing, when the SQL thread stops with
multiple parallel replication domains active, the current GTID
position. Additionally, the current position in the relay logs is moved back
to a point known to be earlier than the current position of any replication
domain. Then when the SQL thread restarts from the earlier position, GTIDs
encountered are compared against the stored GTID position. Any GTID that was
already applied before the stop is skipped to avoid duplicate apply.
This patch should have no effect if multi-domain GTID parallel replication is
not used. Similarly, if both SQL and IO thread are stopped and restarted, the
patch has no effect, as in this case the existing relay logs are removed and
re-fetched from the master at the current global @@gtid_slave_pos.
2015-02-18 12:22:50 +01:00
|
|
|
if (unlikely(typ == FORMAT_DESCRIPTION_EVENT))
|
2014-07-02 12:51:45 +02:00
|
|
|
{
|
|
|
|
Format_description_log_event *fdev=
|
|
|
|
static_cast<Format_description_log_event *>(ev);
|
|
|
|
if (fdev->created)
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
This format description event marks a new binlog after a master server
|
|
|
|
restart. We are going to close all temporary tables to clean up any
|
|
|
|
possible left-overs after a prior master crash.
|
|
|
|
|
|
|
|
Thus we need to wait for all prior events to execute to completion,
|
|
|
|
in case they need access to any of the temporary tables.
|
2014-08-19 14:26:42 +02:00
|
|
|
|
|
|
|
We also need to notify the worker thread running the prior incomplete
|
|
|
|
event group (if any), as such event group signifies an incompletely
|
|
|
|
written group cut short by a master crash, and must be rolled back.
|
2014-07-02 12:51:45 +02:00
|
|
|
*/
|
2014-08-20 10:59:39 +02:00
|
|
|
if (current->queue_master_restart(serial_rgi, fdev) ||
|
|
|
|
wait_for_workers_idle(rli->sql_driver_thd))
|
2014-08-19 14:26:42 +02:00
|
|
|
{
|
|
|
|
delete ev;
|
|
|
|
return 1;
|
|
|
|
}
|
2014-07-02 12:51:45 +02:00
|
|
|
}
|
|
|
|
}
|
MDEV-6589: Incorrect relay log start position when restarting SQL thread after error in parallel replication
The problem occurs in parallel replication in GTID mode, when we are using
multiple replication domains. In this case, if the SQL thread stops, the
slave GTID position may refer to a different point in the relay log for each
domain.
The bug was that when the SQL thread was stopped and restarted (but the IO
thread was kept running), the SQL thread would resume applying the relay log
from the point of the most advanced replication domain, silently skipping all
earlier events within other domains. This caused replication corruption.
This patch solves the problem by storing, when the SQL thread stops with
multiple parallel replication domains active, the current GTID
position. Additionally, the current position in the relay logs is moved back
to a point known to be earlier than the current position of any replication
domain. Then when the SQL thread restarts from the earlier position, GTIDs
encountered are compared against the stored GTID position. Any GTID that was
already applied before the stop is skipped to avoid duplicate apply.
This patch should have no effect if multi-domain GTID parallel replication is
not used. Similarly, if both SQL and IO thread are stopped and restarted, the
patch has no effect, as in this case the existing relay logs are removed and
re-fetched from the master at the current global @@gtid_slave_pos.
2015-02-18 12:22:50 +01:00
|
|
|
else if (unlikely(typ == GTID_LIST_EVENT))
|
|
|
|
{
|
|
|
|
Gtid_list_log_event *glev= static_cast<Gtid_list_log_event *>(ev);
|
|
|
|
rpl_gtid *list= glev->list;
|
|
|
|
uint32 count= glev->count;
|
|
|
|
rli->update_relay_log_state(list, count);
|
|
|
|
while (count)
|
|
|
|
{
|
|
|
|
process_gtid_for_restart_pos(rli, list);
|
|
|
|
++list;
|
|
|
|
--count;
|
|
|
|
}
|
|
|
|
}
|
2014-07-02 12:51:45 +02:00
|
|
|
|
2013-10-08 14:36:06 +02:00
|
|
|
/*
|
|
|
|
Stop queueing additional event groups once the SQL thread is requested to
|
|
|
|
stop.
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
|
|
|
We have to queue any remaining events of any event group that has already
|
|
|
|
been partially queued, but after that we will just ignore any further
|
|
|
|
events the SQL driver thread may try to queue, and eventually it will stop.
|
2013-10-08 14:36:06 +02:00
|
|
|
*/
|
2014-03-04 08:48:32 +01:00
|
|
|
if ((typ == GTID_EVENT || !is_group_event) && rli->abort_slave)
|
2013-10-08 14:36:06 +02:00
|
|
|
sql_thread_stopping= true;
|
|
|
|
if (sql_thread_stopping)
|
2013-10-14 23:17:16 +02:00
|
|
|
{
|
2014-01-03 12:20:53 +01:00
|
|
|
delete ev;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
/*
|
2014-03-04 08:48:32 +01:00
|
|
|
Return "no error"; normal stop is not an error, and otherwise the error
|
|
|
|
has already been recorded.
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
*/
|
2014-03-04 08:48:32 +01:00
|
|
|
return 0;
|
2013-10-14 23:17:16 +02:00
|
|
|
}
|
2013-10-08 14:36:06 +02:00
|
|
|
|
MDEV-6589: Incorrect relay log start position when restarting SQL thread after error in parallel replication
The problem occurs in parallel replication in GTID mode, when we are using
multiple replication domains. In this case, if the SQL thread stops, the
slave GTID position may refer to a different point in the relay log for each
domain.
The bug was that when the SQL thread was stopped and restarted (but the IO
thread was kept running), the SQL thread would resume applying the relay log
from the point of the most advanced replication domain, silently skipping all
earlier events within other domains. This caused replication corruption.
This patch solves the problem by storing, when the SQL thread stops with
multiple parallel replication domains active, the current GTID
position. Additionally, the current position in the relay logs is moved back
to a point known to be earlier than the current position of any replication
domain. Then when the SQL thread restarts from the earlier position, GTIDs
encountered are compared against the stored GTID position. Any GTID that was
already applied before the stop is skipped to avoid duplicate apply.
This patch should have no effect if multi-domain GTID parallel replication is
not used. Similarly, if both SQL and IO thread are stopped and restarted, the
patch has no effect, as in this case the existing relay logs are removed and
re-fetched from the master at the current global @@gtid_slave_pos.
2015-02-18 12:22:50 +01:00
|
|
|
if (unlikely(rli->gtid_skip_flag != GTID_SKIP_NOT) && is_group_event)
|
|
|
|
{
|
|
|
|
if (typ == GTID_EVENT)
|
|
|
|
rli->gtid_skip_flag= GTID_SKIP_NOT;
|
|
|
|
else
|
|
|
|
{
|
|
|
|
if (rli->gtid_skip_flag == GTID_SKIP_STANDALONE)
|
|
|
|
{
|
|
|
|
if (!Log_event::is_part_of_group(typ))
|
|
|
|
rli->gtid_skip_flag= GTID_SKIP_NOT;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
DBUG_ASSERT(rli->gtid_skip_flag == GTID_SKIP_TRANSACTION);
|
|
|
|
if (typ == XID_EVENT ||
|
2016-10-20 18:00:59 +02:00
|
|
|
(typ == QUERY_EVENT && // COMMIT/ROLLBACK are never compressed
|
MDEV-6589: Incorrect relay log start position when restarting SQL thread after error in parallel replication
The problem occurs in parallel replication in GTID mode, when we are using
multiple replication domains. In this case, if the SQL thread stops, the
slave GTID position may refer to a different point in the relay log for each
domain.
The bug was that when the SQL thread was stopped and restarted (but the IO
thread was kept running), the SQL thread would resume applying the relay log
from the point of the most advanced replication domain, silently skipping all
earlier events within other domains. This caused replication corruption.
This patch solves the problem by storing, when the SQL thread stops with
multiple parallel replication domains active, the current GTID
position. Additionally, the current position in the relay logs is moved back
to a point known to be earlier than the current position of any replication
domain. Then when the SQL thread restarts from the earlier position, GTIDs
encountered are compared against the stored GTID position. Any GTID that was
already applied before the stop is skipped to avoid duplicate apply.
This patch should have no effect if multi-domain GTID parallel replication is
not used. Similarly, if both SQL and IO thread are stopped and restarted, the
patch has no effect, as in this case the existing relay logs are removed and
re-fetched from the master at the current global @@gtid_slave_pos.
2015-02-18 12:22:50 +01:00
|
|
|
(((Query_log_event *)ev)->is_commit() ||
|
|
|
|
((Query_log_event *)ev)->is_rollback())))
|
|
|
|
rli->gtid_skip_flag= GTID_SKIP_NOT;
|
|
|
|
}
|
|
|
|
delete_or_keep_event_post_apply(serial_rgi, typ, ev);
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-03-04 08:48:32 +01:00
|
|
|
if (typ == GTID_EVENT)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
MDEV-6589: Incorrect relay log start position when restarting SQL thread after error in parallel replication
The problem occurs in parallel replication in GTID mode, when we are using
multiple replication domains. In this case, if the SQL thread stops, the
slave GTID position may refer to a different point in the relay log for each
domain.
The bug was that when the SQL thread was stopped and restarted (but the IO
thread was kept running), the SQL thread would resume applying the relay log
from the point of the most advanced replication domain, silently skipping all
earlier events within other domains. This caused replication corruption.
This patch solves the problem by storing, when the SQL thread stops with
multiple parallel replication domains active, the current GTID
position. Additionally, the current position in the relay logs is moved back
to a point known to be earlier than the current position of any replication
domain. Then when the SQL thread restarts from the earlier position, GTIDs
encountered are compared against the stored GTID position. Any GTID that was
already applied before the stop is skipped to avoid duplicate apply.
This patch should have no effect if multi-domain GTID parallel replication is
not used. Similarly, if both SQL and IO thread are stopped and restarted, the
patch has no effect, as in this case the existing relay logs are removed and
re-fetched from the master at the current global @@gtid_slave_pos.
2015-02-18 12:22:50 +01:00
|
|
|
rpl_gtid gtid;
|
2014-05-13 13:42:06 +02:00
|
|
|
Gtid_log_event *gtid_ev= static_cast<Gtid_log_event *>(ev);
|
2014-12-05 16:09:48 +01:00
|
|
|
uint32 domain_id= (rli->mi->using_gtid == Master_info::USE_GTID_NO ||
|
2015-02-06 10:02:02 +01:00
|
|
|
rli->mi->parallel_mode <= SLAVE_PARALLEL_MINIMAL ?
|
2014-05-13 13:42:06 +02:00
|
|
|
0 : gtid_ev->domain_id);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
if (!(e= find(domain_id)))
|
|
|
|
{
|
|
|
|
my_error(ER_OUT_OF_RESOURCES, MYF(MY_WME));
|
|
|
|
delete ev;
|
2014-03-04 08:48:32 +01:00
|
|
|
return 1;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
}
|
|
|
|
current= e;
|
MDEV-6589: Incorrect relay log start position when restarting SQL thread after error in parallel replication
The problem occurs in parallel replication in GTID mode, when we are using
multiple replication domains. In this case, if the SQL thread stops, the
slave GTID position may refer to a different point in the relay log for each
domain.
The bug was that when the SQL thread was stopped and restarted (but the IO
thread was kept running), the SQL thread would resume applying the relay log
from the point of the most advanced replication domain, silently skipping all
earlier events within other domains. This caused replication corruption.
This patch solves the problem by storing, when the SQL thread stops with
multiple parallel replication domains active, the current GTID
position. Additionally, the current position in the relay logs is moved back
to a point known to be earlier than the current position of any replication
domain. Then when the SQL thread restarts from the earlier position, GTIDs
encountered are compared against the stored GTID position. Any GTID that was
already applied before the stop is skipped to avoid duplicate apply.
This patch should have no effect if multi-domain GTID parallel replication is
not used. Similarly, if both SQL and IO thread are stopped and restarted, the
patch has no effect, as in this case the existing relay logs are removed and
re-fetched from the master at the current global @@gtid_slave_pos.
2015-02-18 12:22:50 +01:00
|
|
|
|
|
|
|
gtid.domain_id= gtid_ev->domain_id;
|
|
|
|
gtid.server_id= gtid_ev->server_id;
|
|
|
|
gtid.seq_no= gtid_ev->seq_no;
|
|
|
|
rli->update_relay_log_state(>id, 1);
|
|
|
|
if (process_gtid_for_restart_pos(rli, >id))
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
This domain has progressed further into the relay log before the last
|
|
|
|
SQL thread restart. So we need to skip this event group to not doubly
|
|
|
|
apply it.
|
|
|
|
*/
|
|
|
|
rli->gtid_skip_flag= ((gtid_ev->flags2 & Gtid_log_event::FL_STANDALONE) ?
|
|
|
|
GTID_SKIP_STANDALONE : GTID_SKIP_TRANSACTION);
|
|
|
|
delete_or_keep_event_post_apply(serial_rgi, typ, ev);
|
|
|
|
return 0;
|
|
|
|
}
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
}
|
|
|
|
else
|
|
|
|
e= current;
|
|
|
|
|
|
|
|
/*
|
|
|
|
Find a worker thread to queue the event for.
|
|
|
|
Prefer a new thread, so we maximise parallelism (at least for the group
|
|
|
|
commit). But do not exceed a limit of --slave-domain-parallel-threads;
|
|
|
|
instead re-use a thread that we queued for previously.
|
|
|
|
*/
|
|
|
|
cur_thread=
|
2014-06-25 15:17:03 +02:00
|
|
|
e->choose_thread(serial_rgi, &did_enter_cond, &old_stage,
|
|
|
|
typ != GTID_EVENT);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
if (!cur_thread)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
/* This means we were killed. The error is already signalled. */
|
|
|
|
delete ev;
|
2014-03-04 08:48:32 +01:00
|
|
|
return 1;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
if (!(qev= cur_thread->get_qev(ev, event_size, rli)))
|
|
|
|
{
|
|
|
|
abandon_worker_thread(rli->sql_driver_thd, cur_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
&did_enter_cond, &old_stage);
|
2014-01-03 12:20:53 +01:00
|
|
|
delete ev;
|
2014-03-04 08:48:32 +01:00
|
|
|
return 1;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
|
2013-10-08 14:36:06 +02:00
|
|
|
if (typ == GTID_EVENT)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
|
|
|
Gtid_log_event *gtid_ev= static_cast<Gtid_log_event *>(ev);
|
2014-12-05 16:09:48 +01:00
|
|
|
bool new_gco;
|
2015-02-06 10:02:02 +01:00
|
|
|
enum_slave_parallel_mode mode= rli->mi->parallel_mode;
|
2014-12-05 16:09:48 +01:00
|
|
|
uchar gtid_flags= gtid_ev->flags2;
|
|
|
|
group_commit_orderer *gco;
|
|
|
|
uint8 force_switch_flag;
|
|
|
|
enum rpl_group_info::enum_speculation speculation;
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2014-05-08 14:20:18 +02:00
|
|
|
if (!(rgi= cur_thread->get_rgi(rli, gtid_ev, e, event_size)))
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
cur_thread->free_qev(qev);
|
|
|
|
abandon_worker_thread(rli->sql_driver_thd, cur_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
&did_enter_cond, &old_stage);
|
2014-01-03 12:20:53 +01:00
|
|
|
delete ev;
|
2014-03-04 08:48:32 +01:00
|
|
|
return 1;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
/*
|
|
|
|
We queue the event group in a new worker thread, to run in parallel
|
|
|
|
with previous groups.
|
2013-06-24 10:50:25 +02:00
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
To preserve commit order within the replication domain, we set up
|
|
|
|
rgi->wait_commit_sub_id to make the new group commit only after the
|
|
|
|
previous group has committed.
|
2013-12-18 16:26:22 +01:00
|
|
|
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
Event groups that group-committed together on the master can be run
|
|
|
|
in parallel with each other without restrictions. But one batch of
|
|
|
|
group-commits may not start before all groups in the previous batch
|
|
|
|
have initiated their commit phase; we set up rgi->gco to ensure that.
|
|
|
|
*/
|
|
|
|
rgi->wait_commit_sub_id= e->current_sub_id;
|
|
|
|
rgi->wait_commit_group_info= e->current_group_info;
|
|
|
|
|
2014-12-05 16:09:48 +01:00
|
|
|
speculation= rpl_group_info::SPECULATE_NO;
|
|
|
|
new_gco= true;
|
|
|
|
force_switch_flag= 0;
|
|
|
|
gco= e->current_gco;
|
|
|
|
if (likely(gco))
|
|
|
|
{
|
|
|
|
uint8 flags= gco->flags;
|
|
|
|
|
2015-02-06 10:02:02 +01:00
|
|
|
if (mode <= SLAVE_PARALLEL_MINIMAL ||
|
|
|
|
!(gtid_flags & Gtid_log_event::FL_GROUP_COMMIT_ID) ||
|
2014-12-05 16:09:48 +01:00
|
|
|
e->last_commit_id != gtid_ev->commit_id)
|
|
|
|
flags|= group_commit_orderer::MULTI_BATCH;
|
|
|
|
/* Make sure we do not attempt to run DDL in parallel speculatively. */
|
|
|
|
if (gtid_flags & Gtid_log_event::FL_DDL)
|
|
|
|
flags|= (force_switch_flag= group_commit_orderer::FORCE_SWITCH);
|
|
|
|
|
|
|
|
if (!(flags & group_commit_orderer::MULTI_BATCH))
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
Still the same batch of event groups that group-committed together
|
|
|
|
on the master, so we can run in parallel.
|
|
|
|
*/
|
|
|
|
new_gco= false;
|
|
|
|
}
|
2015-02-06 10:02:02 +01:00
|
|
|
else if ((mode >= SLAVE_PARALLEL_OPTIMISTIC) &&
|
2014-12-05 16:09:48 +01:00
|
|
|
!(flags & group_commit_orderer::FORCE_SWITCH))
|
|
|
|
{
|
|
|
|
/*
|
|
|
|
In transactional parallel mode, we optimistically attempt to run
|
|
|
|
non-DDL in parallel. In case of conflicts, we catch the conflict as
|
|
|
|
a deadlock or other error, roll back and retry serially.
|
|
|
|
|
|
|
|
The assumption is that only a few event groups will be
|
|
|
|
non-transactional or otherwise unsuitable for parallel apply. Those
|
|
|
|
transactions are still scheduled in parallel, but we set a flag that
|
|
|
|
will make the worker thread wait for everything before to complete
|
|
|
|
before starting.
|
|
|
|
*/
|
|
|
|
new_gco= false;
|
|
|
|
if (!(gtid_flags & Gtid_log_event::FL_TRANSACTIONAL) ||
|
2015-02-06 10:02:02 +01:00
|
|
|
( (!(gtid_flags & Gtid_log_event::FL_ALLOW_PARALLEL) ||
|
|
|
|
(gtid_flags & Gtid_log_event::FL_WAITED)) &&
|
|
|
|
(mode < SLAVE_PARALLEL_AGGRESSIVE)))
|
2014-12-05 16:09:48 +01:00
|
|
|
{
|
|
|
|
/*
|
|
|
|
This transaction should not be speculatively run in parallel with
|
|
|
|
what came before, either because it cannot safely be rolled back in
|
|
|
|
case of a conflict, or because it was marked as likely to conflict
|
|
|
|
and require expensive rollback and retry.
|
|
|
|
|
|
|
|
Here we mark it as such, and then the worker thread will do a
|
|
|
|
wait_for_prior_commit() before starting it. We do not introduce a
|
|
|
|
new group_commit_orderer, since we still want following transactions
|
|
|
|
to run in parallel with transactions prior to this one.
|
|
|
|
*/
|
|
|
|
speculation= rpl_group_info::SPECULATE_WAIT;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
speculation= rpl_group_info::SPECULATE_OPTIMISTIC;
|
|
|
|
}
|
|
|
|
gco->flags= flags;
|
|
|
|
}
|
2015-05-11 11:55:58 +02:00
|
|
|
else
|
|
|
|
{
|
|
|
|
if (gtid_flags & Gtid_log_event::FL_DDL)
|
|
|
|
force_switch_flag= group_commit_orderer::FORCE_SWITCH;
|
|
|
|
}
|
2014-12-05 16:09:48 +01:00
|
|
|
rgi->speculation= speculation;
|
|
|
|
|
|
|
|
if (gtid_flags & Gtid_log_event::FL_GROUP_COMMIT_ID)
|
|
|
|
e->last_commit_id= gtid_ev->commit_id;
|
|
|
|
else
|
|
|
|
e->last_commit_id= 0;
|
|
|
|
|
|
|
|
if (new_gco)
|
2013-06-24 10:50:25 +02:00
|
|
|
{
|
2013-09-23 14:46:57 +02:00
|
|
|
/*
|
2014-12-05 16:09:48 +01:00
|
|
|
Do not run this event group in parallel with what came before; instead
|
|
|
|
wait for everything prior to at least have started its commit phase, to
|
|
|
|
avoid any risk of performing any conflicting action too early.
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
|
2014-12-05 16:09:48 +01:00
|
|
|
Remember the count that marks the end of the previous batch of event
|
|
|
|
groups that run in parallel, and allocate a new gco.
|
2013-09-23 14:46:57 +02:00
|
|
|
*/
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
uint64 count= e->count_queued_event_groups;
|
2013-07-08 16:47:07 +02:00
|
|
|
|
2015-01-31 21:48:47 +01:00
|
|
|
if (!(gco= cur_thread->get_gco(count, gco, e->current_sub_id)))
|
2013-07-08 16:47:07 +02:00
|
|
|
{
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
cur_thread->free_rgi(rgi);
|
|
|
|
cur_thread->free_qev(qev);
|
|
|
|
abandon_worker_thread(rli->sql_driver_thd, cur_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
&did_enter_cond, &old_stage);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
delete ev;
|
2014-03-04 08:48:32 +01:00
|
|
|
return 1;
|
2013-07-08 16:47:07 +02:00
|
|
|
}
|
2014-12-05 16:09:48 +01:00
|
|
|
gco->flags|= force_switch_flag;
|
|
|
|
e->current_gco= gco;
|
2013-07-08 16:47:07 +02:00
|
|
|
}
|
2014-12-05 16:09:48 +01:00
|
|
|
rgi->gco= gco;
|
|
|
|
|
2013-10-31 14:11:41 +01:00
|
|
|
qev->rgi= e->current_group_info= rgi;
|
2013-07-03 13:46:33 +02:00
|
|
|
e->current_sub_id= rgi->gtid_sub_id;
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
++e->count_queued_event_groups;
|
2013-07-03 13:46:33 +02:00
|
|
|
}
|
2014-03-04 08:48:32 +01:00
|
|
|
else if (!is_group_event)
|
2013-07-03 13:46:33 +02:00
|
|
|
{
|
2013-10-28 13:24:56 +01:00
|
|
|
int err;
|
2013-10-31 14:11:41 +01:00
|
|
|
bool tmp;
|
2013-07-03 13:46:33 +02:00
|
|
|
/*
|
|
|
|
Events like ROTATE and FORMAT_DESCRIPTION. Do not run in worker thread.
|
|
|
|
Same for events not preceeded by GTID (we should not see those normally,
|
|
|
|
but they might be from an old master).
|
|
|
|
*/
|
|
|
|
qev->rgi= serial_rgi;
|
2013-10-23 15:03:03 +02:00
|
|
|
|
2013-10-31 14:11:41 +01:00
|
|
|
tmp= serial_rgi->is_parallel_exec;
|
|
|
|
serial_rgi->is_parallel_exec= true;
|
2013-10-28 13:24:56 +01:00
|
|
|
err= rpt_handle_event(qev, NULL);
|
2013-10-31 14:11:41 +01:00
|
|
|
serial_rgi->is_parallel_exec= tmp;
|
2014-04-09 14:42:46 +02:00
|
|
|
if (ev->is_relay_log_event())
|
|
|
|
qev->future_event_master_log_pos= 0;
|
|
|
|
else if (typ == ROTATE_EVENT)
|
|
|
|
qev->future_event_master_log_pos=
|
|
|
|
(static_cast<Rotate_log_event *>(ev))->pos;
|
|
|
|
else
|
|
|
|
qev->future_event_master_log_pos= ev->log_pos;
|
2014-03-04 08:48:32 +01:00
|
|
|
delete_or_keep_event_post_apply(serial_rgi, typ, ev);
|
2013-07-04 13:17:01 +02:00
|
|
|
|
2013-10-31 14:11:41 +01:00
|
|
|
if (err)
|
|
|
|
{
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
cur_thread->free_qev(qev);
|
|
|
|
abandon_worker_thread(rli->sql_driver_thd, cur_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
&did_enter_cond, &old_stage);
|
2014-03-04 08:48:32 +01:00
|
|
|
return 1;
|
2013-10-31 14:11:41 +01:00
|
|
|
}
|
|
|
|
/*
|
MDEV-6551: Some replication errors are ignored if slave_parallel_threads > 0
The problem occured when using parallel replication, and an error occured that
caused the SQL thread to stop when the IO thread had already reached a
following binlog file from the master (or otherwise performed a relay log
rotation).
In this case, the Rotate Event at the end of the relay log file could still be
executed, even though an earlier event in that relay log file had gotten an
error. This would cause the position to be incorrectly updated, so that upon
restart of the SQL thread, the event that had failed would be silently skipped
and ignored, causing replication corruption.
Fixed by checking before executing Rotate Event, whether an earlier event
has failed. If so, the Rotate Event is not executed, just dequeued, same as
for other normal events following a failing event.
2014-08-15 11:31:13 +02:00
|
|
|
Queue a position update, so that the position will be updated in a
|
2013-10-31 14:11:41 +01:00
|
|
|
reasonable way relative to other events:
|
|
|
|
|
|
|
|
- If the currently executing events are queued serially for a single
|
|
|
|
thread, the position will only be updated when everything before has
|
|
|
|
completed.
|
|
|
|
|
|
|
|
- If we are executing multiple independent events in parallel, then at
|
|
|
|
least the position will not be updated until one of them has reached
|
|
|
|
the current point.
|
|
|
|
*/
|
MDEV-6551: Some replication errors are ignored if slave_parallel_threads > 0
The problem occured when using parallel replication, and an error occured that
caused the SQL thread to stop when the IO thread had already reached a
following binlog file from the master (or otherwise performed a relay log
rotation).
In this case, the Rotate Event at the end of the relay log file could still be
executed, even though an earlier event in that relay log file had gotten an
error. This would cause the position to be incorrectly updated, so that upon
restart of the SQL thread, the event that had failed would be silently skipped
and ignored, causing replication corruption.
Fixed by checking before executing Rotate Event, whether an earlier event
has failed. If so, the Rotate Event is not executed, just dequeued, same as
for other normal events following a failing event.
2014-08-15 11:31:13 +02:00
|
|
|
qev->typ= rpl_parallel_thread::queued_event::QUEUED_POS_UPDATE;
|
|
|
|
qev->entry_for_queued= e;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
qev->rgi= e->current_group_info;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|
2013-07-03 13:46:33 +02:00
|
|
|
|
2013-06-24 10:50:25 +02:00
|
|
|
/*
|
|
|
|
Queue the event for processing.
|
|
|
|
*/
|
2014-05-15 15:52:08 +02:00
|
|
|
qev->ir= rli->last_inuse_relaylog;
|
|
|
|
++qev->ir->queued_count;
|
2013-10-24 12:44:21 +02:00
|
|
|
cur_thread->enqueue(qev);
|
MDEV-5657: Parallel replication.
Clean up and improve the parallel implementation code, mainly related to
scheduling of work to threads and handling of stop and errors.
Fix a lot of bugs in various corner cases that could lead to crashes or
corruption.
Fix that a single replication domain could easily grab all worker threads and
stall all other domains; now a configuration variable
--slave-domain-parallel-threads allows to limit the number of
workers.
Allow next event group to start as soon as previous group begins the commit
phase (as opposed to when it ends it); this allows multiple event groups on
the slave to participate in group commit, even when no other opportunities for
parallelism are available.
Various fixes:
- Fix some races in the rpl.rpl_parallel test case.
- Fix an old incorrect assertion in Log_event iocache read.
- Fix repeated malloc/free of wait_for_commit and rpl_group_info objects.
- Simplify wait_for_commit wakeup logic.
- Fix one case in queue_for_group_commit() where killing one thread would
fail to correctly signal the error to the next, causing loss of the
transaction after slave restart.
- Fix leaking of pthreads (and their allocated stack) due to missing
PTHREAD_CREATE_DETACHED attribute.
- Fix how one batch of group-committed transactions wait for the previous
batch before starting to execute themselves. The old code had a very
complex scheduling where the first transaction was handled differently,
with subtle bugs in corner cases. Now each event group is always scheduled
for a new worker (in a round-robin fashion amongst available workers).
Keep a count of how many transactions have started to commit, and wait for
that counter to reach the appropriate value.
- Fix slave stop to wait for all workers to actually complete processing;
before, the wait was for update of last_committed_sub_id, which happens a
bit earlier, and could leave worker threads potentially accessing bits of
the replication state that is no longer valid after slave stop.
- Fix a couple of places where the test suite would kill a thread waiting
inside enter_cond() in connection with debug_sync; debug_sync + kill can
crash in rare cases due to a race with mysys_var_current_mutex in this
case.
- Fix some corner cases where we had enter_cond() but no exit_cond().
- Fix that we could get failure in wait_for_prior_commit() but forget to flag
the error with my_error().
- Fix slave stop (both for normal stop and stop due to error). Now, at stop
we pick a specific safe point (in terms of event groups executed) and make
sure that all event groups before that point are executed to completion,
and that no event group after start executing; this ensures a safe place to
restart replication, even for non-transactional stuff/DDL. In error stop,
make sure that all prior event groups are allowed to execute to completion,
and that any later event groups that have started are rolled back, if
possible. The old code could leave eg. T1 and T3 committed but T2 not, or
it could even leave half a transaction not rolled back in some random
worker, which would cause big problems when that worker was later reused
after slave restart.
- Fix the accounting of amount of events queued for one worker. Before, the
amount was reduced immediately as soon as the events were dequeued (which
happens all at once); this allowed twice the amount of events to be queued
in memory for each single worker, which is not what users would expect.
- Fix that an error set during execution of one event was sometimes not
cleared before executing the next, causing problems with the error
reporting.
- Fix incorrect handling of thd->killed in worker threads.
2014-02-26 15:02:09 +01:00
|
|
|
unlock_or_exit_cond(rli->sql_driver_thd, &cur_thread->LOCK_rpl_thread,
|
2014-02-26 16:38:42 +01:00
|
|
|
&did_enter_cond, &old_stage);
|
2013-09-23 14:46:57 +02:00
|
|
|
mysql_cond_signal(&cur_thread->COND_rpl_thread);
|
2013-06-24 10:50:25 +02:00
|
|
|
|
2014-03-04 08:48:32 +01:00
|
|
|
return 0;
|
2013-06-24 10:50:25 +02:00
|
|
|
}
|