summaryrefslogtreecommitdiff
path: root/sql/log.h
diff options
context:
space:
mode:
Diffstat (limited to 'sql/log.h')
-rw-r--r--sql/log.h191
1 files changed, 159 insertions, 32 deletions
diff --git a/sql/log.h b/sql/log.h
index c6e5c135b25..1c5e09c5c09 100644
--- a/sql/log.h
+++ b/sql/log.h
@@ -17,7 +17,6 @@
#ifndef LOG_H
#define LOG_H
-#include "unireg.h" // REQUIRED: for other includes
#include "handler.h" /* my_xid */
class Relay_log_info;
@@ -45,10 +44,20 @@ class TC_LOG
virtual int open(const char *opt_name)=0;
virtual void close()=0;
+ /*
+ Transaction coordinator 2-phase commit.
+
+ Must invoke the run_prepare_ordered and run_commit_ordered methods, as
+ described below for these methods.
+
+ In addition, must invoke THD::wait_for_prior_commit(), or equivalent
+ wait, to ensure that one commit waits for another if registered to do so.
+ */
virtual int log_and_order(THD *thd, my_xid xid, bool all,
bool need_prepare_ordered,
bool need_commit_ordered) = 0;
virtual int unlog(ulong cookie, my_xid xid)=0;
+ virtual void commit_checkpoint_notify(void *cookie)= 0;
protected:
/*
@@ -75,9 +84,11 @@ protected:
prepare_ordered() or commit_ordered() methods.
*/
extern mysql_mutex_t LOCK_prepare_ordered;
+extern mysql_cond_t COND_prepare_ordered;
extern mysql_mutex_t LOCK_commit_ordered;
#ifdef HAVE_PSI_INTERFACE
extern PSI_mutex_key key_LOCK_prepare_ordered, key_LOCK_commit_ordered;
+extern PSI_cond_key key_COND_prepare_ordered;
#endif
class TC_LOG_DUMMY: public TC_LOG // use it to disable the logging
@@ -98,8 +109,12 @@ public:
return 1;
}
int unlog(ulong cookie, my_xid xid) { return 0; }
+ void commit_checkpoint_notify(void *cookie) { DBUG_ASSERT(0); };
};
+#define TC_LOG_PAGE_SIZE 8192
+#define TC_LOG_MIN_SIZE (3*TC_LOG_PAGE_SIZE)
+
#ifdef HAVE_MMAP
class TC_LOG_MMAP: public TC_LOG
{
@@ -110,6 +125,12 @@ class TC_LOG_MMAP: public TC_LOG
PS_DIRTY // new xids added since last sync
} PAGE_STATE;
+ struct pending_cookies {
+ uint count;
+ uint pending_count;
+ ulong cookies[TC_LOG_PAGE_SIZE/sizeof(my_xid)];
+ };
+
private:
typedef struct st_page {
struct st_page *next; // page a linked in a fifo queue
@@ -141,7 +162,7 @@ class TC_LOG_MMAP: public TC_LOG
one has to use active->lock.
Same for LOCK_pool and LOCK_sync
*/
- mysql_mutex_t LOCK_active, LOCK_pool, LOCK_sync;
+ mysql_mutex_t LOCK_active, LOCK_pool, LOCK_sync, LOCK_pending_checkpoint;
mysql_cond_t COND_pool, COND_active;
/*
Queue of threads that need to call commit_ordered().
@@ -163,14 +184,16 @@ class TC_LOG_MMAP: public TC_LOG
*/
mysql_cond_t COND_queue_busy;
my_bool commit_ordered_queue_busy;
+ pending_cookies* pending_checkpoint;
public:
- TC_LOG_MMAP(): inited(0) {}
+ TC_LOG_MMAP(): inited(0), pending_checkpoint(0) {}
int open(const char *opt_name);
void close();
int log_and_order(THD *thd, my_xid xid, bool all,
bool need_prepare_ordered, bool need_commit_ordered);
int unlog(ulong cookie, my_xid xid);
+ void commit_checkpoint_notify(void *cookie);
int recover();
private:
@@ -178,6 +201,7 @@ class TC_LOG_MMAP: public TC_LOG
void get_active_from_pool();
int sync();
int overflow();
+ int delete_entry(ulong cookie);
};
#else
#define TC_LOG_MMAP TC_LOG_DUMMY
@@ -354,7 +378,36 @@ private:
time_t last_time;
};
+/*
+ We assign each binlog file an internal ID, used to identify them for unlog().
+ The IDs start from 0 and increment for each new binlog created.
+
+ In unlog() we need to know the ID of the binlog file that the corresponding
+ transaction was written into. We also need a special value for a corner
+ case where there is no corresponding binlog id (since nothing was logged).
+ And we need an error flag to mark that unlog() must return failure.
+
+ We use the following macros to pack all of this information into the single
+ ulong available with log_and_order() / unlog().
+
+ Note that we cannot use the value 0 for cookie, as that is reserved as error
+ return value from log_and_order().
+ */
+#define BINLOG_COOKIE_ERROR_RETURN 0
+#define BINLOG_COOKIE_DUMMY_ID 1
+#define BINLOG_COOKIE_BASE 2
+#define BINLOG_COOKIE_DUMMY(error_flag) \
+ ( (BINLOG_COOKIE_DUMMY_ID<<1) | ((error_flag)&1) )
+#define BINLOG_COOKIE_MAKE(id, error_flag) \
+ ( (((id)+BINLOG_COOKIE_BASE)<<1) | ((error_flag)&1) )
+#define BINLOG_COOKIE_GET_ERROR_FLAG(c) ((c) & 1)
+#define BINLOG_COOKIE_GET_ID(c) ( ((ulong)(c)>>1) - BINLOG_COOKIE_BASE )
+#define BINLOG_COOKIE_IS_DUMMY(c) \
+ ( ((ulong)(c)>>1) == BINLOG_COOKIE_DUMMY_ID )
+
class binlog_cache_mngr;
+struct rpl_gtid;
+struct wait_for_commit;
class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
{
private:
@@ -379,11 +432,10 @@ class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
bool using_stmt_cache;
bool using_trx_cache;
/*
- Extra events (BEGIN, COMMIT/ROLLBACK/XID, and possibly INCIDENT) to be
+ Extra events (COMMIT/ROLLBACK/XID, and possibly INCIDENT) to be
written during group commit. The incident_event is only valid if
trx_data->has_incident() is true.
*/
- Log_event *begin_event;
Log_event *end_event;
Log_event *incident_event;
/* Set during group commit to record any per-thread error. */
@@ -392,12 +444,38 @@ class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
IO_CACHE *error_cache;
/* This is the `all' parameter for ha_commit_ordered(). */
bool all;
+ /*
+ True if we need to increment xid_count in trx_group_commit_leader() and
+ decrement in unlog() (this is needed if there is a participating engine
+ that does not implement the commit_checkpoint_request() handlerton
+ method).
+ */
+ bool need_unlog;
+ /*
+ Fields used to pass the necessary information to the last thread in a
+ group commit, only used when opt_optimize_thread_scheduling is not set.
+ */
+ bool check_purge;
+ /* Flag used to optimise around wait_for_prior_commit. */
+ bool queued_by_other;
+ ulong binlog_id;
};
+ /*
+ When this is set, a RESET MASTER is in progress.
+
+ Then we should not write any binlog checkpoints into the binlog (that
+ could result in deadlock on LOCK_log, and we will delete all binlog files
+ anyway). Instead we should signal COND_xid_list whenever a new binlog
+ checkpoint arrives - when all have arrived, RESET MASTER will complete.
+ */
+ uint reset_master_pending;
+ ulong mark_xid_done_waiting;
+
/* LOCK_log and LOCK_index are inited by init_pthread_objects() */
mysql_mutex_t LOCK_index;
- mysql_mutex_t LOCK_prep_xids;
- mysql_cond_t COND_prep_xids;
+ mysql_mutex_t LOCK_xid_list;
+ mysql_cond_t COND_xid_list;
mysql_cond_t update_cond;
ulonglong bytes_written;
IO_CACHE index_file;
@@ -414,27 +492,14 @@ class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
The max size before rotation (usable only if log_type == LOG_BIN: binary
logs and relay logs).
For a binlog, max_size should be max_binlog_size.
- For a relay log, it should be max_relay_log_size if this is non-zero,
- max_binlog_size otherwise.
max_size is set in init(), and dynamically changed (when one does SET
- GLOBAL MAX_BINLOG_SIZE|MAX_RELAY_LOG_SIZE) by fix_max_binlog_size and
- fix_max_relay_log_size).
+ GLOBAL MAX_BINLOG_SIZE|MAX_RELAY_LOG_SIZE) from sys_vars.cc
*/
ulong max_size;
- long prepared_xids; /* for tc log - number of xids to remember */
// current file sequence number for load data infile binary logging
uint file_id;
uint open_count; // For replication
int readers_count;
- bool need_start_event;
- /*
- no_auto_events means we don't want any of these automatic events :
- Start/Rotate/Stop. That is, in 4.x when we rotate a relay log, we don't
- want a Rotate_log event to be written to the relay log. When we start a
- relay log etc. So in 4.x this is 1 for relay logs, 0 for binlogs.
- In 5.0 it's 0 for relay logs too!
- */
- bool no_auto_events;
/* Queue of transactions queued up to participate in group commit. */
group_commit_entry *group_commit_queue;
/*
@@ -449,6 +514,9 @@ class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
ulonglong num_commits;
/* Number of group commits done. */
ulonglong num_group_commits;
+ /* The reason why the group commit was grouped */
+ ulonglong group_commit_trigger_count, group_commit_trigger_timeout;
+ ulonglong group_commit_trigger_lock_wait;
/* pointer to the sync period variable, for binlog this will be
sync_binlog_period, for relay log this will be
@@ -456,6 +524,8 @@ class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
*/
uint *sync_period_ptr;
uint sync_counter;
+ bool state_file_deleted;
+ bool binlog_state_recover_done;
inline uint get_sync_period()
{
@@ -470,13 +540,42 @@ class MYSQL_BIN_LOG: public TC_LOG, private MYSQL_LOG
*/
int new_file_without_locking();
int new_file_impl(bool need_lock);
- int write_transaction_or_stmt(group_commit_entry *entry);
+ void do_checkpoint_request(ulong binlog_id);
+ void purge();
+ int write_transaction_or_stmt(group_commit_entry *entry, uint64 commit_id);
+ int queue_for_group_commit(group_commit_entry *entry);
bool write_transaction_to_binlog_events(group_commit_entry *entry);
void trx_group_commit_leader(group_commit_entry *leader);
- void mark_xid_done();
- void mark_xids_active(uint xid_count);
+ bool is_xidlist_idle_nolock();
public:
+ /*
+ A list of struct xid_count_per_binlog is used to keep track of how many
+ XIDs are in prepared, but not committed, state in each binlog. And how
+ many commit_checkpoint_request()'s are pending.
+
+ When count drops to zero in a binlog after rotation, it means that there
+ are no more XIDs in prepared state, so that binlog is no longer needed
+ for XA crash recovery, and we can log a new binlog checkpoint event.
+
+ The list is protected against simultaneous access from multiple
+ threads by LOCK_xid_list.
+ */
+ struct xid_count_per_binlog : public ilink {
+ char *binlog_name;
+ uint binlog_name_len;
+ ulong binlog_id;
+ /* Total prepared XIDs and pending checkpoint requests in this binlog. */
+ long xid_count;
+ /* For linking in requests to the binlog background thread. */
+ xid_count_per_binlog *next_in_queue;
+ xid_count_per_binlog(); /* Give link error if constructor used. */
+ };
+ I_List<xid_count_per_binlog> binlog_xid_count_list;
+ mysql_mutex_t LOCK_binlog_background_thread;
+ mysql_cond_t COND_binlog_background_thread;
+ mysql_cond_t COND_binlog_background_thread_end;
+
using MYSQL_LOG::generate_name;
using MYSQL_LOG::is_open;
@@ -534,6 +633,7 @@ public:
*/
char last_commit_pos_file[FN_REFLEN];
my_off_t last_commit_pos_offset;
+ ulong current_binlog_id;
MYSQL_BIN_LOG(uint *sync_period);
/*
@@ -562,7 +662,10 @@ public:
int log_and_order(THD *thd, my_xid xid, bool all,
bool need_prepare_ordered, bool need_commit_ordered);
int unlog(ulong cookie, my_xid xid);
- int recover(IO_CACHE *log, Format_description_log_event *fdle);
+ 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 do_binlog_recovery(const char *opt_name, bool do_xa_recovery);
#if !defined(MYSQL_CLIENT)
int flush_and_set_pending_rows_event(THD *thd, Rows_log_event* event,
@@ -588,17 +691,18 @@ public:
}
void set_max_size(ulong max_size_arg);
void signal_update();
+ void wait_for_sufficient_commits();
+ void binlog_trigger_immediate_group_commit();
void wait_for_update_relay_log(THD* thd);
int wait_for_update_bin_log(THD* thd, const struct timespec * timeout);
- void set_need_start_event() { need_start_event = 1; }
- void init(bool no_auto_events_arg, ulong max_size);
+ void init(ulong max_size);
void init_pthread_objects();
void cleanup();
bool open(const char *log_name,
enum_log_type log_type,
const char *new_name,
enum cache_type io_cache_type_arg,
- bool no_auto_events_arg, ulong max_size,
+ ulong max_size,
bool null_created,
bool need_mutex);
bool open_index_file(const char *index_file_name_arg,
@@ -614,6 +718,7 @@ public:
bool write_incident_already_locked(THD *thd);
bool write_incident(THD *thd);
+ void write_binlog_checkpoint_event_already_locked(const char *name, uint len);
int write_cache(THD *thd, IO_CACHE *cache);
void set_write_error(THD *thd, bool is_transactional);
bool check_write_error(THD *thd);
@@ -628,12 +733,16 @@ public:
*/
bool appendv(const char* buf,uint len,...);
bool append(Log_event* ev);
+ bool append_no_lock(Log_event* ev);
+ void mark_xids_active(ulong cookie, uint xid_count);
+ void mark_xid_done(ulong cookie, bool write_checkpoint);
void make_log_name(char* buf, const char* log_ident);
bool is_active(const char* log_file_name);
+ bool can_purge_log(const char *log_file_name);
int update_log_index(LOG_INFO* linfo, bool need_update_threads);
int rotate(bool force_rotate, bool* check_purge);
- void purge();
+ void checkpoint_and_purge(ulong binlog_id);
int rotate_and_purge(bool force_rotate);
/**
Flush binlog cache and synchronize to disk.
@@ -664,7 +773,8 @@ public:
int register_create_index_entry(const char* entry);
int purge_index_entry(THD *thd, ulonglong *decrease_log_space,
bool need_mutex);
- bool reset_logs(THD* thd);
+ bool reset_logs(THD* thd, bool create_new_log,
+ rpl_gtid *init_state, uint32 init_state_len);
void close(uint exiting);
void clear_inuse_flag_when_closing(File file);
@@ -687,6 +797,21 @@ public:
inline IO_CACHE *get_index_file() { return &index_file;}
inline uint32 get_open_count() { return open_count; }
void set_status_variables(THD *thd);
+ bool is_xidlist_idle();
+ bool write_gtid_event(THD *thd, bool standalone, bool is_transactional,
+ uint64 commit_id);
+ int read_state_from_file();
+ int write_state_to_file();
+ int get_most_recent_gtid_list(rpl_gtid **list, uint32 *size);
+ bool append_state_pos(String *str);
+ bool append_state(String *str);
+ bool is_empty_state();
+ bool find_in_binlog_state(uint32 domain_id, uint32 server_id,
+ rpl_gtid *out_gtid);
+ bool lookup_domain_in_binlog_state(uint32 domain_id, rpl_gtid *out_gtid);
+ int bump_seq_no_counter_if_needed(uint32 domain_id, uint64 seq_no);
+ bool check_strict_gtid_sequence(uint32 domain_id, uint32 server_id,
+ uint64 seq_no);
};
class Log_event_handler
@@ -712,8 +837,8 @@ public:
};
-int check_if_log_table(size_t db_len, const char *db, size_t table_name_len,
- const char *table_name, bool check_if_opened);
+int check_if_log_table(const TABLE_LIST *table, bool check_if_opened,
+ const char *errmsg);
class Log_to_csv_event_handler: public Log_event_handler
{
@@ -881,6 +1006,7 @@ bool general_log_print(THD *thd, enum enum_server_command command,
bool general_log_write(THD *thd, enum enum_server_command command,
const char *query, uint query_length);
+void binlog_report_wait_for(THD *thd, THD *other_thd);
void sql_perror(const char *message);
bool flush_error_log();
@@ -888,6 +1014,7 @@ File open_binlog(IO_CACHE *log, const char *log_file_name,
const char **errmsg);
void make_default_log_name(char **out, const char* log_ext, bool once);
+void binlog_reset_cache(THD *thd);
extern MYSQL_PLUGIN_IMPORT MYSQL_BIN_LOG mysql_bin_log;
extern LOGGER logger;