summaryrefslogtreecommitdiff
path: root/sql/log.cc
diff options
context:
space:
mode:
Diffstat (limited to 'sql/log.cc')
-rw-r--r--sql/log.cc1170
1 files changed, 878 insertions, 292 deletions
diff --git a/sql/log.cc b/sql/log.cc
index c659a3ede2e..336e49b02da 100644
--- a/sql/log.cc
+++ b/sql/log.cc
@@ -25,7 +25,6 @@
#include "mysql_priv.h"
#include "sql_acl.h"
#include "sql_repl.h"
-#include "ha_innodb.h" // necessary to cut the binlog when crash recovery
#include <my_dir.h>
#include <stdarg.h>
@@ -40,23 +39,235 @@ ulong sync_binlog_counter= 0;
static bool test_if_number(const char *str,
long *res, bool allow_wildcards);
+static int binlog_close_connection(THD *thd);
+static int binlog_savepoint_set(THD *thd, void *sv);
+static int binlog_savepoint_rollback(THD *thd, void *sv);
+static int binlog_commit(THD *thd, bool all);
+static int binlog_rollback(THD *thd, bool all);
+static int binlog_prepare(THD *thd, bool all);
+
+static handlerton binlog_hton = {
+ 0,
+ sizeof(my_off_t), /* savepoint size = binlog offset */
+ binlog_close_connection,
+ binlog_savepoint_set,
+ binlog_savepoint_rollback,
+ NULL, /* savepoint_release */
+ binlog_commit,
+ binlog_rollback,
+ binlog_prepare,
+ NULL, /* recover */
+ NULL, /* commit_by_xid */
+ NULL /* rollback_by_xid */
+};
+
+/*
+ this function is mostly a placeholder.
+ conceptually, binlog initialization (now mostly done in MYSQL_LOG::open)
+ should be moved here.
+*/
+
+handlerton *binlog_init()
+{
+ return &binlog_hton;
+}
+
+static int binlog_close_connection(THD *thd)
+{
+ IO_CACHE *trans_log= (IO_CACHE*)thd->ha_data[binlog_hton.slot];
+ close_cached_file(trans_log);
+ my_free((gptr)trans_log, MYF(0));
+ return 0;
+}
+
+static inline void binlog_cleanup_trans(IO_CACHE *trans_log)
+{
+ statistic_increment(binlog_cache_use, &LOCK_status);
+ if (trans_log->disk_writes != 0)
+ {
+ statistic_increment(binlog_cache_disk_use, &LOCK_status);
+ trans_log->disk_writes= 0;
+ }
+ reinit_io_cache(trans_log, WRITE_CACHE, (my_off_t) 0, 0, 1); // cannot fail
+ trans_log->end_of_file= max_binlog_cache_size;
+}
+
+static int binlog_prepare(THD *thd, bool all)
+{
+ /*
+ do nothing.
+ just pretend we can do 2pc, so that MySQL won't
+ switch to 1pc.
+ real work will be done in MYSQL_LOG::log()
+ */
+ return 0;
+}
+
+static int binlog_commit(THD *thd, bool all)
+{
+ int error;
+ IO_CACHE *trans_log= (IO_CACHE*)thd->ha_data[binlog_hton.slot];
+ DBUG_ENTER("binlog_commit");
+ DBUG_ASSERT(mysql_bin_log.is_open() &&
+ (all || !(thd->options & OPTION_NOT_AUTOCOMMIT)));
+
+ if (!my_b_tell(trans_log))
+ {
+ // we're here because trans_log was flushed in MYSQL_LOG::log()
+ DBUG_RETURN(0);
+ }
+
+ /* Update the binary log as we have cached some queries */
+ error= mysql_bin_log.write(thd, trans_log, 1);
+ binlog_cleanup_trans(trans_log);
+ DBUG_RETURN(error);
+}
+
+static int binlog_rollback(THD *thd, bool all)
+{
+ int error=0;
+ IO_CACHE *trans_log= (IO_CACHE*)thd->ha_data[binlog_hton.slot];
+ DBUG_ENTER("binlog_rollback");
+ /*
+ first two conditions here are guaranteed - see trans_register_ha()
+ call below. The third one must be true. If it is not, we're registering
+ unnecessary, doing extra work. The cause should be found and eliminated
+ */
+ DBUG_ASSERT(all && mysql_bin_log.is_open() && my_b_tell(trans_log));
+ /*
+ Update the binary log with a BEGIN/ROLLBACK block if we have
+ cached some queries and we updated some non-transactional
+ table. Such cases should be rare (updating a
+ non-transactional table inside a transaction...)
+ */
+ if (unlikely(thd->options & OPTION_STATUS_NO_TRANS_UPDATE))
+ error= mysql_bin_log.write(thd, trans_log, 0);
+ binlog_cleanup_trans(trans_log);
+ DBUG_RETURN(error);
+}
+
+/*
+ NOTE: how do we handle this (unlikely but legal) case:
+ [transaction] + [update to non-trans table] + [rollback to savepoint] ?
+ The problem occurs when a savepoint is before the update to the
+ non-transactional table. Then when there's a rollback to the savepoint, if we
+ simply truncate the binlog cache, we lose the part of the binlog cache where
+ the update is. If we want to not lose it, we need to write the SAVEPOINT
+ command and the ROLLBACK TO SAVEPOINT command to the binlog cache. The latter
+ is easy: it's just write at the end of the binlog cache, but the former
+ should be *inserted* to the place where the user called SAVEPOINT. The
+ solution is that when the user calls SAVEPOINT, we write it to the binlog
+ cache (so no need to later insert it). As transactions are never intermixed
+ in the binary log (i.e. they are serialized), we won't have conflicts with
+ savepoint names when using mysqlbinlog or in the slave SQL thread.
+ Then when ROLLBACK TO SAVEPOINT is called, if we updated some
+ non-transactional table, we don't truncate the binlog cache but instead write
+ ROLLBACK TO SAVEPOINT to it; otherwise we truncate the binlog cache (which
+ will chop the SAVEPOINT command from the binlog cache, which is good as in
+ that case there is no need to have it in the binlog).
+*/
+
+static int binlog_savepoint_set(THD *thd, void *sv)
+{
+ IO_CACHE *trans_log= (IO_CACHE*)thd->ha_data[binlog_hton.slot];
+ DBUG_ENTER("binlog_savepoint_set");
+ DBUG_ASSERT(mysql_bin_log.is_open() && my_b_tell(trans_log));
+
+ *(my_off_t *)sv= my_b_tell(trans_log);
+ /* Write it to the binary log */
+ Query_log_event qinfo(thd, thd->query, thd->query_length, TRUE, FALSE);
+ DBUG_RETURN(mysql_bin_log.write(&qinfo));
+}
+
+static int binlog_savepoint_rollback(THD *thd, void *sv)
+{
+ IO_CACHE *trans_log= (IO_CACHE*)thd->ha_data[binlog_hton.slot];
+ DBUG_ENTER("binlog_savepoint_rollback");
+ DBUG_ASSERT(mysql_bin_log.is_open() && my_b_tell(trans_log));
+
+ /*
+ Write ROLLBACK TO SAVEPOINT to the binlog cache if we have updated some
+ non-transactional table. Otherwise, truncate the binlog cache starting
+ from the SAVEPOINT command.
+ */
+ if (unlikely(thd->options & OPTION_STATUS_NO_TRANS_UPDATE))
+ {
+ Query_log_event qinfo(thd, thd->query, thd->query_length, TRUE, FALSE);
+ DBUG_RETURN(mysql_bin_log.write(&qinfo));
+ }
+ reinit_io_cache(trans_log, WRITE_CACHE, *(my_off_t *)sv, 0, 0);
+ DBUG_RETURN(0);
+}
+
+int check_binlog_magic(IO_CACHE* log, const char** errmsg)
+{
+ char magic[4];
+ DBUG_ASSERT(my_b_tell(log) == 0);
+
+ if (my_b_read(log, (byte*) magic, sizeof(magic)))
+ {
+ *errmsg = "I/O error reading the header from the binary log";
+ sql_print_error("%s, errno=%d, io cache code=%d", *errmsg, my_errno,
+ log->error);
+ return 1;
+ }
+ if (memcmp(magic, BINLOG_MAGIC, sizeof(magic)))
+ {
+ *errmsg = "Binlog has bad magic number; It's not a binary log file that can be used by this version of MySQL";
+ return 1;
+ }
+ return 0;
+}
+
+File open_binlog(IO_CACHE *log, const char *log_file_name, const char **errmsg)
+{
+ File file;
+ DBUG_ENTER("open_binlog");
+
+ if ((file = my_open(log_file_name, O_RDONLY | O_BINARY, MYF(MY_WME))) < 0)
+ {
+ sql_print_error("Failed to open log (file '%s', errno %d)",
+ log_file_name, my_errno);
+ *errmsg = "Could not open log file";
+ goto err;
+ }
+ if (init_io_cache(log, file, IO_SIZE*2, READ_CACHE, 0, 0,
+ MYF(MY_WME|MY_DONT_CHECK_FILESIZE)))
+ {
+ sql_print_error("Failed to create a cache on log (file '%s')",
+ log_file_name);
+ *errmsg = "Could not open log file";
+ goto err;
+ }
+ if (check_binlog_magic(log,errmsg))
+ goto err;
+ DBUG_RETURN(file);
+
+err:
+ if (file >= 0)
+ {
+ my_close(file,MYF(0));
+ end_io_cache(log);
+ }
+ DBUG_RETURN(-1);
+}
#ifdef __NT__
static int eventSource = 0;
-void setup_windows_event_source()
+void setup_windows_event_source()
{
- HKEY hRegKey= NULL;
+ HKEY hRegKey= NULL;
DWORD dwError= 0;
TCHAR szPath[MAX_PATH];
DWORD dwTypes;
-
+
if (eventSource) // Ensure that we are only called once
return;
eventSource= 1;
// Create the event source registry key
- dwError= RegCreateKey(HKEY_LOCAL_MACHINE,
+ dwError= RegCreateKey(HKEY_LOCAL_MACHINE,
"SYSTEM\\CurrentControlSet\\Services\\EventLog\\Application\\MySQL",
&hRegKey);
@@ -64,9 +275,8 @@ void setup_windows_event_source()
GetModuleFileName(NULL, szPath, MAX_PATH);
/* Register EventMessageFile */
- dwError = RegSetValueEx(hRegKey, "EventMessageFile", 0, REG_EXPAND_SZ,
+ dwError = RegSetValueEx(hRegKey, "EventMessageFile", 0, REG_EXPAND_SZ,
(PBYTE) szPath, strlen(szPath)+1);
-
/* Register supported event types */
dwTypes= (EVENTLOG_ERROR_TYPE | EVENTLOG_WARNING_TYPE |
@@ -129,14 +339,14 @@ static int find_uniq_filename(char *name)
MYSQL_LOG::MYSQL_LOG()
:bytes_written(0), last_time(0), query_start(0), name(0),
file_id(1), open_count(1), log_type(LOG_CLOSED), write_error(0), inited(0),
- need_start_event(1), description_event_for_exec(0),
+ need_start_event(1), prepared_xids(0), description_event_for_exec(0),
description_event_for_queue(0)
{
/*
We don't want to initialize LOCK_Log here as such initialization depends on
safe_mutex (when using safe_mutex) which depends on MY_INIT(), which is
called only in main(). Doing initialization here would make it happen
- before main().
+ before main().
*/
index_file_name[0] = 0;
bzero((char*) &log_file,sizeof(log_file));
@@ -157,7 +367,7 @@ void MYSQL_LOG::cleanup()
if (inited)
{
inited= 0;
- close(LOG_CLOSE_INDEX);
+ close(LOG_CLOSE_INDEX|LOG_CLOSE_STOP_EVENT);
delete description_event_for_queue;
delete description_event_for_exec;
(void) pthread_mutex_destroy(&LOCK_log);
@@ -234,7 +444,7 @@ bool MYSQL_LOG::open(const char *log_name, enum_log_type log_type_arg,
{
char buff[512];
File file= -1, index_file_nr= -1;
- int open_flags = O_CREAT | O_APPEND | O_BINARY;
+ int open_flags = O_CREAT | O_BINARY;
DBUG_ENTER("MYSQL_LOG::open");
DBUG_PRINT("enter",("log_type: %d",(int) log_type));
@@ -251,9 +461,9 @@ bool MYSQL_LOG::open(const char *log_name, enum_log_type log_type_arg,
goto err;
if (io_cache_type == SEQ_READ_APPEND)
- open_flags |= O_RDWR;
+ open_flags |= O_RDWR | O_APPEND;
else
- open_flags |= O_WRONLY;
+ open_flags |= O_WRONLY | (log_type == LOG_BIN ? 0 : O_APPEND);
db[0]=0;
open_count++;
@@ -627,7 +837,7 @@ int MYSQL_LOG::find_next_log(LOG_INFO* linfo, bool need_lock)
error = !index_file.error ? LOG_INFO_EOF : LOG_INFO_IO;
goto err;
}
- fname[length-1]=0; // kill /n
+ fname[length-1]=0; // kill \n
linfo->index_file_offset = my_b_tell(&index_file);
err:
@@ -683,7 +893,7 @@ bool MYSQL_LOG::reset_logs(THD* thd)
error=1;
goto err;
}
-
+
for (;;)
{
my_delete(linfo.log_file_name, MYF(MY_WME));
@@ -1041,6 +1251,24 @@ void MYSQL_LOG::new_file(bool need_lock)
safe_mutex_assert_owner(&LOCK_log);
safe_mutex_assert_owner(&LOCK_index);
+ /*
+ if binlog is used as tc log, be sure all xids are "unlogged",
+ so that on recover we only need to scan one - latest - binlog file
+ for prepared xids. As this is expected to be a rare event,
+ simple wait strategy is enough. We're locking LOCK_log to be sure no
+ new Xid_log_event's are added to the log (and prepared_xids is not
+ increased), and waiting on COND_prep_xids for late threads to
+ catch up.
+ */
+ if (prepared_xids)
+ {
+ tc_log_page_waits++;
+ pthread_mutex_lock(&LOCK_prep_xids);
+ while (prepared_xids)
+ pthread_cond_wait(&COND_prep_xids, &LOCK_prep_xids);
+ pthread_mutex_unlock(&LOCK_prep_xids);
+ }
+
/* Reuse old name if not binlog and not update log */
new_name_ptr= name;
@@ -1093,8 +1321,6 @@ void MYSQL_LOG::new_file(bool need_lock)
open(old_name, save_log_type, new_name_ptr, index_file_name, io_cache_type,
no_auto_events, max_size, 1);
- if (this == &mysql_bin_log)
- report_pos_in_innodb();
my_free(old_name,MYF(0));
end:
@@ -1283,8 +1509,7 @@ inline bool sync_binlog(IO_CACHE *cache)
bool MYSQL_LOG::write(Log_event* event_info)
{
THD *thd=event_info->thd;
- bool called_handler_commit=0;
- bool error=0;
+ bool error=1;
bool should_rotate = 0;
DBUG_ENTER("MYSQL_LOG::write(event)");
@@ -1295,26 +1520,10 @@ bool MYSQL_LOG::write(Log_event* event_info)
mostly called if is_open() *was* true a few instructions before, but it
could have changed since.
*/
- if (is_open())
+ if (likely(is_open()))
{
const char *local_db= event_info->get_db();
IO_CACHE *file= &log_file;
-#ifdef USING_TRANSACTIONS
- /*
- Should we write to the binlog cache or to the binlog on disk?
- Write to the binlog cache if:
- - it is already not empty (meaning we're in a transaction; note that the
- present event could be about a non-transactional table, but still we need
- to write to the binlog cache in that case to handle updates to mixed
- trans/non-trans table types the best possible in binlogging)
- - or if the event asks for it (cache_stmt == true).
- */
- if (opt_using_transactions &&
- (event_info->get_cache_stmt() ||
- (thd && my_b_tell(&thd->transaction.trans_log))))
- file= &thd->transaction.trans_log;
-#endif
- DBUG_PRINT("info",("event type=%d",event_info->get_type_code()));
#ifdef HAVE_REPLICATION
/*
In the future we need to add to the following if tests like
@@ -1330,7 +1539,46 @@ bool MYSQL_LOG::write(Log_event* event_info)
}
#endif /* HAVE_REPLICATION */
- error=1;
+#ifdef USING_TRANSACTIONS
+ /*
+ Should we write to the binlog cache or to the binlog on disk?
+ Write to the binlog cache if:
+ - it is already not empty (meaning we're in a transaction; note that the
+ present event could be about a non-transactional table, but still we need
+ to write to the binlog cache in that case to handle updates to mixed
+ trans/non-trans table types the best possible in binlogging)
+ - or if the event asks for it (cache_stmt == true).
+ */
+ if (opt_using_transactions && thd)
+ {
+ IO_CACHE *trans_log= (IO_CACHE*)thd->ha_data[binlog_hton.slot];
+
+ if (event_info->get_cache_stmt())
+ {
+ if (!trans_log)
+ {
+ thd->ha_data[binlog_hton.slot]= trans_log= (IO_CACHE *)
+ my_malloc(sizeof(IO_CACHE), MYF(MY_ZEROFILL));
+ if (!trans_log || open_cached_file(trans_log, mysql_tmpdir, LOG_PREFIX,
+ binlog_cache_size, MYF(MY_WME)))
+ {
+ my_free((gptr)trans_log, MYF(MY_ALLOW_ZERO_PTR));
+ thd->ha_data[binlog_hton.slot]= trans_log= 0;
+ goto err;
+ }
+ trans_log->end_of_file= max_binlog_cache_size;
+ trans_register_ha(thd, TRUE, &binlog_hton);
+ }
+ else if (!my_b_tell(trans_log))
+ trans_register_ha(thd, TRUE, &binlog_hton);
+ file= trans_log;
+ }
+ else if (trans_log && my_b_tell(trans_log))
+ file= trans_log;
+ }
+#endif
+ DBUG_PRINT("info",("event type=%d",event_info->get_type_code()));
+
/*
No check for auto events flag here - this write method should
never be called if auto-events are enabled
@@ -1423,17 +1671,6 @@ COLLATION_CONNECTION=%u,COLLATION_DATABASE=%u,COLLATION_SERVER=%u",
goto err;
}
}
-#ifdef TO_BE_REMOVED
- if (thd->variables.convert_set)
- {
- char buf[256], *p;
- p= strmov(strmov(buf, "SET CHARACTER SET "),
- thd->variables.convert_set->name);
- Query_log_event e(thd, buf, (ulong) (p - buf), 0, FALSE);
- if (e.write(file))
- goto err;
- }
-#endif
}
/* Write the SQL command */
@@ -1441,71 +1678,12 @@ COLLATION_CONNECTION=%u,COLLATION_DATABASE=%u,COLLATION_SERVER=%u",
if (event_info->write(file))
goto err;
- /*
- Tell for transactional table handlers up to which position in the
- binlog file we wrote. The table handler can store this info, and
- after crash recovery print for the user the offset of the last
- transactions which were recovered. Actually, we must also call
- the table handler commit here, protected by the LOCK_log mutex,
- because otherwise the transactions may end up in a different order
- in the table handler log!
-
- Note that we will NOT call ha_report_binlog_offset_and_commit() if
- there are binlog events cached in the transaction cache. That is
- because then the log event which we write to the binlog here is
- not a transactional event. In versions < 4.0.13 before this fix this
- caused an InnoDB transaction to be committed if in the middle there
- was a MyISAM event!
- */
-
if (file == &log_file) // we are writing to the real log (disk)
{
if (flush_io_cache(file) || sync_binlog(file))
goto err;
- if (opt_using_transactions &&
- !(thd->options & (OPTION_NOT_AUTOCOMMIT | OPTION_BEGIN)))
- {
- /*
- LOAD DATA INFILE in AUTOCOMMIT=1 mode writes to the binlog
- chunks also before it is successfully completed. We only report
- the binlog write and do the commit inside the transactional table
- handler if the log event type is appropriate.
- */
-
- if (event_info->get_type_code() == QUERY_EVENT ||
- event_info->get_type_code() == EXEC_LOAD_EVENT)
- {
-#ifndef DBUG_OFF
- if (unlikely(opt_crash_binlog_innodb))
- {
- /*
- This option is for use in rpl_crash_binlog_innodb.test.
- 1st we want to verify that Binlog_dump thread cannot send the
- event now (because of LOCK_log): we here tell the Binlog_dump
- thread to wake up, sleep for the slave to have time to possibly
- receive data from the master (it should not), and then crash.
- 2nd we want to verify that at crash recovery the rolled back
- event is cut from the binlog.
- */
- if (!(--opt_crash_binlog_innodb))
- {
- signal_update();
- sleep(2);
- fprintf(stderr,"This is a normal crash because of"
- " --crash-binlog-innodb\n");
- assert(0);
- }
- DBUG_PRINT("info",("opt_crash_binlog_innodb: %d",
- opt_crash_binlog_innodb));
- }
-#endif
- error = ha_report_binlog_offset_and_commit(thd, log_file_name,
- file->pos_in_file);
- called_handler_commit=1;
- }
- }
- /* We wrote to the real log, check automatic rotation; */
+ /* check automatic rotation; */
DBUG_PRINT("info",("max_size: %lu",max_size));
should_rotate= (my_b_tell(file) >= (my_off_t) max_size);
}
@@ -1524,7 +1702,7 @@ err:
signal_update();
if (should_rotate)
{
- pthread_mutex_lock(&LOCK_index);
+ pthread_mutex_lock(&LOCK_index);
new_file(0); // inside mutex
pthread_mutex_unlock(&LOCK_index);
}
@@ -1532,15 +1710,6 @@ err:
pthread_mutex_unlock(&LOCK_log);
- /*
- Flush the transactional handler log file now that we have released
- LOCK_log; the flush is placed here to eliminate the bottleneck on the
- group commit
- */
-
- if (called_handler_commit)
- ha_commit_complete(thd);
-
#ifdef HAVE_REPLICATION
if (should_rotate && expire_logs_days)
{
@@ -1568,9 +1737,9 @@ uint MYSQL_LOG::next_file_id()
SYNOPSIS
write()
- thd
+ thd
cache The cache to copy to the binlog
- commit_or_rollback If true, will write "COMMIT" in the end, if false will
+ is_commit If true, will write "COMMIT" in the end, if false will
write "ROLLBACK".
NOTE
@@ -1578,6 +1747,10 @@ uint MYSQL_LOG::next_file_id()
- The thing in the cache is always a complete transaction
- 'cache' needs to be reinitialized after this functions returns.
+ TODO
+ fix it to become atomic - either the complete cache is added to binlog
+ or nothing (other storage engines rely on this, doing a ROLLBACK)
+
IMPLEMENTATION
- To support transaction over replication, we wrap the transaction
with BEGIN/COMMIT or BEGIN/ROLLBACK in the binary log.
@@ -1586,12 +1759,12 @@ uint MYSQL_LOG::next_file_id()
same updates are run on the slave.
*/
-bool MYSQL_LOG::write(THD *thd, IO_CACHE *cache, bool commit_or_rollback)
+bool MYSQL_LOG::write(THD *thd, IO_CACHE *cache, bool is_commit)
{
VOID(pthread_mutex_lock(&LOCK_log));
DBUG_ENTER("MYSQL_LOG::write(cache");
- if (is_open()) // Should always be true
+ if (likely(is_open())) // Should always be true
{
uint length;
@@ -1601,11 +1774,11 @@ bool MYSQL_LOG::write(THD *thd, IO_CACHE *cache, bool commit_or_rollback)
AUTOCOMMIT=1, then MySQL immediately writes each SQL statement to
the binlog when the statement has been completed. No need to add
"BEGIN" ... "COMMIT" around such statements. Otherwise, MySQL uses
- thd->transaction.trans_log to cache the SQL statements until the
- explicit commit, and at the commit writes the contents in .trans_log
- to the binlog.
+ trans_log (that is thd->ha_data[binlog_hton.slot]) to cache
+ the SQL statements until the explicit commit, and at the commit writes
+ the contents in trans_log to the binlog.
- We write the "BEGIN" mark first in the buffer (.trans_log) where we
+ We write the "BEGIN" mark first in the buffer (trans_log) where we
store the SQL statements for a transaction. At the transaction commit
we will add the "COMMIT mark and write the buffer to the binlog.
*/
@@ -1648,8 +1821,8 @@ bool MYSQL_LOG::write(THD *thd, IO_CACHE *cache, bool commit_or_rollback)
{
Query_log_event qinfo(thd,
- commit_or_rollback ? "COMMIT" : "ROLLBACK",
- commit_or_rollback ? 6 : 8,
+ is_commit ? "COMMIT" : "ROLLBACK",
+ is_commit ? 6 : 8,
TRUE, FALSE);
qinfo.error_code= 0;
if (qinfo.write(&log_file) || flush_io_cache(&log_file) ||
@@ -1662,25 +1835,6 @@ bool MYSQL_LOG::write(THD *thd, IO_CACHE *cache, bool commit_or_rollback)
write_error=1; // Don't give more errors
goto err;
}
-#ifndef DBUG_OFF
- if (unlikely(opt_crash_binlog_innodb))
- {
- /* see the previous MYSQL_LOG::write() method for a comment */
- if (!(--opt_crash_binlog_innodb))
- {
- signal_update();
- sleep(2);
- fprintf(stderr, "This is a normal crash because of"
- " --crash-binlog-innodb\n");
- assert(0);
- }
- DBUG_PRINT("info",("opt_crash_binlog_innodb: %d",
- opt_crash_binlog_innodb));
- }
-#endif
- if ((ha_report_binlog_offset_and_commit(thd, log_file_name,
- log_file.pos_in_file)))
- goto err;
signal_update();
DBUG_PRINT("info",("max_size: %lu",max_size));
if (my_b_tell(&log_file) >= (my_off_t) max_size)
@@ -1693,12 +1847,6 @@ bool MYSQL_LOG::write(THD *thd, IO_CACHE *cache, bool commit_or_rollback)
}
VOID(pthread_mutex_unlock(&LOCK_log));
- /* Flush the transactional handler log file now that we have released
- LOCK_log; the flush is placed here to eliminate the bottleneck on the
- group commit */
-
- ha_commit_complete(thd);
-
DBUG_RETURN(0);
err:
@@ -1903,6 +2051,16 @@ void MYSQL_LOG::close(uint exiting)
}
#endif /* HAVE_REPLICATION */
end_io_cache(&log_file);
+
+ /* don't pwrite in a file opened with O_APPEND - it doesn't work */
+ if (log_file.type == WRITE_CACHE)
+ {
+ my_off_t offset= BIN_LOG_HEADER_SIZE + FLAGS_OFFSET;
+ char flags=LOG_EVENT_BINLOG_CLOSED_F;
+ my_pwrite(log_file.file, &flags, 1,
+ BIN_LOG_HEADER_SIZE + FLAGS_OFFSET, MYF(0));
+ }
+
if (my_close(log_file.file,MYF(0)) < 0 && ! write_error)
{
write_error=1;
@@ -2085,145 +2243,6 @@ bool flush_error_log()
return result;
}
-
-/*
- If the server has InnoDB on, and InnoDB has published the position of the
- last committed transaction (which happens only if a crash recovery occured at
- this startup) then truncate the previous binary log at the position given by
- InnoDB. If binlog is shorter than the position, print a message to the error
- log.
-
- SYNOPSIS
- cut_spurious_tail()
-
- RETURN VALUES
- 1 Error
- 0 Ok
-*/
-
-bool MYSQL_LOG::cut_spurious_tail()
-{
- int error= 0;
- DBUG_ENTER("cut_spurious_tail");
-
-#ifdef HAVE_INNOBASE_DB
- if (have_innodb != SHOW_OPTION_YES)
- DBUG_RETURN(0);
- /*
- This is the place where we use information from InnoDB to cut the
- binlog.
- */
- char *name= ha_innobase::get_mysql_bin_log_name();
- ulonglong pos= ha_innobase::get_mysql_bin_log_pos();
- ulonglong actual_size;
- char llbuf1[22], llbuf2[22];
-
- if (name[0] == 0 || pos == ULONGLONG_MAX)
- {
- DBUG_PRINT("info", ("InnoDB has not set binlog info"));
- DBUG_RETURN(0);
- }
- /* The binlog given by InnoDB normally is never an active binlog */
- if (is_open() && is_active(name))
- {
- sql_print_error("Warning: after InnoDB crash recovery, InnoDB says that "
- "the binary log of the previous run has the same name "
- "'%s' as the current one; this is likely to be abnormal.",
- name);
- DBUG_RETURN(1);
- }
- sql_print_error("After InnoDB crash recovery, checking if the binary log "
- "'%s' contains rolled back transactions which must be "
- "removed from it...", name);
- /* If we have a too long binlog, cut. If too short, print error */
- int fd= my_open(name, O_EXCL | O_APPEND | O_BINARY | O_WRONLY, MYF(MY_WME));
- if (fd < 0)
- {
- int save_errno= my_errno;
- sql_print_error("Could not open the binary log '%s' for truncation.",
- name);
- if (save_errno != ENOENT)
- sql_print_error("The binary log '%s' should not be used for "
- "replication.", name);
- DBUG_RETURN(1);
- }
-
- if (pos > (actual_size= my_seek(fd, 0L, MY_SEEK_END, MYF(MY_WME))))
- {
- /*
- Note that when we have MyISAM rollback this error message should be
- reconsidered.
- */
- sql_print_error("The binary log '%s' is shorter than its expected size "
- "(actual: %s, expected: %s) so it misses at least one "
- "committed transaction; so it should not be used for "
- "replication or point-in-time recovery. You would need "
- "to restart slaves from a fresh master's data "
- "snapshot ",
- name, llstr(actual_size, llbuf1),
- llstr(pos, llbuf2));
- error= 1;
- goto err;
- }
- if (pos < actual_size)
- {
- sql_print_error("The binary log '%s' is bigger than its expected size "
- "(actual: %s, expected: %s) so it contains a rolled back "
- "transaction; now truncating that.", name,
- llstr(actual_size, llbuf1), llstr(pos, llbuf2));
- /*
- As on some OS, my_chsize() can only pad with 0s instead of really
- truncating. Then mysqlbinlog (and Binlog_dump thread) will error on
- these zeroes. This is annoying, but not more (you just need to manually
- switch replication to the next binlog). Fortunately, in my_chsize.c, it
- says that all modern machines support real ftruncate().
-
- */
- if ((error= my_chsize(fd, pos, 0, MYF(MY_WME))))
- goto err;
- }
-err:
- if (my_close(fd, MYF(MY_WME)))
- error= 1;
-#endif
- DBUG_RETURN(error);
-}
-
-
-/*
- If the server has InnoDB on, store the binlog name and position into
- InnoDB. This function is used every time we create a new binlog.
-
- SYNOPSIS
- report_pos_in_innodb()
-
- NOTES
- This cannot simply be done in MYSQL_LOG::open(), because when we create
- the first binlog at startup, we have not called ha_init() yet so we cannot
- write into InnoDB yet.
-
- RETURN VALUES
- 1 Error
- 0 Ok
-*/
-
-void MYSQL_LOG::report_pos_in_innodb()
-{
- DBUG_ENTER("report_pos_in_innodb");
-#ifdef HAVE_INNOBASE_DB
- if (is_open() && have_innodb == SHOW_OPTION_YES)
- {
- DBUG_PRINT("info", ("Reporting binlog info into InnoDB - "
- "name: '%s' position: %d",
- log_file_name, my_b_tell(&log_file)));
- innobase_store_binlog_offset_and_flush_log(log_file_name,
- my_b_tell(&log_file));
- }
-#endif
- DBUG_VOID_RETURN;
-}
-
-
void MYSQL_LOG::signal_update()
{
DBUG_ENTER("MYSQL_LOG::signal_update");
@@ -2354,3 +2373,570 @@ void sql_print_information(const char *format, ...)
DBUG_VOID_RETURN;
}
+
+/********* transaction coordinator log for 2pc - mmap() based solution *******/
+
+/*
+ the log consists of a file, mmapped to a memory.
+ file is divided on pages of tc_log_page_size size.
+ (usable size of the first page is smaller because of log header)
+ there's PAGE control structure for each page
+ each page (or rather PAGE control structure) can be in one of three
+ states - active, in_sync, pool.
+ there could be only one page in active or in_sync states,
+ but many in pool - pool is fifo queue.
+ usual lifecycle of a page is pool->active->in_sync->pool
+ "active" page - is a page where new xid's are logged.
+ the page stays active as long as in_sync slot is taken.
+ "in_sync" page is being synced to disk. no new xid can be added to it.
+ when the sync is done the page is moved to a pool and an active page
+ becomes "in_sync".
+
+ when a xid is added to an active page, the thread of this xid waits for
+ a page's condition until the page is synced. when in_sync slot becomes vacant
+ one of these waiters is awaken to take care of syncing. it syncs the page
+ and signals all waiters that the page is synced.
+ PAGE::waiters is used to count these waiters, and a page may never
+ become active again until waiters==0 (that is all waiters from the
+ previous sync have noticed the sync was completed)
+
+ note, that the page becomes "dirty" and has to be synced only when
+ a new xid is added into it. Removing a xid from a page does not make it
+ dirty - we don't sync removals to disk.
+*/
+#define TC_LOG_HEADER_SIZE (sizeof(tc_log_magic)+1)
+
+static const char tc_log_magic[]={254, 0x23, 0x05, 0x74};
+
+uint opt_tc_log_size=TC_LOG_MIN_SIZE;
+uint tc_log_max_pages_used=0, tc_log_page_size=0,
+ tc_log_page_waits=0, tc_log_cur_pages_used=0;
+
+TC_LOG *tc_log;
+TC_LOG_MMAP tc_log_mmap;
+TC_LOG_DUMMY tc_log_dummy;
+
+int TC_LOG_MMAP::open(const char *opt_name)
+{
+ uint i;
+ bool crashed=FALSE;
+ PAGE *pg;
+
+ DBUG_ASSERT(total_ha_2pc);
+ DBUG_ASSERT(opt_name && opt_name[0]);
+
+#ifdef HAVE_GETPAGESIZE
+ tc_log_page_size=getpagesize();
+ DBUG_ASSERT(TC_LOG_PAGE_SIZE % tc_log_page_size == 0);
+#else
+ tc_log_page_size=TC_LOG_PAGE_SIZE;
+#endif
+
+ fn_format(logname,opt_name,mysql_data_home,"",MY_UNPACK_FILENAME);
+ fd=my_open(logname, O_RDWR, MYF(0)); // TODO use O_CREAT and check len==0 ?
+ if (fd == -1)
+ {
+ if (using_heuristic_recover())
+ return 1;
+ fd=my_open(logname, O_RDWR|O_CREAT, MYF(MY_WME));
+ if (fd == -1)
+ goto err;
+ inited=1;
+ file_length=opt_tc_log_size;
+ if (my_chsize(fd, file_length, 0, MYF(MY_WME)))
+ goto err;
+ }
+ else
+ {
+ inited=1;
+ crashed=TRUE;
+ if (tc_heuristic_recover)
+ {
+ sql_print_error("Cannot perform automatic crash recovery when "
+ "--tc-heuristic-recover is used");
+ goto err;
+ }
+ file_length = my_seek(fd, 0L, MY_SEEK_END, MYF(MY_WME+MY_FAE));
+ if (file_length == MY_FILEPOS_ERROR || file_length % tc_log_page_size)
+ goto err;
+ }
+
+ data=(uchar *)my_mmap(0, file_length, PROT_READ|PROT_WRITE,
+ MAP_NOSYNC|MAP_SHARED, fd, 0);
+ if (data == MAP_FAILED)
+ {
+ my_errno=errno;
+ goto err;
+ }
+ inited=2;
+
+ npages=file_length/tc_log_page_size;
+ DBUG_ASSERT(npages >= 3); // to guarantee non-empty pool
+ if (!(pages=(PAGE *)my_malloc(npages*sizeof(PAGE), MYF(MY_WME|MY_ZEROFILL))))
+ goto err;
+ inited=3;
+ for (pg=pages, i=0; i < npages; i++, pg++)
+ {
+ pg->next=pg+1;
+ pg->waiters=0;
+ pg->state=POOL;
+ pthread_mutex_init(&pg->lock, MY_MUTEX_INIT_FAST);
+ pthread_cond_init (&pg->cond, 0);
+ pg->start=(my_xid *)(data + i*tc_log_page_size);
+ pg->end=(my_xid *)(pg->start + tc_log_page_size);
+ pg->size=pg->free=tc_log_page_size/sizeof(my_xid);
+ }
+ pages[0].start=(my_xid *)(data+TC_LOG_HEADER_SIZE);
+ pages[0].size=pages[0].free=
+ (tc_log_page_size-TC_LOG_HEADER_SIZE)/sizeof(my_xid);
+ pages[npages-1].next=0;
+ inited=4;
+
+ if (crashed)
+ {
+ sql_print_error("Recovering after a crash");
+ if (recover())
+ goto err;
+ }
+ memcpy(data, tc_log_magic, sizeof(tc_log_magic));
+ data[sizeof(tc_log_magic)]= total_ha_2pc;
+ my_msync(fd, data, tc_log_page_size, MS_SYNC);
+ inited=5;
+
+ pthread_mutex_init(&LOCK_sync, MY_MUTEX_INIT_FAST);
+ pthread_mutex_init(&LOCK_active, MY_MUTEX_INIT_FAST);
+ pthread_mutex_init(&LOCK_pool, MY_MUTEX_INIT_FAST);
+ pthread_cond_init(&COND_active, 0);
+ pthread_cond_init(&COND_pool, 0);
+
+ inited=6;
+
+ in_sync= 0;
+ active=pages;
+ pool=pages+1;
+ pool_last=pages+npages-1;
+
+ return 0;
+
+err:
+ close();
+ return 1;
+}
+
+/*
+ two strategies here:
+ 1. take the first from the pool
+ 2. if there're waiters - take the one with the most free space
+
+ TODO page merging. try to allocate adjacent page first,
+ so that they can be flushed both in one sync
+*/
+void TC_LOG_MMAP::get_active_from_pool()
+{
+ PAGE **p, **best_p=0;
+ int best_free;
+
+ do
+ {
+ best_p= p= &pool;
+ if ((*p)->waiters == 0)
+ break;
+
+ best_free=0;
+ for (p=&(*p)->next; *p; p=&(*p)->next)
+ {
+ if ((*p)->waiters == 0 && (*p)->free > best_free)
+ {
+ best_free=(*p)->free;
+ best_p=p;
+ }
+ }
+ }
+ while ((*best_p == 0 || best_free == 0) && overflow());
+
+ active=*best_p;
+ if (active->free == active->size)
+ {
+ tc_log_cur_pages_used++;
+ set_if_bigger(tc_log_max_pages_used, tc_log_cur_pages_used);
+ }
+
+ if ((*best_p)->next)
+ *best_p=(*best_p)->next;
+ else
+ pool_last=*best_p;
+}
+
+int TC_LOG_MMAP::overflow()
+{
+ /*
+ simple overflow handling - just wait
+ TODO perhaps, increase log size ?
+ let's check the behaviour of tc_log_page_waits first
+ */
+ tc_log_page_waits++;
+ pthread_cond_wait(&COND_pool, &LOCK_pool);
+ return 1; // returns always 1
+}
+
+/*
+ all access to active page is serialized but it's not a problem, as we're
+ assuming that fsync() will be a bottleneck anyway.
+ That is, parallelizing writes to log pages we'll decrease number of threads
+ waiting for a page, but then all these threads will be waiting for fsync()
+*/
+
+int TC_LOG_MMAP::log(THD *thd, my_xid xid)
+{
+ int err;
+ PAGE *p;
+ ulong cookie;
+
+ pthread_mutex_lock(&LOCK_active);
+
+ while (unlikely(active && active->free == 0))
+ pthread_cond_wait(&COND_active, &LOCK_active);
+
+ if (active == 0)
+ {
+ lock_queue(&LOCK_pool);
+ get_active_from_pool();
+ unlock_queue(&LOCK_pool);
+ }
+
+ p=active;
+ pthread_mutex_lock(&p->lock);
+
+ while (*p->ptr)
+ {
+ p->ptr++;
+ DBUG_ASSERT(p->ptr < p->end); // because p->free > 0
+ }
+
+ cookie= (ulong)((uchar *)p->ptr - data); // Can never be zero
+ *p->ptr++= xid;
+ p->free--;
+ p->state= DIRTY;
+
+ /* to sync or not to sync - this is the question */
+ pthread_mutex_lock(&LOCK_sync);
+ pthread_mutex_unlock(&p->lock);
+
+ if (in_sync)
+ { // somebody's syncing. let's wait
+ pthread_mutex_unlock(&LOCK_active);
+ p->waiters++;
+ do
+ {
+ pthread_cond_wait(&p->cond, &LOCK_sync);
+ } while (p->state == DIRTY && in_sync);
+ p->waiters--;
+ err= p->state == ERROR;
+ if (p->state != DIRTY) // page was synced
+ {
+ if (p->waiters == 0)
+ pthread_cond_signal(&COND_pool); // in case somebody's waiting
+ pthread_mutex_unlock(&LOCK_sync);
+ goto done; // we're done
+ }
+ } // page was not synced! do it now
+ in_sync=p; // place is vacant - take it
+ DBUG_ASSERT(active == p);
+ active=0;
+ pthread_cond_broadcast(&COND_active); // in case somebody's waiting
+ pthread_mutex_unlock(&LOCK_active);
+ pthread_mutex_unlock(&LOCK_sync);
+ err= sync();
+
+done:
+ return err ? 0 : cookie;
+}
+
+int TC_LOG_MMAP::sync()
+{
+ int err;
+
+ DBUG_ASSERT(in_sync != active);
+
+ /*
+ sit down and relax - this can take a while...
+ note - no locks are held at this point
+ */
+ err= my_msync(fd, in_sync->start, 1, MS_SYNC);
+
+ pthread_mutex_lock(&LOCK_pool);
+ pool_last->next=in_sync;
+ pool_last=in_sync;
+ in_sync->next=0;
+ in_sync->state= err ? ERROR : POOL;
+ pthread_cond_broadcast(&in_sync->cond); // signal "sync done"
+ pthread_cond_signal(&COND_pool); // in case somebody's waiting
+ pthread_mutex_unlock(&LOCK_pool);
+
+ pthread_mutex_lock(&LOCK_sync);
+ in_sync=0;
+ pthread_cond_signal(&active->cond); // wake up a new syncer
+ pthread_mutex_unlock(&LOCK_sync);
+ return err;
+}
+
+void TC_LOG_MMAP::unlog(ulong cookie, my_xid xid)
+{
+ PAGE *p=pages+(cookie/tc_log_page_size);
+ my_xid *x=(my_xid *)(data+cookie);
+
+ DBUG_ASSERT(*x == xid);
+ DBUG_ASSERT(x >= p->start && x < p->end);
+ *x=0;
+
+ pthread_mutex_lock(&p->lock);
+ p->free++;
+ DBUG_ASSERT(p->free <= p->size);
+ set_if_smaller(p->ptr, x);
+ if (p->free == p->size)
+ statistic_decrement(tc_log_cur_pages_used, &LOCK_status);
+ if (p->waiters == 0) // the page is in pool and ready to rock
+ pthread_cond_signal(&COND_pool); // ping ... in case somebody's waiting
+ pthread_mutex_unlock(&p->lock);
+}
+
+void TC_LOG_MMAP::close()
+{
+ switch (inited) {
+ case 6:
+ pthread_mutex_destroy(&LOCK_sync);
+ pthread_mutex_destroy(&LOCK_active);
+ pthread_mutex_destroy(&LOCK_pool);
+ pthread_cond_destroy(&COND_pool);
+ case 5:
+ data[0]='A'; // garble the first (signature) byte, in case my_delete fails
+ case 4:
+ for (uint i=0; i < npages; i++)
+ {
+ if (pages[i].ptr == 0)
+ break;
+ pthread_mutex_destroy(&pages[i].lock);
+ pthread_cond_destroy(&pages[i].cond);
+ }
+ case 3:
+ my_free((gptr)pages, MYF(0));
+ case 2:
+ my_munmap(data, file_length);
+ case 1:
+ my_close(fd, MYF(0));
+ }
+ if (inited>=5) // cannot do in the switch because of Windows
+ my_delete(logname, MYF(MY_WME));
+ inited=0;
+}
+
+int TC_LOG_MMAP::recover()
+{
+ HASH xids;
+ PAGE *p=pages, *end_p=pages+npages;
+
+ if (memcmp(data, tc_log_magic, sizeof(tc_log_magic)))
+ {
+ sql_print_error("Bad magic header in tc log");
+ goto err1;
+ }
+
+ /*
+ the first byte after magic signature is set to current
+ number of storage engines on startup
+ */
+ if (data[sizeof(tc_log_magic)] != total_ha_2pc)
+ {
+ sql_print_error("For recovery to work all storage engines and binary log "
+ "must have exactly the same settings as before the crash!");
+ goto err1;
+ }
+
+ if (hash_init(&xids, &my_charset_bin, tc_log_page_size/3, 0,
+ sizeof(my_xid), 0, 0, MYF(0)))
+ goto err1;
+
+ for ( ; p < end_p ; p++)
+ {
+ for (my_xid *x=p->start; x < p->end; x++)
+ if (*x && my_hash_insert(&xids, (byte *)x))
+ goto err2; // OOM
+ }
+
+ if (ha_recover(&xids))
+ goto err2;
+
+ hash_free(&xids);
+ bzero(data, file_length);
+ return 0;
+
+err2:
+ hash_free(&xids);
+err1:
+ sql_print_error("Crash recovery failed. Either correct the problem "
+ "(if it's, for example, out of memory error) and restart, "
+ "or delete tc log and start mysqld with "
+ "--tc-heuristic-recover={commit|rollback}");
+ return 1;
+}
+
+int TC_LOG::using_heuristic_recover()
+{
+ if (!tc_heuristic_recover)
+ return 0;
+
+ sql_print_information("Heuristic crash recovery mode");
+ if (ha_recover(0))
+ sql_print_error("Heuristic crash recovery failed");
+ sql_print_information("Please restart mysqld without --tc-heuristic-recover");
+ return 1;
+}
+
+/****** transaction coordinator log for 2pc - binlog() based solution ******/
+
+/*
+ TODO keep in-memory list of prepared transactions
+ (add to list in log(), remove on unlog())
+ and copy it to the new binlog if rotated
+ but let's check the behaviour of tc_log_page_waits first!
+*/
+
+int MYSQL_LOG::open(const char *opt_name)
+{
+ LOG_INFO log_info, new_log_info;
+
+ DBUG_ASSERT(total_ha_2pc > 1);
+
+ pthread_mutex_init(&LOCK_prep_xids, MY_MUTEX_INIT_FAST);
+ pthread_cond_init (&COND_prep_xids, 0);
+
+ if (using_heuristic_recover())
+ return 1;
+
+ /*
+ read index file to get a last but one binlog filename
+ note - there's no need to lock any mutex, mysqld is only starting
+ up, no other threads are running yet.
+ still, there's safe_mutex_assert_owner() in binlog code, so
+ let's keep it happy.
+ */
+
+ if (find_log_pos(&new_log_info, NullS, 1))
+ goto err; // er ? where's the current entry ?
+
+ if (strcmp(log_file_name, new_log_info.log_file_name))
+ {
+ const char *errmsg;
+ char last_event_type=UNKNOWN_EVENT;
+ IO_CACHE log;
+ File file;
+ int error;
+ Log_event *ev;
+ Format_description_log_event fdle(4);
+
+ if (! fdle.is_valid())
+ goto err;
+
+ do
+ {
+ log_info.index_file_offset=new_log_info.index_file_offset;
+ log_info.index_file_start_offset=new_log_info.index_file_offset;
+ strcpy(log_info.log_file_name, new_log_info.log_file_name);
+ if (find_next_log(&new_log_info, 1))
+ goto err; // er ? where's the current entry ?
+ } while (strcmp(log_file_name, new_log_info.log_file_name));
+
+ if ((file= open_binlog(&log, log_info.log_file_name, &errmsg)) < 0)
+ {
+ sql_print_error("%s", errmsg);
+ goto err;
+ }
+
+ if (((ev= Log_event::read_log_event(&log, 0, &fdle))) &&
+ (ev->get_type_code() == FORMAT_DESCRIPTION_EVENT) &&
+ (ev->flags & LOG_EVENT_BINLOG_CLOSED_F))
+ error=0;
+ else
+ error= recover(&log, (Format_description_log_event *)ev);
+
+ delete ev;
+ end_io_cache(&log);
+ my_close(file, MYF(MY_WME));
+
+ if (error)
+ goto err;
+ }
+
+ return 0;
+
+err:
+ return 1;
+}
+
+void MYSQL_LOG::close()
+{
+ DBUG_ASSERT(prepared_xids==0);
+ pthread_mutex_destroy(&LOCK_prep_xids);
+ pthread_cond_destroy (&COND_prep_xids);
+}
+
+/* TODO group commit */
+int MYSQL_LOG::log(THD *thd, my_xid xid)
+{
+ Xid_log_event xle(thd, xid);
+ if (xle.write((IO_CACHE*)thd->ha_data[binlog_hton.slot]))
+ return 0;
+ thread_safe_increment(prepared_xids, &LOCK_prep_xids);
+ return !binlog_commit(thd,1); // invert return value
+}
+
+void MYSQL_LOG::unlog(ulong cookie, my_xid xid)
+{
+ if (thread_safe_dec_and_test(prepared_xids, &LOCK_prep_xids))
+ pthread_cond_signal(&COND_prep_xids);
+}
+
+int MYSQL_LOG::recover(IO_CACHE *log, Format_description_log_event *fdle)
+{
+ Log_event *ev;
+ HASH xids;
+ MEM_ROOT mem_root;
+
+ if (! fdle->is_valid() ||
+ hash_init(&xids, &my_charset_bin, tc_log_page_size/3, 0,
+ sizeof(my_xid), 0, 0, MYF(0)))
+ goto err1;
+
+ init_alloc_root(&mem_root, tc_log_page_size, tc_log_page_size);
+
+ while ((ev= Log_event::read_log_event(log,0,fdle)) && ev->is_valid())
+ {
+ if (ev->get_type_code() == XID_EVENT)
+ {
+ Xid_log_event *xev=(Xid_log_event *)ev;
+ byte *x=memdup_root(&mem_root, (char *)& xev->xid, sizeof(xev->xid));
+ if (! x)
+ goto err2;
+ my_hash_insert(&xids, x);
+ }
+ delete ev;
+ }
+
+ if (ha_recover(&xids))
+ goto err2;
+
+ free_root(&mem_root, MYF(0));
+ hash_free(&xids);
+ return 0;
+
+err2:
+ free_root(&mem_root, MYF(0));
+ hash_free(&xids);
+err1:
+ sql_print_error("Crash recovery failed. Either correct the problem "
+ "(if it's, for example, out of memory error) and restart, "
+ "or delete (or rename) binary log and start mysqld with "
+ "--tc-heuristic-recover={commit|rollback}");
+ return 1;
+}
+