summaryrefslogtreecommitdiff
path: root/sql/log.cc
diff options
context:
space:
mode:
Diffstat (limited to 'sql/log.cc')
-rw-r--r--sql/log.cc526
1 files changed, 458 insertions, 68 deletions
diff --git a/sql/log.cc b/sql/log.cc
index 78fbd2cf0ab..1295dc087fd 100644
--- a/sql/log.cc
+++ b/sql/log.cc
@@ -12,7 +12,7 @@
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
- Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA */
+ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA */
/**
@@ -119,6 +119,7 @@ static MYSQL_BIN_LOG::xid_count_per_binlog *
static bool start_binlog_background_thread();
+static rpl_binlog_state rpl_global_gtid_binlog_state;
/**
purge logs, master and slave sides both, related error code
@@ -185,7 +186,7 @@ Silence_log_table_errors::handle_condition(THD *,
Sql_condition ** cond_hdl)
{
*cond_hdl= NULL;
- strmake(m_message, msg, sizeof(m_message)-1);
+ strmake_buf(m_message, msg);
return TRUE;
}
@@ -686,7 +687,8 @@ bool Log_to_csv_event_handler::
/* do a write */
if (table->field[1]->store(user_host, user_host_len, client_cs) ||
table->field[2]->store((longlong) thread_id, TRUE) ||
- table->field[3]->store((longlong) server_id, TRUE) ||
+ table->field[3]->store((longlong) global_system_variables.server_id,
+ TRUE) ||
table->field[4]->store(command_type, command_type_len, client_cs))
goto err;
@@ -883,7 +885,7 @@ bool Log_to_csv_event_handler::
table->field[8]->set_notnull();
}
- if (table->field[9]->store((longlong) server_id, TRUE))
+ if (table->field[9]->store((longlong)global_system_variables.server_id, TRUE))
goto err;
table->field[9]->set_notnull();
@@ -2293,7 +2295,7 @@ static int find_uniq_filename(char *name)
DBUG_RETURN(1);
}
file_info= dir_info->dir_entry;
- for (i= dir_info->number_off_files ; i-- ; file_info++)
+ for (i= dir_info->number_of_files ; i-- ; file_info++)
{
if (memcmp(file_info->name, start, length) == 0 &&
test_if_number(file_info->name+length, &number,0))
@@ -2304,7 +2306,7 @@ static int find_uniq_filename(char *name)
my_dirend(dir_info);
/* check if reached the maximum possible extension number */
- if ((max_found == MAX_LOG_UNIQUE_FN_EXT))
+ if (max_found == MAX_LOG_UNIQUE_FN_EXT)
{
sql_print_error("Log filename extension number exhausted: %06lu. \
Please fix this by archiving old logs and \
@@ -2941,7 +2943,7 @@ MYSQL_BIN_LOG::MYSQL_BIN_LOG(uint *sync_period)
bytes_written(0), file_id(1), open_count(1),
group_commit_queue(0), group_commit_queue_busy(FALSE),
num_commits(0), num_group_commits(0),
- sync_period_ptr(sync_period), sync_counter(0),
+ sync_period_ptr(sync_period), sync_counter(0), state_read(false),
is_relay_log(0), signal_cnt(0),
checksum_alg_reset(BINLOG_CHECKSUM_ALG_UNDEF),
relay_log_checksum_alg(BINLOG_CHECKSUM_ALG_UNDEF),
@@ -2968,6 +2970,19 @@ void MYSQL_BIN_LOG::cleanup()
{
xid_count_per_binlog *b;
+ /* Wait for the binlog background thread to stop. */
+ if (!is_relay_log && binlog_background_thread_started)
+ {
+ mysql_mutex_lock(&LOCK_binlog_background_thread);
+ binlog_background_thread_stop= true;
+ mysql_cond_signal(&COND_binlog_background_thread);
+ while (binlog_background_thread_stop)
+ mysql_cond_wait(&COND_binlog_background_thread_end,
+ &LOCK_binlog_background_thread);
+ mysql_mutex_unlock(&LOCK_binlog_background_thread);
+ binlog_background_thread_started= false;
+ }
+
inited= 0;
close(LOG_CLOSE_INDEX|LOG_CLOSE_STOP_EVENT);
delete description_event_for_queue;
@@ -2984,19 +2999,6 @@ void MYSQL_BIN_LOG::cleanup()
my_free(b);
}
- /* Wait for the binlog background thread to stop. */
- if (!is_relay_log && binlog_background_thread_started)
- {
- mysql_mutex_lock(&LOCK_binlog_background_thread);
- binlog_background_thread_stop= true;
- mysql_cond_signal(&COND_binlog_background_thread);
- while (binlog_background_thread_stop)
- mysql_cond_wait(&COND_binlog_background_thread_end,
- &LOCK_binlog_background_thread);
- mysql_mutex_unlock(&LOCK_binlog_background_thread);
- binlog_background_thread_started= false;
- }
-
mysql_mutex_destroy(&LOCK_log);
mysql_mutex_destroy(&LOCK_index);
mysql_mutex_destroy(&LOCK_xid_list);
@@ -3007,6 +3009,14 @@ void MYSQL_BIN_LOG::cleanup()
mysql_cond_destroy(&COND_binlog_background_thread);
mysql_cond_destroy(&COND_binlog_background_thread_end);
}
+
+ /*
+ Free data for global binlog state.
+ We can't do that automaticly as we need to do this before
+ safemalloc is shut down
+ */
+ if (!is_relay_log)
+ rpl_global_gtid_binlog_state.free();
DBUG_VOID_RETURN;
}
@@ -3135,6 +3145,9 @@ bool MYSQL_BIN_LOG::open(const char *log_name,
DBUG_ENTER("MYSQL_BIN_LOG::open");
DBUG_PRINT("enter",("log_type: %d",(int) log_type_arg));
+ if (!is_relay_log && read_state_from_file())
+ DBUG_RETURN(1);
+
if (!is_relay_log && !binlog_background_thread_started &&
start_binlog_background_thread())
DBUG_RETURN(1);
@@ -3248,6 +3261,47 @@ bool MYSQL_BIN_LOG::open(const char *log_name,
if (!is_relay_log)
{
char buf[FN_REFLEN];
+
+ /*
+ Output a Gtid_list_log_event at the start of the binlog file.
+
+ This is used to quickly determine which GTIDs are found in binlog
+ files earlier than this one, and which are found in this (or later)
+ binlogs.
+
+ The list gives a mapping from (domain_id, server_id) -> seq_no (so
+ this means that there is at most one entry for every unique pair
+ (domain_id, server_id) in the list). It indicates that this seq_no is
+ the last one found in an earlier binlog file for this (domain_id,
+ server_id) combination - so any higher seq_no should be search for
+ from this binlog file, or a later one.
+
+ This allows to locate the binlog file containing a given GTID by
+ scanning backwards, reading just the Gtid_list_log_event at the
+ start of each file, and scanning only the relevant binlog file when
+ found, not all binlog files.
+
+ The existence of a given entry (domain_id, server_id, seq_no)
+ guarantees only that this seq_no will not be found in this or any
+ later binlog file. It does not guarantee that it can be found it an
+ earlier binlog file, for example the file may have been purged.
+
+ If there is no entry for a given (domain_id, server_id) pair, then
+ it means that no such GTID exists in any earlier binlog. It is
+ permissible to remove such pair from future Gtid_list_log_events
+ if all previous binlog files containing such GTIDs have been purged
+ (though such optimization is not performed at the time of this
+ writing). So if there is no entry for given GTID it means that such
+ GTID should be search for in this or later binlog file, same as if
+ there had been an entry (domain_id, server_id, 0).
+ */
+
+ Gtid_list_log_event gl_ev(&rpl_global_gtid_binlog_state, 0);
+ if (gl_ev.write(&log_file))
+ goto err;
+
+ /* Output a binlog checkpoint event at the start of the binlog file. */
+
/*
Construct an entry in the binlog_xid_count_list for the new binlog
file (we will not link it into the list until we know the new file
@@ -3334,8 +3388,7 @@ bool MYSQL_BIN_LOG::open(const char *log_name,
mysql_file_sync(log_file.file, MYF(MY_WME|MY_SYNC_FILESIZE)))
goto err;
mysql_mutex_lock(&LOCK_commit_ordered);
- strmake(last_commit_pos_file, log_file_name,
- sizeof(last_commit_pos_file)-1);
+ strmake_buf(last_commit_pos_file, log_file_name);
last_commit_pos_offset= my_b_tell(&log_file);
mysql_mutex_unlock(&LOCK_commit_ordered);
@@ -3423,7 +3476,7 @@ int MYSQL_BIN_LOG::get_current_log(LOG_INFO* linfo)
int MYSQL_BIN_LOG::raw_get_current_log(LOG_INFO* linfo)
{
- strmake(linfo->log_file_name, log_file_name, sizeof(linfo->log_file_name)-1);
+ strmake_buf(linfo->log_file_name, log_file_name);
linfo->pos = my_b_tell(&log_file);
return 0;
}
@@ -3674,7 +3727,8 @@ bool MYSQL_BIN_LOG::reset_logs(THD* thd, bool create_new_log)
const char* save_name;
DBUG_ENTER("reset_logs");
- ha_reset_logs(thd);
+ if (thd)
+ ha_reset_logs(thd);
/*
We need to get both locks to be sure that no one is trying to
write to the index log file.
@@ -3718,17 +3772,10 @@ bool MYSQL_BIN_LOG::reset_logs(THD* thd, bool create_new_log)
/* Now wait for all checkpoint requests and pending unlog() to complete. */
mysql_mutex_lock(&LOCK_xid_list);
- xid_count_per_binlog *b;
for (;;)
{
- I_List_iterator<xid_count_per_binlog> it(binlog_xid_count_list);
- while ((b= it++))
- {
- if (b->xid_count > 0)
- break;
- }
- if (!b)
- break; /* No more pending XIDs */
+ if (is_xidlist_idle_nolock())
+ break;
/*
Wait until signalled that one more binlog dropped to zero, then check
again.
@@ -3777,7 +3824,7 @@ bool MYSQL_BIN_LOG::reset_logs(THD* thd, bool create_new_log)
for (;;)
{
- if ((error= my_delete_allow_opened(linfo.log_file_name, MYF(0))) != 0)
+ if ((error= my_delete(linfo.log_file_name, MYF(0))) != 0)
{
if (my_errno == ENOENT)
{
@@ -3806,9 +3853,14 @@ bool MYSQL_BIN_LOG::reset_logs(THD* thd, bool create_new_log)
break;
}
+ if (!is_relay_log)
+ {
+ rpl_global_gtid_binlog_state.reset();
+ }
+
/* Start logging with a new file */
close(LOG_CLOSE_INDEX | LOG_CLOSE_TO_BE_OPENED);
- if ((error= my_delete_allow_opened(index_file_name, MYF(0)))) // Reset (open will update)
+ if ((error= my_delete(index_file_name, MYF(0)))) // Reset (open will update)
{
if (my_errno == ENOENT)
{
@@ -3945,8 +3997,7 @@ int MYSQL_BIN_LOG::purge_first_log(Relay_log_info* rli, bool included)
Reset rli's coordinates to the current log.
*/
rli->event_relay_log_pos= BIN_LOG_HEADER_SIZE;
- strmake(rli->event_relay_log_name,rli->linfo.log_file_name,
- sizeof(rli->event_relay_log_name)-1);
+ strmake_buf(rli->event_relay_log_name,rli->linfo.log_file_name);
/*
If we removed the rli->group_relay_log_name file,
@@ -3956,8 +4007,7 @@ int MYSQL_BIN_LOG::purge_first_log(Relay_log_info* rli, bool included)
if (included)
{
rli->group_relay_log_pos = BIN_LOG_HEADER_SIZE;
- strmake(rli->group_relay_log_name,rli->linfo.log_file_name,
- sizeof(rli->group_relay_log_name)-1);
+ strmake_buf(rli->group_relay_log_name,rli->linfo.log_file_name);
rli->notify_group_relay_log_name_update();
}
@@ -4465,9 +4515,7 @@ int MYSQL_BIN_LOG::purge_logs_before_date(time_t purge_time)
else
{
if (stat_area.st_mtime < purge_time)
- strmake(to_log,
- log_info.log_file_name,
- sizeof(log_info.log_file_name) - 1);
+ strmake_buf(to_log, log_info.log_file_name);
else
break;
}
@@ -4506,6 +4554,32 @@ MYSQL_BIN_LOG::can_purge_log(const char *log_file_name)
#endif /* HAVE_REPLICATION */
+bool
+MYSQL_BIN_LOG::is_xidlist_idle()
+{
+ bool res;
+ mysql_mutex_lock(&LOCK_xid_list);
+ res= is_xidlist_idle_nolock();
+ mysql_mutex_unlock(&LOCK_xid_list);
+ return res;
+}
+
+
+bool
+MYSQL_BIN_LOG::is_xidlist_idle_nolock()
+{
+ xid_count_per_binlog *b;
+
+ I_List_iterator<xid_count_per_binlog> it(binlog_xid_count_list);
+ while ((b= it++))
+ {
+ if (b->xid_count > 0)
+ return false;
+ }
+ return true;
+}
+
+
/**
Create a new log file name.
@@ -5083,8 +5157,7 @@ binlog_start_consistent_snapshot(handlerton *hton, THD *thd)
binlog_cache_mngr *const cache_mngr= thd->binlog_setup_trx_data();
/* Server layer calls us with LOCK_commit_ordered locked, so this is safe. */
- strmake(cache_mngr->last_commit_pos_file, mysql_bin_log.last_commit_pos_file,
- sizeof(cache_mngr->last_commit_pos_file)-1);
+ strmake_buf(cache_mngr->last_commit_pos_file, mysql_bin_log.last_commit_pos_file);
cache_mngr->last_commit_pos_offset= mysql_bin_log.last_commit_pos_offset;
trans_register_ha(thd, TRUE, hton);
@@ -5294,6 +5367,237 @@ MYSQL_BIN_LOG::flush_and_set_pending_rows_event(THD *thd,
DBUG_RETURN(error);
}
+
+/* Generate a new global transaction ID, and write it to the binlog */
+bool
+MYSQL_BIN_LOG::write_gtid_event(THD *thd, bool standalone,
+ bool is_transactional)
+{
+ rpl_gtid gtid;
+ uint32 domain_id= thd->variables.gtid_domain_id;
+ uint32 server_id= thd->variables.server_id;
+ uint64 seq_no= thd->variables.gtid_seq_no;
+ int err;
+
+ /*
+ Reset the session variable gtid_seq_no, to reduce the risk of accidentally
+ producing a duplicate GTID.
+ */
+ thd->variables.gtid_seq_no= 0;
+ if (seq_no != 0)
+ {
+ /* Use the specified sequence number. */
+ gtid.domain_id= domain_id;
+ gtid.server_id= server_id;
+ gtid.seq_no= seq_no;
+ mysql_mutex_lock(&LOCK_rpl_gtid_state);
+ err= rpl_global_gtid_binlog_state.update(&gtid, opt_gtid_strict_mode);
+ mysql_mutex_unlock(&LOCK_rpl_gtid_state);
+ if (err && thd->get_stmt_da()->sql_errno()==ER_GTID_STRICT_OUT_OF_ORDER)
+ errno= ER_GTID_STRICT_OUT_OF_ORDER;
+ }
+ else
+ {
+ /* Allocate the next sequence number for the GTID. */
+ mysql_mutex_lock(&LOCK_rpl_gtid_state);
+ err= rpl_global_gtid_binlog_state.update_with_next_gtid(domain_id,
+ server_id, &gtid);
+ mysql_mutex_unlock(&LOCK_rpl_gtid_state);
+ seq_no= gtid.seq_no;
+ }
+ if (err)
+ return true;
+
+ Gtid_log_event gtid_event(thd, seq_no, domain_id, standalone,
+ LOG_EVENT_SUPPRESS_USE_F, is_transactional);
+
+ /* Write the event to the binary log. */
+ if (gtid_event.write(&mysql_bin_log.log_file))
+ return true;
+ status_var_add(thd->status_var.binlog_bytes_written, gtid_event.data_written);
+
+ return false;
+}
+
+
+int
+MYSQL_BIN_LOG::write_state_to_file()
+{
+ File file_no;
+ IO_CACHE cache;
+ char buf[FN_REFLEN];
+ int err;
+ bool opened= false;
+ bool inited= false;
+
+ fn_format(buf, opt_bin_logname, mysql_data_home, ".state",
+ MY_UNPACK_FILENAME);
+ if ((file_no= mysql_file_open(key_file_binlog_state, buf,
+ O_RDWR|O_CREAT|O_TRUNC|O_BINARY,
+ MYF(MY_WME))) < 0)
+ {
+ err= 1;
+ goto err;
+ }
+ opened= true;
+ if ((err= init_io_cache(&cache, file_no, IO_SIZE, WRITE_CACHE, 0, 0,
+ MYF(MY_WME|MY_WAIT_IF_FULL))))
+ goto err;
+ inited= true;
+ if ((err= rpl_global_gtid_binlog_state.write_to_iocache(&cache)))
+ goto err;
+ inited= false;
+ if ((err= end_io_cache(&cache)))
+ goto err;
+ if ((err= mysql_file_sync(file_no, MYF(MY_WME|MY_SYNC_FILESIZE))))
+ goto err;
+ goto end;
+
+err:
+ sql_print_error("Error writing binlog state to file '%s'.\n", buf);
+ if (inited)
+ end_io_cache(&cache);
+end:
+ if (opened)
+ mysql_file_close(file_no, MYF(0));
+
+ return err;
+}
+
+
+int
+MYSQL_BIN_LOG::read_state_from_file()
+{
+ File file_no;
+ IO_CACHE cache;
+ char buf[FN_REFLEN];
+ int err;
+ bool opened= false;
+ bool inited= false;
+
+ if (state_read)
+ return 0;
+ state_read= true;
+
+ fn_format(buf, opt_bin_logname, mysql_data_home, ".state",
+ MY_UNPACK_FILENAME);
+ if ((file_no= mysql_file_open(key_file_binlog_state, buf,
+ O_RDONLY|O_BINARY, MYF(0))) < 0)
+ {
+ if (my_errno != ENOENT)
+ {
+ err= 1;
+ goto err;
+ }
+ else
+ {
+ /*
+ If the state file does not exist, this is the first server startup
+ with GTID enabled. So initialize to empty state.
+ */
+ rpl_global_gtid_binlog_state.reset();
+ err= 0;
+ goto end;
+ }
+ }
+ opened= true;
+ if ((err= init_io_cache(&cache, file_no, IO_SIZE, READ_CACHE, 0, 0,
+ MYF(MY_WME|MY_WAIT_IF_FULL))))
+ goto err;
+ inited= true;
+ if ((err= rpl_global_gtid_binlog_state.read_from_iocache(&cache)))
+ goto err;
+ goto end;
+
+err:
+ sql_print_error("Error reading binlog GTID state from file '%s'.\n", buf);
+end:
+ if (inited)
+ end_io_cache(&cache);
+ if (opened)
+ mysql_file_close(file_no, MYF(0));
+
+ return err;
+}
+
+
+int
+MYSQL_BIN_LOG::get_most_recent_gtid_list(rpl_gtid **list, uint32 *size)
+{
+ return rpl_global_gtid_binlog_state.get_most_recent_gtid_list(list, size);
+}
+
+
+bool
+MYSQL_BIN_LOG::append_state_pos(String *str)
+{
+ bool err;
+
+ mysql_mutex_lock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ err= rpl_global_gtid_binlog_state.append_pos(str);
+ mysql_mutex_unlock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ return err;
+}
+
+
+bool
+MYSQL_BIN_LOG::find_in_binlog_state(uint32 domain_id, uint32 server_id,
+ rpl_gtid *out_gtid)
+{
+ rpl_gtid *gtid;
+ mysql_mutex_lock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ if ((gtid= rpl_global_gtid_binlog_state.find(domain_id, server_id)))
+ *out_gtid= *gtid;
+ mysql_mutex_unlock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ return gtid != NULL;
+}
+
+
+bool
+MYSQL_BIN_LOG::lookup_domain_in_binlog_state(uint32 domain_id,
+ rpl_gtid *out_gtid)
+{
+ rpl_gtid *found_gtid;
+ bool res= false;
+
+ mysql_mutex_lock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ if ((found_gtid= rpl_global_gtid_binlog_state.find_most_recent(domain_id)))
+ {
+ *out_gtid= *found_gtid;
+ res= true;
+ }
+ mysql_mutex_unlock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+
+ return res;
+}
+
+
+int
+MYSQL_BIN_LOG::bump_seq_no_counter_if_needed(uint32 domain_id, uint64 seq_no)
+{
+ int err;
+
+ mysql_mutex_lock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ err= rpl_global_gtid_binlog_state.bump_seq_no_if_needed(domain_id, seq_no);
+ mysql_mutex_unlock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ return err;
+}
+
+
+bool
+MYSQL_BIN_LOG::check_strict_gtid_sequence(uint32 domain_id, uint32 server_id,
+ uint64 seq_no)
+{
+ bool err;
+
+ mysql_mutex_lock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ err= rpl_global_gtid_binlog_state.check_strict_sequence(domain_id, server_id,
+ seq_no);
+ mysql_mutex_unlock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ return err;
+}
+
+
/**
Write an event to the binary log. If with_annotate != NULL and
*with_annotate = TRUE write also Annotate_rows before the event
@@ -5363,6 +5667,8 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
my_org_b_tell= my_b_tell(file);
mysql_mutex_lock(&LOCK_log);
prev_binlog_id= current_binlog_id;
+ if (write_gtid_event(thd, true, using_trans))
+ goto err;
}
else
{
@@ -6235,19 +6541,6 @@ MYSQL_BIN_LOG::write_transaction_to_binlog(THD *thd,
break;
}
- /*
- Log "BEGIN" at the beginning of every transaction. Here, a transaction is
- either a BEGIN..COMMIT block or a single statement in autocommit mode.
-
- Create the necessary events here, where we have the correct THD (and
- thread context).
-
- Due to group commit the actual writing to binlog may happen in a different
- thread.
- */
- Query_log_event qinfo(thd, STRING_WITH_LEN("BEGIN"), using_trx_cache, TRUE,
- TRUE, 0);
- entry.begin_event= &qinfo;
entry.end_event= end_ev;
if (cache_mngr->stmt_cache.has_incident() ||
cache_mngr->trx_cache.has_incident())
@@ -6444,10 +6737,10 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
*/
DBUG_ASSERT(!cache_mngr->stmt_cache.empty() || !cache_mngr->trx_cache.empty());
- current->error= write_transaction_or_stmt(current);
+ if ((current->error= write_transaction_or_stmt(current)))
+ current->commit_errno= errno;
- strmake(cache_mngr->last_commit_pos_file, log_file_name,
- sizeof(cache_mngr->last_commit_pos_file)-1);
+ strmake_buf(cache_mngr->last_commit_pos_file, log_file_name);
commit_offset= my_b_write_tell(&log_file);
cache_mngr->last_commit_pos_offset= commit_offset;
if (cache_mngr->using_xa && cache_mngr->xa_xid)
@@ -6623,10 +6916,8 @@ MYSQL_BIN_LOG::write_transaction_or_stmt(group_commit_entry *entry)
{
binlog_cache_mngr *mngr= entry->cache_mngr;
- if (entry->begin_event->write(&log_file))
+ if (write_gtid_event(entry->thd, false, entry->using_trx_cache))
return ER_ERROR_ON_WRITE;
- status_var_add(entry->thd->status_var.binlog_bytes_written,
- entry->begin_event->data_written);
if (entry->using_stmt_cache && !mngr->stmt_cache.empty() &&
write_cache(entry->thd, mngr->get_binlog_cache_log(FALSE)))
@@ -6766,6 +7057,8 @@ int MYSQL_BIN_LOG::wait_for_update_bin_log(THD* thd,
void MYSQL_BIN_LOG::close(uint exiting)
{ // One can't set log_type here!
+ bool failed_to_save_state= false;
+
DBUG_ENTER("MYSQL_BIN_LOG::close");
DBUG_PRINT("enter",("exiting: %d", (int) exiting));
if (log_state == LOG_OPENED)
@@ -6783,6 +7076,27 @@ void MYSQL_BIN_LOG::close(uint exiting)
s.write(&log_file);
bytes_written+= s.data_written;
signal_update();
+
+ /*
+ When we shut down server, write out the binlog state to a separate
+ file so we do not have to scan an entire binlog file to recover it
+ at next server start.
+
+ Note that this must be written and synced to disk before marking the
+ last binlog file as "not crashed".
+ */
+ if (!is_relay_log && write_state_to_file())
+ {
+ sql_print_error("Failed to save binlog GTID state during shutdown. "
+ "Binlog will be marked as crashed, so that crash "
+ "recovery can recover the state at next server "
+ "startup.");
+ /*
+ Leave binlog file marked as crashed, so we can recover state by
+ scanning it now that we failed to write out the state properly.
+ */
+ failed_to_save_state= true;
+ }
}
#endif /* HAVE_REPLICATION */
@@ -6791,7 +7105,8 @@ void MYSQL_BIN_LOG::close(uint exiting)
&& !(exiting & LOG_CLOSE_DELAYED_CLOSE))
{
my_off_t org_position= mysql_file_tell(log_file.file, MYF(0));
- clear_inuse_flag_when_closing(log_file.file);
+ if (!failed_to_save_state)
+ clear_inuse_flag_when_closing(log_file.file);
/*
Restore position so that anything we have in the IO_cache is written
to the correct position.
@@ -7944,7 +8259,7 @@ int TC_LOG_BINLOG::open(const char *opt_name)
do
{
- strmake(log_name, log_info.log_file_name, sizeof(log_name)-1);
+ strmake_buf(log_name, log_info.log_file_name);
} while (!(error= find_next_log(&log_info, 1)));
if (error != LOG_INFO_EOF)
@@ -7967,9 +8282,10 @@ int TC_LOG_BINLOG::open(const char *opt_name)
sql_print_information("Recovering after a crash using %s", opt_name);
error= recover(&log_info, log_name, &log,
(Format_description_log_event *)ev);
+ state_read= true;
}
else
- error=0;
+ error= read_state_from_file();
delete ev;
end_io_cache(&log);
@@ -8219,6 +8535,29 @@ binlog_background_thread(void *arg __attribute__((unused)))
mysql_mutex_unlock(&LOCK_thread_count);
thd->store_globals();
+ /*
+ Load the slave replication GTID state from the mysql.gtid_slave_pos
+ table.
+
+ This is mostly so that we can start our seq_no counter from the highest
+ seq_no seen by a slave. This way, we have a way to tell if a transaction
+ logged by ourselves as master is newer or older than a replicated
+ transaction.
+ */
+#ifdef HAVE_REPLICATION
+ if (rpl_load_gtid_slave_state(thd))
+ sql_print_warning("Failed to load slave replication state from table "
+ "%s.%s: %u: %s", "mysql",
+ rpl_gtid_slave_state_table_name.str,
+ thd->get_stmt_da()->sql_errno(),
+ thd->get_stmt_da()->message());
+#endif
+
+ mysql_mutex_lock(&mysql_bin_log.LOCK_binlog_background_thread);
+ binlog_background_thread_started= true;
+ mysql_cond_signal(&mysql_bin_log.COND_binlog_background_thread_end);
+ mysql_mutex_unlock(&mysql_bin_log.LOCK_binlog_background_thread);
+
for (;;)
{
/*
@@ -8231,6 +8570,13 @@ binlog_background_thread(void *arg __attribute__((unused)))
{
stop= binlog_background_thread_stop;
queue= binlog_background_thread_queue;
+ if (stop && !mysql_bin_log.is_xidlist_idle())
+ {
+ /*
+ Delay stop until all pending binlog checkpoints have been processed.
+ */
+ stop= false;
+ }
if (stop || queue)
break;
mysql_cond_wait(&mysql_bin_log.COND_binlog_background_thread,
@@ -8241,9 +8587,18 @@ binlog_background_thread(void *arg __attribute__((unused)))
mysql_mutex_unlock(&mysql_bin_log.LOCK_binlog_background_thread);
/* Process any incoming commit_checkpoint_notify() calls. */
+ DBUG_EXECUTE_IF("inject_binlog_background_thread_before_mark_xid_done",
+ DBUG_ASSERT(!debug_sync_set_action(
+ thd,
+ STRING_WITH_LEN("binlog_background_thread_before_mark_xid_done "
+ "SIGNAL injected_binlog_background_thread "
+ "WAIT_FOR something_that_will_never_happen "
+ "TIMEOUT 2")));
+ );
while (queue)
{
THD_STAGE_INFO(thd, stage_binlog_processing_checkpoint_notify);
+ DEBUG_SYNC(current_thd, "binlog_background_thread_before_mark_xid_done");
/* Grab next pointer first, as mark_xid_done() may free the element. */
next= queue->next_in_queue;
mysql_bin_log.mark_xid_done(queue->binlog_id, true);
@@ -8301,7 +8656,16 @@ start_binlog_background_thread()
binlog_background_thread, NULL))
return 1;
- binlog_background_thread_started= true;
+ /*
+ Wait for the thread to have started (so we know that the slave replication
+ state is loaded and we have correct global_gtid_counter).
+ */
+ mysql_mutex_lock(&mysql_bin_log.LOCK_binlog_background_thread);
+ while (!binlog_background_thread_started)
+ mysql_cond_wait(&mysql_bin_log.COND_binlog_background_thread_end,
+ &mysql_bin_log.LOCK_binlog_background_thread);
+ mysql_mutex_unlock(&mysql_bin_log.LOCK_binlog_background_thread);
+
return 0;
}
@@ -8380,6 +8744,32 @@ int TC_LOG_BINLOG::recover(LOG_INFO *linfo, const char *last_log_name,
}
break;
}
+ case GTID_LIST_EVENT:
+ if (first_round)
+ {
+ Gtid_list_log_event *glev= (Gtid_list_log_event *)ev;
+
+ /* Initialise the binlog state from the Gtid_list event. */
+ if (rpl_global_gtid_binlog_state.load(glev->list, glev->count))
+ goto err2;
+ }
+ break;
+
+ case GTID_EVENT:
+ if (first_round)
+ {
+ Gtid_log_event *gev= (Gtid_log_event *)ev;
+ rpl_gtid gtid;
+
+ /* Update the binlog state with any GTID logged after Gtid_list. */
+ gtid.domain_id= gev->domain_id;
+ gtid.server_id= gev->server_id;
+ gtid.seq_no= gev->seq_no;
+ if (rpl_global_gtid_binlog_state.update(&gtid, false))
+ goto err2;
+ }
+ break;
+
default:
/* Nothing. */
break;
@@ -8596,7 +8986,7 @@ static void
set_binlog_snapshot_file(const char *src)
{
int dir_len = dirname_length(src);
- strmake(binlog_snapshot_file, src + dir_len, sizeof(binlog_snapshot_file)-1);
+ strmake_buf(binlog_snapshot_file, src + dir_len);
}
/*