summaryrefslogtreecommitdiff
path: root/sql/slave.cc
diff options
context:
space:
mode:
Diffstat (limited to 'sql/slave.cc')
-rw-r--r--sql/slave.cc137
1 files changed, 105 insertions, 32 deletions
diff --git a/sql/slave.cc b/sql/slave.cc
index cb093560456..83f138b78f0 100644
--- a/sql/slave.cc
+++ b/sql/slave.cc
@@ -788,7 +788,7 @@ bool init_slave_skip_errors(const char* arg)
if (!arg || !*arg) // No errors defined
goto end;
- if (unlikely(my_bitmap_init(&slave_error_mask,0,MAX_SLAVE_ERROR,0)))
+ if (my_bitmap_init(&slave_error_mask,0,MAX_SLAVE_ERROR))
DBUG_RETURN(1);
use_slave_mask= 1;
@@ -2329,11 +2329,11 @@ past_checksum:
/* Announce MariaDB slave capabilities. */
DBUG_EXECUTE_IF("simulate_slave_capability_none", goto after_set_capability;);
{
- int rc= DBUG_EVALUATE_IF("simulate_slave_capability_old_53",
+ int rc= DBUG_IF("simulate_slave_capability_old_53") ?
mysql_real_query(mysql, STRING_WITH_LEN("SET @mariadb_slave_capability="
- STRINGIFY_ARG(MARIA_SLAVE_CAPABILITY_ANNOTATE))),
+ STRINGIFY_ARG(MARIA_SLAVE_CAPABILITY_ANNOTATE))) :
mysql_real_query(mysql, STRING_WITH_LEN("SET @mariadb_slave_capability="
- STRINGIFY_ARG(MARIA_SLAVE_CAPABILITY_MINE))));
+ STRINGIFY_ARG(MARIA_SLAVE_CAPABILITY_MINE)));
if (unlikely(rc))
{
err_code= mysql_errno(mysql);
@@ -3932,6 +3932,10 @@ apply_event_and_update_pos_apply(Log_event* ev, THD* thd, rpl_group_info *rgi,
DBUG_PRINT("info", ("apply_event error = %d", exec_res));
if (exec_res == 0)
{
+ if (thd->rgi_slave && (thd->rgi_slave->gtid_ev_flags_extra &
+ Gtid_log_event::FL_START_ALTER_E1) &&
+ thd->rgi_slave->get_finish_event_group_called())
+ DBUG_RETURN(exec_res ? 1 : 0);
int error= ev->update_pos(rgi);
#ifdef DBUG_TRACE
DBUG_PRINT("info", ("update_pos error = %d", error));
@@ -4055,6 +4059,11 @@ int
apply_event_and_update_pos_for_parallel(Log_event* ev, THD* thd,
rpl_group_info *rgi)
{
+ int rc= 0;
+ ulong retries= 0;
+ bool is_sa= rgi->gtid_ev_flags_extra == Gtid_log_event::FL_START_ALTER_E1;
+ bool is_sa_temp_err= false;
+
mysql_mutex_assert_not_owner(&rgi->rli->data_lock);
int reason= apply_event_and_update_pos_setup(ev, thd, rgi);
/*
@@ -4066,7 +4075,51 @@ apply_event_and_update_pos_for_parallel(Log_event* ev, THD* thd,
Calling sql_delay_event() was handled in the SQL driver thread when
doing parallel replication.
*/
- return apply_event_and_update_pos_apply(ev, thd, rgi, reason);
+ do
+ {
+ rc= apply_event_and_update_pos_apply(ev, thd, rgi, reason);
+ if (rc && is_sa)
+ {
+ is_sa_temp_err=
+ is_parallel_retry_error(rgi, thd->get_stmt_da()->sql_errno());
+ }
+ }
+ while(is_sa_temp_err && retries++ < slave_trans_retries);
+
+ if (is_sa_temp_err)
+ {
+ Master_info *mi= rgi->rli->mi;
+ mysql_mutex_lock(&mi->start_alter_lock);
+
+ DBUG_ASSERT(!rgi->sa_info->direct_commit_alter);
+ /*
+ Give up retrying to hand the whole ALTER execution over to
+ the "Complete" ALTER.
+ */
+ rgi->sa_info->direct_commit_alter= true;
+ rgi->sa_info->state= start_alter_state::COMPLETED;
+ mysql_cond_broadcast(&rgi->sa_info->start_alter_cond);
+ mysql_mutex_unlock(&mi->start_alter_lock);
+ if (global_system_variables.log_warnings > 2)
+ {
+ rpl_gtid *gtid= &rgi->current_gtid;
+ sql_print_information("Start Alter Query '%s' "
+ "GTID %u-%u-%llu having a temporary error %d code "
+ "has been unsuccessfully retried %lu times; its "
+ "parallel optimistic execution now proceeds in "
+ "legacy mode",
+ static_cast<Query_log_event*>(ev)->query,
+ gtid->domain_id, gtid->server_id, gtid->seq_no,
+ thd->get_stmt_da()->sql_errno(), retries - 1);
+ }
+ thd->clear_error();
+ thd->reset_killed();
+ rgi->killed_for_retry = rpl_group_info::RETRY_KILL_NONE;
+
+ rc= false;
+ }
+
+ return rc;
}
@@ -4705,7 +4758,7 @@ pthread_handler_t handle_slave_io(void *arg)
}
thd->variables.wsrep_on= 0;
- if (DBUG_EVALUATE_IF("failed_slave_start", 1, 0)
+ if (DBUG_IF("failed_slave_start")
|| repl_semisync_slave.slave_start(mi))
{
mi->report(ERROR_LEVEL, ER_SLAVE_FATAL_ERROR, NULL,
@@ -4982,7 +5035,7 @@ Stopping slave I/O thread due to out-of-memory error from master");
(!repl_semisync_slave.get_slave_enabled() ||
(!(mi->semi_ack & SEMI_SYNC_SLAVE_DELAY_SYNC) ||
(mi->semi_ack & (SEMI_SYNC_NEED_ACK)))) &&
- (DBUG_EVALUATE_IF("failed_flush_master_info", 1, 0) ||
+ (DBUG_IF("failed_flush_master_info") ||
flush_master_info(mi, TRUE, TRUE)))
{
sql_print_error("Failed to flush master info file");
@@ -5611,8 +5664,10 @@ pthread_handler_t handle_slave_sql(void *arg)
err:
if (mi->using_parallel())
+ {
rli->parallel.wait_for_done(thd, rli);
- /* Gtid_list_log_event::do_apply_event has already reported the GTID until */
+ };
+ /* Gtid_list_log_event::do_apply_event has already reported the GTID until */
if (rli->stop_for_until && rli->until_condition != Relay_log_info::UNTIL_GTID)
{
if (global_system_variables.log_warnings > 2)
@@ -6327,12 +6382,19 @@ static int queue_event(Master_info* mi, const uchar *buf, ulong event_len)
Rotate_log_event rev(buf, checksum_alg != BINLOG_CHECKSUM_ALG_OFF ?
event_len - BINLOG_CHECKSUM_LEN : event_len,
mi->rli.relay_log.description_event_for_queue);
-
- if (unlikely(mi->gtid_reconnect_event_skip_count) &&
- unlikely(!mi->gtid_event_seen) &&
- rev.is_artificial_event() &&
- (mi->prev_master_id != mi->master_id ||
- strcmp(rev.new_log_ident, mi->master_log_name) != 0))
+ bool master_changed= false;
+ bool maybe_crashed= false;
+ // Exclude server start scenario
+ if ((mi->prev_master_id && mi->master_id) &&
+ (mi->prev_master_id != mi->master_id))
+ master_changed= true;
+ if ((mi->master_log_name[0]!='\0') &&
+ (strcmp(rev.new_log_ident, mi->master_log_name) != 0))
+ maybe_crashed= true;
+
+ if (unlikely((mi->gtid_reconnect_event_skip_count && master_changed) ||
+ maybe_crashed) &&
+ unlikely(!mi->gtid_event_seen) && rev.is_artificial_event())
{
/*
Artificial Rotate_log_event is the first event we receive at the start
@@ -6368,26 +6430,37 @@ static int queue_event(Master_info* mi, const uchar *buf, ulong event_len)
case likewise rollback the partially received event group.
*/
Format_description_log_event fdle(4);
+ fdle.checksum_alg= checksum_alg;
+
+ /*
+ Possible crash is flagged in being created FD' common header
+ to conduct any necessary cleanup by the slave applier.
+ */
+ if (maybe_crashed)
+ fdle.flags |= LOG_EVENT_BINLOG_IN_USE_F;
- if (mi->prev_master_id != mi->master_id)
- sql_print_warning("The server_id of master server changed in the "
- "middle of GTID %u-%u-%llu. Assuming a change of "
- "master server, so rolling back the previously "
- "received partial transaction. Expected: %lu, "
- "received: %lu", mi->last_queued_gtid.domain_id,
- mi->last_queued_gtid.server_id,
- mi->last_queued_gtid.seq_no,
- mi->prev_master_id, mi->master_id);
- else if (strcmp(rev.new_log_ident, mi->master_log_name) != 0)
- sql_print_warning("Unexpected change of master binlog file name in the "
- "middle of GTID %u-%u-%llu, assuming that master has "
- "crashed and rolling back the transaction. Expected: "
- "'%s', received: '%s'",
- mi->last_queued_gtid.domain_id,
- mi->last_queued_gtid.server_id,
- mi->last_queued_gtid.seq_no,
- mi->master_log_name, rev.new_log_ident);
+ if (mi->gtid_reconnect_event_skip_count)
+ {
+ if (master_changed)
+ sql_print_warning("The server_id of master server changed in the "
+ "middle of GTID %u-%u-%llu. Assuming a change of "
+ "master server, so rolling back the previously "
+ "received partial transaction. Expected: %lu, "
+ "received: %lu", mi->last_queued_gtid.domain_id,
+ mi->last_queued_gtid.server_id,
+ mi->last_queued_gtid.seq_no,
+ mi->prev_master_id, mi->master_id);
+ else
+ sql_print_warning("Unexpected change of master binlog file name in "
+ "the middle of GTID %u-%u-%llu, assuming that "
+ "master has crashed and rolling back the "
+ "transaction. Expected: '%s', received: '%s'",
+ mi->last_queued_gtid.domain_id,
+ mi->last_queued_gtid.server_id,
+ mi->last_queued_gtid.seq_no, mi->master_log_name,
+ rev.new_log_ident);
+ }
mysql_mutex_lock(log_lock);
if (likely(!rli->relay_log.write_event(&fdle) &&
!rli->relay_log.flush_and_sync(NULL)))