diff options
Diffstat (limited to 'sql/slave.cc')
-rw-r--r-- | sql/slave.cc | 3144 |
1 files changed, 2231 insertions, 913 deletions
diff --git a/sql/slave.cc b/sql/slave.cc index f2f37807128..b8689a28a54 100644 --- a/sql/slave.cc +++ b/sql/slave.cc @@ -20,48 +20,140 @@ #include <myisam.h> #include "mini_client.h" #include "slave.h" +#include "sql_repl.h" +#include "repl_failsafe.h" #include <thr_alarm.h> #include <my_dir.h> +#include <assert.h> -#define RPL_LOG_NAME (glob_mi.log_file_name[0] ? glob_mi.log_file_name :\ - "FIRST") - -volatile bool slave_running = 0; -pthread_t slave_real_id; -MASTER_INFO glob_mi; -MY_BITMAP slave_error_mask; bool use_slave_mask = 0; +MY_BITMAP slave_error_mask; + +typedef bool (*CHECK_KILLED_FUNC)(THD*,void*); + +volatile bool slave_sql_running = 0, slave_io_running = 0; +char* slave_load_tmpdir = 0; +MASTER_INFO main_mi; +MASTER_INFO* active_mi; +volatile int active_mi_in_use = 0; HASH replicate_do_table, replicate_ignore_table; DYNAMIC_ARRAY replicate_wild_do_table, replicate_wild_ignore_table; bool do_table_inited = 0, ignore_table_inited = 0; bool wild_do_table_inited = 0, wild_ignore_table_inited = 0; bool table_rules_on = 0; -uint32 slave_skip_counter = 0; -static TABLE* save_temporary_tables = 0; -THD* slave_thd = 0; -// when slave thread exits, we need to remember the temporary tables so we -// can re-use them on slave start - -static int last_slave_errno = 0; -static char last_slave_error[1024] = ""; -#ifndef DBUG_OFF -int disconnect_slave_event_count = 0, abort_slave_event_count = 0; -static int events_till_disconnect = -1, events_till_abort = -1; -static int stuck_count = 0; -#endif +ulonglong relay_log_space_limit = 0; + +/* + When slave thread exits, we need to remember the temporary tables so we + can re-use them on slave start. -inline void skip_load_data_infile(NET* net); -inline bool slave_killed(THD* thd); -static int init_slave_thread(THD* thd); + TODO: move the vars below under MASTER_INFO +*/ + +int disconnect_slave_event_count = 0, abort_slave_event_count = 0; +int events_till_abort = -1; +static int events_till_disconnect = -1; + +typedef enum { SLAVE_THD_IO, SLAVE_THD_SQL} SLAVE_THD_TYPE; + +void skip_load_data_infile(NET* net); +static int process_io_rotate(MASTER_INFO* mi, Rotate_log_event* rev); +static int process_io_create_file(MASTER_INFO* mi, Create_file_log_event* cev); +static bool wait_for_relay_log_space(RELAY_LOG_INFO* rli); +static inline bool io_slave_killed(THD* thd,MASTER_INFO* mi); +static inline bool sql_slave_killed(THD* thd,RELAY_LOG_INFO* rli); +static int count_relay_log_space(RELAY_LOG_INFO* rli); +static int init_slave_thread(THD* thd, SLAVE_THD_TYPE thd_type); static int safe_connect(THD* thd, MYSQL* mysql, MASTER_INFO* mi); static int safe_reconnect(THD* thd, MYSQL* mysql, MASTER_INFO* mi, bool suppress_warnings); -static int safe_sleep(THD* thd, int sec); -static int request_table_dump(MYSQL* mysql, char* db, char* table); +static int connect_to_master(THD* thd, MYSQL* mysql, MASTER_INFO* mi, + bool reconnect, bool suppress_warnings); +static int safe_sleep(THD* thd, int sec, CHECK_KILLED_FUNC thread_killed, + void* thread_killed_arg); +static int request_table_dump(MYSQL* mysql, const char* db, const char* table); static int create_table_from_dump(THD* thd, NET* net, const char* db, const char* table_name); -inline char* rewrite_db(char* db); -static int check_expected_error(THD* thd, int expected_error); +static int check_master_version(MYSQL* mysql, MASTER_INFO* mi); +char* rewrite_db(char* db); + + +/* + Get a bit mask for which threads are running so that we later can + restart these threads +*/ + +void init_thread_mask(int* mask,MASTER_INFO* mi,bool inverse) +{ + bool set_io = mi->slave_running, set_sql = mi->rli.slave_running; + register int tmp_mask=0; + if (set_io) + tmp_mask |= SLAVE_IO; + if (set_sql) + tmp_mask |= SLAVE_SQL; + if (inverse) + tmp_mask^= (SLAVE_IO | SLAVE_SQL); + *mask = tmp_mask; +} + + +void lock_slave_threads(MASTER_INFO* mi) +{ + //TODO: see if we can do this without dual mutex + pthread_mutex_lock(&mi->run_lock); + pthread_mutex_lock(&mi->rli.run_lock); +} + +void unlock_slave_threads(MASTER_INFO* mi) +{ + //TODO: see if we can do this without dual mutex + pthread_mutex_unlock(&mi->rli.run_lock); + pthread_mutex_unlock(&mi->run_lock); +} + + +int init_slave() +{ + DBUG_ENTER("init_slave"); + + /* + TODO: re-write this to interate through the list of files + for multi-master + */ + active_mi = &main_mi; + + /* + If master_host is not specified, try to read it from the master_info file. + If master_host is specified, create the master_info file if it doesn't + exists. + */ + if (init_master_info(active_mi,master_info_file,relay_log_info_file, + !master_host)) + { + sql_print_error("Warning: failed to initialized master info"); + DBUG_RETURN(0); + } + + /* + make sure slave thread gets started if server_id is set, + valid master.info is present, and master_host has not been specified + */ + if (server_id && !master_host && active_mi->host[0]) + master_host= active_mi->host; + + if (master_host && !opt_skip_slave_start) + { + if (start_slave_threads(1 /* need mutex */, + 0 /* no wait for start*/, + active_mi, + master_info_file, + relay_log_info_file, + SLAVE_IO | SLAVE_SQL)) + sql_print_error("Warning: Can't create threads to handle slave"); + } + DBUG_RETURN(0); +} + static void free_table_ent(TABLE_RULE_ENT* e) { @@ -75,10 +167,122 @@ static byte* get_table_key(TABLE_RULE_ENT* e, uint* len, return (byte*)e->db; } + +/* + Open the given relay log + + SYNOPSIS + init_relay_log_pos() + rli Relay information (will be initialized) + log Name of relay log file to read from. NULL = First log + pos Position in relay log file + need_data_lock Set to 1 if this functions should do mutex locks + errmsg Store pointer to error message here + + DESCRIPTION + - Close old open relay log files. + - If we are using the same relay log as the running IO-thread, then set + rli->cur_log to point to the same IO_CACHE entry. + - If not, open the 'log' binary file. + + TODO + - check proper initialization of master_log_name/master_log_pos + - We may always want to delete all logs before 'log'. + Currently if we are not calling this with 'log' as NULL or the first + log we will never delete relay logs. + If we want this we should not set skip_log_purge to 1. + + RETURN VALUES + 0 ok + 1 error. errmsg is set to point to the error message +*/ + +int init_relay_log_pos(RELAY_LOG_INFO* rli,const char* log, + ulonglong pos, bool need_data_lock, + const char** errmsg) +{ + DBUG_ENTER("init_relay_log_pos"); + + *errmsg=0; + if (rli->log_pos_current) // TODO: When can this happen ? + DBUG_RETURN(0); + pthread_mutex_t *log_lock=rli->relay_log.get_log_lock(); + pthread_mutex_lock(log_lock); + if (need_data_lock) + pthread_mutex_lock(&rli->data_lock); + + /* Close log file and free buffers if it's already open */ + if (rli->cur_log_fd >= 0) + { + end_io_cache(&rli->cache_buf); + my_close(rli->cur_log_fd, MYF(MY_WME)); + rli->cur_log_fd = -1; + } + + rli->relay_log_pos = pos; + + /* + Test to see if the previous run was with the skip of purging + If yes, we do not purge when we restart + */ + if (rli->relay_log.find_log_pos(&rli->linfo, NullS, 1)) + { + *errmsg="Could not find first log during relay log initialization"; + goto err; + } + + if (log) // If not first log + { + if (strcmp(log, rli->linfo.log_file_name)) + rli->skip_log_purge=1; // Different name; Don't purge + if (rli->relay_log.find_log_pos(&rli->linfo, log, 1)) + { + *errmsg="Could not find target log during relay log initialization"; + goto err; + } + } + strmake(rli->relay_log_name,rli->linfo.log_file_name, + sizeof(rli->relay_log_name)-1); + if (rli->relay_log.is_active(rli->linfo.log_file_name)) + { + /* + The IO thread is using this log file. + In this case, we will use the same IO_CACHE pointer to + read data as the IO thread is using to write data. + */ + if (my_b_tell((rli->cur_log=rli->relay_log.get_log_file())) == 0 && + check_binlog_magic(rli->cur_log,errmsg)) + goto err; + rli->cur_log_old_open_count=rli->relay_log.get_open_count(); + } + else + { + /* + Open the relay log and set rli->cur_log to point at this one + */ + if ((rli->cur_log_fd=open_binlog(&rli->cache_buf, + rli->linfo.log_file_name,errmsg)) < 0) + goto err; + rli->cur_log = &rli->cache_buf; + } + if (pos > BIN_LOG_HEADER_SIZE) + my_b_seek(rli->cur_log,(off_t)pos); + rli->log_pos_current=1; + +err: + pthread_cond_broadcast(&rli->data_cond); + if (need_data_lock) + pthread_mutex_unlock(&rli->data_lock); + pthread_mutex_unlock(log_lock); + DBUG_RETURN ((*errmsg) ? 1 : 0); +} + + /* called from get_options() in mysqld.cc on start-up */ -void init_slave_skip_errors(char* arg) + +void init_slave_skip_errors(const char* arg) { - char* p; + const char *p; if (bitmap_init(&slave_error_mask,MAX_SLAVE_ERROR,0)) { fprintf(stderr, "Badly out of memory, please check your system status\n"); @@ -104,11 +308,257 @@ void init_slave_skip_errors(char* arg) } } + +/* + We assume we have a run lock on rli and that both slave thread + are not running +*/ + +int purge_relay_logs(RELAY_LOG_INFO* rli, THD *thd, bool just_reset, + const char** errmsg) +{ + int error=0; + DBUG_ENTER("purge_relay_logs"); + if (!rli->inited) + DBUG_RETURN(0); /* successfully do nothing */ + + DBUG_ASSERT(rli->slave_running == 0); + DBUG_ASSERT(rli->mi->slave_running == 0); + + rli->slave_skip_counter=0; + pthread_mutex_lock(&rli->data_lock); + rli->pending=0; + rli->master_log_name[0]=0; + rli->master_log_pos=0; // 0 means uninitialized + if (rli->relay_log.reset_logs(thd)) + { + *errmsg = "Failed during log reset"; + error=1; + goto err; + } + /* Save name of used relay log file */ + strmake(rli->relay_log_name, rli->relay_log.get_log_fname(), + sizeof(rli->relay_log_name)-1); + // Just first log with magic number and nothing else + rli->log_space_total= BIN_LOG_HEADER_SIZE; + rli->relay_log_pos= BIN_LOG_HEADER_SIZE; + rli->relay_log.reset_bytes_written(); + rli->log_pos_current=0; + if (!just_reset) + error= init_relay_log_pos(rli, rli->relay_log_name, rli->relay_log_pos, + 0 /* do not need data lock */, errmsg); + +err: +#ifndef DBUG_OFF + char buf[22]; +#endif + DBUG_PRINT("info",("log_space_total: %s",llstr(rli->log_space_total,buf))); + pthread_mutex_unlock(&rli->data_lock); + DBUG_RETURN(error); +} + + +int terminate_slave_threads(MASTER_INFO* mi,int thread_mask,bool skip_lock) +{ + if (!mi->inited) + return 0; /* successfully do nothing */ + int error,force_all = (thread_mask & SLAVE_FORCE_ALL); + pthread_mutex_t *sql_lock = &mi->rli.run_lock, *io_lock = &mi->run_lock; + pthread_mutex_t *sql_cond_lock,*io_cond_lock; + DBUG_ENTER("terminate_slave_threads"); + + sql_cond_lock=sql_lock; + io_cond_lock=io_lock; + + if (skip_lock) + { + sql_lock = io_lock = 0; + } + if ((thread_mask & (SLAVE_IO|SLAVE_FORCE_ALL)) && mi->slave_running) + { + DBUG_PRINT("info",("Terminating IO thread")); + mi->abort_slave=1; + if ((error=terminate_slave_thread(mi->io_thd,io_lock, + io_cond_lock, + &mi->stop_cond, + &mi->slave_running)) && + !force_all) + DBUG_RETURN(error); + } + if ((thread_mask & (SLAVE_SQL|SLAVE_FORCE_ALL)) && mi->rli.slave_running) + { + DBUG_PRINT("info",("Terminating SQL thread")); + DBUG_ASSERT(mi->rli.sql_thd != 0) ; + mi->rli.abort_slave=1; + if ((error=terminate_slave_thread(mi->rli.sql_thd,sql_lock, + sql_cond_lock, + &mi->rli.stop_cond, + &mi->rli.slave_running)) && + !force_all) + DBUG_RETURN(error); + } + DBUG_RETURN(0); +} + + +int terminate_slave_thread(THD* thd, pthread_mutex_t* term_lock, + pthread_mutex_t *cond_lock, + pthread_cond_t* term_cond, + volatile bool* slave_running) +{ + if (term_lock) + { + pthread_mutex_lock(term_lock); + if (!*slave_running) + { + pthread_mutex_unlock(term_lock); + return ER_SLAVE_NOT_RUNNING; + } + } + DBUG_ASSERT(thd != 0); + /* + Is is criticate to test if the slave is running. Otherwise, we might + be referening freed memory trying to kick it + */ + THD_CHECK_SENTRY(thd); + if (*slave_running) + { + KICK_SLAVE(thd); + } + while (*slave_running) + { + /* + There is a small chance that slave thread might miss the first + alarm. To protect againts it, resend the signal until it reacts + */ + struct timespec abstime; + set_timespec(abstime,2); + pthread_cond_timedwait(term_cond, cond_lock, &abstime); + if (*slave_running) + { + KICK_SLAVE(thd); + } + } + if (term_lock) + pthread_mutex_unlock(term_lock); + return 0; +} + + +int start_slave_thread(pthread_handler h_func, pthread_mutex_t *start_lock, + pthread_mutex_t *cond_lock, + pthread_cond_t *start_cond, + volatile bool *slave_running, + volatile ulong *slave_run_id, + MASTER_INFO* mi) +{ + pthread_t th; + ulong start_id; + DBUG_ASSERT(mi->inited); + DBUG_ENTER("start_slave_thread"); + + if (start_lock) + pthread_mutex_lock(start_lock); + if (!server_id) + { + if (start_cond) + pthread_cond_broadcast(start_cond); + if (start_lock) + pthread_mutex_unlock(start_lock); + sql_print_error("Server id not set, will not start slave"); + DBUG_RETURN(ER_BAD_SLAVE); + } + + if (*slave_running) + { + if (start_cond) + pthread_cond_broadcast(start_cond); + if (start_lock) + pthread_mutex_unlock(start_lock); + DBUG_RETURN(ER_SLAVE_MUST_STOP); + } + start_id= *slave_run_id; + DBUG_PRINT("info",("Creating new slave thread")); + if (pthread_create(&th, &connection_attrib, h_func, (void*)mi)) + { + if (start_lock) + pthread_mutex_unlock(start_lock); + DBUG_RETURN(ER_SLAVE_THREAD); + } + if (start_cond && cond_lock) + { + THD* thd = current_thd; + while (start_id == *slave_run_id) + { + DBUG_PRINT("sleep",("Waiting for slave thread to start")); + const char* old_msg = thd->enter_cond(start_cond,cond_lock, + "Waiting for slave thread to start"); + pthread_cond_wait(start_cond,cond_lock); + thd->exit_cond(old_msg); + if (thd->killed) + { + pthread_mutex_unlock(cond_lock); + DBUG_RETURN(ER_SERVER_SHUTDOWN); + } + } + } + if (start_lock) + pthread_mutex_unlock(start_lock); + DBUG_RETURN(0); +} + + +/* + SLAVE_FORCE_ALL is not implemented here on purpose since it does not make + sense to do that for starting a slave - we always care if it actually + started the threads that were not previously running +*/ + +int start_slave_threads(bool need_slave_mutex, bool wait_for_start, + MASTER_INFO* mi, const char* master_info_fname, + const char* slave_info_fname, int thread_mask) +{ + pthread_mutex_t *lock_io=0,*lock_sql=0,*lock_cond_io=0,*lock_cond_sql=0; + pthread_cond_t* cond_io=0,*cond_sql=0; + int error=0; + DBUG_ENTER("start_slave_threads"); + + if (need_slave_mutex) + { + lock_io = &mi->run_lock; + lock_sql = &mi->rli.run_lock; + } + if (wait_for_start) + { + cond_io = &mi->start_cond; + cond_sql = &mi->rli.start_cond; + lock_cond_io = &mi->run_lock; + lock_cond_sql = &mi->rli.run_lock; + } + + if (thread_mask & SLAVE_IO) + error=start_slave_thread(handle_slave_io,lock_io,lock_cond_io, + cond_io, + &mi->slave_running, &mi->slave_run_id, + mi); + if (!error && (thread_mask & SLAVE_SQL)) + { + error=start_slave_thread(handle_slave_sql,lock_sql,lock_cond_sql, + cond_sql, + &mi->rli.slave_running, &mi->rli.slave_run_id, + mi); + if (error) + terminate_slave_threads(mi, thread_mask & SLAVE_IO, 0); + } + DBUG_RETURN(error); +} + + void init_table_rule_hash(HASH* h, bool* h_inited) { hash_init(h, TABLE_RULE_HASH_SIZE,0,0, (hash_get_key) get_table_key, - (void (*)(void*)) free_table_ent, 0); + (hash_free_key) free_table_ent, 0); *h_inited = 1; } @@ -124,11 +574,11 @@ static TABLE_RULE_ENT* find_wild(DYNAMIC_ARRAY *a, const char* key, int len) uint i; const char* key_end = key + len; - for(i = 0; i < a->elements; i++) + for (i = 0; i < a->elements; i++) { TABLE_RULE_ENT* e ; get_dynamic(a, (gptr)&e, i); - if(!wild_case_compare(key, key_end, (const char*)e->db, + if (!wild_case_compare(key, key_end, (const char*)e->db, (const char*)(e->db + e->key_len),'\\')) return e; } @@ -152,7 +602,7 @@ int tables_ok(THD* thd, TABLE_LIST* tables) if (hash_search(&replicate_do_table, (byte*) hash_key, len)) return 1; } - if (ignore_table_inited) // if there are any do's + if (ignore_table_inited) // if there are any ignores { if (hash_search(&replicate_ignore_table, (byte*) hash_key, len)) return 0; @@ -165,9 +615,10 @@ int tables_ok(THD* thd, TABLE_LIST* tables) return 0; } - // if no explicit rule found - // and there was a do list, do not replicate. If there was - // no do list, go ahead + /* + If no explicit rule found and there was a do list, do not replicate. + If there was no do list, go ahead + */ return !do_table_inited && !wild_do_table_inited; } @@ -175,12 +626,12 @@ int tables_ok(THD* thd, TABLE_LIST* tables) int add_table_rule(HASH* h, const char* table_spec) { const char* dot = strchr(table_spec, '.'); - if(!dot) return 1; + if (!dot) return 1; // len is always > 0 because we know the there exists a '.' uint len = (uint)strlen(table_spec); TABLE_RULE_ENT* e = (TABLE_RULE_ENT*)my_malloc(sizeof(TABLE_RULE_ENT) + len, MYF(MY_WME)); - if(!e) return 1; + if (!e) return 1; e->db = (char*)e + sizeof(TABLE_RULE_ENT); e->tbl_name = e->db + (dot - table_spec) + 1; e->key_len = len; @@ -192,11 +643,11 @@ int add_table_rule(HASH* h, const char* table_spec) int add_wild_table_rule(DYNAMIC_ARRAY* a, const char* table_spec) { const char* dot = strchr(table_spec, '.'); - if(!dot) return 1; + if (!dot) return 1; uint len = (uint)strlen(table_spec); TABLE_RULE_ENT* e = (TABLE_RULE_ENT*)my_malloc(sizeof(TABLE_RULE_ENT) + len, MYF(MY_WME)); - if(!e) return 1; + if (!e) return 1; e->db = (char*)e + sizeof(TABLE_RULE_ENT); e->tbl_name = e->db + (dot - table_spec) + 1; e->key_len = len; @@ -208,7 +659,7 @@ int add_wild_table_rule(DYNAMIC_ARRAY* a, const char* table_spec) static void free_string_array(DYNAMIC_ARRAY *a) { uint i; - for(i = 0; i < a->elements; i++) + for (i = 0; i < a->elements; i++) { char* p; get_dynamic(a, (gptr) &p, i); @@ -217,100 +668,130 @@ static void free_string_array(DYNAMIC_ARRAY *a) delete_dynamic(a); } +#ifdef NOT_USED_YET + +static int end_slave_on_walk(MASTER_INFO* mi, gptr /*unused*/) +{ + end_master_info(mi); + return 0; +} +#endif + void end_slave() { - pthread_mutex_lock(&LOCK_slave); - if (slave_running) - { - abort_slave = 1; - thr_alarm_kill(slave_real_id); -#ifdef SIGNAL_WITH_VIO_CLOSE - slave_thd->close_active_vio(); -#endif - while (slave_running) - pthread_cond_wait(&COND_slave_stopped, &LOCK_slave); - } - pthread_mutex_unlock(&LOCK_slave); - - end_master_info(&glob_mi); - if(do_table_inited) + /* + TODO: replace the line below with + list_walk(&master_list, (list_walk_action)end_slave_on_walk,0); + once multi-master code is ready. + */ + terminate_slave_threads(active_mi,SLAVE_FORCE_ALL); + end_master_info(active_mi); + if (do_table_inited) hash_free(&replicate_do_table); - if(ignore_table_inited) + if (ignore_table_inited) hash_free(&replicate_ignore_table); - if(wild_do_table_inited) + if (wild_do_table_inited) free_string_array(&replicate_wild_do_table); - if(wild_ignore_table_inited) + if (wild_ignore_table_inited) free_string_array(&replicate_wild_ignore_table); } -inline bool slave_killed(THD* thd) + +static bool io_slave_killed(THD* thd, MASTER_INFO* mi) +{ + DBUG_ASSERT(mi->io_thd == thd); + DBUG_ASSERT(mi->slave_running == 1); // tracking buffer overrun + return mi->abort_slave || abort_loop || thd->killed; +} + + +static bool sql_slave_killed(THD* thd, RELAY_LOG_INFO* rli) +{ + DBUG_ASSERT(rli->sql_thd == thd); + DBUG_ASSERT(rli->slave_running == 1);// tracking buffer overrun + return rli->abort_slave || abort_loop || thd->killed; +} + + +void slave_print_error(RELAY_LOG_INFO* rli, int err_code, const char* msg, ...) { - return abort_slave || abort_loop || thd->killed; + va_list args; + va_start(args,msg); + my_vsnprintf(rli->last_slave_error, + sizeof(rli->last_slave_error), msg, args); + sql_print_error("Slave: %s, error_code=%d", rli->last_slave_error, + err_code); + rli->last_slave_errno = err_code; } -inline void skip_load_data_infile(NET* net) + +void skip_load_data_infile(NET* net) { (void)my_net_write(net, "\xfb/dev/null", 10); (void)net_flush(net); - (void)my_net_read(net); // discard response - send_ok(net); // the master expects it + (void)my_net_read(net); // discard response + send_ok(net); // the master expects it } -inline char* rewrite_db(char* db) + +char* rewrite_db(char* db) { - if(replicate_rewrite_db.is_empty() || !db) return db; + if (replicate_rewrite_db.is_empty() || !db) + return db; I_List_iterator<i_string_pair> it(replicate_rewrite_db); i_string_pair* tmp; - while((tmp=it++)) - { - if(!strcmp(tmp->key, db)) - return tmp->val; - } - + while ((tmp=it++)) + { + if (!strcmp(tmp->key, db)) + return tmp->val; + } return db; } + int db_ok(const char* db, I_List<i_string> &do_list, I_List<i_string> &ignore_list ) { - if(do_list.is_empty() && ignore_list.is_empty()) + if (do_list.is_empty() && ignore_list.is_empty()) return 1; // ok to replicate if the user puts no constraints - // if the user has specified restrictions on which databases to replicate - // and db was not selected, do not replicate - if(!db) + /* + If the user has specified restrictions on which databases to replicate + and db was not selected, do not replicate. + */ + if (!db) return 0; - if(!do_list.is_empty()) // if the do's are not empty - { - I_List_iterator<i_string> it(do_list); - i_string* tmp; + if (!do_list.is_empty()) // if the do's are not empty + { + I_List_iterator<i_string> it(do_list); + i_string* tmp; - while((tmp=it++)) - { - if(!strcmp(tmp->ptr, db)) - return 1; // match - } - return 0; + while ((tmp=it++)) + { + if (!strcmp(tmp->ptr, db)) + return 1; // match } + return 0; + } else // there are some elements in the don't, otherwise we cannot get here - { - I_List_iterator<i_string> it(ignore_list); - i_string* tmp; + { + I_List_iterator<i_string> it(ignore_list); + i_string* tmp; - while((tmp=it++)) - { - if(!strcmp(tmp->ptr, db)) - return 0; // match - } - - return 1; + while ((tmp=it++)) + { + if (!strcmp(tmp->ptr, db)) + return 0; // match } + return 1; + } } -static int init_strvar_from_file(char* var, int max_size, IO_CACHE* f, - char* default_val) + +static int init_strvar_from_file(char *var, int max_size, IO_CACHE *f, + const char *default_val) { uint length; if ((length=my_b_gets(f,var, max_size))) @@ -320,10 +801,12 @@ static int init_strvar_from_file(char* var, int max_size, IO_CACHE* f, *last_p = 0; // if we stopped on newline, kill it else { - // if we truncated a line or stopped on last char, remove all chars - // up to and including newline + /* + If we truncated a line or stopped on last char, remove all chars + up to and including newline. + */ int c; - while( ((c=my_b_get(f)) != '\n' && c != my_b_EOF)); + while (((c=my_b_get(f)) != '\n' && c != my_b_EOF)); } return 0; } @@ -335,6 +818,7 @@ static int init_strvar_from_file(char* var, int max_size, IO_CACHE* f, return 1; } + static int init_intvar_from_file(int* var, IO_CACHE* f, int default_val) { char buf[32]; @@ -344,7 +828,7 @@ static int init_intvar_from_file(int* var, IO_CACHE* f, int default_val) *var = atoi(buf); return 0; } - else if(default_val) + else if (default_val) { *var = default_val; return 0; @@ -353,15 +837,42 @@ static int init_intvar_from_file(int* var, IO_CACHE* f, int default_val) } +static int check_master_version(MYSQL* mysql, MASTER_INFO* mi) +{ + const char* errmsg= 0; + + switch (*mysql->server_version) { + case '3': + mi->old_format = 1; + break; + case '4': + case '5': + mi->old_format = 0; + break; + default: + errmsg = "Master reported unrecognized MySQL version"; + break; + } + + if (errmsg) + { + sql_print_error(errmsg); + return 1; + } + return 0; +} + + static int create_table_from_dump(THD* thd, NET* net, const char* db, const char* table_name) { - uint packet_len = my_net_read(net); // read create table statement + ulong packet_len = my_net_read(net); // read create table statement Vio* save_vio; HA_CHECK_OPT check_opt; TABLE_LIST tables; int error= 1; handler *file; + ulong save_options; if (packet_len == packet_error) { @@ -387,12 +898,17 @@ static int create_table_from_dump(THD* thd, NET* net, const char* db, thd->current_tablenr = 0; thd->query_error = 0; thd->net.no_send_ok = 1; + + /* we do not want to log create table statement */ + save_options = thd->options; + thd->options &= ~(ulong) (OPTION_BIN_LOG); thd->proc_info = "Creating table from master dump"; // save old db in case we are creating in a different database char* save_db = thd->db; - thd->db = thd->last_nx_db; + thd->db = (char*)db; mysql_parse(thd, thd->query, packet_len); // run create table thd->db = save_db; // leave things the way the were before + thd->options = save_options; if (thd->query_error) goto err; // mysql_parse took care of the error send @@ -420,12 +936,13 @@ static int create_table_from_dump(THD* thd, NET* net, const char* db, } check_opt.init(); - check_opt.flags|= T_VERY_SILENT | T_CALC_CHECKSUM; - check_opt.quick = 1; + check_opt.flags|= T_VERY_SILENT | T_CALC_CHECKSUM | T_QUICK; thd->proc_info = "Rebuilding the index on master dump table"; - // we do not want repair() to spam us with messages - // just send them to the error log, and report the failure in case of - // problems + /* + We do not want repair() to spam us with messages + just send them to the error log, and report the failure in case of + problems. + */ save_vio = thd->net.vio; thd->net.vio = 0; error=file->repair(thd,&check_opt) != 0; @@ -439,100 +956,310 @@ err: return error; } -int fetch_nx_table(THD* thd, MASTER_INFO* mi) +int fetch_master_table(THD *thd, const char *db_name, const char *table_name, + MASTER_INFO *mi, MYSQL *mysql) { - MYSQL* mysql = mc_mysql_init(NULL); - int error = 1; - int nx_errno = 0; - if (!mysql) + int error= 1; + const char *errmsg=0; + bool called_connected= (mysql != NULL); + DBUG_ENTER("fetch_master_table"); + DBUG_PRINT("enter", ("db_name: '%s' table_name: '%s'", + db_name,table_name)); + + if (!called_connected) + { + if (!(mysql = mc_mysql_init(NULL))) + { + send_error(&thd->net); // EOM + DBUG_RETURN(1); + } + if (connect_to_master(thd, mysql, mi)) + { + net_printf(&thd->net, ER_CONNECT_TO_MASTER, mc_mysql_error(mysql)); + mc_mysql_close(mysql); + DBUG_RETURN(1); + } + if (thd->killed) + goto err; + } + + if (request_table_dump(mysql, db_name, table_name)) { - sql_print_error("fetch_nx_table: Error in mysql_init()"); - nx_errno = ER_GET_ERRNO; + error= ER_UNKNOWN_ERROR; + errmsg= "Failed on table dump request"; goto err; } + if (create_table_from_dump(thd, &mysql->net, db_name, + table_name)) + goto err; // create_table_from_dump will have sent the error already + error = 0; - if (!mi->host || !*mi->host) + err: + thd->net.no_send_ok = 0; // Clear up garbage after create_table_from_dump + if (!called_connected) + mc_mysql_close(mysql); + if (errmsg && thd->net.vio) + send_error(&thd->net, error, errmsg); + DBUG_RETURN(test(error)); // Return 1 on error +} + + +void end_master_info(MASTER_INFO* mi) +{ + DBUG_ENTER("end_master_info"); + + if (!mi->inited) + DBUG_VOID_RETURN; + end_relay_log_info(&mi->rli); + if (mi->fd >= 0) { - nx_errno = ER_BAD_HOST_ERROR; - goto err; + end_io_cache(&mi->file); + (void)my_close(mi->fd, MYF(MY_WME)); + mi->fd = -1; } + mi->inited = 0; + + DBUG_VOID_RETURN; +} - safe_connect(thd, mysql, mi); - if (slave_killed(thd)) - goto err; - if (request_table_dump(mysql, thd->last_nx_db, thd->last_nx_table)) +int init_relay_log_info(RELAY_LOG_INFO* rli, const char* info_fname) +{ + char fname[FN_REFLEN+128]; + int info_fd; + const char* msg = 0; + int error = 0; + DBUG_ENTER("init_relay_log_info"); + + if (rli->inited) // Set if this function called + DBUG_RETURN(0); + fn_format(fname, info_fname, mysql_data_home, "", 4+32); + pthread_mutex_lock(&rli->data_lock); + info_fd = rli->info_fd; + rli->pending = 0; + rli->cur_log_fd = -1; + rli->slave_skip_counter=0; + rli->log_pos_current=0; + rli->abort_pos_wait=0; + rli->skip_log_purge=0; + rli->log_space_limit = relay_log_space_limit; + rli->log_space_total = 0; + + // TODO: make this work with multi-master + if (!opt_relay_logname) { - nx_errno = ER_GET_ERRNO; - sql_print_error("fetch_nx_table: failed on table dump request "); - goto err; + char tmp[FN_REFLEN]; + /* + TODO: The following should be using fn_format(); We just need to + first change fn_format() to cut the file name if it's too long. + */ + strmake(tmp,glob_hostname,FN_REFLEN-5); + strmov(strcend(tmp,'.'),"-relay-bin"); + opt_relay_logname=my_strdup(tmp,MYF(MY_WME)); } + if (open_log(&rli->relay_log, glob_hostname, opt_relay_logname, + "-relay-bin", opt_relaylog_index_name, + LOG_BIN, 1 /* read_append cache */, + 1 /* no auto events */)) + DBUG_RETURN(1); + + /* if file does not exist */ + if (access(fname,F_OK)) + { + /* + If someone removed the file from underneath our feet, just close + the old descriptor and re-create the old file + */ + if (info_fd >= 0) + my_close(info_fd, MYF(MY_WME)); + if ((info_fd = my_open(fname, O_CREAT|O_RDWR|O_BINARY, MYF(MY_WME))) < 0 || + init_io_cache(&rli->info_file, info_fd, IO_SIZE*2, READ_CACHE, 0L,0, + MYF(MY_WME))) + { + msg= current_thd->net.last_error; + goto err; + } - if (create_table_from_dump(thd, &mysql->net, thd->last_nx_db, - thd->last_nx_table)) + /* Init relay log with first entry in the relay index file */ + if (init_relay_log_pos(rli,NullS,BIN_LOG_HEADER_SIZE,0 /* no data lock */, + &msg)) + goto err; + rli->master_log_pos = 0; // uninitialized + rli->info_fd = info_fd; + } + else // file exists { - // create_table_from_dump will have sent the error alread - sql_print_error("fetch_nx_table: failed on create table "); + if (info_fd >= 0) + reinit_io_cache(&rli->info_file, READ_CACHE, 0L,0,0); + else if ((info_fd = my_open(fname, O_RDWR|O_BINARY, MYF(MY_WME))) < 0 || + init_io_cache(&rli->info_file, info_fd, + IO_SIZE*2, READ_CACHE, 0L, 0, MYF(MY_WME))) + { + if (info_fd >= 0) + my_close(info_fd, MYF(0)); + rli->info_fd= -1; + rli->relay_log.close(1); + pthread_mutex_unlock(&rli->data_lock); + DBUG_RETURN(1); + } + + rli->info_fd = info_fd; + int relay_log_pos, master_log_pos; + if (init_strvar_from_file(rli->relay_log_name, + sizeof(rli->relay_log_name), &rli->info_file, + "") || + init_intvar_from_file(&relay_log_pos, + &rli->info_file, BIN_LOG_HEADER_SIZE) || + init_strvar_from_file(rli->master_log_name, + sizeof(rli->master_log_name), &rli->info_file, + "") || + init_intvar_from_file(&master_log_pos, &rli->info_file, 0)) + { + msg="Error reading slave log configuration"; + goto err; + } + rli->relay_log_pos= relay_log_pos; + rli->master_log_pos= master_log_pos; + + if (init_relay_log_pos(rli, + rli->relay_log_name, + rli->relay_log_pos, + 0 /* no data lock*/, + &msg)) + goto err; + } + DBUG_ASSERT(rli->relay_log_pos >= BIN_LOG_HEADER_SIZE); + DBUG_ASSERT(my_b_tell(rli->cur_log) == rli->relay_log_pos); + /* + Now change the cache from READ to WRITE - must do this + before flush_relay_log_info + */ + reinit_io_cache(&rli->info_file, WRITE_CACHE,0L,0,1); + error= flush_relay_log_info(rli); + if (count_relay_log_space(rli)) + { + msg="Error counting relay log space"; goto err; } - - error = 0; + rli->inited= 1; + pthread_mutex_unlock(&rli->data_lock); + DBUG_RETURN(error); - err: - if (mysql) - mc_mysql_close(mysql); - if (nx_errno && thd->net.vio) - send_error(&thd->net, nx_errno, "Error in fetch_nx_table"); - thd->net.no_send_ok = 0; // Clear up garbage after create_table_from_dump - return error; +err: + sql_print_error(msg); + end_io_cache(&rli->info_file); + if (info_fd >= 0) + my_close(info_fd, MYF(0)); + rli->info_fd= -1; + rli->relay_log.close(1); + pthread_mutex_unlock(&rli->data_lock); + DBUG_RETURN(1); } -void end_master_info(MASTER_INFO* mi) + +static inline int add_relay_log(RELAY_LOG_INFO* rli,LOG_INFO* linfo) { - if(mi->fd >= 0) - { - end_io_cache(&mi->file); - (void)my_close(mi->fd, MYF(MY_WME)); - mi->fd = -1; - } - mi->inited = 0; + MY_STAT s; + DBUG_ENTER("add_relay_log"); + if (!my_stat(linfo->log_file_name,&s,MYF(0))) + { + sql_print_error("log %s listed in the index, but failed to stat", + linfo->log_file_name); + DBUG_RETURN(1); + } + rli->log_space_total += s.st_size; +#ifndef DBUG_OFF + char buf[22]; + DBUG_PRINT("info",("log_space_total: %s", llstr(rli->log_space_total,buf))); +#endif + DBUG_RETURN(0); } -int init_master_info(MASTER_INFO* mi) + +static bool wait_for_relay_log_space(RELAY_LOG_INFO* rli) { - if (mi->inited) - return 0; - int fd,length,error; - MY_STAT stat_area; + bool slave_killed=0; + MASTER_INFO* mi = rli->mi; + const char* save_proc_info; + THD* thd = mi->io_thd; + + DBUG_ENTER("wait_for_relay_log_space"); + pthread_mutex_lock(&rli->log_space_lock); + save_proc_info = thd->proc_info; + thd->proc_info = "Waiting for relay log space to free"; + while (rli->log_space_limit < rli->log_space_total && + !(slave_killed=io_slave_killed(thd,mi))) + { + pthread_cond_wait(&rli->log_space_cond, &rli->log_space_lock); + } + thd->proc_info = save_proc_info; + pthread_mutex_unlock(&rli->log_space_lock); + DBUG_RETURN(slave_killed); +} + + +static int count_relay_log_space(RELAY_LOG_INFO* rli) +{ + LOG_INFO linfo; + DBUG_ENTER("count_relay_log_space"); + rli->log_space_total = 0; + if (rli->relay_log.find_log_pos(&linfo, NullS, 1)) + { + sql_print_error("Could not find first log while counting relay log space"); + DBUG_RETURN(1); + } + do + { + if (add_relay_log(rli,&linfo)) + DBUG_RETURN(1); + } while (!rli->relay_log.find_next_log(&linfo, 1)); + DBUG_RETURN(0); +} + + +int init_master_info(MASTER_INFO* mi, const char* master_info_fname, + const char* slave_info_fname, + bool abort_if_no_master_info_file) +{ + int fd,error; char fname[FN_REFLEN+128]; - const char *msg; - fn_format(fname, master_info_file, mysql_data_home, "", 4+16+32); + DBUG_ENTER("init_master_info"); - // we need a mutex while we are changing master info parameters to - // keep other threads from reading bogus info + if (mi->inited) + DBUG_RETURN(0); + mi->mysql=0; + mi->file_id=1; + mi->ignore_stop_event=0; + fn_format(fname, master_info_fname, mysql_data_home, "", 4+32); - pthread_mutex_lock(&mi->lock); - mi->pending = 0; + /* + We need a mutex while we are changing master info parameters to + keep other threads from reading bogus info + */ + + pthread_mutex_lock(&mi->data_lock); fd = mi->fd; - // we do not want any messages if the file does not exist - if (!my_stat(fname, &stat_area, MYF(0))) + if (access(fname,F_OK)) { - // if someone removed the file from underneath our feet, just close - // the old descriptor and re-create the old file - if (fd >= 0) - my_close(fd, MYF(MY_WME)); - if ((fd = my_open(fname, O_CREAT|O_RDWR|O_BINARY, MYF(MY_WME))) < 0 - || init_io_cache(&mi->file, fd, IO_SIZE*2, READ_CACHE, 0L,0, - MYF(MY_WME))) + if (abort_if_no_master_info_file) { - if(fd >= 0) - my_close(fd, MYF(0)); - pthread_mutex_unlock(&mi->lock); - return 1; + pthread_mutex_unlock(&mi->data_lock); + DBUG_RETURN(0); } - mi->log_file_name[0] = 0; - mi->pos = 4; // skip magic number + /* + if someone removed the file from underneath our feet, just close + the old descriptor and re-create the old file + */ + if (fd >= 0) + my_close(fd, MYF(MY_WME)); + if ((fd = my_open(fname, O_CREAT|O_RDWR|O_BINARY, MYF(MY_WME))) < 0 || + init_io_cache(&mi->file, fd, IO_SIZE*2, READ_CACHE, 0L,0, + MYF(MY_WME))) + goto err; + + mi->master_log_name[0] = 0; + mi->master_log_pos = BIN_LOG_HEADER_SIZE; // skip magic number mi->fd = fd; if (master_host) @@ -546,240 +1273,318 @@ int init_master_info(MASTER_INFO* mi) } else // file exists { - if(fd >= 0) + if (fd >= 0) reinit_io_cache(&mi->file, READ_CACHE, 0L,0,0); - else if((fd = my_open(fname, O_RDWR|O_BINARY, MYF(MY_WME))) < 0 - || init_io_cache(&mi->file, fd, IO_SIZE*2, READ_CACHE, 0L, - 0, MYF(MY_WME))) - { - if(fd >= 0) - my_close(fd, MYF(0)); - pthread_mutex_unlock(&mi->lock); - return 1; - } - - if ((length=my_b_gets(&mi->file, mi->log_file_name, - sizeof(mi->log_file_name))) < 1) - { - msg="Error reading log file name from master info file "; - goto error; - } - - mi->log_file_name[length-1]= 0; // kill \n - /* Reuse fname buffer */ - if(!my_b_gets(&mi->file, fname, sizeof(fname))) - { - msg="Error reading log file position from master info file"; - goto error; - } - mi->pos = strtoull(fname,(char**) 0, 10); + else if ((fd = my_open(fname, O_RDWR|O_BINARY, MYF(MY_WME))) < 0 || + init_io_cache(&mi->file, fd, IO_SIZE*2, READ_CACHE, 0L, + 0, MYF(MY_WME))) + goto err; mi->fd = fd; - if(init_strvar_from_file(mi->host, sizeof(mi->host), &mi->file, - master_host) || - init_strvar_from_file(mi->user, sizeof(mi->user), &mi->file, - master_user) || - init_strvar_from_file(mi->password, HASH_PASSWORD_LENGTH+1, &mi->file, - master_password) || - init_intvar_from_file((int*)&mi->port, &mi->file, master_port) || - init_intvar_from_file((int*)&mi->connect_retry, &mi->file, - master_connect_retry)) + int port, connect_retry, master_log_pos; + + if (init_strvar_from_file(mi->master_log_name, + sizeof(mi->master_log_name), &mi->file, + "") || + init_intvar_from_file(&master_log_pos, &mi->file, 4) || + init_strvar_from_file(mi->host, sizeof(mi->host), &mi->file, + master_host) || + init_strvar_from_file(mi->user, sizeof(mi->user), &mi->file, + master_user) || + init_strvar_from_file(mi->password, HASH_PASSWORD_LENGTH+1, &mi->file, + master_password) || + init_intvar_from_file(&port, &mi->file, master_port) || + init_intvar_from_file(&connect_retry, &mi->file, + master_connect_retry)) { - msg="Error reading master configuration"; - goto error; + sql_print_error("Error reading master configuration"); + goto err; } + /* + This has to be handled here as init_intvar_from_file can't handle + my_off_t types + */ + mi->master_log_pos= (my_off_t) master_log_pos; + mi->port= (uint) port; + mi->connect_retry= (uint) connect_retry; } - + DBUG_PRINT("master_info",("log_file_name: %s position: %ld", + mi->master_log_name, + (ulong) mi->master_log_pos)); + + if (init_relay_log_info(&mi->rli, slave_info_fname)) + goto err; + mi->rli.mi = mi; + mi->inited = 1; - // now change the cache from READ to WRITE - must do this - // before flush_master_info - reinit_io_cache(&mi->file, WRITE_CACHE, 0L,0,1); + // now change cache READ -> WRITE - must do this before flush_master_info + reinit_io_cache(&mi->file, WRITE_CACHE,0L,0,1); error=test(flush_master_info(mi)); - pthread_mutex_unlock(&mi->lock); - return error; + pthread_mutex_unlock(&mi->data_lock); + DBUG_RETURN(error); -error: - sql_print_error(msg); - end_io_cache(&mi->file); - my_close(fd, MYF(0)); - pthread_mutex_unlock(&mi->lock); - return 1; +err: + if (fd >= 0) + { + my_close(fd, MYF(0)); + end_io_cache(&mi->file); + } + mi->fd= -1; + pthread_mutex_unlock(&mi->data_lock); + DBUG_RETURN(1); } -int show_master_info(THD* thd) + +int register_slave_on_master(MYSQL* mysql) { + String packet; + char buf[4]; + + if (!report_host) + return 0; + + int4store(buf, server_id); + packet.append(buf, 4); + + net_store_data(&packet, report_host); + if (report_user) + net_store_data(&packet, report_user); + else + packet.append((char)0); + + if (report_password) + net_store_data(&packet, report_user); + else + packet.append((char)0); + + int2store(buf, (uint16)report_port); + packet.append(buf, 2); + int4store(buf, rpl_recovery_rank); + packet.append(buf, 4); + int4store(buf, 0); /* tell the master will fill in master_id */ + packet.append(buf, 4); + + if (mc_simple_command(mysql, COM_REGISTER_SLAVE, (char*)packet.ptr(), + packet.length(), 0)) + { + sql_print_error("Error on COM_REGISTER_SLAVE: %d '%s'", + mc_mysql_errno(mysql), + mc_mysql_error(mysql)); + return 1; + } + + return 0; +} + +int show_master_info(THD* thd, MASTER_INFO* mi) +{ + // TODO: fix this for multi-master DBUG_ENTER("show_master_info"); List<Item> field_list; field_list.push_back(new Item_empty_string("Master_Host", - sizeof(glob_mi.host))); + sizeof(mi->host))); field_list.push_back(new Item_empty_string("Master_User", - sizeof(glob_mi.user))); + sizeof(mi->user))); field_list.push_back(new Item_empty_string("Master_Port", 6)); field_list.push_back(new Item_empty_string("Connect_retry", 6)); - field_list.push_back( new Item_empty_string("Log_File", + field_list.push_back(new Item_empty_string("Master_Log_File", + FN_REFLEN)); + field_list.push_back(new Item_empty_string("Read_Master_Log_Pos", 12)); + field_list.push_back(new Item_empty_string("Relay_Log_File", + FN_REFLEN)); + field_list.push_back(new Item_empty_string("Relay_Log_Pos", 12)); + field_list.push_back(new Item_empty_string("Relay_Master_Log_File", FN_REFLEN)); - field_list.push_back(new Item_empty_string("Pos", 12)); - field_list.push_back(new Item_empty_string("Slave_Running", 3)); + field_list.push_back(new Item_empty_string("Slave_IO_Running", 3)); + field_list.push_back(new Item_empty_string("Slave_SQL_Running", 3)); field_list.push_back(new Item_empty_string("Replicate_do_db", 20)); field_list.push_back(new Item_empty_string("Replicate_ignore_db", 20)); field_list.push_back(new Item_empty_string("Last_errno", 4)); field_list.push_back(new Item_empty_string("Last_error", 20)); field_list.push_back(new Item_empty_string("Skip_counter", 12)); - if(send_fields(thd, field_list, 1)) + field_list.push_back(new Item_empty_string("Exec_master_log_pos", 12)); + field_list.push_back(new Item_empty_string("Relay_log_space", 12)); + if (send_fields(thd, field_list, 1)) DBUG_RETURN(-1); - String* packet = &thd->packet; - packet->length(0); + if (mi->host[0]) + { + String *packet= &thd->packet; + packet->length(0); - pthread_mutex_lock(&glob_mi.lock); - net_store_data(packet, glob_mi.host); - net_store_data(packet, glob_mi.user); - net_store_data(packet, (uint32) glob_mi.port); - net_store_data(packet, (uint32) glob_mi.connect_retry); - net_store_data(packet, glob_mi.log_file_name); - net_store_data(packet, (uint32) glob_mi.pos); // QQ: Should be fixed - pthread_mutex_unlock(&glob_mi.lock); - pthread_mutex_lock(&LOCK_slave); - net_store_data(packet, slave_running ? "Yes":"No"); - pthread_mutex_unlock(&LOCK_slave); - net_store_data(packet, &replicate_do_db); - net_store_data(packet, &replicate_ignore_db); - net_store_data(packet, (uint32)last_slave_errno); - net_store_data(packet, last_slave_error); - net_store_data(packet, slave_skip_counter); + pthread_mutex_lock(&mi->data_lock); + pthread_mutex_lock(&mi->rli.data_lock); + net_store_data(packet, mi->host); + net_store_data(packet, mi->user); + net_store_data(packet, (uint32) mi->port); + net_store_data(packet, (uint32) mi->connect_retry); + net_store_data(packet, mi->master_log_name); + net_store_data(packet, (longlong) mi->master_log_pos); + net_store_data(packet, mi->rli.relay_log_name + + dirname_length(mi->rli.relay_log_name)); + net_store_data(packet, (longlong) mi->rli.relay_log_pos); + net_store_data(packet, mi->rli.master_log_name); + net_store_data(packet, mi->slave_running ? "Yes":"No"); + net_store_data(packet, mi->rli.slave_running ? "Yes":"No"); + net_store_data(packet, &replicate_do_db); + net_store_data(packet, &replicate_ignore_db); + net_store_data(packet, (uint32)mi->rli.last_slave_errno); + net_store_data(packet, mi->rli.last_slave_error); + net_store_data(packet, mi->rli.slave_skip_counter); + net_store_data(packet, (longlong) mi->rli.master_log_pos); + net_store_data(packet, (longlong) mi->rli.log_space_total); + pthread_mutex_unlock(&mi->rli.data_lock); + pthread_mutex_unlock(&mi->data_lock); - if (my_net_write(&thd->net, (char*)thd->packet.ptr(), packet->length())) - DBUG_RETURN(-1); - + if (my_net_write(&thd->net, (char*)thd->packet.ptr(), packet->length())) + DBUG_RETURN(-1); + } send_eof(&thd->net); DBUG_RETURN(0); } -int flush_master_info(MASTER_INFO* mi) + +bool flush_master_info(MASTER_INFO* mi) { IO_CACHE* file = &mi->file; char lbuf[22]; - + DBUG_ENTER("flush_master_info"); + DBUG_PRINT("enter",("master_pos: %ld", (long) mi->master_log_pos)); + my_b_seek(file, 0L); my_b_printf(file, "%s\n%s\n%s\n%s\n%s\n%d\n%d\n", - mi->log_file_name, llstr(mi->pos, lbuf), mi->host, mi->user, - mi->password, mi->port, mi->connect_retry); + mi->master_log_name, llstr(mi->master_log_pos, lbuf), + mi->host, mi->user, + mi->password, mi->port, mi->connect_retry + ); flush_io_cache(file); - return 0; + DBUG_RETURN(0); } -int st_master_info::wait_for_pos(THD* thd, String* log_name, ulonglong log_pos) + +int st_relay_log_info::wait_for_pos(THD* thd, String* log_name, + ulonglong log_pos) { - if (!inited) return -1; - bool pos_reached; + if (!inited) + return -1; int event_count = 0; - pthread_mutex_lock(&lock); - while(!thd->killed) + ulong init_abort_pos_wait; + DBUG_ENTER("wait_for_pos"); + DBUG_PRINT("enter",("master_log_name: '%s' pos: %ld", + master_log_name, (ulong) master_log_pos)); + + pthread_mutex_lock(&data_lock); + // abort only if master info changes during wait + init_abort_pos_wait= abort_pos_wait; + + while (!thd->killed && + init_abort_pos_wait == abort_pos_wait && + mi->slave_running) { - int cmp_result; - if (*log_file_name) + bool pos_reached; + int cmp_result= 0; + DBUG_ASSERT(*master_log_name || master_log_pos == 0); + if (*master_log_name) { /* - We should use dirname_length() here when we have a version of - this that doesn't modify the argument */ - char *basename = strrchr(log_file_name, FN_LIBCHAR); - if (basename) - ++basename; - else - basename = log_file_name; + TODO: + Replace strncmp() with a comparison function that + can handle comparison of the following files: + mysqlbin.999 + mysqlbin.1000 + */ + char *basename= master_log_name + dirname_length(master_log_name); cmp_result = strncmp(basename, log_name->ptr(), log_name->length()); } - else - cmp_result = 0; - - pos_reached = ((!cmp_result && pos >= log_pos) || cmp_result > 0); + pos_reached = ((!cmp_result && master_log_pos >= log_pos) || + cmp_result > 0); if (pos_reached || thd->killed) break; - const char* msg = thd->enter_cond(&cond, &lock, + DBUG_PRINT("info",("Waiting for master update")); + const char* msg = thd->enter_cond(&data_cond, &data_lock, "Waiting for master update"); - pthread_cond_wait(&cond, &lock); + pthread_cond_wait(&data_cond, &data_lock); thd->exit_cond(msg); event_count++; } - pthread_mutex_unlock(&lock); - return thd->killed ? -1 : event_count; + pthread_mutex_unlock(&data_lock); + DBUG_PRINT("exit",("killed: %d abort: %d slave_running: %d", + (int) thd->killed, + (int) (init_abort_pos_wait != abort_pos_wait), + (int) mi->slave_running)); + DBUG_RETURN((thd->killed || init_abort_pos_wait != abort_pos_wait || + !mi->slave_running) ? + -1 : event_count); } -static int init_slave_thread(THD* thd) +static int init_slave_thread(THD* thd, SLAVE_THD_TYPE thd_type) { DBUG_ENTER("init_slave_thread"); thd->system_thread = thd->bootstrap = 1; thd->client_capabilities = 0; my_net_init(&thd->net, 0); - thd->net.timeout = slave_net_timeout; - thd->max_packet_length=thd->net.max_packet; + thd->net.read_timeout = slave_net_timeout; thd->master_access= ~0; thd->priv_user = 0; thd->slave_thread = 1; thd->options = (((opt_log_slave_updates) ? OPTION_BIN_LOG:0) | OPTION_AUTO_IS_NULL) ; thd->system_thread = 1; thd->client_capabilities = CLIENT_LOCAL_FILES; - slave_real_id=thd->real_id=pthread_self(); + thd->real_id=pthread_self(); pthread_mutex_lock(&LOCK_thread_count); thd->thread_id = thread_id++; pthread_mutex_unlock(&LOCK_thread_count); - if (init_thr_lock() || - my_pthread_setspecific_ptr(THR_THD, thd) || - my_pthread_setspecific_ptr(THR_MALLOC, &thd->mem_root) || - my_pthread_setspecific_ptr(THR_NET, &thd->net)) + if (init_thr_lock() || thd->store_globals()) { - close_connection(&thd->net,ER_OUT_OF_RESOURCES); // is this needed? end_thread(thd,0); DBUG_RETURN(-1); } - thd->mysys_var=my_thread_var; - thd->dbug_thread_id=my_thread_id(); #if !defined(__WIN__) && !defined(OS2) sigset_t set; VOID(sigemptyset(&set)); // Get mask in use VOID(pthread_sigmask(SIG_UNBLOCK,&set,&thd->block_signals)); #endif - thd->mem_root.free=thd->mem_root.used=0; // Probably not needed - if (thd->max_join_size == (ulong) ~0L) + if (thd->variables.max_join_size == HA_POS_ERROR) thd->options |= OPTION_BIG_SELECTS; - thd->proc_info="Waiting for master update"; + if (thd_type == SLAVE_THD_SQL) + thd->proc_info= "Waiting for the next event in slave queue"; + else + thd->proc_info= "Waiting for master update"; thd->version=refresh_version; thd->set_time(); - DBUG_RETURN(0); } -static int safe_sleep(THD* thd, int sec) + +static int safe_sleep(THD* thd, int sec, CHECK_KILLED_FUNC thread_killed, + void* thread_killed_arg) { + int nap_time; thr_alarm_t alarmed; thr_alarm_init(&alarmed); time_t start_time= time((time_t*) 0); time_t end_time= start_time+sec; - ALARM alarm_buff; - while (start_time < end_time) + while ((nap_time= (int) (end_time - start_time)) > 0) { - int nap_time = (int) (end_time - start_time); + ALARM alarm_buff; /* - the only reason we are asking for alarm is so that + The only reason we are asking for alarm is so that we will be woken up in case of murder, so if we do not get killed, set the alarm so it goes off after we wake up naturally */ - thr_alarm(&alarmed, 2 * nap_time,&alarm_buff); + thr_alarm(&alarmed, 2 * nap_time, &alarm_buff); sleep(nap_time); - // if we wake up before the alarm goes off, hit the button - // so it will not wake up the wife and kids :-) - if (thr_alarm_in_use(&alarmed)) - thr_end_alarm(&alarmed); + thr_end_alarm(&alarmed); - if (slave_killed(thd)) + if ((*thread_killed)(thd,thread_killed_arg)) return 1; start_time=time((time_t*) 0); } @@ -787,41 +1592,52 @@ static int safe_sleep(THD* thd, int sec) } -static int request_dump(MYSQL* mysql, MASTER_INFO* mi) +static int request_dump(MYSQL* mysql, MASTER_INFO* mi, + bool *suppress_warnings) { char buf[FN_REFLEN + 10]; int len; int binlog_flags = 0; // for now - char* logname = mi->log_file_name; - int4store(buf, mi->pos); + char* logname = mi->master_log_name; + DBUG_ENTER("request_dump"); + + // TODO if big log files: Change next to int8store() + int4store(buf, (longlong) mi->master_log_pos); int2store(buf + 4, binlog_flags); int4store(buf + 6, server_id); len = (uint) strlen(logname); memcpy(buf + 10, logname,len); if (mc_simple_command(mysql, COM_BINLOG_DUMP, buf, len + 10, 1)) { - // something went wrong, so we will just reconnect and retry later - // in the future, we should do a better error analysis, but for - // now we just fill up the error log :-) - sql_print_error("Error on COM_BINLOG_DUMP: %s, will retry in %d secs", - mc_mysql_error(mysql), master_connect_retry); - return 1; + /* + Something went wrong, so we will just reconnect and retry later + in the future, we should do a better error analysis, but for + now we just fill up the error log :-) + */ + if (mc_mysql_errno(mysql) == ER_NET_READ_INTERRUPTED) + *suppress_warnings= 1; // Suppress reconnect warning + else + sql_print_error("Error on COM_BINLOG_DUMP: %d %s, will retry in %d secs", + mc_mysql_errno(mysql), mc_mysql_error(mysql), + master_connect_retry); + DBUG_RETURN(1); } - return 0; + DBUG_RETURN(0); } -static int request_table_dump(MYSQL* mysql, char* db, char* table) + +static int request_table_dump(MYSQL* mysql, const char* db, const char* table) { char buf[1024]; char * p = buf; uint table_len = (uint) strlen(table); uint db_len = (uint) strlen(db); - if(table_len + db_len > sizeof(buf) - 2) - { - sql_print_error("request_table_dump: Buffer overrun"); - return 1; - } + if (table_len + db_len > sizeof(buf) - 2) + { + sql_print_error("request_table_dump: Buffer overrun"); + return 1; + } *p++ = db_len; memcpy(p, db, db_len); @@ -841,26 +1657,40 @@ command"); /* - We set suppress_warnings TRUE when a normal net read timeout has - caused us to try a reconnect. We do not want to print anything to - the error log in this case because this a anormal event in an idle - server. + read one event from the master + + SYNOPSIS + read_event() + mysql MySQL connection + mi Master connection information + suppress_warnings TRUE when a normal net read timeout has caused us to + try a reconnect. We do not want to print anything to + the error log in this case because this a anormal + event in an idle server. + + RETURN VALUES + 'packet_error' Error + number Length of packet + */ -static uint read_event(MYSQL* mysql, MASTER_INFO *mi, bool* suppress_warnings) +static ulong read_event(MYSQL* mysql, MASTER_INFO *mi, bool* suppress_warnings) { - uint len = packet_error; + ulong len; - // my_real_read() will time us out - // we check if we were told to die, and if not, try reading again + *suppress_warnings= 0; + /* + my_real_read() will time us out + We check if we were told to die, and if not, try reading again + + TODO: Move 'events_till_disconnect' to the MASTER_INFO structure + */ #ifndef DBUG_OFF if (disconnect_slave_event_count && !(events_till_disconnect--)) return packet_error; #endif - *suppress_warnings= 0; - + len = mc_net_safe_read(mysql); - if (len == packet_error || (long) len < 1) { if (mc_mysql_errno(mysql) == ER_NET_READ_INTERRUPTED) @@ -892,351 +1722,82 @@ server_errno=%d)", return len - 1; } -static int check_expected_error(THD* thd, int expected_error) + +int check_expected_error(THD* thd, RELAY_LOG_INFO* rli, int expected_error) { - switch(expected_error) - { - case ER_NET_READ_ERROR: - case ER_NET_ERROR_ON_WRITE: - case ER_SERVER_SHUTDOWN: - case ER_NEW_ABORTING_CONNECTION: - my_snprintf(last_slave_error, sizeof(last_slave_error), - "Slave: query '%s' partially completed on the master \ + switch (expected_error) { + case ER_NET_READ_ERROR: + case ER_NET_ERROR_ON_WRITE: + case ER_SERVER_SHUTDOWN: + case ER_NEW_ABORTING_CONNECTION: + my_snprintf(rli->last_slave_error, sizeof(rli->last_slave_error), + "Slave: query '%s' partially completed on the master \ and was aborted. There is a chance that your master is inconsistent at this \ point. If you are sure that your master is ok, run this query manually on the\ - slave and then restart the slave with SET SQL_SLAVE_SKIP_COUNTER=1;\ + slave and then restart the slave with SET GLOBAL SQL_SLAVE_SKIP_COUNTER=1;\ SLAVE START;", thd->query); - last_slave_errno = expected_error; - sql_print_error("%s",last_slave_error); - return 1; - default: - return 0; - } + rli->last_slave_errno = expected_error; + sql_print_error("%s",rli->last_slave_error); + return 1; + default: + return 0; + } } -inline int ignored_error_code(int err_code) -{ - return use_slave_mask && bitmap_is_set(&slave_error_mask, err_code); -} -static int exec_event(THD* thd, NET* net, MASTER_INFO* mi, int event_len) +static int exec_relay_log_event(THD* thd, RELAY_LOG_INFO* rli) { - Log_event * ev = Log_event::read_log_event((const char*)net->read_pos + 1, - event_len); - char llbuff[22]; - - mi->event_len = event_len; /* Added by Heikki: InnoDB internally stores the - master log position it has processed so far; - position to store is really - mi->pos + mi->pending + mi->event_len - since we must store the pos of the END of the - current log event */ + DBUG_ASSERT(rli->sql_thd==thd); + Log_event * ev = next_event(rli); + DBUG_ASSERT(rli->sql_thd==thd); + if (sql_slave_killed(thd,rli)) + return 1; if (ev) { int type_code = ev->get_type_code(); - if (ev->server_id == ::server_id || slave_skip_counter) + int exec_res; + pthread_mutex_lock(&rli->data_lock); + + /* + Skip queries originating from this server or number of + queries specified by the user in slave_skip_counter + We can't however skip event's that has something to do with the + log files themselves. + */ + + if (ev->server_id == (uint32) ::server_id || + (rli->slave_skip_counter && type_code != ROTATE_EVENT)) { - if(type_code == LOAD_EVENT) - skip_load_data_infile(net); - - mi->inc_pos(event_len); - flush_master_info(mi); - if(slave_skip_counter && /* protect against common user error of - setting the counter to 1 instead of 2 - while recovering from an failed - auto-increment insert */ - !(type_code == INTVAR_EVENT && - slave_skip_counter == 1)) - --slave_skip_counter; + /* TODO: I/O thread should not even log events with the same server id */ + rli->inc_pos(ev->get_event_len(), + type_code != STOP_EVENT ? ev->log_pos : LL(0), + 1/* skip lock*/); + flush_relay_log_info(rli); + + /* + Protect against common user error of setting the counter to 1 + instead of 2 while recovering from an failed auto-increment insert + */ + if (rli->slave_skip_counter && + !((type_code == INTVAR_EVENT || type_code == STOP_EVENT) && + rli->slave_skip_counter == 1)) + --rli->slave_skip_counter; + pthread_mutex_unlock(&rli->data_lock); delete ev; return 0; // avoid infinite update loops } + pthread_mutex_unlock(&rli->data_lock); thd->server_id = ev->server_id; // use the original server id for logging thd->set_time(); // time the query - if(!ev->when) + if (!ev->when) ev->when = time(NULL); - - switch(type_code) { - case QUERY_EVENT: - { - Query_log_event* qev = (Query_log_event*)ev; - int q_len = qev->q_len; - int expected_error,actual_error = 0; - init_sql_alloc(&thd->mem_root, 8192,0); - thd->db = rewrite_db((char*)qev->db); - if (db_ok(thd->db, replicate_do_db, replicate_ignore_db)) - { - thd->query = (char*)qev->query; - thd->set_time((time_t)qev->when); - thd->current_tablenr = 0; - VOID(pthread_mutex_lock(&LOCK_thread_count)); - thd->query_id = query_id++; - VOID(pthread_mutex_unlock(&LOCK_thread_count)); - thd->last_nx_table = thd->last_nx_db = 0; - thd->query_error = 0; // clear error - thd->net.last_errno = 0; - thd->net.last_error[0] = 0; - thd->slave_proxy_id = qev->thread_id; // for temp tables - - // sanity check to make sure the master did not get a really bad - // error on the query - if (ignored_error_code((expected_error=qev->error_code)) || - !check_expected_error(thd, expected_error)) - { - mysql_parse(thd, thd->query, q_len); - if (expected_error != - (actual_error = thd->net.last_errno) && expected_error && - !ignored_error_code(actual_error)) - { - const char* errmsg = "Slave: did not get the expected error\ - running query from master - expected: '%s' (%d), got '%s' (%d)"; - sql_print_error(errmsg, ER_SAFE(expected_error), - expected_error, - actual_error ? thd->net.last_error:"no error", - actual_error); - thd->query_error = 1; - } - else if (expected_error == actual_error || - ignored_error_code(actual_error)) - { - thd->query_error = 0; - *last_slave_error = 0; - last_slave_errno = 0; - } - } - else - { - // master could be inconsistent, abort and tell DBA to check/fix it - thd->db = thd->query = 0; - thd->convert_set = 0; - close_thread_tables(thd); - free_root(&thd->mem_root,0); - delete ev; - return 1; - } - } - thd->db = 0; // prevent db from being freed - thd->query = 0; // just to be sure - // assume no convert for next query unless set explictly - thd->convert_set = 0; - close_thread_tables(thd); - - if (thd->query_error || thd->fatal_error) - { - sql_print_error("Slave: error running query '%s' ", - qev->query); - last_slave_errno = actual_error ? actual_error : -1; - my_snprintf(last_slave_error, sizeof(last_slave_error), - "error '%s' on query '%s'", - actual_error ? thd->net.last_error : - "unexpected success or fatal error", - qev->query - ); - free_root(&thd->mem_root,0); - delete ev; - return 1; - } - free_root(&thd->mem_root,0); - delete ev; - - mi->inc_pos(event_len); - - if (!(thd->options & OPTION_BEGIN)) { - - /* We only flush the master info position to the master.info file if - the transaction is not open any more: an incomplete transaction will - be rolled back automatically in crash recovery in transactional - table handlers */ - - flush_master_info(mi); - } - break; - } - - case LOAD_EVENT: - { - Load_log_event* lev = (Load_log_event*)ev; - init_sql_alloc(&thd->mem_root, 8192,0); - thd->db = rewrite_db((char*)lev->db); - thd->query = 0; - thd->query_error = 0; - - if(db_ok(thd->db, replicate_do_db, replicate_ignore_db)) - { - thd->set_time((time_t)lev->when); - thd->current_tablenr = 0; - VOID(pthread_mutex_lock(&LOCK_thread_count)); - thd->query_id = query_id++; - VOID(pthread_mutex_unlock(&LOCK_thread_count)); - - TABLE_LIST tables; - bzero((char*) &tables,sizeof(tables)); - tables.db = thd->db; - tables.alias= tables.real_name= (char*)lev->table_name; - tables.lock_type = TL_WRITE; - // the table will be opened in mysql_load - if(table_rules_on && !tables_ok(thd, &tables)) - { - skip_load_data_infile(net); - } - else - { - enum enum_duplicates handle_dup = DUP_IGNORE; - if(lev->sql_ex.opt_flags && REPLACE_FLAG) - handle_dup = DUP_REPLACE; - sql_exchange ex((char*)lev->fname, lev->sql_ex.opt_flags && - DUMPFILE_FLAG ); - String field_term(&lev->sql_ex.field_term, 1), - enclosed(&lev->sql_ex.enclosed, 1), - line_term(&lev->sql_ex.line_term,1), - escaped(&lev->sql_ex.escaped, 1), - line_start(&lev->sql_ex.line_start, 1); - - ex.field_term = &field_term; - if(lev->sql_ex.empty_flags & FIELD_TERM_EMPTY) - ex.field_term->length(0); - - ex.enclosed = &enclosed; - if(lev->sql_ex.empty_flags & ENCLOSED_EMPTY) - ex.enclosed->length(0); - - ex.line_term = &line_term; - if(lev->sql_ex.empty_flags & LINE_TERM_EMPTY) - ex.line_term->length(0); - - ex.line_start = &line_start; - if(lev->sql_ex.empty_flags & LINE_START_EMPTY) - ex.line_start->length(0); - - ex.escaped = &escaped; - if(lev->sql_ex.empty_flags & ESCAPED_EMPTY) - ex.escaped->length(0); - - ex.opt_enclosed = (lev->sql_ex.opt_flags & OPT_ENCLOSED_FLAG); - if(lev->sql_ex.empty_flags & FIELD_TERM_EMPTY) - ex.field_term->length(0); - - ex.skip_lines = lev->skip_lines; - - - List<Item> fields; - lev->set_fields(fields); - thd->slave_proxy_id = thd->thread_id; - thd->net.vio = net->vio; - // mysql_load will use thd->net to read the file - thd->net.pkt_nr = net->pkt_nr; - // make sure the client does not get confused - // about the packet sequence - if(mysql_load(thd, &ex, &tables, fields, handle_dup, 1, - TL_WRITE)) - thd->query_error = 1; - if(thd->cuted_fields) - sql_print_error("Slave: load data infile at position %s in log \ -'%s' produced %d warning(s)", llstr(glob_mi.pos,llbuff), RPL_LOG_NAME, - thd->cuted_fields ); - net->pkt_nr = thd->net.pkt_nr; - } - } - else - { - // we will just ask the master to send us /dev/null if we do not - // want to load the data :-) - skip_load_data_infile(net); - } - - thd->net.vio = 0; - thd->db = 0;// prevent db from being freed - close_thread_tables(thd); - if(thd->query_error) - { - int sql_error = thd->net.last_errno; - if(!sql_error) - sql_error = ER_UNKNOWN_ERROR; - - sql_print_error("Slave: Error '%s' running load data infile ", - ER(sql_error)); - delete ev; - free_root(&thd->mem_root,0); - return 1; - } - - delete ev; - free_root(&thd->mem_root,0); - - if(thd->fatal_error) - { - sql_print_error("Slave: Fatal error running query '%s' ", - thd->query); - return 1; - } - - mi->inc_pos(event_len); - - if (!(thd->options & OPTION_BEGIN)) - flush_master_info(mi); - - break; - } - - /* Question: in a START or STOP event, what happens if we have transaction - open? */ - - case START_EVENT: - mi->inc_pos(event_len); - flush_master_info(mi); - delete ev; - break; - - case STOP_EVENT: - if(mi->pos > 4) // stop event should be ignored after rotate event - { - close_temporary_tables(thd); - mi->inc_pos(event_len); - flush_master_info(mi); - } - delete ev; - break; - case ROTATE_EVENT: - { - Rotate_log_event* rev = (Rotate_log_event*)ev; - int ident_len = rev->ident_len; - pthread_mutex_lock(&mi->lock); - memcpy(mi->log_file_name, rev->new_log_ident,ident_len ); - mi->log_file_name[ident_len] = 0; - mi->pos = 4; // skip magic number - pthread_cond_broadcast(&mi->cond); - pthread_mutex_unlock(&mi->lock); - - if (!(thd->options & OPTION_BEGIN)) - flush_master_info(mi); -#ifndef DBUG_OFF - if(abort_slave_event_count) - ++events_till_abort; -#endif - delete ev; - break; - } - - case INTVAR_EVENT: - { - Intvar_log_event* iev = (Intvar_log_event*)ev; - switch(iev->type) - { - case LAST_INSERT_ID_EVENT: - thd->last_insert_id_used = 1; - thd->last_insert_id = iev->val; - break; - case INSERT_ID_EVENT: - thd->next_insert_id = iev->val; - break; - - } - mi->inc_pending(event_len); - delete ev; - break; - } - } + ev->thd = thd; + thd->log_pos = ev->log_pos; + exec_res = ev->exec_event(rli); + DBUG_ASSERT(rli->sql_thd==thd); + delete ev; + return exec_res; } else { @@ -1246,269 +1807,272 @@ This may also be a network problem, or just a bug in the master or slave code.\ "); return 1; } - return 0; } - -// slave thread -pthread_handler_decl(handle_slave,arg __attribute__((unused))) + +/* slave I/O thread */ +extern "C" pthread_handler_decl(handle_slave_io,arg) { -#ifndef DBUG_OFF - slave_begin: -#endif THD *thd; // needs to be first for thread_stack - MYSQL *mysql = NULL ; + MYSQL *mysql; + MASTER_INFO *mi = (MASTER_INFO*)arg; char llbuff[22]; - - pthread_mutex_lock(&LOCK_slave); - if (!server_id) - { - pthread_cond_broadcast(&COND_slave_start); - pthread_mutex_unlock(&LOCK_slave); - sql_print_error("Server id not set, will not start slave"); - pthread_exit((void*)1); - } + uint retry_count; - if(slave_running) - { - pthread_cond_broadcast(&COND_slave_start); - pthread_mutex_unlock(&LOCK_slave); - pthread_exit((void*)1); // safety just in case - } - slave_running = 1; - abort_slave = 0; + // needs to call my_thread_init(), otherwise we get a coredump in DBUG_ stuff + my_thread_init(); + +#ifndef DBUG_OFF +slave_begin: +#endif + DBUG_ASSERT(mi->inited); + mysql= NULL ; + retry_count= 0; + + pthread_mutex_lock(&mi->run_lock); + /* Inform waiting threads that slave has started */ + mi->slave_run_id++; + #ifndef DBUG_OFF - events_till_abort = abort_slave_event_count; + mi->events_till_abort = abort_slave_event_count; #endif - pthread_cond_broadcast(&COND_slave_start); - pthread_mutex_unlock(&LOCK_slave); - - // int error = 1; - bool retried_once = 0; - ulonglong last_failed_pos = 0; - // needs to call my_thread_init(), otherwise we get a coredump in DBUG_ stuff - my_thread_init(); - slave_thd = thd = new THD; // note that contructor of THD uses DBUG_ ! - thd->set_time(); - DBUG_ENTER("handle_slave"); + thd= new THD; // note that contructor of THD uses DBUG_ ! + DBUG_ENTER("handle_slave_io"); + THD_CHECK_SENTRY(thd); pthread_detach_this_thread(); - if (init_slave_thread(thd) || init_master_info(&glob_mi)) - { - sql_print_error("Failed during slave thread initialization"); - goto err; - } + if (init_slave_thread(thd, SLAVE_THD_IO)) + { + pthread_cond_broadcast(&mi->start_cond); + pthread_mutex_unlock(&mi->run_lock); + sql_print_error("Failed during slave I/O thread initialization"); + goto err; + } + mi->io_thd = thd; thd->thread_stack = (char*)&thd; // remember where our stack is - thd->temporary_tables = save_temporary_tables; // restore temp tables - (void) pthread_mutex_lock(&LOCK_thread_count); + pthread_mutex_lock(&LOCK_thread_count); threads.append(thd); - (void) pthread_mutex_unlock(&LOCK_thread_count); - glob_mi.pending = 0; //this should always be set to 0 when the slave thread - // is started + pthread_mutex_unlock(&LOCK_thread_count); + mi->slave_running = 1; + mi->abort_slave = 0; + pthread_mutex_unlock(&mi->run_lock); + pthread_cond_broadcast(&mi->start_cond); - DBUG_PRINT("info",("master info: log_file_name=%s, position=%s", - glob_mi.log_file_name, llstr(glob_mi.pos,llbuff))); - + DBUG_PRINT("master_info",("log_file_name: '%s' position: %s", + mi->master_log_name, + llstr(mi->master_log_pos,llbuff))); - if (!(mysql = mc_mysql_init(NULL))) + if (!(mi->mysql = mysql = mc_mysql_init(NULL))) { - sql_print_error("Slave thread: error in mc_mysql_init()"); + sql_print_error("Slave I/O thread: error in mc_mysql_init()"); goto err; } thd->proc_info = "connecting to master"; -#ifndef DBUG_OFF - sql_print_error("Slave thread initialized"); -#endif // we can get killed during safe_connect - if (!safe_connect(thd, mysql, &glob_mi)) - sql_print_error("Slave: connected to master '%s@%s:%d',\ - replication started in log '%s' at position %s", glob_mi.user, - glob_mi.host, glob_mi.port, - RPL_LOG_NAME, - llstr(glob_mi.pos,llbuff)); + if (!safe_connect(thd, mysql, mi)) + sql_print_error("Slave I/O thread: connected to master '%s@%s:%d',\ + replication started in log '%s' at position %s", mi->user, + mi->host, mi->port, + IO_RPL_LOG_NAME, + llstr(mi->master_log_pos,llbuff)); else { - sql_print_error("Slave thread killed while connecting to master"); + sql_print_error("Slave I/O thread killed while connecting to master"); goto err; } - + connected: - mysql->net.timeout=slave_net_timeout; - while (!slave_killed(thd)) + thd->slave_net = &mysql->net; + thd->proc_info = "Checking master version"; + if (check_master_version(mysql, mi)) + goto err; + if (!mi->old_format) { - thd->proc_info = "Requesting binlog dump"; - if(request_dump(mysql, &glob_mi)) - { - sql_print_error("Failed on request_dump()"); - if(slave_killed(thd)) - { - sql_print_error("Slave thread killed while requesting master \ + /* + Register ourselves with the master. + If fails, this is not fatal - we just print the error message and go + on with life. + */ + thd->proc_info = "Registering slave on master"; + if (register_slave_on_master(mysql) || update_slave_list(mysql)) + goto err; + } + + DBUG_PRINT("info",("Starting reading binary log from master")); + while (!io_slave_killed(thd,mi)) + { + bool suppress_warnings= 0; + thd->proc_info = "Requesting binlog dump"; + if (request_dump(mysql, mi, &suppress_warnings)) + { + sql_print_error("Failed on request_dump()"); + if (io_slave_killed(thd,mi)) + { + sql_print_error("Slave I/O thread killed while requesting master \ dump"); - goto err; - } - - thd->proc_info = "Waiiting to reconnect after a failed dump request"; - if(mysql->net.vio) - vio_close(mysql->net.vio); - // first time retry immediately, assuming that we can recover - // right away - if first time fails, sleep between re-tries - // hopefuly the admin can fix the problem sometime - if(retried_once) - safe_sleep(thd, glob_mi.connect_retry); - else - retried_once = 1; + goto err; + } - if(slave_killed(thd)) - { - sql_print_error("Slave thread killed while retrying master \ + thd->proc_info = "Waiiting to reconnect after a failed dump request"; + mc_end_server(mysql); + /* + First time retry immediately, assuming that we can recover + right away - if first time fails, sleep between re-tries + hopefuly the admin can fix the problem sometime + */ + if (retry_count++) + { + if (retry_count > master_retry_count) + goto err; // Don't retry forever + safe_sleep(thd,mi->connect_retry,(CHECK_KILLED_FUNC)io_slave_killed, + (void*)mi); + } + if (io_slave_killed(thd,mi)) + { + sql_print_error("Slave I/O thread killed while retrying master \ dump"); - goto err; - } - - thd->proc_info = "Reconnecting after a failed dump request"; - last_failed_pos=glob_mi.pos; - sql_print_error("Slave: failed dump request, reconnecting to \ -try again, log '%s' at postion %s", RPL_LOG_NAME, - llstr(last_failed_pos,llbuff)); - if(safe_reconnect(thd, mysql, &glob_mi, 0) || slave_killed(thd)) - { - sql_print_error("Slave thread killed during or after reconnect"); - goto err; - } - - goto connected; - } + goto err; + } - while(!slave_killed(thd)) - { - bool suppress_warnings= 0; - - thd->proc_info = "Reading master update"; - uint event_len = read_event(mysql, &glob_mi, &suppress_warnings); - - if(slave_killed(thd)) - { - sql_print_error("Slave thread killed while reading event"); - goto err; - } + thd->proc_info = "Reconnecting after a failed dump request"; + if (!suppress_warnings) + sql_print_error("Slave I/O thread: failed dump request, \ +reconnecting to try again, log '%s' at postion %s", IO_RPL_LOG_NAME, + llstr(mi->master_log_pos,llbuff)); + if (safe_reconnect(thd, mysql, mi, suppress_warnings) || + io_slave_killed(thd,mi)) + { + sql_print_error("Slave I/O thread killed during or \ +after reconnect"); + goto err; + } + + goto connected; + } + + while (!io_slave_killed(thd,mi)) + { + bool suppress_warnings= 0; + thd->proc_info = "Reading master update"; + ulong event_len = read_event(mysql, mi, &suppress_warnings); + if (io_slave_killed(thd,mi)) + { + if (global_system_variables.log_warnings) + sql_print_error("Slave I/O thread killed while reading event"); + goto err; + } - if (event_len == packet_error) - { - if(mc_mysql_errno(mysql) == ER_NET_PACKET_TOO_LARGE) - { - sql_print_error("Log entry on master is longer than \ -max_allowed_packet on slave. Slave thread will be aborted. If the entry is \ -really supposed to be that long, restart the server with a higher value of \ -max_allowed_packet. The current value is %ld", max_allowed_packet); - goto err; - } - - thd->proc_info = "Waiting to reconnect after a failed read"; - if(mysql->net.vio) - vio_close(mysql->net.vio); - if(retried_once) // punish repeat offender with sleep - safe_sleep(thd, glob_mi.connect_retry); - else - retried_once = 1; - - if(slave_killed(thd)) - { - sql_print_error("Slave thread killed while waiting to \ + if (event_len == packet_error) + { + uint mysql_error_number= mc_mysql_errno(mysql); + if (mysql_error_number == ER_NET_PACKET_TOO_LARGE) + { + sql_print_error("\ +Log entry on master is longer than max_allowed_packet (%ld) on \ +slave. If the entry is correct, restart the server with a higher value of \ +max_allowed_packet", + thd->variables.max_allowed_packet); + goto err; + } + if (mysql_error_number == ER_MASTER_FATAL_ERROR_READING_BINLOG) + { + sql_print_error(ER(mysql_error_number), mysql_error_number, + mc_mysql_error(mysql)); + goto err; + } + thd->proc_info = "Waiting to reconnect after a failed read"; + mc_end_server(mysql); + if (retry_count++) + { + if (retry_count > master_retry_count) + goto err; // Don't retry forever + safe_sleep(thd,mi->connect_retry,(CHECK_KILLED_FUNC)io_slave_killed, + (void*) mi); + } + if (io_slave_killed(thd,mi)) + { + if (global_system_variables.log_warnings) + sql_print_error("Slave I/O thread killed while waiting to \ reconnect after a failed read"); - goto err; - } - thd->proc_info = "Reconnecting after a failed read"; - last_failed_pos= glob_mi.pos; - - if (!suppress_warnings) - sql_print_error("Slave: Failed reading log event, \ -reconnecting to retry, log '%s' position %s", RPL_LOG_NAME, - llstr(last_failed_pos, llbuff)); - if(safe_reconnect(thd, mysql, &glob_mi, - suppress_warnings) - || slave_killed(thd)) - { - sql_print_error("Slave thread killed during or after a \ + goto err; + } + thd->proc_info = "Reconnecting after a failed read"; + if (!suppress_warnings) + sql_print_error("Slave I/O thread: Failed reading log event, \ +reconnecting to retry, log '%s' position %s", IO_RPL_LOG_NAME, + llstr(mi->master_log_pos, llbuff)); + if (safe_reconnect(thd, mysql, mi, suppress_warnings) || + io_slave_killed(thd,mi)) + { + if (global_system_variables.log_warnings) + sql_print_error("Slave I/O thread killed during or after a \ reconnect done to recover from failed read"); - goto err; - } - - goto connected; - } // if(event_len == packet_error) - - thd->proc_info = "Processing master log event"; - if(exec_event(thd, &mysql->net, &glob_mi, event_len)) - { - sql_print_error("\ -Error running query, slave aborted. Fix the problem, and re-start \ -the slave thread with \"mysqladmin start-slave\". We stopped at log \ -'%s' position %s", - RPL_LOG_NAME, llstr(glob_mi.pos, llbuff)); - goto err; - // there was an error running the query - // abort the slave thread, when the problem is fixed, the user - // should restart the slave with mysqladmin start-slave - } -#ifndef DBUG_OFF - if(abort_slave_event_count && !--events_till_abort) - { - sql_print_error("Slave: debugging abort"); - goto err; - } -#endif + goto err; + } + goto connected; + } // if (event_len == packet_error) - // successful exec with offset advance, - // the slave repents and his sins are forgiven! - if(glob_mi.pos > last_failed_pos) - { - retried_once = 0; + retry_count=0; // ok event, reset retry counter + thd->proc_info = "Queueing event from master"; + if (queue_event(mi,(const char*)mysql->net.read_pos + 1, + event_len)) + { + sql_print_error("Slave I/O thread could not queue event from master"); + goto err; + } + flush_master_info(mi); + if (mi->rli.log_space_limit && mi->rli.log_space_limit < + mi->rli.log_space_total) + if (wait_for_relay_log_space(&mi->rli)) + { + sql_print_error("Slave I/O thread aborted while waiting for relay \ +log space"); + goto err; + } + // TODO: check debugging abort code #ifndef DBUG_OFF - stuck_count = 0; + if (abort_slave_event_count && !--events_till_abort) + { + sql_print_error("Slave I/O thread: debugging abort"); + goto err; + } #endif - } -#ifndef DBUG_OFF - else - { - // show a little mercy, allow slave to read one more event - // before cutting him off - otherwise he gets stuck - // on Intvar events, since they do not advance the offset - // immediately - if (++stuck_count > 2) - events_till_disconnect++; - } -#endif - } // while(!slave_killed(thd)) - read/exec loop - } // while(!slave_killed(thd)) - slave loop + } + } // error = 0; - err: - // print the current replication position - sql_print_error("Slave thread exiting, replication stopped in log '%s' at \ -position %s", - RPL_LOG_NAME, llstr(glob_mi.pos,llbuff)); +err: + // print the current replication position + sql_print_error("Slave I/O thread exiting, read up to log '%s', position %s", + IO_RPL_LOG_NAME, llstr(mi->master_log_pos,llbuff)); thd->query = thd->db = 0; // extra safety - if(mysql) - mc_mysql_close(mysql); + if (mysql) + { + mc_mysql_close(mysql); + mi->mysql=0; + } thd->proc_info = "Waiting for slave mutex on exit"; - pthread_mutex_lock(&LOCK_slave); - slave_running = 0; - abort_slave = 0; - save_temporary_tables = thd->temporary_tables; - thd->temporary_tables = 0; // remove tempation from destructor to close them - pthread_cond_broadcast(&COND_slave_stopped); // tell the world we are done - pthread_mutex_unlock(&LOCK_slave); - net_end(&thd->net); // destructor will not free it, because we are weird - slave_thd = 0; - (void) pthread_mutex_lock(&LOCK_thread_count); + pthread_mutex_lock(&mi->run_lock); + mi->slave_running = 0; + mi->io_thd = 0; + // TODO: make rpl_status part of MASTER_INFO + change_rpl_status(RPL_ACTIVE_SLAVE,RPL_IDLE_SLAVE); + mi->abort_slave = 0; // TODO: check if this is needed + DBUG_ASSERT(thd->net.buff != 0); + net_end(&thd->net); // destructor will not free it, because net.vio is 0 + pthread_mutex_lock(&LOCK_thread_count); + THD_CHECK_SENTRY(thd); delete thd; - (void) pthread_mutex_unlock(&LOCK_thread_count); - my_thread_end(); + pthread_mutex_unlock(&LOCK_thread_count); + my_thread_end(); // clean-up before broadcast + pthread_cond_broadcast(&mi->stop_cond); // tell the world we are done + pthread_mutex_unlock(&mi->run_lock); #ifndef DBUG_OFF - if(abort_slave_event_count && !events_till_abort) + if (abort_slave_event_count && !events_till_abort) goto slave_begin; #endif pthread_exit(0); @@ -1516,92 +2080,846 @@ position %s", } -/* try to connect until successful or slave killed */ +/* slave SQL logic thread */ -static int safe_connect(THD* thd, MYSQL* mysql, MASTER_INFO* mi) +extern "C" pthread_handler_decl(handle_slave_sql,arg) { - int slave_was_killed; + THD *thd; /* needs to be first for thread_stack */ + char llbuff[22],llbuff1[22]; + RELAY_LOG_INFO* rli = &((MASTER_INFO*)arg)->rli; + const char *errmsg; + + // needs to call my_thread_init(), otherwise we get a coredump in DBUG_ stuff + my_thread_init(); + #ifndef DBUG_OFF - events_till_disconnect = disconnect_slave_event_count; +slave_begin: #endif - while(!(slave_was_killed = slave_killed(thd)) && - !mc_mysql_connect(mysql, mi->host, mi->user, mi->password, 0, - mi->port, 0, 0)) + + DBUG_ASSERT(rli->inited); + pthread_mutex_lock(&rli->run_lock); + DBUG_ASSERT(!rli->slave_running); + errmsg= 0; +#ifndef DBUG_OFF + rli->events_till_abort = abort_slave_event_count; +#endif + DBUG_ENTER("handle_slave_sql"); + + thd = new THD; // note that contructor of THD uses DBUG_ ! + THD_CHECK_SENTRY(thd); + /* Inform waiting threads that slave has started */ + rli->slave_run_id++; + + pthread_detach_this_thread(); + if (init_slave_thread(thd, SLAVE_THD_SQL)) { - sql_print_error("Slave thread: error connecting to master: %s (%d),\ - retry in %d sec", mc_mysql_error(mysql), errno, mi->connect_retry); - safe_sleep(thd, mi->connect_retry); + /* + TODO: this is currently broken - slave start and change master + will be stuck if we fail here + */ + pthread_cond_broadcast(&rli->start_cond); + pthread_mutex_unlock(&rli->run_lock); + sql_print_error("Failed during slave thread initialization"); + goto err; } + rli->sql_thd= thd; + thd->temporary_tables = rli->save_temporary_tables; // restore temp tables + thd->thread_stack = (char*)&thd; // remember where our stack is + pthread_mutex_lock(&LOCK_thread_count); + threads.append(thd); + pthread_mutex_unlock(&LOCK_thread_count); + rli->slave_running = 1; + rli->abort_slave = 0; + pthread_mutex_unlock(&rli->run_lock); + pthread_cond_broadcast(&rli->start_cond); + // This should always be set to 0 when the slave thread is started + rli->pending = 0; + if (init_relay_log_pos(rli, + rli->relay_log_name, + rli->relay_log_pos, + 1 /*need data lock*/, &errmsg)) + { + sql_print_error("Error initializing relay log position: %s", + errmsg); + goto err; + } + THD_CHECK_SENTRY(thd); + DBUG_ASSERT(rli->relay_log_pos >= BIN_LOG_HEADER_SIZE); + DBUG_ASSERT(my_b_tell(rli->cur_log) == rli->relay_log_pos); + DBUG_ASSERT(rli->sql_thd == thd); + + DBUG_PRINT("master_info",("log_file_name: %s position: %s", + rli->master_log_name, + llstr(rli->master_log_pos,llbuff))); + if (global_system_variables.log_warnings) + sql_print_error("Slave SQL thread initialized, starting replication in \ +log '%s' at position %s, relay log '%s' position: %s", RPL_LOG_NAME, + llstr(rli->master_log_pos,llbuff),rli->relay_log_name, + llstr(rli->relay_log_pos,llbuff1)); + + /* Read queries from the IO/THREAD until this thread is killed */ + + while (!sql_slave_killed(thd,rli)) + { + thd->proc_info = "Processing master log event"; + DBUG_ASSERT(rli->sql_thd == thd); + THD_CHECK_SENTRY(thd); + if (exec_relay_log_event(thd,rli)) + { + // do not scare the user if SQL thread was simply killed or stopped + if (!sql_slave_killed(thd,rli)) + sql_print_error("\ +Error running query, slave SQL thread aborted. Fix the problem, and restart \ +the slave SQL thread with \"SLAVE START\". We stopped at log \ +'%s' position %s", + RPL_LOG_NAME, llstr(rli->master_log_pos, llbuff)); + goto err; + } + } + + /* Thread stopped. Print the current replication position to the log */ + sql_print_error("Slave SQL thread exiting, replication stopped in log \ + '%s' at position %s", + RPL_LOG_NAME, llstr(rli->master_log_pos,llbuff)); + + err: + thd->query = thd->db = 0; // extra safety + thd->proc_info = "Waiting for slave mutex on exit"; + pthread_mutex_lock(&rli->run_lock); + DBUG_ASSERT(rli->slave_running == 1); // tracking buffer overrun + rli->slave_running = 0; + rli->save_temporary_tables = thd->temporary_tables; + + /* + TODO: see if we can do this conditionally in next_event() instead + to avoid unneeded position re-init + */ + rli->log_pos_current=0; + thd->temporary_tables = 0; // remove tempation from destructor to close them + DBUG_ASSERT(thd->net.buff != 0); + net_end(&thd->net); // destructor will not free it, because we are weird + DBUG_ASSERT(rli->sql_thd == thd); + THD_CHECK_SENTRY(thd); + rli->sql_thd= 0; + pthread_mutex_lock(&LOCK_thread_count); + THD_CHECK_SENTRY(thd); + delete thd; + pthread_mutex_unlock(&LOCK_thread_count); + my_thread_end(); // clean-up before broadcasting termination + pthread_cond_broadcast(&rli->stop_cond); + // tell the world we are done + pthread_mutex_unlock(&rli->run_lock); +#ifndef DBUG_OFF // TODO: reconsider the code below + if (abort_slave_event_count && !rli->events_till_abort) + goto slave_begin; +#endif + pthread_exit(0); + DBUG_RETURN(0); // Can't return anything here +} + +static int process_io_create_file(MASTER_INFO* mi, Create_file_log_event* cev) +{ + int error = 1; + ulong num_bytes; + bool cev_not_written; + THD* thd; + NET* net = &mi->mysql->net; + DBUG_ENTER("process_io_create_file"); + + if (unlikely(!cev->is_valid())) + DBUG_RETURN(1); + /* + TODO: fix to honor table rules, not only db rules + */ + if (!db_ok(cev->db, replicate_do_db, replicate_ignore_db)) + { + skip_load_data_infile(net); + DBUG_RETURN(0); + } + DBUG_ASSERT(cev->inited_from_old); + thd = mi->io_thd; + thd->file_id = cev->file_id = mi->file_id++; + thd->server_id = cev->server_id; + cev_not_written = 1; + + if (unlikely(net_request_file(net,cev->fname))) + { + sql_print_error("Slave I/O: failed requesting download of '%s'", + cev->fname); + goto err; + } + + /* this dummy block is so we could instantiate Append_block_log_event + once and then modify it slightly instead of doing it multiple times + in the loop + */ + { + Append_block_log_event aev(thd,0,0,0); - if(!slave_was_killed) + for (;;) { - mysql_log.write(thd, COM_CONNECT_OUT, "%s@%s:%d", - mi->user, mi->host, mi->port); -#ifdef SIGNAL_WITH_VIO_CLOSE - thd->set_active_vio(mysql->net.vio); -#endif + if (unlikely((num_bytes=my_net_read(net)) == packet_error)) + { + sql_print_error("Network read error downloading '%s' from master", + cev->fname); + goto err; + } + if (unlikely(!num_bytes)) /* eof */ + { + send_ok(net); /* 3.23 master wants it */ + Execute_load_log_event xev(thd,0); + xev.log_pos = mi->master_log_pos; + if (unlikely(mi->rli.relay_log.append(&xev))) + { + sql_print_error("Slave I/O: error writing Exec_load event to \ +relay log"); + goto err; + } + mi->rli.relay_log.harvest_bytes_written(&mi->rli.log_space_total); + break; + } + if (unlikely(cev_not_written)) + { + cev->block = (char*)net->read_pos; + cev->block_len = num_bytes; + cev->log_pos = mi->master_log_pos; + if (unlikely(mi->rli.relay_log.append(cev))) + { + sql_print_error("Slave I/O: error writing Create_file event to \ +relay log"); + goto err; + } + cev_not_written=0; + mi->rli.relay_log.harvest_bytes_written(&mi->rli.log_space_total); + } + else + { + aev.block = (char*)net->read_pos; + aev.block_len = num_bytes; + aev.log_pos = mi->master_log_pos; + if (unlikely(mi->rli.relay_log.append(&aev))) + { + sql_print_error("Slave I/O: error writing Append_block event to \ +relay log"); + goto err; + } + mi->rli.relay_log.harvest_bytes_written(&mi->rli.log_space_total) ; + } + } + } + error=0; +err: + DBUG_RETURN(error); +} + +/* + Start using a new binary log on the master + + SYNOPSIS + process_io_rotate() + mi master_info for the slave + rev The rotate log event read from the binary log + + DESCRIPTION + Updates the master info and relay data with the place in the next binary + log where we should start reading. + + NOTES + We assume we already locked mi->data_lock + + RETURN VALUES + 0 ok + 1 Log event is illegal +*/ + +static int process_io_rotate(MASTER_INFO *mi, Rotate_log_event *rev) +{ + int return_val= 1; + DBUG_ENTER("process_io_rotate"); + safe_mutex_assert_owner(&mi->data_lock); + + if (unlikely(!rev->is_valid())) + DBUG_RETURN(1); + + memcpy(mi->master_log_name, rev->new_log_ident, rev->ident_len+1); + mi->master_log_pos= rev->pos; + DBUG_PRINT("info", ("master_log_pos: '%s' %d", + mi->master_log_name, (ulong) mi->master_log_pos)); +#ifndef DBUG_OFF + /* + If we do not do this, we will be getting the first + rotate event forever, so we need to not disconnect after one. + */ + if (disconnect_slave_event_count) + events_till_disconnect++; +#endif + DBUG_RETURN(0); +} + +/* + TODO: + Test this code before release - it has to be tested on a separate + setup with 3.23 master +*/ + +static int queue_old_event(MASTER_INFO *mi, const char *buf, + ulong event_len) +{ + const char *errmsg = 0; + ulong inc_pos; + bool ignore_event= 0; + char *tmp_buf = 0; + RELAY_LOG_INFO *rli= &mi->rli; + DBUG_ENTER("queue_old_event"); + + /* + If we get Load event, we need to pass a non-reusable buffer + to read_log_event, so we do a trick + */ + if (buf[EVENT_TYPE_OFFSET] == LOAD_EVENT) + { + if (unlikely(!(tmp_buf=(char*)my_malloc(event_len+1,MYF(MY_WME))))) + { + sql_print_error("Slave I/O: out of memory for Load event"); + DBUG_RETURN(1); + } + memcpy(tmp_buf,buf,event_len); + tmp_buf[event_len]=0; // Create_file constructor wants null-term buffer + buf = (const char*)tmp_buf; + } + Log_event *ev = Log_event::read_log_event(buf,event_len, &errmsg, + 1 /*old format*/ ); + if (unlikely(!ev)) + { + sql_print_error("Read invalid event from master: '%s',\ + master could be corrupt but a more likely cause of this is a bug", + errmsg); + my_free((char*) tmp_buf, MYF(MY_ALLOW_ZERO_PTR)); + DBUG_RETURN(1); + } + pthread_mutex_lock(&mi->data_lock); + ev->log_pos = mi->master_log_pos; + switch (ev->get_type_code()) { + case STOP_EVENT: + ignore_event= mi->ignore_stop_event; + mi->ignore_stop_event=0; + inc_pos= event_len; + break; + case ROTATE_EVENT: + if (unlikely(process_io_rotate(mi,(Rotate_log_event*)ev))) + { + delete ev; + pthread_mutex_unlock(&mi->data_lock); + DBUG_RETURN(1); + } + mi->ignore_stop_event=1; + inc_pos= 0; + break; + case CREATE_FILE_EVENT: + { + /* We come here when and only when tmp_buf != 0 */ + DBUG_ASSERT(tmp_buf); + int error = process_io_create_file(mi,(Create_file_log_event*)ev); + delete ev; + mi->master_log_pos += event_len; + DBUG_PRINT("info", ("master_log_pos: %d", (ulong) mi->master_log_pos)); + pthread_mutex_unlock(&mi->data_lock); + my_free((char*)tmp_buf, MYF(0)); + DBUG_RETURN(error); + } + default: + mi->ignore_stop_event=0; + inc_pos= event_len; + break; + } + if (likely(!ignore_event)) + { + if (unlikely(rli->relay_log.append(ev))) + { + delete ev; + pthread_mutex_unlock(&mi->data_lock); + DBUG_RETURN(1); } + rli->relay_log.harvest_bytes_written(&rli->log_space_total); + } + delete ev; + mi->master_log_pos+= inc_pos; + DBUG_PRINT("info", ("master_log_pos: %d", (ulong) mi->master_log_pos)); + pthread_mutex_unlock(&mi->data_lock); + DBUG_RETURN(0); +} + +/* + TODO: verify the issue with stop events, see if we need them at all + in the relay log +*/ + +int queue_event(MASTER_INFO* mi,const char* buf, ulong event_len) +{ + int error= 0; + ulong inc_pos; + bool ignore_event= 0; + RELAY_LOG_INFO *rli= &mi->rli; + DBUG_ENTER("queue_event"); + + if (mi->old_format) + DBUG_RETURN(queue_old_event(mi,buf,event_len)); + + pthread_mutex_lock(&mi->data_lock); + + /* + TODO: figure out if other events in addition to Rotate + require special processing + */ + switch (buf[EVENT_TYPE_OFFSET]) { + case STOP_EVENT: + ignore_event= mi->ignore_stop_event; + mi->ignore_stop_event= 0; + inc_pos= event_len; + break; + case ROTATE_EVENT: + { + Rotate_log_event rev(buf,event_len,0); + if (unlikely(process_io_rotate(mi,&rev))) + { + pthread_mutex_unlock(&mi->data_lock); + DBUG_RETURN(1); + } + mi->ignore_stop_event= 1; + inc_pos= 0; + break; + } + default: + mi->ignore_stop_event= 0; + inc_pos= event_len; + break; + } - return slave_was_killed; + if (likely(!ignore_event && + !(error= rli->relay_log.appendv(buf,event_len,0)))) + { + mi->master_log_pos+= inc_pos; + DBUG_PRINT("info", ("master_log_pos: %d", (ulong) mi->master_log_pos)); + rli->relay_log.harvest_bytes_written(&rli->log_space_total); + } + pthread_mutex_unlock(&mi->data_lock); + DBUG_RETURN(error); } + +void end_relay_log_info(RELAY_LOG_INFO* rli) +{ + DBUG_ENTER("end_relay_log_info"); + + if (!rli->inited) + DBUG_VOID_RETURN; + if (rli->info_fd >= 0) + { + end_io_cache(&rli->info_file); + (void) my_close(rli->info_fd, MYF(MY_WME)); + rli->info_fd = -1; + } + if (rli->cur_log_fd >= 0) + { + end_io_cache(&rli->cache_buf); + (void)my_close(rli->cur_log_fd, MYF(MY_WME)); + rli->cur_log_fd = -1; + } + rli->inited = 0; + rli->log_pos_current=0; + rli->relay_log.close(1); + DBUG_VOID_RETURN; +} + +/* try to connect until successful or slave killed */ +static int safe_connect(THD* thd, MYSQL* mysql, MASTER_INFO* mi) +{ + return connect_to_master(thd, mysql, mi, 0, 0); +} + + /* Try to connect until successful or slave killed or we have retried master_retry_count times */ -static int safe_reconnect(THD* thd, MYSQL* mysql, MASTER_INFO* mi, - bool suppress_warnings) +static int connect_to_master(THD* thd, MYSQL* mysql, MASTER_INFO* mi, + bool reconnect, bool suppress_warnings) { int slave_was_killed; int last_errno= -2; // impossible error ulong err_count=0; char llbuff[22]; + DBUG_ENTER("connect_to_master"); - /* - If we lost connection after reading a state set event - we will be re-reading it, so pending needs to be cleared - */ - mi->pending = 0; #ifndef DBUG_OFF events_till_disconnect = disconnect_slave_event_count; #endif - while (!(slave_was_killed = slave_killed(thd)) && mc_mysql_reconnect(mysql)) + uint client_flag=0; + if (opt_slave_compressed_protocol) + client_flag=CLIENT_COMPRESS; /* We will use compression */ + + while (!(slave_was_killed = io_slave_killed(thd,mi)) && + (reconnect ? mc_mysql_reconnect(mysql) != 0: + !mc_mysql_connect(mysql, mi->host, mi->user, mi->password, 0, + mi->port, 0, client_flag, + thd->variables.net_read_timeout))) { /* Don't repeat last error */ if (mc_mysql_errno(mysql) != last_errno) { + last_errno=mc_mysql_errno(mysql); suppress_warnings= 0; - sql_print_error("Slave thread: error re-connecting to master: \ -%s, last_errno=%d, retry in %d sec", - mc_mysql_error(mysql), last_errno=mc_mysql_errno(mysql), - mi->connect_retry); + sql_print_error("Slave I/O thread: error %s to master \ +'%s@%s:%d': \ +Error: '%s' errno: %d retry-time: %d retries: %d", + (reconnect ? "reconnecting" : "connecting"), + mi->user,mi->host,mi->port, + mc_mysql_error(mysql), last_errno, + mi->connect_retry, + master_retry_count); } - safe_sleep(thd, mi->connect_retry); - /* if master_retry_count is not set, keep trying until success */ - if (master_retry_count && err_count++ == master_retry_count) + /* + By default we try forever. The reason is that failure will trigger + master election, so if the user did not set master_retry_count we + do not want to have election triggered on the first failure to + connect + */ + if (++err_count == master_retry_count) { slave_was_killed=1; + if (reconnect) + change_rpl_status(RPL_ACTIVE_SLAVE,RPL_LOST_SOLDIER); break; } + safe_sleep(thd,mi->connect_retry,(CHECK_KILLED_FUNC)io_slave_killed, + (void*)mi); } if (!slave_was_killed) { - if (!suppress_warnings) - sql_print_error("Slave: reconnected to master '%s@%s:%d',\ -replication resumed in log '%s' at position %s", glob_mi.user, - glob_mi.host, glob_mi.port, - RPL_LOG_NAME, - llstr(glob_mi.pos,llbuff)); + if (reconnect) + { + if (!suppress_warnings && global_system_variables.log_warnings) + sql_print_error("Slave: connected to master '%s@%s:%d',\ +replication resumed in log '%s' at position %s", mi->user, + mi->host, mi->port, + IO_RPL_LOG_NAME, + llstr(mi->master_log_pos,llbuff)); + } + else + { + change_rpl_status(RPL_IDLE_SLAVE,RPL_ACTIVE_SLAVE); + mysql_log.write(thd, COM_CONNECT_OUT, "%s@%s:%d", + mi->user, mi->host, mi->port); + } #ifdef SIGNAL_WITH_VIO_CLOSE thd->set_active_vio(mysql->net.vio); #endif } + DBUG_PRINT("exit",("slave_was_killed: %d", slave_was_killed)); + DBUG_RETURN(slave_was_killed); +} + + +/* + Try to connect until successful or slave killed or we have retried + master_retry_count times +*/ - return slave_was_killed; +static int safe_reconnect(THD* thd, MYSQL* mysql, MASTER_INFO* mi, + bool suppress_warnings) +{ + return connect_to_master(thd, mysql, mi, 1, suppress_warnings); } + +/* + Store the file and position where the execute-slave thread are in the + relay log. + + SYNOPSIS + flush_relay_log_info() + rli Relay log information + + NOTES + - As this is only called by the slave thread, we don't need to + have a lock on this. + - If there is an active transaction, then we don't update the position + in the relay log. This is to ensure that we re-execute statements + if we die in the middle of an transaction that was rolled back. + - As a transaction never spans binary logs, we don't have to handle the + case where we do a relay-log-rotation in the middle of the transaction. + If this would not be the case, we would have to ensure that we + don't delete the relay log file where the transaction started when + we switch to a new relay log file. + + TODO + - Change the log file information to a binary format to avoid calling + longlong2str. + + RETURN VALUES + 0 ok + 1 write error +*/ + +bool flush_relay_log_info(RELAY_LOG_INFO* rli) +{ + bool error=0; + IO_CACHE *file = &rli->info_file; + char buff[FN_REFLEN*2+22*2+4], *pos; + + /* sql_thd is not set when calling from init_slave() */ + if ((rli->sql_thd && rli->sql_thd->options & OPTION_BEGIN)) + return 0; // Wait for COMMIT + + my_b_seek(file, 0L); + pos=strmov(buff, rli->relay_log_name); + *pos++='\n'; + pos=longlong2str(rli->relay_log_pos, pos, 10); + *pos++='\n'; + pos=strmov(pos, rli->master_log_name); + *pos++='\n'; + pos=longlong2str(rli->master_log_pos, pos, 10); + *pos='\n'; + if (my_b_write(file, (byte*) buff, (ulong) (pos-buff)+1)) + error=1; + if (flush_io_cache(file)) + error=1; + if (flush_io_cache(rli->cur_log)) // QQ Why this call ? + error=1; + return error; +} + + +/* + This function is called when we notice that the current "hot" log + got rotated under our feet. +*/ + +static IO_CACHE *reopen_relay_log(RELAY_LOG_INFO *rli, const char **errmsg) +{ + DBUG_ASSERT(rli->cur_log != &rli->cache_buf); + DBUG_ASSERT(rli->cur_log_fd == -1); + DBUG_ENTER("reopen_relay_log"); + + IO_CACHE *cur_log = rli->cur_log=&rli->cache_buf; + if ((rli->cur_log_fd=open_binlog(cur_log,rli->relay_log_name, + errmsg)) <0) + DBUG_RETURN(0); + /* + We want to start exactly where we was before: + relay_log_pos Current log pos + pending Number of bytes already processed from the event + */ + my_b_seek(cur_log,rli->relay_log_pos + rli->pending); + DBUG_RETURN(cur_log); +} + + +Log_event* next_event(RELAY_LOG_INFO* rli) +{ + Log_event* ev; + IO_CACHE* cur_log = rli->cur_log; + pthread_mutex_t *log_lock = rli->relay_log.get_log_lock(); + const char* errmsg=0; + THD* thd = rli->sql_thd; + DBUG_ENTER("next_event"); + DBUG_ASSERT(thd != 0); + + /* + For most operations we need to protect rli members with data_lock, + so we will hold it for the most of the loop below + However, we will release it whenever it is worth the hassle, + and in the cases when we go into a pthread_cond_wait() with the + non-data_lock mutex + */ + pthread_mutex_lock(&rli->data_lock); + + while (!sql_slave_killed(thd,rli)) + { + /* + We can have two kinds of log reading: + hot_log: + rli->cur_log points at the IO_CACHE of relay_log, which + is actively being updated by the I/O thread. We need to be careful + in this case and make sure that we are not looking at a stale log that + has already been rotated. If it has been, we reopen the log. + + The other case is much simpler: + We just have a read only log that nobody else will be updating. + */ + bool hot_log; + if ((hot_log = (cur_log != &rli->cache_buf))) + { + DBUG_ASSERT(rli->cur_log_fd == -1); // foreign descriptor + pthread_mutex_lock(log_lock); + + /* + Reading xxx_file_id is safe because the log will only + be rotated when we hold relay_log.LOCK_log + */ + if (rli->relay_log.get_open_count() != rli->cur_log_old_open_count) + { + // The master has switched to a new log file; Reopen the old log file + cur_log=reopen_relay_log(rli, &errmsg); + pthread_mutex_unlock(log_lock); + if (!cur_log) // No more log files + goto err; + hot_log=0; // Using old binary log + } + } + DBUG_ASSERT(my_b_tell(cur_log) >= BIN_LOG_HEADER_SIZE); + DBUG_ASSERT(my_b_tell(cur_log) == rli->relay_log_pos + rli->pending); + /* + Relay log is always in new format - if the master is 3.23, the + I/O thread will convert the format for us + */ + if ((ev=Log_event::read_log_event(cur_log,0,(bool)0 /* new format */))) + { + DBUG_ASSERT(thd==rli->sql_thd); + if (hot_log) + pthread_mutex_unlock(log_lock); + pthread_mutex_unlock(&rli->data_lock); + DBUG_RETURN(ev); + } + DBUG_ASSERT(thd==rli->sql_thd); + if (opt_reckless_slave) // For mysql-test + cur_log->error = 0; + if (cur_log->error < 0) + { + errmsg = "slave SQL thread aborted because of I/O error"; + if (hot_log) + pthread_mutex_unlock(log_lock); + goto err; + } + if (!cur_log->error) /* EOF */ + { + /* + On a hot log, EOF means that there are no more updates to + process and we must block until I/O thread adds some and + signals us to continue + */ + if (hot_log) + { + DBUG_ASSERT(rli->relay_log.get_open_count() == rli->cur_log_old_open_count); + /* + We can, and should release data_lock while we are waiting for + update. If we do not, show slave status will block + */ + pthread_mutex_unlock(&rli->data_lock); + /* Note that wait_for_update unlocks lock_log ! */ + rli->relay_log.wait_for_update(rli->sql_thd); + + // re-acquire data lock since we released it earlier + pthread_mutex_lock(&rli->data_lock); + continue; + } + /* + If the log was not hot, we need to move to the next log in + sequence. The next log could be hot or cold, we deal with both + cases separately after doing some common initialization + */ + end_io_cache(cur_log); + DBUG_ASSERT(rli->cur_log_fd >= 0); + my_close(rli->cur_log_fd, MYF(MY_WME)); + rli->cur_log_fd = -1; + + /* + TODO: make skip_log_purge a start-up option. At this point this + is not critical priority + */ + if (!rli->skip_log_purge) + { + // purge_first_log will properly set up relay log coordinates in rli + if (rli->relay_log.purge_first_log(rli)) + { + errmsg = "Error purging processed log"; + goto err; + } + } + else + { + /* + If hot_log is set, then we already have a lock on + LOCK_log. If not, we have to get the lock. + + According to Sasha, the only time this code will ever be executed + is if we are recovering from a bug. + */ + if (rli->relay_log.find_next_log(&rli->linfo, !hot_log)) + { + errmsg = "error switching to the next log"; + goto err; + } + rli->relay_log_pos = BIN_LOG_HEADER_SIZE; + rli->pending=0; + strmake(rli->relay_log_name,rli->linfo.log_file_name, + sizeof(rli->relay_log_name)-1); + flush_relay_log_info(rli); + } + + // next log is hot + if (rli->relay_log.is_active(rli->linfo.log_file_name)) + { +#ifdef EXTRA_DEBUG + sql_print_error("next log '%s' is currently active", + rli->linfo.log_file_name); +#endif + rli->cur_log= cur_log= rli->relay_log.get_log_file(); + rli->cur_log_old_open_count= rli->relay_log.get_open_count(); + DBUG_ASSERT(rli->cur_log_fd == -1); + + /* + Read pointer has to be at the start since we are the only + reader + */ + if (check_binlog_magic(cur_log,&errmsg)) + goto err; + continue; + } + /* + if we get here, the log was not hot, so we will have to + open it ourselves + */ +#ifdef EXTRA_DEBUG + sql_print_error("next log '%s' is not active", + rli->linfo.log_file_name); +#endif + // open_binlog() will check the magic header + if ((rli->cur_log_fd=open_binlog(cur_log,rli->linfo.log_file_name, + &errmsg)) <0) + goto err; + } + else + { + /* + Read failed with a non-EOF error. + TODO: come up with something better to handle this error + */ + if (hot_log) + pthread_mutex_unlock(log_lock); + sql_print_error("Slave SQL thread: I/O error reading \ +event(errno: %d cur_log->error: %d)", + my_errno,cur_log->error); + // set read position to the beginning of the event + my_b_seek(cur_log,rli->relay_log_pos+rli->pending); + /* otherwise, we have had a partial read */ + errmsg = "Aborting slave SQL thread because of partial event read"; + break; // To end of function + } + } + if (!errmsg && global_system_variables.log_warnings) + errmsg = "slave SQL thread was killed"; + +err: + pthread_mutex_unlock(&rli->data_lock); + if (errmsg) + sql_print_error("Error reading relay log event: %s", errmsg); + DBUG_RETURN(0); +} + + #ifdef __GNUC__ template class I_List_iterator<i_string>; template class I_List_iterator<i_string_pair>; |