summaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
Diffstat (limited to 'sql')
-rw-r--r--sql/handler.cc143
-rw-r--r--sql/handler.h11
-rw-r--r--sql/log.cc592
-rw-r--r--sql/log.h7
-rw-r--r--sql/log_event.cc49
-rw-r--r--sql/log_event.h14
-rw-r--r--sql/mysqld.cc2
-rw-r--r--sql/sql_class.h1
8 files changed, 794 insertions, 25 deletions
diff --git a/sql/handler.cc b/sql/handler.cc
index 76a187f4312..42cd048afc8 100644
--- a/sql/handler.cc
+++ b/sql/handler.cc
@@ -1249,6 +1249,29 @@ int ha_prepare(THD *thd)
DBUG_RETURN(error);
}
+/*
+ Returns counted number of
+ read-write recoverable transaction participants optionally limited to two.
+ Also optionally returns the last found rw ha_info through the 2nd argument.
+*/
+uint ha_count_rw_all(THD *thd, Ha_trx_info **ptr_ha_info, bool count_through)
+{
+ unsigned rw_ha_count= 0;
+
+ for (Ha_trx_info * ha_info= thd->transaction.all.ha_list; ha_info;
+ ha_info= ha_info->next())
+ {
+ if (ha_info->is_trx_read_write() && ha_info->ht()->recover)
+ {
+ if (ptr_ha_info)
+ *ptr_ha_info= ha_info;
+ if (++rw_ha_count > 1 && !count_through)
+ break;
+ }
+ }
+ return rw_ha_count;
+}
+
/**
Check if we can skip the two-phase commit.
@@ -1866,7 +1889,7 @@ static char* xid_to_str(char *buf, XID *xid)
recover() step of xa.
@note
- there are three modes of operation:
+ there are four modes of operation:
- automatic recover after a crash
in this case commit_list != 0, tc_heuristic_recover==0
all xids from commit_list are committed, others are rolled back
@@ -1877,6 +1900,9 @@ static char* xid_to_str(char *buf, XID *xid)
- no recovery (MySQL did not detect a crash)
in this case commit_list==0, tc_heuristic_recover == 0
there should be no prepared transactions in this case.
+ - recovery to truncated binlog to the last committed transaction
+ in any engine. Other prepared following binlog order transactions are
+ rolled back.
*/
struct xarecover_st
{
@@ -1884,8 +1910,95 @@ struct xarecover_st
XID *list;
HASH *commit_list;
bool dry_run;
+ MEM_ROOT *mem_root;
+ bool error;
};
+#ifdef HAVE_REPLICATION
+/*
+ Inserts a new hash member.
+
+ returns a successfully created and inserted @c xid_recovery_member
+ into hash @c hash_arg,
+ or NULL.
+*/
+static xid_recovery_member*
+xid_member_insert(HASH *hash_arg, my_xid xid_arg, MEM_ROOT *ptr_mem_root)
+{
+ xid_recovery_member *member= (xid_recovery_member*)
+ alloc_root(ptr_mem_root, sizeof(xid_recovery_member));
+ if (!member)
+ return NULL;
+
+ member->xid= xid_arg;
+ member->in_engine_prepare= 1;
+ return my_hash_insert(hash_arg, (uchar*) member) ? NULL : member;
+}
+
+/*
+ Inserts a new or updates an existing hash member.
+
+ returns false on success,
+ true otherwise.
+*/
+static bool xid_member_replace(HASH *hash_arg, my_xid xid_arg,
+ MEM_ROOT *ptr_mem_root)
+{
+ /*
+ Search if XID is already present in recovery_list. If found
+ and the state is 'XA_PREPRAED' mark it as XA_COMPLETE.
+ Effectively, there won't be XA-prepare event group replay.
+ */
+ xid_recovery_member* member;
+ if ((member= (xid_recovery_member *)
+ my_hash_search(hash_arg, (uchar *)& xid_arg, sizeof(xid_arg))))
+ member->in_engine_prepare++;
+ else
+ member= xid_member_insert(hash_arg, xid_arg, ptr_mem_root);
+
+ return member == NULL;
+}
+
+/*
+ Hash iterate function to complete with commit or rollback as
+ decided at binlog scanning.
+*/
+static my_bool xarecover_do_commit_or_rollback(void *member_arg,
+ void *hton_arg)
+{
+ xid_recovery_member *member= (xid_recovery_member*) member_arg;
+ handlerton *hton= (handlerton*) hton_arg;
+ xid_t x;
+ my_bool rc;
+
+ x.set(member->xid);
+ rc= member->in_engine_prepare > 0 ?
+ hton->rollback_by_xid(hton, &x) : hton->commit_by_xid(hton, &x);
+
+ return rc;
+}
+
+static my_bool xarecover_binlog_truncate_handlerton(THD *unused,
+ plugin_ref plugin,
+ void *arg)
+{
+ handlerton *hton= plugin_hton(plugin);
+
+ if (hton->state == SHOW_OPTION_YES && hton->recover)
+ {
+ my_hash_iterate((HASH*) arg, xarecover_do_commit_or_rollback, hton);
+ }
+
+ return FALSE;
+}
+
+void ha_recover_binlog_truncate_complete(HASH *commit_list)
+{
+ plugin_foreach(NULL, xarecover_binlog_truncate_handlerton,
+ MYSQL_STORAGE_ENGINE_PLUGIN, commit_list);
+}
+#endif
+
static my_bool xarecover_handlerton(THD *unused, plugin_ref plugin,
void *arg)
{
@@ -1893,13 +2006,16 @@ static my_bool xarecover_handlerton(THD *unused, plugin_ref plugin,
struct xarecover_st *info= (struct xarecover_st *) arg;
int got;
+ if (info->error)
+ return TRUE;
+
if (hton->state == SHOW_OPTION_YES && hton->recover)
{
while ((got= hton->recover(hton, info->list, info->len)) > 0 )
{
sql_print_information("Found %d prepared transaction(s) in %s",
got, hton_name(hton)->str);
- for (int i=0; i < got; i ++)
+ for (int i=0; i < got && !info->error; i ++)
{
my_xid x= WSREP_ON && wsrep_is_wsrep_xid(&info->list[i]) ?
wsrep_xid_seqno(info->list[i]) :
@@ -1936,7 +2052,7 @@ static my_bool xarecover_handlerton(THD *unused, plugin_ref plugin,
}
#endif
}
- else
+ else if (tc_heuristic_recover != TC_RECOVER_BINLOG_TRUNCATE)
{
#ifndef DBUG_OFF
int rc=
@@ -1951,6 +2067,17 @@ static my_bool xarecover_handlerton(THD *unused, plugin_ref plugin,
}
#endif
}
+#ifdef HAVE_REPLICATION
+ else
+ {
+ if (xid_member_replace(info->commit_list, x, info->mem_root))
+ {
+ info->error= true;
+ sql_print_error("Error in memory allocation at xarecover_handlerton");
+ break;
+ }
+ }
+#endif
}
if (got < info->len)
break;
@@ -1959,7 +2086,7 @@ static my_bool xarecover_handlerton(THD *unused, plugin_ref plugin,
return FALSE;
}
-int ha_recover(HASH *commit_list)
+int ha_recover(HASH *commit_list, MEM_ROOT *arg_mem_root)
{
struct xarecover_st info;
DBUG_ENTER("ha_recover");
@@ -1967,9 +2094,12 @@ int ha_recover(HASH *commit_list)
info.commit_list= commit_list;
info.dry_run= (info.commit_list==0 && tc_heuristic_recover==0);
info.list= NULL;
+ info.mem_root= arg_mem_root;
+ info.error= false;
/* commit_list and tc_heuristic_recover cannot be set both */
- DBUG_ASSERT(info.commit_list==0 || tc_heuristic_recover==0);
+ DBUG_ASSERT(info.commit_list==0 || tc_heuristic_recover==0 ||
+ tc_heuristic_recover > 2);
/* if either is set, total_ha_2pc must be set too */
DBUG_ASSERT(info.dry_run ||
(failed_ha_2pc + total_ha_2pc) > (ulong)opt_bin_log);
@@ -2011,6 +2141,9 @@ int ha_recover(HASH *commit_list)
info.found_my_xids, opt_tc_log_file);
DBUG_RETURN(1);
}
+ if (info.error)
+ DBUG_RETURN(1);
+
if (info.commit_list)
sql_print_information("Crash recovery finished.");
DBUG_RETURN(0);
diff --git a/sql/handler.h b/sql/handler.h
index c6cac5dfbbe..73af3b550fd 100644
--- a/sql/handler.h
+++ b/sql/handler.h
@@ -638,6 +638,13 @@ struct xid_t {
};
typedef struct xid_t XID;
+/* struct for heuristic binlog truncate recovery */
+struct xid_recovery_member
+{
+ my_xid xid;
+ uint in_engine_prepare; // number of engines that have xid prepared
+};
+
/* for recover() handlerton call */
#define MIN_XID_LIST_SIZE 128
#define MAX_XID_LIST_SIZE (1024*128)
@@ -4270,7 +4277,8 @@ int ha_commit_one_phase(THD *thd, bool all);
int ha_commit_trans(THD *thd, bool all);
int ha_rollback_trans(THD *thd, bool all);
int ha_prepare(THD *thd);
-int ha_recover(HASH *commit_list);
+int ha_recover(HASH *commit_list, MEM_ROOT *mem_root= NULL);
+void ha_recover_binlog_truncate_complete(HASH *commit_list);
/* transactions: these functions never call handlerton functions directly */
int ha_enable_transaction(THD *thd, bool on);
@@ -4326,4 +4334,5 @@ void print_keydup_error(TABLE *table, KEY *key, myf errflag);
int del_global_index_stat(THD *thd, TABLE* table, KEY* key_info);
int del_global_table_stat(THD *thd, LEX_STRING *db, LEX_STRING *table);
+uint ha_count_rw_all(THD *thd, Ha_trx_info **ptr_ha_info, bool count_through);
#endif /* HANDLER_INCLUDED */
diff --git a/sql/log.cc b/sql/log.cc
index f2fe0d852d1..aa6b1293200 100644
--- a/sql/log.cc
+++ b/sql/log.cc
@@ -4746,7 +4746,7 @@ int MYSQL_BIN_LOG::purge_index_entry(THD *thd, ulonglong *reclaimed_space,
}
goto err;
}
-
+
error= 0;
DBUG_PRINT("info",("purging %s",log_info.log_file_name));
@@ -7882,6 +7882,7 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
first= false;
}
+ DEBUG_SYNC(leader->thd, "commit_before_update_end_pos");
/* update binlog_end_pos so it can be read by dump thread
*
* note: must be _after_ the RUN_HOOK(after_flush) or else
@@ -8969,7 +8970,7 @@ int TC_LOG_MMAP::open(const char *opt_name)
{
if (my_errno != ENOENT)
goto err;
- if (using_heuristic_recover())
+ if (using_heuristic_recover(opt_name))
return 1;
if ((fd= mysql_file_create(key_file_tclog, logname, CREATE_MODE,
O_RDWR | O_CLOEXEC, MYF(MY_WME))) < 0)
@@ -9500,23 +9501,596 @@ TC_LOG_MMAP tc_log_mmap;
0 no heuristic recovery was requested
@retval
1 heuristic recovery was performed
+ 2 heuristic recovery failed
*/
-int TC_LOG::using_heuristic_recover()
+int TC_LOG::using_heuristic_recover(const char* opt_name)
{
+ LOG_INFO log_info;
+ int error= 0;
+ HASH xids, *ptr_xids= NULL;
+ MEM_ROOT mem_root, *ptr_mem_root= NULL;
+ int rc= 1;
+
if (!tc_heuristic_recover)
- return 0;
+ return (rc= 0);
sql_print_information("Heuristic crash recovery mode");
- if (ha_recover(0))
+ if (tc_heuristic_recover == TC_RECOVER_BINLOG_TRUNCATE)
+ {
+ (void) my_hash_init(&xids, &my_charset_bin, TC_LOG_PAGE_SIZE/3, 0,
+ sizeof(my_xid), 0, 0, MYF(0));
+ (void) init_alloc_root(&mem_root,
+ TC_LOG_PAGE_SIZE, TC_LOG_PAGE_SIZE, MYF(0));
+ ptr_xids= &xids;
+ ptr_mem_root= &mem_root;
+ }
+ if ((error= ha_recover(ptr_xids, ptr_mem_root)))
+ {
sql_print_error("Heuristic crash recovery failed");
+ goto err;
+ }
+
+ if (!strcmp(opt_name, opt_bin_logname) &&
+ tc_heuristic_recover == TC_RECOVER_BINLOG_TRUNCATE)
+ {
+ if ((error= mysql_bin_log.find_log_pos(&log_info, NullS, 1)))
+ sql_print_error("tc-heuristic-recover failed to find binlog file in "
+ "the index file. Error: %d; consider recovering "
+ "the index file and retry", error);
+ else if ((error= heuristic_binlog_rollback(&xids)))
+ sql_print_error("Heuristic BINLOG_TRUNCATE crash recovery failed. "
+ "Error: %d", error);
+ if (error > 0)
+ rc= 2;
+ }
+
+err:
sql_print_information("Please restart mysqld without --tc-heuristic-recover");
- return 1;
+ if (tc_heuristic_recover == TC_RECOVER_BINLOG_TRUNCATE)
+ {
+ free_root(&mem_root, MYF(0));
+ my_hash_free(&xids);
+ }
+ return rc;
}
/****** transaction coordinator log for 2pc - binlog() based solution ******/
#define TC_LOG_BINLOG MYSQL_BIN_LOG
+/**
+ Truncates the current binlog to specified position. Removes the rest of binlogs
+ which are present after this binlog file.
+
+ @param truncate_file Holds the binlog name to be truncated
+ @param truncate_pos Position within binlog from where it needs to
+ truncated.
+
+ @retval true ok
+ @retval false error
+
+*/
+bool MYSQL_BIN_LOG::truncate_and_remove_binlogs(const char *truncate_file,
+ my_off_t truncate_pos)
+{
+ int error= 0;
+#ifdef HAVE_REPLICATION
+ LOG_INFO log_info;
+ THD *thd= current_thd;
+ my_off_t index_file_offset= 0;
+ File file= -1;
+ IO_CACHE cache;
+ MY_STAT s;
+ my_off_t binlog_size;
+
+ if ((error= find_log_pos(&log_info, truncate_file, 1)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to locate binary log file:%s."
+ "Error:%d", truncate_file, error);
+ goto end;
+ }
+
+ while (!(error= find_next_log(&log_info, 1)))
+ {
+ if (!index_file_offset)
+ {
+ index_file_offset= log_info.index_file_start_offset;
+ if ((error= open_purge_index_file(TRUE)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to open purge index "
+ "file:%s. Error:%d", purge_index_file_name, error);
+ goto end;
+ }
+ }
+ if ((error= register_purge_index_entry(log_info.log_file_name)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to copy %s to purge index"
+ " file. Error:%d", log_info.log_file_name, error);
+ goto end;
+ }
+ }
+
+ if (error != LOG_INFO_EOF)
+ {
+ sql_print_error("tc-heuristic-recover: Failed to find the next binlog to "
+ "add to purge index register. Error:%d", error);
+ goto end;
+ }
+
+ if (is_inited_purge_index_file())
+ {
+ if (!index_file_offset)
+ index_file_offset= log_info.index_file_start_offset;
+
+ if ((error= sync_purge_index_file()))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to flush purge index "
+ "file. Error:%d", error);
+ goto end;
+ }
+
+ // Trim index file
+ if ((error=
+ mysql_file_chsize(index_file.file, index_file_offset, '\n',
+ MYF(MY_WME))) ||
+ (error=
+ mysql_file_sync(index_file.file, MYF(MY_WME|MY_SYNC_FILESIZE))))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to trim binlog index "
+ "file:%s to offset:%llu. Error:%d", index_file_name,
+ index_file_offset);
+ mysql_file_close(index_file.file, MYF(MY_WME));
+ goto end;
+ }
+
+ /* Reset data in old index cache */
+ if ((error= reinit_io_cache(&index_file, READ_CACHE, (my_off_t) 0, 0, 1)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to reinit binlog index "
+ "file. Error:%d", error);
+ mysql_file_close(index_file.file, MYF(MY_WME));
+ goto end;
+ }
+
+ /* Read each entry from purge_index_file and delete the file. */
+ if ((error= purge_index_entry(thd, NULL, TRUE)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to process registered "
+ "files that would be purged.");
+ goto end;
+ }
+ }
+
+ DBUG_ASSERT(truncate_pos);
+
+ if ((file= mysql_file_open(key_file_binlog, truncate_file,
+ O_RDWR | O_BINARY, MYF(MY_WME))) < 0)
+ {
+ error= 1;
+ sql_print_error("tc-heuristic-recover: Failed to open binlog file:%s for "
+ "truncation.", truncate_file);
+ goto end;
+ }
+ my_stat(truncate_file, &s, MYF(0));
+ binlog_size= s.st_size;
+
+ /* Change binlog file size to truncate_pos */
+ if ((error=
+ mysql_file_chsize(file, truncate_pos, 0, MYF(MY_WME))) ||
+ (error= mysql_file_sync(file, MYF(MY_WME|MY_SYNC_FILESIZE))))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to trim the "
+ "binlog file:%s to size:%llu. Error:%d",
+ truncate_file, truncate_pos, error);
+ goto end;
+ }
+ else
+ {
+ sql_print_information("tc-heuristic-recover: Truncated binlog "
+ "File: %s of Size:%llu, to Position:%llu.",
+ truncate_file, binlog_size, truncate_pos);
+ }
+ if (!(error= init_io_cache(&cache, file, IO_SIZE, WRITE_CACHE,
+ (my_off_t) truncate_pos, 0, MYF(MY_WME|MY_NABP))))
+ {
+ /*
+ Write Stop_log_event to ensure clean end point for the binary log being
+ truncated.
+ */
+ Stop_log_event se;
+ se.checksum_alg= (enum_binlog_checksum_alg) binlog_checksum_options;
+ if ((error= write_event(&se, &cache)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to write stop event to "
+ "binary log. Errno:%d",
+ (cache.error == -1) ? my_errno : error);
+ goto end;
+ }
+ if ((error= flush_io_cache(&cache)) ||
+ (error= mysql_file_sync(file, MYF(MY_WME|MY_SYNC_FILESIZE))))
+ {
+ sql_print_error("tc-heuristic-recover: Faild to write stop event to "
+ "binary log. Errno:%d",
+ (cache.error == -1) ? my_errno : error);
+ }
+ }
+ else
+ sql_print_error("tc-heuristic-recover: Failed to initialize binary log "
+ "cache for writing stop event. Errno:%d",
+ (cache.error == -1) ? my_errno : error);
+
+end:
+ if (file >= 0)
+ {
+ end_io_cache(&cache);
+ mysql_file_close(file, MYF(MY_WME));
+ }
+ error= error || close_purge_index_file();
+#endif
+ return error > 0;
+}
+
+/**
+ Returns the checkpoint binlog file name found in the lastest binlog file.
+
+ @param checkpoint_file Holds the binlog checkpoint file name.
+
+ @retval 0 ok
+ @retval 1 error
+
+*/
+int TC_LOG_BINLOG::get_binlog_checkpoint_file(char* checkpoint_file)
+{
+ Log_event *ev= NULL;
+ bool binlog_checkpoint_found= false;
+ LOG_INFO log_info;
+ const char *errmsg;
+ IO_CACHE log;
+ File file;
+ Format_description_log_event fdle(BINLOG_VERSION);
+ char log_name[FN_REFLEN];
+ int error=1;
+
+ if (!fdle.is_valid())
+ return 1;
+
+ if ((error= find_log_pos(&log_info, NullS, 1)))
+ {
+ sql_print_error("tc-heuristic-recover: find_log_pos() failed to read first "
+ "binary log entry from index file.(error: %d)", error);
+ return error;
+ }
+
+ // Move to the last binary log.
+ do
+ {
+ strmake_buf(log_name, log_info.log_file_name);
+ } while (!(error= find_next_log(&log_info, 1)));
+
+ if (error != LOG_INFO_EOF)
+ {
+ sql_print_error("tc-heuristic-recover: find_next_log() failed "
+ "(error: %d)", error);
+ return error;
+ }
+ if ((file= open_binlog(&log, log_name, &errmsg)) < 0)
+ {
+ sql_print_error("tc-heuristic-recover failed to open the binlog:%s for "
+ "reading checkpoint file name. Error: %s",
+ log_info.log_file_name, errmsg);
+ return error;
+ }
+ while (!binlog_checkpoint_found &&
+ (ev=
+ Log_event::read_log_event(&log, 0, &fdle, opt_master_verify_checksum))
+ && ev->is_valid())
+ {
+ enum Log_event_type typ= ev->get_type_code();
+ if (typ == BINLOG_CHECKPOINT_EVENT)
+ {
+ size_t dir_len;
+ Binlog_checkpoint_log_event *cev= (Binlog_checkpoint_log_event *)ev;
+ if (cev->binlog_file_len >= FN_REFLEN)
+ {
+ sql_print_error("tc-heuristic-recover: Incorrect binlog checkpoint "
+ "event with too long file name found.");
+ delete ev;
+ ev= NULL;
+ end_io_cache(&log);
+ mysql_file_close(file, MYF(MY_WME));
+ return 1;
+ }
+ else
+ {
+ dir_len= dirname_length(log_name);
+ strmake(strnmov(checkpoint_file, log_name, dir_len),
+ cev->binlog_file_name, FN_REFLEN - 1 - dir_len);
+ binlog_checkpoint_found= true;
+ }
+ }
+ delete ev;
+ ev= NULL;
+ } // End of while
+ end_io_cache(&log);
+ mysql_file_close(file, MYF(MY_WME));
+ file= -1;
+ /*
+ Old binary log without checkpoint found, binlog truncation is not
+ possible. Hence return error.
+ */
+ if (!binlog_checkpoint_found)
+ return 1;
+
+ return 0;
+}
+
+
+/**
+ Truncates the binary log, according to the transactions that got rolled
+ back from engines, during --heuristic-recover=BINLOG_TRUNCATE.
+ Global GTID state is adjusted as per the truncated binlog.
+
+ Called from @c TC_LOG::using_heuristic_recover(const char* opt_name)
+
+ @param opt_name The base name of binary log.
+
+ @return indicates success or failure of binlog rollback
+ @retval 0 success
+ @retval 1 failure
+
+*/
+int TC_LOG_BINLOG::heuristic_binlog_rollback(HASH *xids)
+{
+ int error=0;
+#ifdef HAVE_REPLICATION
+ Log_event *ev= NULL;
+ char binlog_truncate_file_name[FN_REFLEN] = {0};
+ char checkpoint_file[FN_REFLEN];
+ my_off_t binlog_truncate_pos= 0;
+ LOG_INFO log_info;
+ const char *errmsg;
+ IO_CACHE log;
+ File file=-1;
+ Format_description_log_event fdle(BINLOG_VERSION);
+ bool is_safe= true;
+ my_off_t tmp_truncate_pos= 0, tmp_pos= 0;
+ rpl_gtid last_gtid;
+ bool last_gtid_standalone= false;
+ bool last_gtid_valid= false;
+ uint last_gtid_engines= 0;
+
+ if ((error= get_binlog_checkpoint_file(checkpoint_file)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to read latest checkpoint "
+ "binary log name.");
+ goto end;
+ }
+ sql_print_information("tc-heuristic-recover: Initialising heuristic "
+ "rollback of binary log using last checkpoint "
+ "file:%s.", checkpoint_file);
+ if ((error= find_log_pos(&log_info, checkpoint_file, 1)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to locate binary log file:%s "
+ "in index file. Error:%d", checkpoint_file, error);
+ goto end;
+ }
+ if ((file= open_binlog(&log, log_info.log_file_name, &errmsg)) < 0 ||
+ DBUG_EVALUATE_IF("fault_injection_opening_binlog", (errmsg="Unknown"),
+ FALSE))
+ {
+ error= 1;
+ sql_print_error("tc-heuristic-recover: Failed to open the binlog:%s for "
+ "recovery. Error:%s", log_info.log_file_name, errmsg);
+ goto end;
+ }
+
+
+ error= read_state_from_file();
+ if (error && error != 2)
+ {
+ sql_print_error("tc-heuristic-recover: Failed to load global gtid binlog "
+ "state from file");
+ goto end;
+ }
+ if (!fdle.is_valid())
+ {
+ error= 1;
+ sql_print_error("tc-heuristic-recover: Failed due to invalid format "
+ "description log event.");
+ goto end;
+ }
+
+ for(;;)
+ {
+ while (is_safe &&
+ (ev= Log_event::read_log_event(&log, 0, &fdle,
+ opt_master_verify_checksum)) && ev->is_valid())
+ {
+ enum Log_event_type typ= ev->get_type_code();
+ switch (typ) {
+ case XID_EVENT:
+ {
+ xid_recovery_member *member;
+
+ if ((member= (xid_recovery_member*)
+ my_hash_search(xids,
+ (uchar*) &static_cast<Xid_log_event*>(ev)->xid,
+ sizeof(my_xid))) != NULL)
+ {
+ // Possible truncate candidate validation follows as:
+ // in_engine_prepare is examined and set or left to stay
+ // either to/as 0 for to-commit mark, or non-zero for rollback
+ if (member->in_engine_prepare > last_gtid_engines)
+ {
+ sql_print_error("Error to recovery multi-engine transaction: "
+ "the number of engines %du exceeds the "
+ "respective number %du in its GTID event",
+ member->in_engine_prepare, last_gtid_engines);
+ error= 1;
+ goto end;
+ }
+ else if (member->in_engine_prepare < last_gtid_engines)
+ {
+ member->in_engine_prepare= 0; // partly committed, to complete
+ }
+ else
+ {
+ DBUG_ASSERT(binlog_truncate_pos == 0);
+
+ binlog_truncate_pos= tmp_truncate_pos; // ascertained now
+ }
+ }
+ }
+ break;
+ case GTID_LIST_EVENT:
+ {
+ Gtid_list_log_event *glev= (Gtid_list_log_event *)ev;
+ /* Initialise the binlog state from the Gtid_list event. */
+ if (binlog_truncate_pos == 0 && glev->count > 0 &&
+ rpl_global_gtid_binlog_state.load(glev->list, glev->count))
+ {
+ error= 1;
+ sql_print_error("tc-heuristic-recover: Failed to read GTID List "
+ "event.");
+ goto end;
+ }
+ }
+ break;
+ case GTID_EVENT:
+ {
+ Gtid_log_event *gev= (Gtid_log_event *)ev;
+ /* Update the binlog state with any GTID logged after Gtid_list. */
+ last_gtid.domain_id= gev->domain_id;
+ last_gtid.server_id= gev->server_id;
+ last_gtid.seq_no= gev->seq_no;
+ last_gtid_standalone=
+ ((gev->flags2 & Gtid_log_event::FL_STANDALONE) ? true : false);
+ last_gtid_valid= true;
+ last_gtid_engines= gev->extra_engines + 1;
+ if (gev->flags2 & Gtid_log_event::FL_TRANSACTIONAL &&
+ binlog_truncate_pos == 0)
+ {
+ strmake_buf(binlog_truncate_file_name, log_info.log_file_name);
+ tmp_truncate_pos= tmp_pos; // yet only a candidate
+ }
+ else
+ {
+ if (binlog_truncate_pos > 0)
+ is_safe= false;
+ }
+ }
+ break;
+ default:
+ /* Nothing. */
+ break;
+ } // End switch
+
+ if (binlog_truncate_pos == 0 && last_gtid_valid &&
+ ((last_gtid_standalone && !ev->is_part_of_group(typ)) ||
+ (!last_gtid_standalone &&
+ (typ == XID_EVENT ||
+ (typ == QUERY_EVENT &&
+ (((Query_log_event *)ev)->is_commit() ||
+ ((Query_log_event *)ev)->is_rollback()))))))
+ {
+ if ((error= rpl_global_gtid_binlog_state.update_nolock(&last_gtid,
+ false)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to update GTID within "
+ "global gtid state.");
+ goto end;
+ }
+ last_gtid_valid= false;
+ }
+ // Used to identify the last group specific end position.
+ tmp_pos= ev->log_pos;
+ delete ev;
+ ev= NULL;
+ } // End While
+ if (file >= 0)
+ {
+ end_io_cache(&log);
+ mysql_file_close(file, MYF(MY_WME));
+ file= -1;
+ }
+ if (is_safe)
+ {
+ if ((error= find_next_log(&log_info, 1)))
+ {
+ if (error != LOG_INFO_EOF)
+ {
+ sql_print_error("tc-heuristic-recover: Failed to read next binary "
+ "log during recovery.");
+ goto end;
+ }
+ else
+ {
+ error= 0; // LOG_INFO_EOF= -1 is not an error.
+ break;
+ }
+ }
+ if ((file= open_binlog(&log, log_info.log_file_name, &errmsg)) < 0)
+ {
+ error= 1;
+ sql_print_error("tc-heuristic-recover: Failed to open the binlog:%s for "
+ "recovery. Error:%s", log_info.log_file_name, errmsg);
+ goto end;
+ }
+ }
+ else
+ break;
+ } //end of for(;;)
+
+ /* complete with xids transactions in engines (regadless of is_safe) */
+ (void) ha_recover_binlog_truncate_complete(xids);
+
+ if (binlog_truncate_pos == 0)
+ goto end; // Nothing to truncate
+ else
+ {
+ DBUG_ASSERT(binlog_truncate_pos > 0);
+
+ sql_print_information("tc-heuristic-recover: Binary log to be truncated "
+ "File:%s Pos:%llu.", binlog_truncate_file_name,
+ binlog_truncate_pos);
+ }
+
+ if (is_safe)
+ {
+ if ((error= truncate_and_remove_binlogs(binlog_truncate_file_name,
+ binlog_truncate_pos)))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to trim the binary log to "
+ "File:%s Pos:%llu.", binlog_truncate_file_name,
+ binlog_truncate_pos);
+ goto end;
+ }
+ }
+ else
+ {
+ sql_print_warning("tc-heuristic-recover cannot trim the binary log to "
+ "File:%s Pos:%llu as unsafe statements (non-trans/DDL) "
+ "statements are found beyond the truncation position.",
+ binlog_truncate_file_name, binlog_truncate_pos);
+ }
+ if ((error= write_state_to_file()))
+ {
+ sql_print_error("tc-heuristic-recover: Failed to write global gtid state "
+ "to file");
+ goto end;
+ }
+
+end:
+ if (file >= 0)
+ {
+ end_io_cache(&log);
+ mysql_file_close(file, MYF(MY_WME));
+ }
+#endif
+
+ return error;
+}
+
int TC_LOG_BINLOG::open(const char *opt_name)
{
int error= 1;
@@ -9531,11 +10105,13 @@ int TC_LOG_BINLOG::open(const char *opt_name)
return 1;
}
- if (using_heuristic_recover())
+ if (using_heuristic_recover(opt_name))
{
mysql_mutex_lock(&LOCK_log);
/* generate a new binlog to mask a corrupted one */
- open(opt_name, LOG_BIN, 0, 0, WRITE_CACHE, max_binlog_size, 0, TRUE);
+ DBUG_EVALUATE_IF("skip_new_binlog_create", 0,
+ open(opt_name, LOG_BIN, 0, 0, WRITE_CACHE,
+ max_binlog_size, 0, TRUE));
mysql_mutex_unlock(&LOCK_log);
cleanup();
return 1;
diff --git a/sql/log.h b/sql/log.h
index 277e5c6f69c..00806c46cc5 100644
--- a/sql/log.h
+++ b/sql/log.h
@@ -41,7 +41,8 @@ bool stmt_has_updated_non_trans_table(const THD* thd);
class TC_LOG
{
public:
- int using_heuristic_recover();
+ int using_heuristic_recover(const char* opt_name);
+ virtual int heuristic_binlog_rollback(HASH *commit_hash) { return 0; };
TC_LOG() {}
virtual ~TC_LOG() {}
@@ -694,6 +695,7 @@ public:
void commit_checkpoint_notify(void *cookie);
int recover(LOG_INFO *linfo, const char *last_log_name, IO_CACHE *first_log,
Format_description_log_event *fdle, bool do_xa);
+ int heuristic_binlog_rollback(HASH *commit_hash);
int do_binlog_recovery(const char *opt_name, bool do_xa_recovery);
#if !defined(MYSQL_CLIENT)
@@ -794,6 +796,9 @@ public:
int purge_first_log(Relay_log_info* rli, bool included);
int set_purge_index_file_name(const char *base_file_name);
int open_purge_index_file(bool destroy);
+ bool truncate_and_remove_binlogs(const char *truncate_file,
+ my_off_t truncate_pos);
+ int get_binlog_checkpoint_file(char* checkpoint_file);
bool is_inited_purge_index_file();
int close_purge_index_file();
int clean_purge_index_file();
diff --git a/sql/log_event.cc b/sql/log_event.cc
index 146d8bd5dbe..ba254f784c6 100644
--- a/sql/log_event.cc
+++ b/sql/log_event.cc
@@ -55,6 +55,7 @@
#define my_b_write_string(A, B) my_b_write((A), (uchar*)(B), (uint) (sizeof(B) - 1))
using std::max;
+using std::min;
/**
BINLOG_CHECKSUM variable.
@@ -6703,10 +6704,13 @@ bool Binlog_checkpoint_log_event::write()
Gtid_log_event::Gtid_log_event(const char *buf, uint event_len,
const Format_description_log_event *description_event)
- : Log_event(buf, description_event), seq_no(0), commit_id(0)
+ : Log_event(buf, description_event), seq_no(0), commit_id(0),
+ flags_extra(0), extra_engines(0)
{
uint8 header_size= description_event->common_header_len;
uint8 post_header_len= description_event->post_header_len[GTID_EVENT-1];
+ const char *buf_0= buf;
+
if (event_len < header_size + post_header_len ||
post_header_len < GTID_HEADER_LEN)
return;
@@ -6717,6 +6721,7 @@ Gtid_log_event::Gtid_log_event(const char *buf, uint event_len,
domain_id= uint4korr(buf);
buf+= 4;
flags2= *buf;
+ ++buf;
if (flags2 & FL_GROUP_COMMIT_ID)
{
if (event_len < (uint)header_size + GTID_HEADER_LEN + 2)
@@ -6724,9 +6729,26 @@ Gtid_log_event::Gtid_log_event(const char *buf, uint event_len,
seq_no= 0; // So is_valid() returns false
return;
}
- ++buf;
commit_id= uint8korr(buf);
+ buf+= 8;
}
+ // the extra flags check and actions
+ if (buf - buf_0 < event_len)
+ {
+ flags_extra= *buf;
+ ++buf;
+ /* extra flags presence is identifed by non-zero byte value at this point */
+ if (flags_extra & FL_EXTRA_MULTI_ENGINE)
+ {
+ extra_engines= uint4korr(buf);
+ buf += 4;
+
+ DBUG_ASSERT(extra_engines > 0);
+ }
+ }
+ /* the '<' part of the assert corresponds to zero-padded trailing bytes */
+ DBUG_ASSERT(buf - buf_0 <= event_len);
+ DBUG_ASSERT(buf - buf_0 == event_len || buf_0[event_len - 1] == 0);
}
@@ -6738,12 +6760,13 @@ Gtid_log_event::Gtid_log_event(THD *thd_arg, uint64 seq_no_arg,
uint64 commit_id_arg)
: Log_event(thd_arg, flags_arg, is_transactional),
seq_no(seq_no_arg), commit_id(commit_id_arg), domain_id(domain_id_arg),
- flags2((standalone ? FL_STANDALONE : 0) | (commit_id_arg ? FL_GROUP_COMMIT_ID : 0))
+ flags2((standalone ? FL_STANDALONE : 0) |
+ (commit_id_arg ? FL_GROUP_COMMIT_ID : 0)),
+ flags_extra(0)
{
cache_type= Log_event::EVENT_NO_CACHE;
if (thd_arg->transaction.stmt.trans_did_wait() ||
- thd_arg->transaction.all.trans_did_wait())
- flags2|= FL_WAITED;
+ thd_arg->transaction.all.trans_did_wait())flags2|= FL_WAITED;
if (thd_arg->transaction.stmt.trans_did_ddl() ||
thd_arg->transaction.stmt.has_created_dropped_temp_table() ||
thd_arg->transaction.all.trans_did_ddl() ||
@@ -6756,6 +6779,8 @@ Gtid_log_event::Gtid_log_event(THD *thd_arg, uint64 seq_no_arg,
/* Preserve any DDL or WAITED flag in the slave's binlog. */
if (thd_arg->rgi_slave)
flags2|= (thd_arg->rgi_slave->gtid_ev_flags2 & (FL_DDL|FL_WAITED));
+ if ((extra_engines= (min<uint>(2, ha_count_rw_all(thd, NULL, true)) - 2) > 0))
+ flags_extra|= FL_EXTRA_MULTI_ENGINE;
}
@@ -6798,8 +6823,8 @@ Gtid_log_event::peek(const char *event_start, size_t event_len,
bool
Gtid_log_event::write()
{
- uchar buf[GTID_HEADER_LEN+2];
- size_t write_len;
+ uchar buf[GTID_HEADER_LEN+2 + /* flags_extra: */ 1+4];
+ size_t write_len= 13;
int8store(buf, seq_no);
int4store(buf+8, domain_id);
@@ -6809,9 +6834,15 @@ Gtid_log_event::write()
int8store(buf+13, commit_id);
write_len= GTID_HEADER_LEN + 2;
}
- else
+ if (flags & FL_EXTRA_MULTI_ENGINE)
+ {
+ int4store(buf, extra_engines);
+ write_len += 4;
+ }
+
+ if (write_len < GTID_HEADER_LEN)
{
- bzero(buf+13, GTID_HEADER_LEN-13);
+ bzero(buf+write_len, GTID_HEADER_LEN-write_len);
write_len= GTID_HEADER_LEN;
}
return write_header(write_len) ||
diff --git a/sql/log_event.h b/sql/log_event.h
index 58e1281c179..0f74eb21029 100644
--- a/sql/log_event.h
+++ b/sql/log_event.h
@@ -3193,6 +3193,12 @@ public:
uint64 commit_id;
uint32 domain_id;
uchar flags2;
+ uint flags_extra; // more flags area placed after the regular flags2's one
+ /*
+ Extra (to binlog and a "base" engine) recoverable engines participating
+ in the transaction. Zero, when none.
+ */
+ uint extra_engines;
/* Flags2. */
@@ -3221,6 +3227,14 @@ public:
/* FL_DDL is set for event group containing DDL. */
static const uchar FL_DDL= 32;
+ /* Flags_extra. */
+
+ /*
+ FL_EXTRA_MULTI_ENGINE is set for event group comprising a transaction
+ involving multiple storage engines.
+ */
+ static const uchar FL_EXTRA_MULTI_ENGINE= 1;
+
#ifdef MYSQL_SERVER
Gtid_log_event(THD *thd_arg, uint64 seq_no, uint32 domain_id, bool standalone,
uint16 flags, bool is_transactional, uint64 commit_id);
diff --git a/sql/mysqld.cc b/sql/mysqld.cc
index 8d00b5af948..0fa0419c880 100644
--- a/sql/mysqld.cc
+++ b/sql/mysqld.cc
@@ -288,7 +288,7 @@ const char *show_comp_option_name[]= {"YES", "NO", "DISABLED"};
static const char *tc_heuristic_recover_names[]=
{
- "OFF", "COMMIT", "ROLLBACK", NullS
+ "OFF", "COMMIT", "ROLLBACK", "BINLOG_TRUNCATE", NullS
};
static TYPELIB tc_heuristic_recover_typelib=
{
diff --git a/sql/sql_class.h b/sql/sql_class.h
index 7ca3896a69d..ca6990d53cc 100644
--- a/sql/sql_class.h
+++ b/sql/sql_class.h
@@ -187,6 +187,7 @@ public:
#define TC_HEURISTIC_RECOVER_COMMIT 1
#define TC_HEURISTIC_RECOVER_ROLLBACK 2
+#define TC_RECOVER_BINLOG_TRUNCATE 3
extern ulong tc_heuristic_recover;
typedef struct st_user_var_events