Commit graph

58 commits

Author SHA1 Message Date
unknown
eb2302ec78 MDEV-5825: Assertion `! is_set() || m_can_overwrite_status' fails in Diagnostics_area::set_error_status on executing rpl.rpl_parallel test
In parallel replication, there was an error case where we could call
my_error() in-between events. This causes the assertion, as the previous event
has reported ok status, but the following event has not yet reset the
diagnostics area. This happened when a worker thread detects that the SQL
driver thread is aborting, and when it gets an error from a prior commit at
the same time in wait_for_prior_commit().

Since this is already an error case, the code should be using
unregister_wait_for_prior_commit() instead of wait_for_prior_commit(). But
unregister is already done a bit later (from finish_event_group()), so just
removing the redundant call to wait_for_prior_commit() fixes the issue.
2014-03-24 11:22:16 +01:00
unknown
b352969118 MDEV-5914: Parallel replication deadlock due to InnoDB lock conflicts
Due to how gap locks work, two transactions could group commit together on the
master, but get lock conflicts and then deadlock due to different thread
scheduling order on slave.

For now, remove these deadlocks by running the parallel slave in READ
COMMITTED mode. And let InnoDB/XtraDB allow statement-based binlogging for the
parallel slave in READ COMMITTED.

We are also investigating a different solution long-term, which is based on
relaxing the gap locks only between the transactions running in parallel for
one slave, but not against possibly external transactions.
2014-03-21 13:30:55 +01:00
unknown
a5418c5540 MDEV-5921: In parallel replication, an error is not correctly signalled to the next transaction
When a transaction fails in parallel replication, it should signal the error
to any following transactions doing wait_for_prior_commit() on it. But the
code for this was incorrect, and would not correctly remember a prior error
when sending the signal. This caused corruption when slave stopped due to an
error.

Fix by remembering the error code when we first get an error, and passing the
saved error code to wakeup_subsequent_commits().

Thanks to nanyi607rao who reported this bug on
maria-developers@lists.launchpad.net and analysed the root cause.
2014-03-21 10:11:28 +01:00
unknown
8b9b7ec395 MDEV-5804: If same GTID is received on multiple master connections in multi-source replication, the event is double-executed causing corruption or replication failure
Some fixes, mainly to make it work in non-parallel replication mode also
(--slave-parallel-threads=0).

Patch should be fairly complete now.
2014-03-12 00:14:49 +01:00
unknown
2c2478b822 MDEV-5804: If same GTID is received on multiple master connections in multi-source replication, the event is double-executed causing corruption or replication failure
Before, the arrival of same GTID twice in multi-source replication
would cause double-apply or in gtid strict mode an error.

Keep the behaviour, but add an option --gtid-ignore-duplicates which
allows to correctly handle duplicates, ignoring all but the first.
This relies on the user ensuring correct configuration so that
sequence numbers are strictly increasing within each replication
domain; then duplicates can be detected simply by comparing the
sequence numbers against what is already applied.

Only one master connection (but possibly multiple parallel worker
threads within that connection) is allowed to apply events within
one replication domain at a time; any other connection that
receives a GTID in the same domain either discards it (if it is
already applied) or waits for the other connection to not have
any events to apply.

Intermediate patch, as proof-of-concept for testing. The main limitation
is that currently it is only implemented for parallel replication,
@@slave_parallel_threads > 0.
2014-03-09 10:27:38 +01:00
unknown
fd25574645 MDEV-5789: race between rpl_parallel_change_thread_count and slave start upon server start without --skip-slave-start
Make sure to signal the condition variable for the thread pool after
the new threads have been added to the pool.

Thanks to user nanyi607rao, who reported this bug on maria-developers@.
2014-03-07 12:08:38 +01:00
unknown
2aa619ec68 MDEV-5788: Incorrect free of rgi->deferred_events in parallel replication
When an rpl_group_info object was returned from the free list, the
rgi->deferred_events_collecting and rgi->deferred_events was not correctly
re-inited. Additionally, the rgi->deferred_events was incorrectly freed in
free_rgi(), which causes unnecessary malloc/free (or crash when re-init is not
done).

Thanks to user nanyi607rao, who reported this bug on maria-developers@.
2014-03-07 12:02:09 +01:00
unknown
5ec49e6452 Merge MDEV-5754, MDEV-5769, and MDEV-5764 into 10.0 2014-03-04 14:32:42 +01:00
unknown
ec374f1e53 MDEV-5769: Slave crashes on attempt to do parallel replication from an older master
Older master has no GTID events, so such events are not available for
deciding on scheduling of event groups and so on.

With this patch, we run such events from old masters single-threaded, in the
sql driver thread.

This seems better than trying to make the parallel code handle the data from
older masters; while possible, this would require a lot of testing (as well as
possibly some extra overhead in the scheduling of events), which hardly seems
worthwhile.
2014-03-04 08:48:32 +01:00
unknown
641feed481 MDEV-5764: START SLAVE UNTIL does not work with parallel replication
With parallel replication, there can be any number of events queued on
in-memory lists in the worker threads.

For normal STOP SLAVE, we want to skip executing any remaining events on those
lists and stop as quickly as possible.

However, for START SLAVE UNTIL, when the UNTIL position is reached in the SQL
driver thread, we must _not_ stop until all already queued events for the
workers have been executed - otherwise we would stop too early, before the
actual UNTIL position had been completely reached.

The code did not handle UNTIL correctly, stopping too early due to not
executing the queued events to completion. Fix this, and also implement that
an explicit STOP SLAVE in the middle (when the SQL driver thread has reached
the UNTIL position but the workers have not) _will_ cause an immediate stop.
2014-03-03 12:13:55 +01:00
unknown
20959fa09c Merge MDEV-5657 (parallel replication) to 10.0 2014-02-26 16:38:42 +01:00
unknown
e90f68c0ba 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
unknown
dd93ec5633 Merge MariaDB 10.0-base to 10.0. 2014-02-10 15:12:17 +01:00
Sergei Golubchik
72c20282db 10.0-base merge 2014-02-03 15:22:39 +01:00
unknown
8cc6e90d74 MDEV-5509: Seconds_behind_master incorrect in parallel replication
The problem was a race between the SQL driver thread and the worker threads.
The SQL driver thread would set rli->last_master_timestamp to zero to
mark that it has caught up with the master, while the worker threads would
set it to the timestamp of the executed event. This can happen out-of-order
in parallel replication, causing the "caught up" status to be overwritten
and Seconds_Behind_Master to wrongly grow when the slave is idle.

To fix, introduce a separate flag rli->sql_thread_caught_up to mark that the
SQL driver thread is caught up. This avoids issues with worker threads
overwriting the SQL driver thread status. In parallel replication, we then
make SHOW SLAVE STATUS check in addition that all worker threads are idle
before showing Seconds_Behind_Master as 0 due to slave idle.
2014-01-08 11:00:44 +01:00
unknown
c4e76b20a4 MDEV-5363: Make parallel replication waits killable
Add another test case. This one for killing the SQL driver thread while it is
waiting for room in the list of events queued for a worker thread.

Fix bugs found:

 - Several memory leaks in various error cases.

 - SQL error code was not set (for SHOW SLAVE STATUS etc.) when killed.
2014-01-03 12:20:53 +01:00
unknown
86a2c03b51 MDEV-5363: Make parallel replication waits killable
Add another test case. This one for killing a worker while its transaction is
waiting to start until the previous transaction has committed.

Fix setting reading_or_writing to 0 in worker threads so SHOW SLAVE STATUS can
show something more useful than "Reading from net".
2013-12-18 16:26:22 +01:00
unknown
dbfe5f4774 MDEV-5363: Make parallel replication waits killable
Add a test case for killing a waiting query in parallel replication.

Fix several bugs found:

 - We should not wakeup_subsequent_commits() in ha_rollback_trans(), since we
   do not know the right wakeup_error() to give.

 - When a wait_for_prior_commit() is killed, we must unregister from the
   waitee so we do not race and get an extra (non-kill) wakeup.

 - We need to deal with error propagation correctly in queue_for_group_commit
   when one thread is killed.

 - Fix one locking issue in queue_for_group_commit(), we could unlock the
   waitee lock too early and this end up processing wakeup() with insufficient
   locking.

 - Fix Xid_log_event::do_apply_event; if commit fails it must not update the
   in-memory @@gtid_slave_pos state.

 - Fix and cleanup some things in the rpl_parallel.cc error handling.

 - Add a missing check for killed in the slave sql driver thread, to avoid a
   race.
2013-12-13 14:26:51 +01:00
unknown
b7ae65ef86 MDEV-5363: Make parallel replication waits killable
A couple of more parallel replication waits made killable.
2013-12-06 13:28:23 +01:00
unknown
4d6ee2d119 MDEV-5363: Make parallel replication waits killable
Make wait_for_prior_commit killable, and handle the error if
killed.
2013-12-05 14:36:09 +01:00
unknown
55de9b0468 merge 10-base->10.0 2013-11-11 23:40:53 +02:00
unknown
2ea0e59937 MDEV-4506: Parallel replication
Tested manually that crash in the middle of writing transaction on the master
does correctly cause a rollback on slave, so remove the corresponding ToDo.
2013-11-08 11:41:13 +01:00
unknown
273bcb92c1 Merge 10.0-base to 10.0 2013-11-07 07:52:40 +01:00
unknown
dcb3650d63 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
unknown
bdbf90b969 MDEV-4506: Parallel replication
MDEV-5217: Incorrect event pos update leading to corruption of reading of events from relay log

The rli->event_relay_log_pos was sometimes undated incorrectly when using
parallel replication, especially around relay log rotates. This could cause
the SQL thread to seek into an invalid position in the relay log, resulting in
errors about invalid events or even random corruption in some cases.
2013-11-06 10:18:04 +01:00
unknown
c834242ad4 MDEV-4506: Parallel replication
MDEV-5217: SQL thread hangs during stop if error occurs in the middle of an event group

Normally, when we stop the slave SQL thread in parallel replication, we want
the worker threads to continue processing events until the end of the current
event group. But if we stop due to an error that prevents further events from
being queued, such as an error reading the relay log, no more events can be
queued for the workers, so they have to abort even if they are in the middle
of an event group. There was a bug that we would deadlock, the workers
waiting for more events to be queued for the event group, the SQL thread
stopped and waiting for the workers to complete their current event group
before exiting.

Fixed by now signalling from the SQL thread to all workers when it is about
to exit, and cleaning up in all workers when so signalled.

This patch fixes one of multiple problems reported in MDEV-5217.
2013-11-05 12:01:26 +01:00
unknown
57a267a8c0 Merge from 10.0-base to 10.0 the feature MDEV-4506: Parallel replication.
The merge is still missing a few hunks related to temporary tables and
InnoDB log file size. The associated code did not seem to exist in
10.0, so the merge of that needs more work. Until this is fixed, there
are a number of test failures as a result.
2013-11-01 12:00:11 +01:00
unknown
39df665a33 MDEV-5206: Incorrect slave old-style position in MDEV-4506, parallel replication.
In parallel replication, there are two kinds of events which are
executed in different ways.

Normal events that are part of event groups/transactions are executed
asynchroneously by being queued for a worker thread.

Other events like format description and rotate and such are executed
directly in the driver SQL thread.

If the direct execution of the other events were to update the old-style
position, then the position gets updated too far ahead, before the normal
events that have been queued for a worker thread have been executed. So
this patch adds some special cases to prevent such position updates ahead
of time, and instead queues dummy events for the worker threads, so that
they will at an appropriate time do the position updates instead.

(Also fix a race in a test case that happened to trigger while running
tests for this patch).
2013-10-31 14:11:41 +01:00
unknown
9c8da4ed76 MDEV-5196: Server hangs or assertion `!thd->wait_for_commit_ptr' fails on MASTER_POS_WAIT with slave-parallel-threads > 0
Fix a couple of issues in MDEV-4506, Parallel replication:

 - Missing mysql_cond_signal(), which could cause hangs.

 - Fix incorrect update of old-style replication position.

 - Change assertion to error handling (can trigger on manipulated/
   corrupt binlog).
2013-10-30 07:52:30 +01:00
unknown
f2799c6828 MDEV-5195: Race when switching relay log causing crash
In parallel replication, when the IO thread switches relay log,
the SQL thread re-opens the current relaylog and seeks to the
current position. There was a race that would cause it to
sometimes seek to the wrong position, causing corruption and
crash.
2013-10-29 11:52:16 +01:00
unknown
2fbd1c7307 MDEV-4506: Parallel replication.
MDEV-5189: Error handling in parallel replication.

Fix error handling in parallel worker threads when a query fails:

 - Report the error to the error log.

 - Return the error back, and set rli->abort_slave.

 - Stop executing more events after the error.
2013-10-28 13:24:56 +01:00
unknown
6a38b59475 MDEV-5189: Incorrect parallel apply in parallel replication
Two problems were fixed:

1. When not in GTID mode (master_use_gtid=no), then we must not apply events
   in different domains in parallel (in non-GTID mode we are not capable of
   restarting at different points in different domains).

2. When transactions B and C group commit together, but after and separate
   from A, we can apply B and C in parallel, but both B and C must not start
   until A has committed. Fix sub_id to be globally increasing (not just
   per-domain increasing) so that this wait (which is based on sub_id) can be
   done correctly.
2013-10-25 21:17:14 +02:00
unknown
80d0dd7bab MDEV-4506: Parallel replication.
Do not update relay-log.info and master.info on disk after every event
when using GTID mode:

 - relay-log.info and master.info are not crash-safe, and are not used
   when slave restarts in GTID mode (slave connects with GTID position
   instead and immediately rewrites the file with the new, correct
   information found).

 - When using GTID and parallel replication, the position in
   relay-log.info is misleading at best and simply wrong at worst.

 - When using parallel replication, the fact that every single
   transaction needs to do a write() syscall to the same file is
   likely to become a serious bottleneck.

The files are still written at normal slave stop.

In non-GTID mode, the files are written as normal (this is needed to
be able to restart after slave crash, even if such restart is then not
crash-safe, no change).
2013-10-25 12:56:12 +02:00
unknown
ee8a816208 MDEV-4506: Parallel replication.
Implement --slave-parallel-max-queue to limit memory usage
of SQL thread read-ahead in the relay log.
2013-10-24 12:44:21 +02:00
unknown
96a4f1f628 MDEV-4506: Parallel replication: Update some comments. 2013-10-24 08:53:48 +02:00
unknown
a09d2b105f MDEV-4506: Parallel replication.
Fix some more parts of old-style position updates.
Now we save in rgi some coordinates for master log and relay log, so
that in do_update_pos() we can use the right set of coordinates with
the right events.

The Rotate_log_event::do_update_pos() is fixed in the parallel case
to not directly update relay-log.info (as Rotate event runs directly
in the driver SQL thread, ahead of actual event execution). Instead,
group_master_log_file is updated as part of do_update_pos() in each
event execution.

In the parallel case, position updates happen in parallel without
any ordering, but taking care that position is not updated backwards.
Since position update happens only after event execution this leads
to the right result.

Also fix an access-after-free introduced in an earlier commit.
2013-10-23 15:03:03 +02:00
unknown
7681c6aa78 MDEV-4506: Parallel replication: Intermediate commit.
Fix some part of update of old-style coordinates in parallel replication:

 - Ignore XtraDB request for old-style coordinates, not meaningful for
   parallel replication (must use GTID to get crash-safe parallel slave).

 - Only update relay log coordinates forward, not backwards, to ensure
   that parallel threads do not conflict with each other.

 - Move future_event_relay_log_pos to rgi.
2013-10-17 14:11:19 +02:00
Michael Widenius
5748eb3ec6 Moved the remaining variables, that depends on sql execution, from Relay_log_info to rpl_group_info:
-row_stmt_start_timestamp
-last_event_start_time
-long_find_row_note
-trans_retries

Added slave_executed_entries_lock to protect rli->executed_entries
Added primitives for thread safe 64 bit increment
Update rli->executed_entries when event has executed, not when event has been sent to sql execution thread


sql/log_event.cc:
  row_stmt_start and long_find_row_note is now in rpl_group_info
sql/mysqld.cc:
  Added slave_executed_entries_lock to protect rli->executed_entries
sql/mysqld.h:
  Added slave_executed_entries_lock to protect rli->executed_entries
  Added primitives for thread safe 64 bit increment
sql/rpl_parallel.cc:
  Update rli->executed_entries when event has executed, not when event has been sent to sql execution thread
sql/rpl_rli.cc:
  Moved row_stmt_start_timestamp, last_event_start_time and long_find_row_note from Relay_log_info to rpl_group_info
sql/rpl_rli.h:
  Moved trans_retries, row_stmt_start_timestamp, last_event_start_time and long_find_row_note from Relay_log_info to rpl_group_info
sql/slave.cc:
  Use rgi for trans_retries and last_event_start_time
  Update rli->executed_entries when event has executed, not when event has been sent to sql execution thread
  Reset trans_retries when object is created
2013-10-15 00:17:16 +03:00
unknown
2842f6b5dc MDEV-4506: Parallel replication: error handling.
Add an error code to the wait_for_commit facility.

Now, when a transaction fails, it can signal the error to
any subsequent transaction that is waiting for it to commit.
The waiting transactions then receive the error code back from
wait_for_prior_commit() and can handle the error appropriately.

Also fix one race that could cause crash if @@slave_parallel_threads
were changed several times quickly in succession.
2013-10-14 15:28:16 +02:00
Michael Widenius
2e100cc5a4 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-14 00:24:05 +03:00
unknown
12c760ef71 MDEV-4506: Parallel replication.
Improve STOP SLAVE in parallel mode.

Now, the parallel part will queue the current event group to the
end, and then stop queing any more events. Each worker will
complete the current event group, and then just skip any further
queued events.
2013-10-08 14:36:06 +02:00
unknown
45c3c71513 MDEV-4506: Parallel replication. .result file updates + a few comment updates. 2013-09-30 10:41:41 +02:00
unknown
976606d031 MDEV-4506: Parallel replication: After-review fixes. 2013-09-23 14:46:57 +02:00
unknown
39794dc72c MDEV-4506: Parallel replication.
Add another test case, using DEBUG_SYNC.
Fix one bug found.
2013-09-17 14:07:21 +02:00
unknown
5633dd8227 MDEV-4506: parallel replication.
Add a simple test case.
Fix bugs found.
2013-09-16 14:33:49 +02:00
unknown
d107bdaa01 MDEV-4506, parallel replication.
Some after-review fixes.
2013-09-13 15:09:57 +02:00
unknown
47f8e0ef6e MDEV-4506: Parallel replication: Intermediate commit
Remove Relay_log_info::group_info. (It is not thread safe).
2013-07-12 14:42:48 +02:00
unknown
ba4b937af2 MDEV-4506: Parallel replication: Intermediate commit
Move the deferred event stuff from Relay_log_info to rpl_group_info
to make it thread safe for parallel replication.
2013-07-12 14:36:20 +02:00
unknown
6d5f237e09 MDEV-4506: Parallel replication: Intermediate commit.
Fix a number of failures in the test suite.
2013-07-09 13:15:53 +02:00
unknown
a99356fbe7 MDEV-4506: Parallel replication: intermediate commit.
Fix a bunch of issues found with locking, ordering, and non-thread-safe stuff
in Relay_log_info.

Now able to do a simple benchmark, showing 4.5 times speedup for applying a
binlog with 10000 REPLACE statements.
2013-07-08 16:47:07 +02:00