MySQL 8.3.0
Source Code Documentation
Replication
Collaboration diagram for Replication:

Modules

 Binlog Event Header Flags
 

Files

file  decompressing_event_object_istream.h
 Stream class that yields Log_event objects, including events contained in Transaction_payload_log_events.
 
file  rpl_replica.cc
 Code to run the io thread and the sql thread on the replication slave.
 
file  rpl_replica.h
 
file  rpl_trx_boundary_parser.h
 Replication transaction boundary parser.
 

Namespaces

namespace  binlog
 
namespace  mysql
 
namespace  mysql::binlog
 
namespace  mysql::binlog::event
 The namespace contains classes representing events that can occur in a replication stream.
 

Classes

class  List< T >
 
struct  Mts_db_names
 
class  Log_event
 This is the abstract base class for binary log events. More...
 
class  Query_log_event
 A Query event is written to the binary log whenever the database is modified on the master, unless row based logging is used. More...
 
class  Format_description_log_event
 For binlog version 4. More...
 
class  Intvar_log_event
 The class derives from the class Intvar_event in Binlog API, defined in the header binlog_event.h. More...
 
class  Rand_log_event
 Logs random seed used by the next RAND(), and by PASSWORD() in 4.1.0. More...
 
class  Xid_apply_log_event
 
class  Xid_log_event
 This is the subclass of Xid_event defined in libbinlogevent, An XID event is generated for a commit of a transaction that modifies one or more tables of an XA-capable storage engine Logs xid of the transaction-to-be-committed in the 2pc protocol. More...
 
class  XA_prepare_log_event
 Similar to Xid_log_event except that. More...
 
class  User_var_log_event
 Every time a query uses the value of a user variable, a User_var_log_event is written before the Query_log_event, to set the user variable. More...
 
class  Stop_log_event
 
class  Rotate_log_event
 This will be deprecated when we move to using sequence ids. More...
 
class  Append_block_log_event
 This event is created to contain the file data. More...
 
class  Delete_file_log_event
 Delete_file_log_event is created when the LOAD_DATA query fails on the master for some reason, and the slave should be notified to abort the load. More...
 
class  Begin_load_query_log_event
 Event for the first block of file to be loaded, its only difference from Append_block event is that this event creates or truncates existing file before writing data. More...
 
class  Execute_load_query_log_event
 Event responsible for LOAD DATA execution, it similar to Query_log_event but before executing the query it substitutes original filename in LOAD DATA query with name of temporary file. More...
 
class  Load_query_generator
 
class  Table_map_log_event
 Table_map_log_event which maps a table definition to a number. More...
 
class  Rows_applier_psi_stage
 
class  Rows_log_event
 Common base class for all row-containing log events. More...
 
class  Write_rows_log_event
 Log row insertions and updates. More...
 
class  Update_rows_log_event
 Log row updates with a before image. More...
 
class  Delete_rows_log_event
 Log row deletions. More...
 
class  Incident_log_event
 Class representing an incident, an occurrence out of the ordinary, that happened on the master. More...
 
class  Ignorable_log_event
 Base class for ignorable log events is Ignorable_event. More...
 
class  Rows_query_log_event
 It is used to record the original query for the rows events in RBR. More...
 
class  Heartbeat_log_event
 
class  Heartbeat_log_event_v2
 
class  Transaction_payload_log_event
 
class  Gtid_log_event
 This is a subclass if Gtid_event and Log_event. More...
 
class  Previous_gtids_log_event
 This is the subclass of Previous_gtids_event and Log_event It is used to record the gtid_executed in the last binary log file, for ex after flush logs, or at the starting of the binary log file. More...
 
class  Transaction_context_log_event
 This is the subclass of Transaction_context_event and Log_event This class encodes the transaction_context_log_event. More...
 
class  View_change_log_event
 This is the subclass of View_change_log_event and Log_event This class created the view_change_log_event which is used as a marker in case a new node joins or leaves the group. More...
 
struct  Reconnect_messages
 Encapsulates the messages and thread stages used for a specific call to try_to_reconnect. More...
 
class  ReplicaInitializer
 Helper class used to initialize the replica (includes init_replica()) More...
 
class  Replication_transaction_boundary_parser
 This is the class for verifying transaction boundaries in a replication event stream. More...
 

Macros

#define PREFIX_SQL_LOAD   "SQL_LOAD-"
 
#define TEMP_FILE_MAX_LEN   UUID_LENGTH + 38
 Maximum length of the name of a temporary file PREFIX LENGTH - 9 UUID - UUID_LENGTH SEPARATORS - 2 SERVER ID - 10 (range of server ID 1 to (2^32)-1 = 4,294,967,295) FILE ID - 10 (uint) EXTENSION - 7 (Assuming that the extension is always less than 7 characters) More...
 
#define ASSERT_OR_RETURN_ERROR(COND, ERRNO)   assert(COND)
 Either assert or return an error. More...
 
#define LOG_EVENT_OFFSET   4
 
#define NUM_LOAD_DELIM_STRS   5
 
#define MAX_LOG_EVENT_HEADER
 
#define SL_MASTER_PORT_OFFSET   8
 
#define SL_MASTER_POS_OFFSET   0
 
#define SL_MASTER_HOST_OFFSET   10
 
#define I_TYPE_OFFSET   0
 
#define I_VAL_OFFSET   1
 
#define BINLOG_MAGIC   "\xfe\x62\x69\x6e"
 
#define BINLOG_MAGIC_SIZE   4
 
#define OPTIONS_WRITTEN_TO_BIN_LOG
 OPTIONS_WRITTEN_TO_BIN_LOG are the bits of thd->options which must be written to the binlog. More...
 
#define EXPECTED_OPTIONS    ((1ULL << 14) | (1ULL << 26) | (1ULL << 27) | (1ULL << 19))
 
#define FLAGSTR(V, F)   ((V) & (F) ? #F " " : "")
 
#define SLAVE_WAIT_GROUP_DONE   60
 
#define SLAVE_MAX_HEARTBEAT_PERIOD   4294967
 Some of defines are need in parser even though replication is not compiled in (embedded). More...
 
#define REPLICA_NET_TIMEOUT   60
 
#define MAX_SLAVE_ERROR   14000
 
#define MTS_WORKER_UNDEF   ((ulong)-1)
 
#define MTS_MAX_WORKERS   1024
 
#define MAX_SLAVE_RETRY_PAUSE   5
 
#define MTS_MAX_BITS_IN_GROUP   ((1L << 19) - 8) /* 524280 */
 
#define MYSQL_SLAVE_NOT_RUN   0
 
#define MYSQL_SLAVE_RUN_NOT_CONNECT   1
 
#define MYSQL_SLAVE_RUN_CONNECT   2
 
#define SLAVE_FORCE_ALL   4
 

Typedefs

using ColumnViewPtr = std::unique_ptr< cs::util::ReplicatedColumnsView >
 
using sql_mode_t = uint64_t
 

Enumerations

enum  enum_slave_apply_event_and_update_pos_retval {
  SLAVE_APPLY_EVENT_AND_UPDATE_POS_OK = 0 , SLAVE_APPLY_EVENT_AND_UPDATE_POS_APPLY_ERROR = 1 , SLAVE_APPLY_EVENT_AND_UPDATE_POS_UPDATE_POS_ERROR = 2 , SLAVE_APPLY_EVENT_AND_UPDATE_POS_APPEND_JOB_ERROR = 3 ,
  SLAVE_APPLY_EVENT_RETRY = 4 , SLAVE_APPLY_EVENT_UNTIL_REACHED = 5 , SLAVE_APPLY_EVENT_AND_UPDATE_POS_MAX
}
 
enum  enum_read_rotate_from_relay_log_status { FOUND_ROTATE , NOT_FOUND_ROTATE , ERROR }
 
enum  enum_command_status { COMMAND_STATUS_OK , COMMAND_STATUS_ERROR , COMMAND_STATUS_ALLOWED_ERROR }
 
enum  QUEUE_EVENT_RESULT { QUEUE_EVENT_OK = 0 , QUEUE_EVENT_ERROR_QUEUING , QUEUE_EVENT_ERROR_FLUSHING_INFO }
 

Functions

int ignored_error_code (int err_code)
 Ignore error code specified on command line. More...
 
int get_rpl_part_id (partition_info *part_info)
 This method is used to extract the partition_id from a partitioned table. More...
 
char * str_to_hex (char *to, const char *from, size_t len)
 Transforms a string into "" or its expression in 0x... form. More...
 
static bool copy_event_cache_to_file_and_reinit (IO_CACHE *cache, FILE *file, bool flush_stream)
 
bool slave_execute_deferred_events (THD *thd)
 The function is called by slave applier in case there are active table filtering rules to force gathering events associated with Query-log-event into an array to execute them once the fate of the Query is determined for execution. More...
 
int append_query_string (const THD *thd, const CHARSET_INFO *csinfo, String const *from, String *to)
 Append a version of the 'from' string suitable for use in a query to the 'to' string. More...
 
bool is_any_gtid_event (const Log_event *evt)
 
bool is_session_control_event (Log_event *evt)
 Check if the given event is a session control event, one of User_var_event, Intvar_event or Rand_event. More...
 
bool is_atomic_ddl_event (Log_event const *evt)
 The function checks the argument event properties to deduce whether it represents an atomic DDL. More...
 
bool is_atomic_ddl (THD *thd, bool using_trans)
 The function lists all DDL instances that are supported for crash-recovery (WL9175). More...
 
template<class EVENT >
bool binary_event_serialize (EVENT *ev, Basic_ostream *ostream)
 Serialize an binary event to the given output stream. More...
 
size_t my_strmov_quoted_identifier (THD *thd, char *buffer, const char *identifier, size_t length)
 
size_t my_strmov_quoted_identifier_helper (int q, char *buffer, const char *identifier, size_t length)
 
template<typename T >
bool net_field_length_checked (const uchar **packet, size_t *max_length, T *out)
 Read an integer in net_field_length format, guarding against read out of bounds and advancing the position. More...
 
std::pair< bool, mysql::binlog::event::Log_event_basic_infoextract_log_event_basic_info (Log_event *log_event)
 Extract basic info about an event: type, query, is it ignorable. More...
 
std::pair< bool, mysql::binlog::event::Log_event_basic_infoextract_log_event_basic_info (const char *buf, size_t length, const mysql::binlog::event::Format_description_event *fd_event)
 Extract basic info about an event: type, query, is it ignorable. More...
 
static int process_io_rotate (Master_info *mi, Rotate_log_event *rev)
 Used by the slave IO thread when it receives a rotate event from the master. More...
 
static bool wait_for_relay_log_space (Relay_log_info *rli)
 
static bool io_slave_killed (THD *thd, Master_info *mi)
 
static bool monitor_io_replica_killed (THD *thd, Master_info *mi)
 
static bool is_autocommit_off (THD *thd)
 Check if in multi-statement transaction mode. More...
 
static void print_replica_skip_errors (void)
 Convert slave skip errors bitmap into a printable string. More...
 
static int safe_connect (THD *thd, MYSQL *mysql, Master_info *mi, const std::string &host=std::string(), const uint port=0)
 
static int safe_reconnect (THD *thd, MYSQL *mysql, Master_info *mi, bool suppress_warnings, const std::string &host=std::string(), const uint port=0)
 
static int get_master_version_and_clock (MYSQL *mysql, Master_info *mi)
 
static int get_master_uuid (MYSQL *mysql, Master_info *mi)
 Get master's uuid on connecting. More...
 
int io_thread_init_commands (MYSQL *mysql, Master_info *mi)
 Set user variables after connecting to the master. More...
 
static int terminate_slave_thread (THD *thd, mysql_mutex_t *term_lock, mysql_cond_t *term_cond, std::atomic< uint > *slave_running, ulong *stop_wait_timeout, bool need_lock_term, bool force)
 Wait for a slave thread to terminate. More...
 
static bool check_io_slave_killed (THD *thd, Master_info *mi, const char *info)
 
static int mts_event_coord_cmp (LOG_POS_COORD *id1, LOG_POS_COORD *id2)
 Orders jobs by comparing relay log information. More...
 
static int check_slave_sql_config_conflict (const Relay_log_info *rli)
 Check if there is any slave SQL config conflict. More...
 
static void group_replication_cleanup_after_clone ()
 Purge Group Replication channels relay logs after this server being a recipient of clone. More...
 
static void check_replica_configuration_restrictions ()
 (end of group Replication) More...
 
static bool check_replica_configuration_errors (Master_info *mi, int thread_mask)
 Checks the current replica configuration when starting a replication thread If some incompatibility is found an error is thrown. More...
 
static void set_thd_tx_priority (THD *thd, int priority)
 
static void set_thd_write_set_options (THD *thd, bool ignore_limit, bool allow_drop_write_set)
 Set for the thread options about the memory and size limits when transactions collect write sets. More...
 
static void set_replica_max_allowed_packet (THD *thd, MYSQL *mysql)
 
bool start_slave (THD *thd)
 Function to start a slave for all channels. More...
 
int stop_slave (THD *thd)
 Function to stop a slave for all channels. More...
 
bool start_slave_cmd (THD *thd)
 Entry point to the START SLAVE command. More...
 
bool stop_slave_cmd (THD *thd)
 Entry point for the STOP SLAVE command. More...
 
static enum_read_rotate_from_relay_log_status read_rotate_from_relay_log (char *filename, char *master_log_file, my_off_t *master_log_pos)
 Parse the given relay log and identify the rotate event from the master. More...
 
static int find_first_relay_log_with_rotate_from_master (Relay_log_info *rli)
 Reads relay logs one by one starting from the first relay log. More...
 
static void recover_relay_log (Master_info *mi)
 
int init_recovery (Master_info *mi)
 
static int fill_mts_gaps_and_recover (Master_info *mi)
 
int load_mi_and_rli_from_repositories (Master_info *mi, bool ignore_if_no_info, int thread_mask, bool skip_received_gtid_set_recovery=false, bool force_load=false)
 Call mi->init_info() and/or mi->rli->init_info(), which will read the replication configuration from repositories. More...
 
void end_info (Master_info *mi)
 
void clear_info (Master_info *mi)
 Clear the information regarding the Master_info and Relay_log_info objects represented by the parameter, meaning, setting to NULL all attributes that are not meant to be kept between slave resets. More...
 
int remove_info (Master_info *mi)
 
bool reset_info (Master_info *mi)
 Resets the information regarding the Master_info and Relay_log_info objects represented by the parameter, meaning, setting to NULL all attributes that are not meant to be kept between slave resets and persisting all other attribute values in the repository. More...
 
int flush_master_info (Master_info *mi, bool force, bool need_lock=true, bool flush_relay_log=true, bool skip_repo_persistence=false)
 This method flushes the current configuration for the channel into the connection metadata repository. More...
 
void set_replica_skip_errors (char **replica_skip_errors_ptr)
 Change arg to the string with the nice, human-readable skip error values. More...
 
static void init_replica_skip_errors ()
 Init function to set up array for errors that should be skipped for slave. More...
 
static void add_replica_skip_errors (const uint *errors, uint n_errors)
 
void add_replica_skip_errors (const char *arg)
 
static void set_thd_in_use_temporary_tables (Relay_log_info *rli)
 
int terminate_slave_threads (Master_info *mi, int thread_mask, ulong stop_wait_timeout, bool need_lock_term=true)
 Terminates the slave threads according to the given mask. More...
 
bool start_slave_thread (PSI_thread_key thread_key, my_start_routine h_func, mysql_mutex_t *start_lock, mysql_mutex_t *cond_lock, mysql_cond_t *start_cond, std::atomic< uint > *slave_running, std::atomic< ulong > *slave_run_id, Master_info *mi)
 
bool start_slave_threads (bool need_lock_slave, bool wait_for_start, Master_info *mi, int thread_mask)
 
void end_slave ()
 
void delete_slave_info_objects ()
 Free all resources used by slave threads at time of executing shutdown. More...
 
bool sql_slave_killed (THD *thd, Relay_log_info *rli)
 The function analyzes a possible killed status and makes a decision whether to accept it or not. More...
 
bool net_request_file (NET *net, const char *fname)
 
const char * print_slave_db_safe (const char *db)
 
bool is_network_error (uint errorno)
 
static enum_command_status io_thread_init_command (Master_info *mi, const char *query, int allowed_error, MYSQL_RES **master_res=nullptr, MYSQL_ROW *master_row=nullptr)
 Execute an initialization query for the IO thread. More...
 
static int write_rotate_to_master_pos_into_relay_log (THD *thd, Master_info *mi, bool force_flush_mi_info)
 
static int write_ignored_events_info_to_relay_log (THD *thd, Master_info *mi)
 
static int register_slave_on_master (MYSQL *mysql, Master_info *mi, bool *suppress_warnings)
 
static void show_slave_status_metadata (mem_root_deque< Item * > *field_list, int io_gtid_set_size, int sql_gtid_set_size)
 Function that fills the metadata required for SHOW REPLICA STATUS. More...
 
static bool show_slave_status_send_data (THD *thd, Master_info *mi, char *io_gtid_set_buffer, char *sql_gtid_set_buffer)
 Send the data to the client of a Master_info during show_slave_status() This function has to be called after calling show_slave_status_metadata(). More...
 
bool show_slave_status (THD *thd)
 Method to the show the replication status in all channels. More...
 
bool show_slave_status (THD *thd, Master_info *mi)
 Execute a SHOW REPLICA STATUS statement. More...
 
bool show_slave_status_cmd (THD *thd)
 Entry point for SHOW REPLICA STATUS command. More...
 
void set_slave_thread_options (THD *thd)
 Set slave thread default options. More...
 
void set_slave_thread_default_charset (THD *thd, Relay_log_info const *rli)
 
int init_replica_thread (THD *thd, SLAVE_THD_TYPE thd_type)
 
template<typename killed_func , typename rpl_info >
static bool slave_sleep (THD *thd, time_t seconds, killed_func func, rpl_info info)
 Sleep for a given amount of time or until killed. More...
 
static void fix_gtid_set (MYSQL_RPL *rpl, uchar *packet_gtid_set)
 Callback function for mysql_binlog_open(). More...
 
static int request_dump (THD *thd, MYSQL *mysql, MYSQL_RPL *rpl, Master_info *mi, bool *suppress_warnings)
 
static ulong read_event (MYSQL *mysql, MYSQL_RPL *rpl, Master_info *mi, bool *suppress_warnings)
 Read one event from the master. More...
 
static int sql_delay_event (Log_event *ev, THD *thd, Relay_log_info *rli)
 If this is a lagging slave (specified with CHANGE MASTER TO MASTER_DELAY = X), delays accordingly. More...
 
static enum enum_slave_apply_event_and_update_pos_retval apply_event_and_update_pos (Log_event **ptr_ev, THD *thd, Relay_log_info *rli)
 Applies the given event and advances the relay log position. More...
 
static bool coord_handle_partial_binlogged_transaction (Relay_log_info *rli, const Log_event *ev)
 Let the worker applying the current group to rollback and gracefully finish its work before. More...
 
static int exec_relay_log_event (THD *thd, Relay_log_info *rli, Rpl_applier_reader *applier_reader, Log_event *in)
 Top-level function for executing the next event in the relay log. More...
 
static int try_to_reconnect (THD *thd, MYSQL *mysql, Master_info *mi, uint *retry_count, bool suppress_warnings, const Reconnect_messages &messages)
 Try to reconnect slave IO thread. More...
 
void * handle_slave_io (void *arg)
 Slave IO thread entry point. More...
 
static int check_temp_dir (char *tmp_file, const char *channel_name)
 
static void * handle_slave_worker (void *arg)
 
bool mts_recovery_groups (Relay_log_info *rli)
 
bool mta_checkpoint_routine (Relay_log_info *rli, bool force)
 Processing rli->gaq to find out the low-water-mark (lwm) coordinates which is stored into the central recovery table. More...
 
static int slave_start_single_worker (Relay_log_info *rli, ulong i)
 Instantiation of a Slave_worker and forking out a single Worker thread. More...
 
static int slave_start_workers (Relay_log_info *rli, ulong n, bool *mts_inited)
 Initialization of the central rli members for Coordinator's role, communication channels such as Assigned Partition Hash (APH), and starting the Worker pool. More...
 
static void slave_stop_workers (Relay_log_info *rli, bool *mts_inited)
 
static int report_apply_event_error (THD *thd, Relay_log_info *rli)
 Processes the outcome of applying an event, logs it properly if it's an error and return the proper error code to trigger. More...
 
void * handle_slave_sql (void *arg)
 Slave SQL thread entry point. More...
 
int heartbeat_queue_event (bool is_valid, Master_info *&mi, std::string binlog_name, uint64_t position, unsigned long &inc_pos, bool &do_flush_mi)
 
QUEUE_EVENT_RESULT queue_event (Master_info *mi, const char *buf, ulong event_len, bool do_flush_mi)
 Store an event received from the master connection into the relay log. More...
 
void slave_io_thread_detach_vio ()
 Hook to detach the active VIO before closing a connection handle. More...
 
static int connect_to_master_via_namespace (THD *thd, MYSQL *mysql, Master_info *mi, bool reconnect, bool suppress_warnings, const std::string &host, const uint port)
 
int connect_to_master (THD *thd, MYSQL *mysql, Master_info *mi, bool reconnect, bool suppress_warnings, const std::string &host, const uint port, bool is_io_thread)
 
int rotate_relay_log (Master_info *mi, bool log_master_fd, bool need_lock, bool need_log_space_lock)
 
int flush_relay_logs (Master_info *mi, THD *thd)
 flushes the relay logs of a replication channel. More...
 
bool flush_relay_logs_cmd (THD *thd)
 Entry point for FLUSH RELAYLOGS command or to flush relaylogs for the FLUSH LOGS command. More...
 
bool reencrypt_relay_logs ()
 Re-encrypt previous relay logs with current master key for all slave channels. More...
 
bool start_slave (THD *thd, LEX_SLAVE_CONNECTION *connection_param, LEX_MASTER_INFO *master_param, int thread_mask_input, Master_info *mi, bool set_mts_settings)
 Executes a START SLAVE statement. More...
 
int stop_slave (THD *thd, Master_info *mi, bool net_report, bool for_one_channel, bool *push_temp_tables_warning)
 Execute a STOP SLAVE statement. More...
 
int reset_slave (THD *thd)
 Execute a RESET SLAVE (for all channels), used in Multisource replication. More...
 
int reset_slave (THD *thd, Master_info *mi, bool reset_all)
 Execute a RESET REPLICA statement. More...
 
bool reset_slave_cmd (THD *thd)
 Entry function for RESET SLAVE command. More...
 
static bool have_change_replication_source_receive_option (const LEX_MASTER_INFO *lex_mi)
 This function checks if the given CHANGE MASTER/REPLICATION SOURCE command has any receive option being set or changed. More...
 
static bool have_change_replication_source_execute_option (const LEX_MASTER_INFO *lex_mi, bool *need_relay_log_purge)
 This function checks if the given CHANGE MASTER/REPLICATION SOURCE command has any execute option being set or changed. More...
 
static bool have_change_replication_source_applier_and_receive_option (const LEX_MASTER_INFO *lex_mi)
 This function checks if the given CHANGE REPLICATION SOURCE command has any option that affect both the receiver and the applier. More...
 
static bool change_master_set_compression (THD *, const LEX_MASTER_INFO *lex_mi, Master_info *mi)
 This function checks all possible cases in which compression algorithm, compression level can be configured for a channel. More...
 
static int change_receive_options (THD *thd, LEX_MASTER_INFO *lex_mi, Master_info *mi)
 This function is called if the change master command had at least one receive option. More...
 
static bool change_execute_options (LEX_MASTER_INFO *lex_mi, Master_info *mi)
 This function is called if the change master command had at least one execute option. More...
 
static bool change_applier_receiver_options (THD *thd, LEX_MASTER_INFO *lex_mi, Master_info *mi)
 This function is called if the change replication source command had at least one option that affects both the receiver and applier parts. More...
 
static int validate_gtid_option_restrictions (const LEX_MASTER_INFO *lex_mi, Master_info *mi)
 This function validates that change replication source options are valid according to the current GTID_MODE. More...
 
bool is_option_enabled_or_will_be (bool base_value, int option_value)
 This is an helper method for boolean vars like SOURCE_AUTO_POSITION REQUIRE_ROW_FORMAT SOURCE_CONNECTION_AUTO_FAILOVER It tells if the variable is already enabled or will be by the command. More...
 
int evaluate_inter_option_dependencies (const LEX_MASTER_INFO *lex_mi, Master_info *mi)
 This method evaluates if the different options given to CHANGE REPLICATION SOURCE TO are compatible with the current configuration and with one another. More...
 
static void log_invalid_position_warning (THD *thd, const LEX_MASTER_INFO *lex_mi, Master_info *mi)
 Log a warning in case GTID_ONLY or SOURCE AUTO POSITION are disabled and the server contains invalid positions. More...
 
static std::pair< bool, bool > validate_change_replication_source_options (THD *thd, const LEX_MASTER_INFO *lex_mi, Master_info *mi, int thread_mask)
 This method aggregates the validation checks made for the command CHANGE REPLICATION SOURCE. More...
 
static bool update_change_replication_source_options (THD *thd, LEX_MASTER_INFO *lex_mi, Master_info *mi, bool have_both_receive_execute_option, bool have_execute_option, bool have_receive_option)
 This method aggregates the the instantiation of options for the command CHANGE REPLICATION SOURCE. More...
 
int change_master (THD *thd, Master_info *mi, LEX_MASTER_INFO *lex_mi, bool preserve_logs)
 Execute a CHANGE MASTER statement. More...
 
int add_new_channel (Master_info **mi, const char *channel)
 This function is first called when the Master_info object corresponding to a channel in a multisourced slave does not exist. More...
 
static bool is_invalid_change_master_for_group_replication_recovery (const LEX_MASTER_INFO *lex_mi)
 Method used to check if the user is trying to update any other option for the change master apart from the MASTER_USER and MASTER_PASSWORD. More...
 
static bool is_invalid_change_master_for_group_replication_applier (const LEX_MASTER_INFO *lex_mi)
 Method used to check if the user is trying to update any other option for the change master apart from the PRIVILEGE_CHECKS_USER. More...
 
bool change_master_cmd (THD *thd)
 Entry point for the CHANGE MASTER command. More...
 
int ReplicaInitializer::get_initialization_code () const
 Gets initialization code set-up at replica initialization. More...
 
 ReplicaInitializer::ReplicaInitializer (bool opt_initialize, bool opt_skip_replica_start, Rpl_channel_filters &filters, char **replica_skip_erors)
 Constructor, calls init_replica() More...
 
void ReplicaInitializer::print_channel_info () const
 In case debug mode is on, prints channel information. More...
 
void ReplicaInitializer::start_replication_threads (bool skip_replica_start=true)
 This function starts replication threads. More...
 
void ReplicaInitializer::start_threads ()
 This function starts replication threads. More...
 
static void ReplicaInitializer::init_replica_psi_keys ()
 Initializes replica PSI keys in case PSI interface is available. More...
 
int ReplicaInitializer::init_replica ()
 Performs replica initialization, creates default replication channel and sets channel filters. More...
 

Variables

PSI_memory_key key_memory_Incident_log_event_message
 
PSI_memory_key key_memory_Rows_query_log_event_rows_query
 
MYSQL_PLUGIN_IMPORT ulong server_id
 
MYSQL_PLUGIN_IMPORT char server_version [SERVER_VERSION_LENGTH]
 
const int64 SEQ_MAX_TIMESTAMP = LLONG_MAX
 Maximum value of binlog logical timestamp. More...
 
TYPELIB binlog_checksum_typelib
 
bool use_slave_mask = false
 
MY_BITMAP slave_error_mask
 
char slave_skip_error_names [SHOW_VAR_FUNC_BUFF_SIZE]
 
char * replica_load_tmpdir = nullptr
 
bool replicate_same_server_id
 
ulonglong relay_log_space_limit = 0
 
const char * relay_log_index = nullptr
 
const char * relay_log_basename = nullptr
 
const ulong mts_slave_worker_queue_len_max = 16384
 
const ulong mts_coordinator_basic_nap = 5
 
const ulong mts_worker_underrun_level = 10
 
static thread_local Master_infoRPL_MASTER_INFO = nullptr
 
static Reconnect_messages reconnect_messages_after_failed_registration
 
static Reconnect_messages reconnect_messages_after_failed_dump
 
static Reconnect_messages reconnect_messages_after_failed_event_read
 
static PSI_memory_key key_memory_rli_mta_coor
 
static PSI_thread_key key_thread_replica_io
 
static PSI_thread_key key_thread_replica_sql
 
static PSI_thread_key key_thread_replica_worker
 
static PSI_thread_key key_thread_replica_monitor_io
 
static PSI_thread_info all_slave_threads []
 
static PSI_memory_info all_slave_memory []
 
uint sql_replica_skip_counter
 a copy of active_mi->rli->slave_skip_counter, for showing in SHOW GLOBAL VARIABLES, INFORMATION_SCHEMA.GLOBAL_VARIABLES and @sql_replica_skip_counter without taking all the mutexes needed to access active_mi->rli->slave_skip_counter properly. More...
 
bool server_id_supplied
 
ulong master_retry_count
 
MY_BITMAP slave_error_mask
 
char slave_skip_error_names []
 
bool use_slave_mask
 
char * replica_load_tmpdir
 
char * opt_relay_logname
 
char * opt_relaylog_index_name
 
bool opt_relaylog_index_name_supplied
 
bool opt_relay_logname_supplied
 
char * opt_binlog_index_name
 
bool opt_skip_replica_start
 If set, slave is not autostarted. More...
 
bool opt_log_replica_updates
 
char * opt_replica_skip_errors
 
ulonglong relay_log_space_limit
 
const char * relay_log_index
 
const char * relay_log_basename
 
bool replicate_same_server_id
 
uint report_port
 
char * report_user
 
char * report_host
 
char * report_password
 

Detailed Description

How replication of field metadata works.

When a table map is created, the master first calls Table_map_log_event::save_field_metadata() which calculates how many values will be in the field metadata.Only those fields that require the extra data are added. The method also loops through all of the fields in the table calling the method Field::save_field_metadata() which returns the values for the field that will be saved in the metadata and replicated to the slave. Once all fields have been processed, the table map is written to the binlog adding the size of the field metadata and the field metadata to the end of the body of the table map.When a table map is read on the slave, the field metadata is read from the table map and passed to the table_def class constructor which saves the field metadata from the table map into an array based on the type of the field. Field metadata values not present (those fields that do not use extra data) in the table map are initialized as zero (0). The array size is the same as the columns for the table on the slave.Additionally, values saved for field metadata on the master are saved as a string of bytes (uchar) in the binlog. A field may require 1 or more bytes to store the information. In cases where values require multiple bytes (e.g. values > 255), the endian-safe methods are used to properly encode the values on the master and decode them on the slave. When the field metadata values are captured on the slave, they are stored in an array of type uint. This allows the least number of casts to prevent casting bugs when the field metadata is used in comparisons of field attributes. When the field metadata is used for calculating addresses in pointer math, the type used is uint32.

Macro Definition Documentation

◆ ASSERT_OR_RETURN_ERROR

#define ASSERT_OR_RETURN_ERROR (   COND,
  ERRNO 
)    assert(COND)

Either assert or return an error.

In debug build, the condition will be checked, but in non-debug builds, the error code given will be returned instead.

Parameters
CONDCondition to check
ERRNOError number to return in non-debug builds

◆ BINLOG_MAGIC

#define BINLOG_MAGIC   "\xfe\x62\x69\x6e"

◆ BINLOG_MAGIC_SIZE

#define BINLOG_MAGIC_SIZE   4

◆ EXPECTED_OPTIONS

#define EXPECTED_OPTIONS    ((1ULL << 14) | (1ULL << 26) | (1ULL << 27) | (1ULL << 19))

◆ FLAGSTR

#define FLAGSTR (   V,
 
)    ((V) & (F) ? #F " " : "")

◆ I_TYPE_OFFSET

#define I_TYPE_OFFSET   0

◆ I_VAL_OFFSET

#define I_VAL_OFFSET   1

◆ LOG_EVENT_OFFSET

#define LOG_EVENT_OFFSET   4

◆ MAX_LOG_EVENT_HEADER

#define MAX_LOG_EVENT_HEADER
Value:
( /* in order of Query_log_event::write */ \
(LOG_EVENT_HEADER_LEN + /* write_header */ \
*/ \
mysql::binlog::event::Binary_log_event:: \
EXECUTE_LOAD_QUERY_EXTRA_HEADER_LEN) + /*write_post_header_for_derived \
*/ \
MAX_SIZE_LOG_EVENT_STATUS + /* status */ \
1)
#define LOG_EVENT_HEADER_LEN
Definition: binlog_event.h:441
@ QUERY_HEADER_LEN
Definition: binlog_event.h:874
#define MAX_SIZE_LOG_EVENT_STATUS
Max number of possible extra bytes in a replication event compared to a packet (i....
Definition: binlog_event.h:136
#define NAME_LEN
Definition: mysql_com.h:66

◆ MAX_SLAVE_ERROR

#define MAX_SLAVE_ERROR   14000

◆ MAX_SLAVE_RETRY_PAUSE

#define MAX_SLAVE_RETRY_PAUSE   5

◆ MTS_MAX_BITS_IN_GROUP

#define MTS_MAX_BITS_IN_GROUP   ((1L << 19) - 8) /* 524280 */

◆ MTS_MAX_WORKERS

#define MTS_MAX_WORKERS   1024

◆ MTS_WORKER_UNDEF

#define MTS_WORKER_UNDEF   ((ulong)-1)

◆ MYSQL_SLAVE_NOT_RUN

#define MYSQL_SLAVE_NOT_RUN   0

◆ MYSQL_SLAVE_RUN_CONNECT

#define MYSQL_SLAVE_RUN_CONNECT   2

◆ MYSQL_SLAVE_RUN_NOT_CONNECT

#define MYSQL_SLAVE_RUN_NOT_CONNECT   1

◆ NUM_LOAD_DELIM_STRS

#define NUM_LOAD_DELIM_STRS   5

◆ OPTIONS_WRITTEN_TO_BIN_LOG

#define OPTIONS_WRITTEN_TO_BIN_LOG
Value:
OPTION_RELAXED_UNIQUE_CHECKS | OPTION_NOT_AUTOCOMMIT)
#define OPTION_NO_FOREIGN_KEY_CHECKS
The following can be set when importing tables in a 'wrong order' to suppress foreign key checks.
Definition: query_options.h:84
#define OPTION_AUTO_IS_NULL
Definition: query_options.h:68
#define OPTION_NOT_AUTOCOMMIT
Definition: query_options.h:73

OPTIONS_WRITTEN_TO_BIN_LOG are the bits of thd->options which must be written to the binlog.

OPTIONS_WRITTEN_TO_BIN_LOG could be written into the Format_description_log_event, so that if later we don't want to replicate a variable we did replicate, or the contrary, it's doable. But it should not be too hard to decide once for all of what we replicate and what we don't, among the fixed 32 bits of thd->options.

I (Guilhem) have read through every option's usage, and it looks like OPTION_AUTO_IS_NULL and OPTION_NO_FOREIGN_KEYS are the only ones which alter how the query modifies the table. It's good to replicate OPTION_RELAXED_UNIQUE_CHECKS too because otherwise, the slave may insert data slower than the master, in InnoDB. OPTION_BIG_SELECTS is not needed (the slave thread runs with max_join_size=HA_POS_ERROR) and OPTION_BIG_TABLES is not needed either, as the manual says (because a too big in-memory temp table is automatically written to disk).

◆ PREFIX_SQL_LOAD

#define PREFIX_SQL_LOAD   "SQL_LOAD-"

◆ REPLICA_NET_TIMEOUT

#define REPLICA_NET_TIMEOUT   60

◆ SL_MASTER_HOST_OFFSET

#define SL_MASTER_HOST_OFFSET   10

◆ SL_MASTER_PORT_OFFSET

#define SL_MASTER_PORT_OFFSET   8

◆ SL_MASTER_POS_OFFSET

#define SL_MASTER_POS_OFFSET   0

◆ SLAVE_FORCE_ALL

#define SLAVE_FORCE_ALL   4

◆ SLAVE_MAX_HEARTBEAT_PERIOD

#define SLAVE_MAX_HEARTBEAT_PERIOD   4294967

Some of defines are need in parser even though replication is not compiled in (embedded).

The maximum is defined as (ULONG_MAX/1000) with 4 bytes ulong

◆ SLAVE_WAIT_GROUP_DONE

#define SLAVE_WAIT_GROUP_DONE   60

◆ TEMP_FILE_MAX_LEN

#define TEMP_FILE_MAX_LEN   UUID_LENGTH + 38

Maximum length of the name of a temporary file PREFIX LENGTH - 9 UUID - UUID_LENGTH SEPARATORS - 2 SERVER ID - 10 (range of server ID 1 to (2^32)-1 = 4,294,967,295) FILE ID - 10 (uint) EXTENSION - 7 (Assuming that the extension is always less than 7 characters)

Typedef Documentation

◆ ColumnViewPtr

◆ sql_mode_t

using sql_mode_t = uint64_t

Enumeration Type Documentation

◆ enum_command_status

Enumerator
COMMAND_STATUS_OK 
COMMAND_STATUS_ERROR 
COMMAND_STATUS_ALLOWED_ERROR 

◆ enum_read_rotate_from_relay_log_status

Enumerator
FOUND_ROTATE 
NOT_FOUND_ROTATE 
ERROR 

◆ enum_slave_apply_event_and_update_pos_retval

Enumerator
SLAVE_APPLY_EVENT_AND_UPDATE_POS_OK 
SLAVE_APPLY_EVENT_AND_UPDATE_POS_APPLY_ERROR 
SLAVE_APPLY_EVENT_AND_UPDATE_POS_UPDATE_POS_ERROR 
SLAVE_APPLY_EVENT_AND_UPDATE_POS_APPEND_JOB_ERROR 
SLAVE_APPLY_EVENT_RETRY 
SLAVE_APPLY_EVENT_UNTIL_REACHED 
SLAVE_APPLY_EVENT_AND_UPDATE_POS_MAX 

◆ QUEUE_EVENT_RESULT

Enumerator
QUEUE_EVENT_OK 
QUEUE_EVENT_ERROR_QUEUING 
QUEUE_EVENT_ERROR_FLUSHING_INFO 

Function Documentation

◆ ReplicaInitializer()

ReplicaInitializer::ReplicaInitializer ( bool  opt_initialize,
bool  opt_skip_replica_start,
Rpl_channel_filters filters,
char **  replica_skip_erors 
)

Constructor, calls init_replica()

Parameters
[in]opt_initializeServer option used to indicate whether mysqld has been started with –initialize
[in]opt_skip_replica_startWhen true, skips the start of replication threads
[in]filtersReplication filters
[in]replica_skip_erorsTBD

◆ add_new_channel()

int add_new_channel ( Master_info **  mi,
const char *  channel 
)

This function is first called when the Master_info object corresponding to a channel in a multisourced slave does not exist.

But before a new channel is created, certain conditions have to be met. The below function apriorily checks if all such conditions are met. If all the conditions are met then it creates a channel i.e mi<->rli

Parameters
[in,out]miWhen new {mi,rli} are created, the reference is stored in *mi
[in]channelThe channel on which the change master was introduced.

◆ add_replica_skip_errors() [1/2]

void add_replica_skip_errors ( const char *  arg)

◆ add_replica_skip_errors() [2/2]

static void add_replica_skip_errors ( const uint *  errors,
uint  n_errors 
)
static

◆ append_query_string()

int append_query_string ( const THD thd,
const CHARSET_INFO csinfo,
String const *  from,
String to 
)

Append a version of the 'from' string suitable for use in a query to the 'to' string.

To generate a correct escaping, the character set information in 'csinfo' is used.

◆ apply_event_and_update_pos()

static enum enum_slave_apply_event_and_update_pos_retval apply_event_and_update_pos ( Log_event **  ptr_ev,
THD thd,
Relay_log_info rli 
)
static

Applies the given event and advances the relay log position.

This is needed by the sql thread to execute events from the binlog, and by clients executing BINLOG statements. Conceptually, this function does:

ev->apply_event(rli);
ev->update_pos(rli);

It also does the following maintenance:

  • Initializes the thread's server_id and time; and the event's thread.
  • If !rli->belongs_to_client() (i.e., if it belongs to the slave sql thread instead of being used for executing BINLOG statements), it does the following things: (1) skips events if it is needed according to the server id or slave_skip_counter; (2) unlocks rli->data_lock; (3) sleeps if required by 'CHANGE MASTER TO MASTER_DELAY=X'; (4) maintains the running state of the sql thread (rli->thread_state).
  • Reports errors as needed.
Parameters
ptr_eva pointer to a reference to the event to apply.
thdThe client thread that executes the event (i.e., the slave sql thread if called from a replication slave, or the client thread if called to execute a BINLOG statement).
rliThe relay log info (i.e., the slave's rli if called from a replication slave, or the client's thd->rli_fake if called to execute a BINLOG statement).
Note
MTS can store NULL to ptr_ev location to indicate the event is taken over by a Worker.
Return values
SLAVE_APPLY_EVENT_AND_UPDATE_POS_OKOK.
SLAVE_APPLY_EVENT_AND_UPDATE_POS_APPLY_ERRORError calling ev->apply_event().
SLAVE_APPLY_EVENT_AND_UPDATE_POS_UPDATE_POS_ERRORNo error calling ev->apply_event(), but error calling ev->update_pos().
SLAVE_APPLY_EVENT_AND_UPDATE_POS_APPEND_JOB_ERRORappend_item_to_jobs() failed, thread was killed while waiting for successful enqueue on worker.

◆ binary_event_serialize()

template<class EVENT >
bool binary_event_serialize ( EVENT *  ev,
Basic_ostream ostream 
)

Serialize an binary event to the given output stream.

It is more general than call ev->write() directly. The caller will not be affected if any change happens in serialization process. For example, serializing the event in different format.

◆ change_applier_receiver_options()

static bool change_applier_receiver_options ( THD thd,
LEX_MASTER_INFO lex_mi,
Master_info mi 
)
static

This function is called if the change replication source command had at least one option that affects both the receiver and applier parts.

Pure execute option(s) are handled in change_execute_options() The receive options are handled in the function change_receive_options()

  • used in change_master().
  • Both receiver and applier threads should be stopped on invocation
Parameters
lex_mistructure that holds all change replication source options
miPointer to Master_info object belonging to the replica channel to be configured
Returns
false if successfully set, true otherwise.

◆ change_execute_options()

static bool change_execute_options ( LEX_MASTER_INFO lex_mi,
Master_info mi 
)
static

This function is called if the change master command had at least one execute option.

This function then sets or alters the execute option(s) given in the command. The receive options are handled in the function change_receive_options()

  • used in change_master().
  • Execute threads should be stopped before this function is called.
Parameters
lex_mistructure that holds all change master options given on the change master command.
miPointer to Master_info object belonging to the replica channel that will be configured
Returns
false if the execute options were successfully set and true, otherwise.

◆ change_master()

int change_master ( THD thd,
Master_info mi,
LEX_MASTER_INFO lex_mi,
bool  preserve_logs 
)

Execute a CHANGE MASTER statement.

Apart from changing the receive/execute configurations/positions, this function also does the following:

  • May leave replicated open temporary table after warning.
  • Purges relay logs if no threads running and no relay log file/pos options.
  • Delete worker info in mysql.slave_worker_info table if applier not running.
Parameters
thdPointer to THD object for the client thread executing the statement.
miPointer to Master_info object belonging to the slave's IO thread.
lex_miLex information with master connection data. Coming from the an executing statement or set directly this shall contain connection settings like hostname, user, password and other settings like the number of connection retries.
preserve_logsIf the decision of purging the logs should be always be false even if no relay log name/position is given to the method. The preserve_logs parameter will not be respected when the relay log info repository is not initialized.
Return values
0success
!=0error

Is there a an error during validation

◆ change_master_cmd()

bool change_master_cmd ( THD thd)

Entry point for the CHANGE MASTER command.

Function decides to create a new channel or create an existing one.

Parameters
[in]thdthe client thread that issued the command.
Return values
truefail
falsesuccess.

◆ change_master_set_compression()

static bool change_master_set_compression ( THD ,
const LEX_MASTER_INFO lex_mi,
Master_info mi 
)
static

This function checks all possible cases in which compression algorithm, compression level can be configured for a channel.

  • used in change_receive_options
Parameters
lex_mipointer to structure holding all options specified as part of change master to statement
mipointer to structure holding all options specified as part of change master to statement after performing necessary checks
Return values
falsein case of success
truein case of failures

◆ change_receive_options()

static int change_receive_options ( THD thd,
LEX_MASTER_INFO lex_mi,
Master_info mi 
)
static

This function is called if the change master command had at least one receive option.

This function then sets or alters the receive option(s) given in the command. The execute options are handled in the function change_execute_options()

  • used in change_master().
  • Receiver threads should be stopped when this function is called.
Parameters
thdPointer to THD object for the client thread executing the statement.
lex_mistructure that holds all change master options given on the change master command. Coming from the an executing statement or set directly this shall contain connection settings like hostname, user, password and other settings like the number of connection retries.
miPointer to Master_info object belonging to the replica channel to be configured
Return values
0no error i.e., success.
!=0error.

◆ check_io_slave_killed()

static bool check_io_slave_killed ( THD thd,
Master_info mi,
const char *  info 
)
static

◆ check_replica_configuration_errors()

static bool check_replica_configuration_errors ( Master_info mi,
int  thread_mask 
)
static

Checks the current replica configuration when starting a replication thread If some incompatibility is found an error is thrown.

Parameters
mipointer to the source info repository object
thread_maskwhat replication threads are running
Returns
true if an error occurs, false otherwise

◆ check_replica_configuration_restrictions()

static void check_replica_configuration_restrictions ( )
static

(end of group Replication)

Checks the current replica configuration against the server GTID mode If some incompatibility is found a warning is logged.

◆ check_slave_sql_config_conflict()

static int check_slave_sql_config_conflict ( const Relay_log_info rli)
static

Check if there is any slave SQL config conflict.

Parameters
[in]rliThe slave's rli object.
Returns
0 is returned if there is no conflict, otherwise 1 is returned.

◆ check_temp_dir()

static int check_temp_dir ( char *  tmp_file,
const char *  channel_name 
)
static

◆ clear_info()

void clear_info ( Master_info mi)

Clear the information regarding the Master_info and Relay_log_info objects represented by the parameter, meaning, setting to NULL all attributes that are not meant to be kept between slave resets.

Parameters
mithe Master_info reference that holds both Master_info and Relay_log_info data.

◆ connect_to_master()

int connect_to_master ( THD thd,
MYSQL mysql,
Master_info mi,
bool  reconnect,
bool  suppress_warnings,
const std::string &  host,
const uint  port,
bool  is_io_thread 
)

◆ connect_to_master_via_namespace()

static int connect_to_master_via_namespace ( THD thd,
MYSQL mysql,
Master_info mi,
bool  reconnect,
bool  suppress_warnings,
const std::string &  host,
const uint  port 
)
static

◆ coord_handle_partial_binlogged_transaction()

static bool coord_handle_partial_binlogged_transaction ( Relay_log_info rli,
const Log_event ev 
)
static

Let the worker applying the current group to rollback and gracefully finish its work before.

Parameters
rliThe slave's relay log info.
eva pointer to the event on hold before applying this rollback procedure.
Return values
falseThe rollback succeeded.
trueThere was an error while injecting events.

◆ copy_event_cache_to_file_and_reinit()

static bool copy_event_cache_to_file_and_reinit ( IO_CACHE cache,
FILE *  file,
bool  flush_stream 
)
inlinestatic

◆ delete_slave_info_objects()

void delete_slave_info_objects ( )

Free all resources used by slave threads at time of executing shutdown.

The routine must be called after all possible users of channel_map have left.

◆ end_info()

void end_info ( Master_info mi)

◆ end_slave()

void end_slave ( )

◆ evaluate_inter_option_dependencies()

int evaluate_inter_option_dependencies ( const LEX_MASTER_INFO lex_mi,
Master_info mi 
)

This method evaluates if the different options given to CHANGE REPLICATION SOURCE TO are compatible with the current configuration and with one another.

Example: SOURCE_CONNECTION_AUTO_FAILOVER = 1 requires SOURCE_AUTO_POSITION to be already enabled or to be enabled on this command.

Parameters
lex_mistructure that holds all change replication source options given on the command
miPointer to Master_info object for the channel that holds the the configuration
Returns
0 if no issues are found != 0 the error number associated to the issue, if one is found

We first define the variables used and then we group the checks for readability

◆ exec_relay_log_event()

static int exec_relay_log_event ( THD thd,
Relay_log_info rli,
Rpl_applier_reader applier_reader,
Log_event in 
)
static

Top-level function for executing the next event in the relay log.

This is called from the SQL thread.

This function reads the event from the relay log, executes it, and advances the relay log position. It also handles errors, etc.

This function may fail to apply the event for the following reasons:

  • The position specified by the UNTIL condition of the START SLAVE command is reached.
  • It was not possible to read the event from the log.
  • The slave is killed.
  • An error occurred when applying the event, and the event has been tried slave_trans_retries times. If the event has been retried fewer times, 0 is returned.
  • init_info or init_relay_log_pos failed. (These are called if a failure occurs when applying the event.)
  • An error occurred when updating the binlog position.
Return values
0The event was applied.
1The event was not applied.

The following failure injecion works in cooperation with tests setting @global.debug= 'd,incomplete_group_in_relay_log'. Xid or Commit events are not executed to force the slave sql read hanging if the relay log does not have any more events.

◆ extract_log_event_basic_info() [1/2]

std::pair< bool, mysql::binlog::event::Log_event_basic_info > extract_log_event_basic_info ( const char *  buf,
size_t  length,
const mysql::binlog::event::Format_description_event fd_event 
)

Extract basic info about an event: type, query, is it ignorable.

Parameters
bufThe event info buffer
lengthThe length of the buffer
fd_eventThe Format description event associated
Returns
a pair first param is true if an error occurred, false otherwise second param is the event info

◆ extract_log_event_basic_info() [2/2]

std::pair< bool, mysql::binlog::event::Log_event_basic_info > extract_log_event_basic_info ( Log_event log_event)

Extract basic info about an event: type, query, is it ignorable.

Parameters
log_eventthe event to extract info from
Returns
a pair first param is true if an error occurred, false otherwise second param is the event info

◆ fill_mts_gaps_and_recover()

static int fill_mts_gaps_and_recover ( Master_info mi)
inlinestatic

◆ find_first_relay_log_with_rotate_from_master()

static int find_first_relay_log_with_rotate_from_master ( Relay_log_info rli)
static

Reads relay logs one by one starting from the first relay log.

Looks for the first rotate event from the master. If rotate is not found in the relay log search continues to next relay log. If rotate event from master is found then the extracted master_log_file and master_log_pos are used to set rli->group_master_log_name and rli->group_master_log_pos. If an error has occurred the error code is returned back.

Parameters
rliRelay_log_info object to read relay log files and to set group_master_log_name and group_master_log_pos.
Return values
0Success - Rotate event was found
1Failure - Found some events replicated but no rotate event was found
2When no rotate event from master was found. This can happen when slave server was restarted immediately after executing CHANGE MASTER

◆ fix_gtid_set()

static void fix_gtid_set ( MYSQL_RPL rpl,
uchar packet_gtid_set 
)
static

Callback function for mysql_binlog_open().

Sets gtid data in the command packet.

Parameters
rplReplication stream information.
packet_gtid_setPointer to command packet where gtid data should be stored.

◆ flush_master_info()

int flush_master_info ( Master_info mi,
bool  force,
bool  need_lock = true,
bool  flush_relay_log = true,
bool  skip_repo_persistence = false 
)

This method flushes the current configuration for the channel into the connection metadata repository.

It will also flush the current contents of the relay log file if instructed to.

Parameters
mithe Master_info reference that holds both Master_info and Relay_log_info data.
forceshall the method ignore the server settings that limit flushes to this repository
need_lockshall the method take the associated data lock and log lock if false ownership is asserted
flush_relay_logshould the method also flush the relay log file
skip_repo_persistenceif this method shall skip the repository flush This won't skip the relay log flush if flush_relay_log = true
Returns
0 if no error occurred, !=0 if an error occurred

◆ flush_relay_logs()

int flush_relay_logs ( Master_info mi,
THD thd 
)

flushes the relay logs of a replication channel.

Parameters
[in]miMaster_info corresponding to the channel.
[in]thdthe client thread carrying the command.
Return values
1fail
0ok
-1deferred flush

◆ flush_relay_logs_cmd()

bool flush_relay_logs_cmd ( THD thd)

Entry point for FLUSH RELAYLOGS command or to flush relaylogs for the FLUSH LOGS command.

FLUSH LOGS or FLUSH RELAYLOGS needs to flush the relaylogs of all the replciaiton channels in multisource replication. FLUSH RELAYLOGS FOR CHANNEL flushes only the relaylogs pertaining to a channel.

Parameters
[in]thdthe client thread carrying the command.
Return values
truefail
falsesuccess

◆ get_initialization_code()

int ReplicaInitializer::get_initialization_code ( ) const

Gets initialization code set-up at replica initialization.

Returns
Error code obtained during the replica initialization

◆ get_master_uuid()

static int get_master_uuid ( MYSQL mysql,
Master_info mi 
)
static

Get master's uuid on connecting.

Parameters
mysqlMYSQL to request uuid from master.
miMaster_info to set master_uuid
Returns
0: Success, 1: Fatal error, 2: Transient network error.

◆ get_master_version_and_clock()

static int get_master_version_and_clock ( MYSQL mysql,
Master_info mi 
)
static

◆ get_rpl_part_id()

int get_rpl_part_id ( partition_info part_info)

This method is used to extract the partition_id from a partitioned table.

Parameters
part_infoan object of class partition_info it will be used to call the methods responsible for returning the value of partition_id
Return values
Thereturn value is the partition_id.

◆ group_replication_cleanup_after_clone()

static void group_replication_cleanup_after_clone ( )
static

Purge Group Replication channels relay logs after this server being a recipient of clone.

◆ handle_slave_io()

void * handle_slave_io ( void *  arg)

Slave IO thread entry point.

Parameters
argPointer to Master_info struct that holds information for the IO thread.
Returns
Always 0.

◆ handle_slave_sql()

void * handle_slave_sql ( void *  arg)

Slave SQL thread entry point.

Parameters
argPointer to Relay_log_info object that holds information for the SQL thread.
Returns
Always 0.

success, we read the next event.

fall through

this will make the main loop abort in the next iteration

fall through

single threaded applier has to retry. Next iteration reads the same event.

fall through

fall through

◆ handle_slave_worker()

static void * handle_slave_worker ( void *  arg)
static

◆ have_change_replication_source_applier_and_receive_option()

static bool have_change_replication_source_applier_and_receive_option ( const LEX_MASTER_INFO lex_mi)
static

This function checks if the given CHANGE REPLICATION SOURCE command has any option that affect both the receiver and the applier.

  • used in change_master().
Parameters
lex_mistructure that holds all options given on the change replication source command.
Return values
falseno option that affects both applier and receiver was found
trueAt least one option affects both the applier and receiver.

◆ have_change_replication_source_execute_option()

static bool have_change_replication_source_execute_option ( const LEX_MASTER_INFO lex_mi,
bool *  need_relay_log_purge 
)
static

This function checks if the given CHANGE MASTER/REPLICATION SOURCE command has any execute option being set or changed.

  • used in change_master().
Parameters
lex_mistructure that holds all options given on the change replication source command.
[out]need_relay_log_purge
  • If relay_log_file/relay_log_pos options are used, we won't delete relaylogs. We set this boolean flag to false.
  • If relay_log_file/relay_log_pos options are NOT used, we return the boolean flag UNCHANGED.
  • Used in change_receive_options() and change_master().
Return values
falseNo change replication source execute option.
trueAt least one execute option was there.

◆ have_change_replication_source_receive_option()

static bool have_change_replication_source_receive_option ( const LEX_MASTER_INFO lex_mi)
static

This function checks if the given CHANGE MASTER/REPLICATION SOURCE command has any receive option being set or changed.

  • used in change_master().
Parameters
lex_mistructure that holds all options given on the change replication source command.
Return values
falseNo change replication source receive options were found.
trueAt least one receive option was found.

◆ heartbeat_queue_event()

int heartbeat_queue_event ( bool  is_valid,
Master_info *&  mi,
std::string  binlog_name,
uint64_t  position,
unsigned long &  inc_pos,
bool &  do_flush_mi 
)

◆ ignored_error_code()

int ignored_error_code ( int  err_code)

Ignore error code specified on command line.

◆ init_recovery()

int init_recovery ( Master_info mi)

◆ init_replica()

int ReplicaInitializer::init_replica ( )
private

Performs replica initialization, creates default replication channel and sets channel filters.

Returns
Error code

◆ init_replica_psi_keys()

void ReplicaInitializer::init_replica_psi_keys ( )
staticprivate

Initializes replica PSI keys in case PSI interface is available.

◆ init_replica_skip_errors()

static void init_replica_skip_errors ( )
static

Init function to set up array for errors that should be skipped for slave.

◆ init_replica_thread()

int init_replica_thread ( THD thd,
SLAVE_THD_TYPE  thd_type 
)

◆ io_slave_killed()

static bool io_slave_killed ( THD thd,
Master_info mi 
)
inlinestatic

◆ io_thread_init_command()

static enum_command_status io_thread_init_command ( Master_info mi,
const char *  query,
int  allowed_error,
MYSQL_RES **  master_res = nullptr,
MYSQL_ROW master_row = nullptr 
)
static

Execute an initialization query for the IO thread.

If there is an error, then this function calls mysql_free_result; otherwise the MYSQL object holds the result after this call. If there is an error other than allowed_error, then this function prints a message and returns -1.

Parameters
miMaster_info object.
queryQuery string.
allowed_errorAllowed error code, or 0 if no errors are allowed.
[out]master_resIf this is not NULL and there is no error, then mysql_store_result() will be called and the result stored in this pointer.
[out]master_rowIf this is not NULL and there is no error, then mysql_fetch_row() will be called and the result stored in this pointer.
Return values
COMMAND_STATUS_OKNo error.
COMMAND_STATUS_ALLOWED_ERRORThere was an error and the error code was 'allowed_error'.
COMMAND_STATUS_ERRORThere was an error and the error code was not 'allowed_error'.

◆ io_thread_init_commands()

int io_thread_init_commands ( MYSQL mysql,
Master_info mi 
)

Set user variables after connecting to the master.

Parameters
mysqlMYSQL to request uuid from master.
miMaster_info to set master_uuid
Returns
0: Success, 1: Fatal error, 2: Transient network error.

◆ is_any_gtid_event()

bool is_any_gtid_event ( const Log_event evt)
inline

◆ is_atomic_ddl()

bool is_atomic_ddl ( THD thd,
bool  using_trans 
)

The function lists all DDL instances that are supported for crash-recovery (WL9175).

todo: the supported feature list is supposed to grow. Once a feature has been readied for 2pc through WL7743,9536(7141/7016) etc it needs registering in the function.

Parameters
thdan Query-log-event creator thread handle
using_transThe caller must specify the value according to the following rules: true when
  • on master the current statement is not processing a table in SE which does not support atomic DDL
  • on slave the relay-log repository is transactional. false otherwise.
Returns
true when the being created (master) or handled (slave) event is 2pc-capable, false otherwise.

◆ is_atomic_ddl_event()

bool is_atomic_ddl_event ( Log_event const *  evt)
inline

The function checks the argument event properties to deduce whether it represents an atomic DDL.

Parameters
evta reference to Log_event
Returns
true when the DDL properties are found, false otherwise

◆ is_autocommit_off()

static bool is_autocommit_off ( THD thd)
inlinestatic

Check if in multi-statement transaction mode.

Parameters
thdTHD object
Return values
trueSuccess
falseFailure

◆ is_invalid_change_master_for_group_replication_applier()

static bool is_invalid_change_master_for_group_replication_applier ( const LEX_MASTER_INFO lex_mi)
static

Method used to check if the user is trying to update any other option for the change master apart from the PRIVILEGE_CHECKS_USER.

In case user tries to update any other parameter apart from this one, this method will return error.

Parameters
lex_mistructure that holds all change master options given on the change master command.
Return values
true- The CHANGE MASTER is updating a unsupported parameter for the recovery channel.
false- Everything is fine. The CHANGE MASTER can execute with the given option(s) for the recovery channel.

◆ is_invalid_change_master_for_group_replication_recovery()

static bool is_invalid_change_master_for_group_replication_recovery ( const LEX_MASTER_INFO lex_mi)
static

Method used to check if the user is trying to update any other option for the change master apart from the MASTER_USER and MASTER_PASSWORD.

In case user tries to update any other parameter apart from these two, this method will return error.

Parameters
lex_mistructure that holds all change master options given on the change master command.
Return values
true- The CHANGE MASTER is updating a unsupported parameter for the recovery channel.
false- Everything is fine. The CHANGE MASTER can execute with the given option(s) for the recovery channel.

◆ is_network_error()

bool is_network_error ( uint  errorno)

◆ is_option_enabled_or_will_be()

bool is_option_enabled_or_will_be ( bool  base_value,
int  option_value 
)

This is an helper method for boolean vars like SOURCE_AUTO_POSITION REQUIRE_ROW_FORMAT SOURCE_CONNECTION_AUTO_FAILOVER It tells if the variable is already enabled or will be by the command.

Parameters
base_valuethe current variable value
option_valuethe configuration input value (UNCHANGED,ENABLED,DISABLE)
Returns
true if the option was already enable or will be. false otherwise

◆ is_session_control_event()

bool is_session_control_event ( Log_event evt)
inline

Check if the given event is a session control event, one of User_var_event, Intvar_event or Rand_event.

Parameters
evtThe event to check.
Returns
true if the given event is of type User_var_event, Intvar_event or Rand_event, false otherwise.

◆ load_mi_and_rli_from_repositories()

int load_mi_and_rli_from_repositories ( Master_info mi,
bool  ignore_if_no_info,
int  thread_mask,
bool  skip_received_gtid_set_recovery = false,
bool  force_load = false 
)

Call mi->init_info() and/or mi->rli->init_info(), which will read the replication configuration from repositories.

This takes care of creating a transaction context in case table repository is needed.

Parameters
miThe Master_info object to use.
ignore_if_no_infoIf this is false, and the repository does not exist, it will be created. If this is true, and the repository does not exist, nothing is done.
thread_maskIndicate which repositories will be initialized: if (thread_mask&SLAVE_IO)!=0, then mi->init_info is called; if (thread_mask&SLAVE_SQL)!=0, then mi->rli->init_info is called.
force_loadrepositories will only read information if they are not yet initialized. When true this flag forces the repositories to load information from table or file.
skip_received_gtid_set_recoveryWhen true, skips the received GTID set recovery.
Return values
0Success
nonzeroError

◆ log_invalid_position_warning()

static void log_invalid_position_warning ( THD thd,
const LEX_MASTER_INFO lex_mi,
Master_info mi 
)
static

Log a warning in case GTID_ONLY or SOURCE AUTO POSITION are disabled and the server contains invalid positions.

Parameters
thdthe associated thread object
lex_mistructure that holds all change replication source options given on the command
miPointer to Master_info object

◆ monitor_io_replica_killed()

static bool monitor_io_replica_killed ( THD thd,
Master_info mi 
)
inlinestatic

◆ mta_checkpoint_routine()

bool mta_checkpoint_routine ( Relay_log_info rli,
bool  force 
)

Processing rli->gaq to find out the low-water-mark (lwm) coordinates which is stored into the central recovery table.

rli->data_lock will be required, so the caller should not hold rli->data_lock.

Parameters
rlipointer to Relay-log-info of Coordinator
forceif true then hang in a loop till some progress
Return values
falseSuccess
trueError

◆ mts_event_coord_cmp()

int mts_event_coord_cmp ( LOG_POS_COORD id1,
LOG_POS_COORD id2 
)
static

Orders jobs by comparing relay log information.

◆ mts_recovery_groups()

bool mts_recovery_groups ( Relay_log_info rli)

◆ my_strmov_quoted_identifier()

size_t my_strmov_quoted_identifier ( THD thd,
char *  buffer,
const char *  identifier,
size_t  length 
)

◆ my_strmov_quoted_identifier_helper()

size_t my_strmov_quoted_identifier_helper ( int  q,
char *  buffer,
const char *  identifier,
size_t  length 
)

◆ net_field_length_checked()

template<typename T >
bool net_field_length_checked ( const uchar **  packet,
size_t *  max_length,
T *  out 
)

Read an integer in net_field_length format, guarding against read out of bounds and advancing the position.

Parameters
[in,out]packetPointer to buffer to read from. On successful return, the buffer position will be incremented to point to the next byte after what was read.
[in,out]max_lengthPointer to the number of bytes in the buffer. If the function would need to look at more than *max_length bytes in order to decode the number, the function will do nothing and return true.
[out]outPointer where the value will be stored.
Return values
falseSuccess.
trueFailure, i.e., reached end of buffer.

◆ net_request_file()

bool net_request_file ( NET net,
const char *  fname 
)

◆ print_channel_info()

void ReplicaInitializer::print_channel_info ( ) const
private

In case debug mode is on, prints channel information.

◆ print_replica_skip_errors()

static void print_replica_skip_errors ( void  )
static

Convert slave skip errors bitmap into a printable string.

◆ print_slave_db_safe()

const char * print_slave_db_safe ( const char *  db)

◆ process_io_rotate()

static int process_io_rotate ( Master_info mi,
Rotate_log_event rev 
)
static

Used by the slave IO thread when it receives a rotate event from the master.

Updates the master info with the place in the next binary log where we should start reading. Rotate the relay log to avoid mixed-format relay logs.

Parameters
mimaster_info for the slave
revThe rotate log event read from the master
Note
The caller must hold mi->data_lock before invoking this function.
Return values
0ok
1error

◆ queue_event()

QUEUE_EVENT_RESULT queue_event ( Master_info mi,
const char *  buf,
ulong  event_len,
bool  do_flush_mi 
)

Store an event received from the master connection into the relay log.

Parameters
miThe Master_info object representing this connection.
bufPointer to the event data.
event_lenLength of event data.
do_flush_miTrue to flush master info after successfully queuing the event.
Return values
QUEUE_EVENT_OKon success.
QUEUE_EVENT_ERROR_QUEUINGif there was an error while queuing.
QUEUE_EVENT_ERROR_FLUSHING_INFOif there was an error while flushing master info.

◆ read_event()

static ulong read_event ( MYSQL mysql,
MYSQL_RPL rpl,
Master_info mi,
bool *  suppress_warnings 
)
static

Read one event from the master.

Parameters
mysqlMySQL connection.
rplReplication stream information.
miMaster connection information.
suppress_warningstrue 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 an abnormal event in an idle server.
Return values
'packet_error'Error.
numberLength of packet.

◆ read_rotate_from_relay_log()

static enum_read_rotate_from_relay_log_status read_rotate_from_relay_log ( char *  filename,
char *  master_log_file,
my_off_t master_log_pos 
)
static

Parse the given relay log and identify the rotate event from the master.

Ignore the Format description event, Previous_gtid log event, ignorable event and Stop event within the relay log as they are generated by slave. When a rotate event is found check if it is a rotate that is originated from the master based on the server_id. Ignore the event if the rotate is from slave or if it is a fake rotate event. If any other events are encountered apart from the above events generate an error. From the rotate event extract the master's binary log name and position.

Parameters
filenameRelay log name which needs to be parsed.
[out]master_log_fileSet the master_log_file to the log file name that is extracted from rotate event. The master_log_file should contain string of len FN_REFLEN.
[out]master_log_posSet the master_log_pos to the log position extracted from rotate event.
Return values
FOUND_ROTATEWhen rotate event is found in the relay log
NOT_FOUND_ROTATEWhen rotate event is not found in the relay log
ERROROn error

◆ recover_relay_log()

static void recover_relay_log ( Master_info mi)
static

◆ reencrypt_relay_logs()

bool reencrypt_relay_logs ( )

Re-encrypt previous relay logs with current master key for all slave channels.

Return values
falseSuccess.
trueError.

◆ register_slave_on_master()

static int register_slave_on_master ( MYSQL mysql,
Master_info mi,
bool *  suppress_warnings 
)
static

◆ remove_info()

int remove_info ( Master_info mi)

◆ report_apply_event_error()

static int report_apply_event_error ( THD thd,
Relay_log_info rli 
)
static

Processes the outcome of applying an event, logs it properly if it's an error and return the proper error code to trigger.

Returns
the error code to bubble up in the execution stack.

◆ request_dump()

static int request_dump ( THD thd,
MYSQL mysql,
MYSQL_RPL rpl,
Master_info mi,
bool *  suppress_warnings 
)
static

◆ reset_info()

bool reset_info ( Master_info mi)

Resets the information regarding the Master_info and Relay_log_info objects represented by the parameter, meaning, setting to NULL all attributes that are not meant to be kept between slave resets and persisting all other attribute values in the repository.

Parameters
mithe Master_info reference that holds both Master_info and Relay_log_info data.
Returns
true if an error occurred and false otherwiser.

◆ reset_slave() [1/2]

int reset_slave ( THD thd)

Execute a RESET SLAVE (for all channels), used in Multisource replication.

If resetting of a particular channel fails, it exits out.

Parameters
[in]thdTHD object of the client.
Return values
0success
1error

◆ reset_slave() [2/2]

int reset_slave ( THD thd,
Master_info mi,
bool  reset_all 
)

Execute a RESET REPLICA statement.

Locks slave threads and unlocks the slave threads after executing reset slave. The method also takes the mi->channel_wrlock; if this {mi} object is deleted (when the parameter reset_all is true) its destructor unlocks the lock. In case of error, the method shall always unlock the mi channel lock.

Parameters
thdPointer to THD object of the client thread executing the statement.
miPointer to Master_info object for the slave.
reset_allDo a full reset or only clean master info structures
Return values
0success
!=0error

◆ reset_slave_cmd()

bool reset_slave_cmd ( THD thd)

Entry function for RESET SLAVE command.

Function either resets the slave for all channels or for a single channel. When RESET SLAVE ALL is given, the slave_info_objects (mi, rli & workers) are destroyed.

Parameters
[in]thdthe client thread with the command.
Return values
falseOK
truenot OK

◆ rotate_relay_log()

int rotate_relay_log ( Master_info mi,
bool  log_master_fd,
bool  need_lock,
bool  need_log_space_lock 
)

◆ safe_connect()

static int safe_connect ( THD thd,
MYSQL mysql,
Master_info mi,
const std::string &  host = std::string(),
const uint  port = 0 
)
static

◆ safe_reconnect()

static int safe_reconnect ( THD thd,
MYSQL mysql,
Master_info mi,
bool  suppress_warnings,
const std::string &  host = std::string(),
const uint  port = 0 
)
static

◆ set_replica_max_allowed_packet()

static void set_replica_max_allowed_packet ( THD thd,
MYSQL mysql 
)
static

◆ set_replica_skip_errors()

void set_replica_skip_errors ( char **  replica_skip_errors_ptr)

Change arg to the string with the nice, human-readable skip error values.

Parameters
replica_skip_errors_ptrThe pointer to be changed

◆ set_slave_thread_default_charset()

void set_slave_thread_default_charset ( THD thd,
Relay_log_info const *  rli 
)

◆ set_slave_thread_options()

void set_slave_thread_options ( THD thd)

Set slave thread default options.

Parameters
[in]thdThe thread

◆ set_thd_in_use_temporary_tables()

static void set_thd_in_use_temporary_tables ( Relay_log_info rli)
static

◆ set_thd_tx_priority()

static void set_thd_tx_priority ( THD thd,
int  priority 
)
static

◆ set_thd_write_set_options()

static void set_thd_write_set_options ( THD thd,
bool  ignore_limit,
bool  allow_drop_write_set 
)
static

Set for the thread options about the memory and size limits when transactions collect write sets.

Parameters
thdThread handler
ignore_limitif the memory limits should be ignored
allow_drop_write_setif this thread does not require WS to always be logged

◆ show_slave_status() [1/2]

bool show_slave_status ( THD thd)

Method to the show the replication status in all channels.

Parameters
[in]thdthe client thread
Return values
0success
1Error

◆ show_slave_status() [2/2]

bool show_slave_status ( THD thd,
Master_info mi 
)

Execute a SHOW REPLICA STATUS statement.

Parameters
thdPointer to THD object for the client thread executing the statement.
miPointer to Master_info object for the IO thread.
Return values
falsesuccess
truefailure

Currently, show slave status works for a channel too, in multisource replication. But using performance schema tables is better.

◆ show_slave_status_cmd()

bool show_slave_status_cmd ( THD thd)

Entry point for SHOW REPLICA STATUS command.

Function displays the slave status for all channels or for a single channel based on the FOR CHANNEL clause.

Parameters
[in]thdthe client thread.
Return values
falseok
truenot ok

◆ show_slave_status_metadata()

static void show_slave_status_metadata ( mem_root_deque< Item * > *  field_list,
int  io_gtid_set_size,
int  sql_gtid_set_size 
)
static

Function that fills the metadata required for SHOW REPLICA STATUS.

This function shall be used in two cases: 1) SHOW REPLICA STATUS FOR ALL CHANNELS 2) SHOW REPLICA STATUS for a channel

Parameters
[in,out]field_listfield_list to fill the metadata
[in]io_gtid_set_sizethe size to be allocated to store the retrieved gtid set
[in]sql_gtid_set_sizethe size to be allocated to store the executed gtid set

◆ show_slave_status_send_data()

static bool show_slave_status_send_data ( THD thd,
Master_info mi,
char *  io_gtid_set_buffer,
char *  sql_gtid_set_buffer 
)
static

Send the data to the client of a Master_info during show_slave_status() This function has to be called after calling show_slave_status_metadata().

Just before sending the data, thd->get_protocol() is prepared to (re)send;

Parameters
[in]thdclient thread
[in]mithe master info. In the case of multisource replication, this master info corresponds to a channel.
[in]io_gtid_set_bufferbuffer related to Retrieved GTID set for each channel.
[in]sql_gtid_set_bufferbuffer related to Executed GTID set for each channel.
Return values
0success
1Error

◆ slave_execute_deferred_events()

bool slave_execute_deferred_events ( THD thd)

The function is called by slave applier in case there are active table filtering rules to force gathering events associated with Query-log-event into an array to execute them once the fate of the Query is determined for execution.

The function is called by slave applier in case there are active table filtering rules to force gathering events associated with Query-log-event into an array to execute them once the fate of the Query is determined for execution.

Parameters
thdTHD handle
Returns
false on success, true if a failure in an event applying occurred.

◆ slave_io_thread_detach_vio()

void slave_io_thread_detach_vio ( )

Hook to detach the active VIO before closing a connection handle.

The client API might close the connection (and associated data) in case it encounters a unrecoverable (network) error. This hook is called from the client code before the VIO handle is deleted allows the thread to detach the active vio so it does not point to freed memory.

Other calls to THD::clear_active_vio throughout this module are redundant due to the hook but are left in place for illustrative purposes.

◆ slave_sleep()

template<typename killed_func , typename rpl_info >
static bool slave_sleep ( THD thd,
time_t  seconds,
killed_func  func,
rpl_info  info 
)
inlinestatic

Sleep for a given amount of time or until killed.

Parameters
thdThread context of the current thread.
secondsThe number of seconds to sleep.
funcFunction object to check if the thread has been killed.
infoThe Rpl_info object associated with this sleep.
Return values
Trueif the thread has been killed, false otherwise.

◆ slave_start_single_worker()

static int slave_start_single_worker ( Relay_log_info rli,
ulong  i 
)
static

Instantiation of a Slave_worker and forking out a single Worker thread.

Parameters
rliCoordinator's Relay_log_info pointer
iidentifier of the Worker
Returns
0 suppress or 1 if fails

◆ slave_start_workers()

static int slave_start_workers ( Relay_log_info rli,
ulong  n,
bool *  mts_inited 
)
static

Initialization of the central rli members for Coordinator's role, communication channels such as Assigned Partition Hash (APH), and starting the Worker pool.

Parameters
rliPointer to Coordinator's Relay_log_info instance.
nNumber of configured Workers in the upcoming session.
[out]mts_initedIf the initialization processed was started.
Returns
0 success non-zero as failure

gtid_monitoring_info must be cleared when MTS is enabled or workers_copy_pfs has elements

◆ slave_stop_workers()

static void slave_stop_workers ( Relay_log_info rli,
bool *  mts_inited 
)
static

◆ sql_delay_event()

static int sql_delay_event ( Log_event ev,
THD thd,
Relay_log_info rli 
)
static

If this is a lagging slave (specified with CHANGE MASTER TO MASTER_DELAY = X), delays accordingly.

Also unlocks rli->data_lock.

Design note: this is the place to unlock rli->data_lock. The lock must be held when reading delay info from rli, but it should not be held while sleeping.

Parameters
evEvent that is about to be executed.
thdThe sql thread's THD object.
rliThe sql thread's Relay_log_info structure.
Return values
0If the delay timed out and the event shall be executed.
nonzeroIf the delay was interrupted and the event shall be skipped.

◆ sql_slave_killed()

bool sql_slave_killed ( THD thd,
Relay_log_info rli 
)

The function analyzes a possible killed status and makes a decision whether to accept it or not.

Normally upon accepting the sql thread goes to shutdown. In the event of deferring decision rli->last_event_start_time waiting timer is set to force the killed status be accepted upon its expiration.

Notice Multi-Threaded-Slave behaves similarly in that when it's being stopped and the current group of assigned events has not yet scheduled completely, Coordinator defers to accept to leave its read-distribute state. The above timeout ensures waiting won't last endlessly, and in such case an error is reported.

Parameters
thdpointer to a THD instance
rlipointer to Relay_log_info instance
Returns
true the killed status is recognized, false a possible killed status is deferred.

◆ start_replication_threads()

void ReplicaInitializer::start_replication_threads ( bool  skip_replica_start = true)
private

This function starts replication threads.

Parameters
[in]skip_replica_startWhen true, skips the start of replication threads threads

◆ start_slave() [1/2]

bool start_slave ( THD thd)

Function to start a slave for all channels.

Used in Multisource replication.

Parameters
[in]thdTHD object of the client.
Return values
falsesuccess
trueerror

◆ start_slave() [2/2]

bool start_slave ( THD thd,
LEX_SLAVE_CONNECTION connection_param,
LEX_MASTER_INFO master_param,
int  thread_mask_input,
Master_info mi,
bool  set_mts_settings 
)

Executes a START SLAVE statement.

Parameters
thdPointer to THD object for the client thread executing the statement.
connection_paramConnection parameters for starting threads
master_paramMaster parameters used for starting threads
thread_mask_inputThe thread mask that identifies which threads to start. If 0 is passed (start no thread) then this parameter is ignored and all stopped threads are started
miPointer to Master_info object for the slave's IO thread.
set_mts_settingsIf true, the channel uses the server MTS configured settings when starting the applier thread.
Return values
falsesuccess
trueerror

◆ start_slave_cmd()

bool start_slave_cmd ( THD thd)

Entry point to the START SLAVE command.

The function decides to start replication threads on several channels or a single given channel.

Parameters
[in]thdthe client thread carrying the command.
Return values
falseok
truenot ok.

◆ start_slave_thread()

bool start_slave_thread ( PSI_thread_key  thread_key,
my_start_routine  h_func,
mysql_mutex_t start_lock,
mysql_mutex_t cond_lock,
mysql_cond_t start_cond,
std::atomic< uint > *  slave_running,
std::atomic< ulong > *  slave_run_id,
Master_info mi 
)

◆ start_slave_threads()

bool start_slave_threads ( bool  need_lock_slave,
bool  wait_for_start,
Master_info mi,
int  thread_mask 
)

SQL AFTER MTS GAPS has no effect when GTID_MODE=ON and SOURCE_AUTO_POS=1 as no gaps information was collected.

◆ start_threads()

void ReplicaInitializer::start_threads ( )
private

This function starts replication threads.

◆ stop_slave() [1/2]

int stop_slave ( THD thd)

Function to stop a slave for all channels.

Used in Multisource replication.

Parameters
[in]thdTHD object of the client.
Return values
0success
1error

◆ stop_slave() [2/2]

int stop_slave ( THD thd,
Master_info mi,
bool  net_report,
bool  for_one_channel,
bool *  push_temp_tables_warning 
)

Execute a STOP SLAVE statement.

Parameters
thdPointer to THD object for the client thread executing the statement.
miPointer to Master_info object for the slave's IO thread.
net_reportIf true, saves the exit status into Diagnostics_area.
for_one_channelIf the method is being invoked only for one channel
push_temp_tables_warningIf it should push a "have temp tables warning" once having open temp tables. This avoids multiple warnings when there is more than one channel with open temp tables. This parameter can be removed when the warning is issued with per-channel information.
Return values
0success
1error

◆ stop_slave_cmd()

bool stop_slave_cmd ( THD thd)

Entry point for the STOP SLAVE command.

This function stops replication threads for all channels or a single channel based on the command options supplied.

Parameters
[in]thdthe client thread.
Return values
falseok
truenot ok.

◆ str_to_hex()

char * str_to_hex ( char *  to,
const char *  from,
size_t  len 
)

Transforms a string into "" or its expression in 0x... form.

◆ terminate_slave_thread()

static int terminate_slave_thread ( THD thd,
mysql_mutex_t term_lock,
mysql_cond_t term_cond,
std::atomic< uint > *  slave_running,
ulong *  stop_wait_timeout,
bool  need_lock_term,
bool  force 
)
static

Wait for a slave thread to terminate.

This function is called after requesting the thread to terminate (by setting abort_slave member of Relay_log_info or Master_info structure to 1). Termination of the thread is controlled with the the predicate *slave_running.

Function will acquire term_lock before waiting on the condition unless need_lock_term is false in which case the mutex should be owned by the caller of this function and will remain acquired after return from the function.

Parameters
thdCurrent session.
term_lockAssociated lock to use when waiting for term_cond
term_condCondition that is signalled when the thread has terminated
slave_runningPointer to predicate to check for slave thread termination
stop_wait_timeoutA pointer to a variable that denotes the time the thread has to stop before we time out and throw an error.
need_lock_termIf false the lock will not be acquired before waiting on the condition. In this case, it is assumed that the calling function acquires the lock before calling this function.
forceForce the slave thread to stop by sending a KILL_CONNECTION signal to it. This is used to forcefully stop the I/O thread when it is waiting for disk space and the server is shutting down.
Return values
0All OK, 1 on "STOP REPLICA" command timeout, ER_REPLICA_CHANNEL_NOT_RUNNING otherwise.
Note
If the executing thread has to acquire term_lock (need_lock_term is true, the negative running status does not represent any issue therefore no error is reported.

◆ terminate_slave_threads()

int terminate_slave_threads ( Master_info mi,
int  thread_mask,
ulong  stop_wait_timeout,
bool  need_lock_term = true 
)

Terminates the slave threads according to the given mask.

Parameters
mithe master info repository
thread_maskthe mask identifying which thread(s) to terminate
stop_wait_timeoutthe timeout after which the method returns and error
need_lock_termIf false the lock will not be acquired before waiting on the condition. In this case, it is assumed that the calling function acquires the lock before calling this function.
Returns
the operation status
Return values
0OK
ER_REPLICA_NOT_RUNNINGThe slave is already stopped
ER_STOP_REPLICA_SQL_THREAD_TIMEOUTThere was a timeout when stopping the SQL thread
ER_STOP_REPLICA_IO_THREAD_TIMEOUTThere was a timeout when stopping the IO thread
ER_ERROR_DURING_FLUSH_LOGSThere was an error while flushing the log/repositories

◆ try_to_reconnect()

static int try_to_reconnect ( THD thd,
MYSQL mysql,
Master_info mi,
uint *  retry_count,
bool  suppress_warnings,
const Reconnect_messages messages 
)
static

Try to reconnect slave IO thread.

Terminates current connection to master, sleeps for mi->connect_retry msecs and initiates new connection with safe_reconnect(). Variable pointed by retry_count is increased - if it exceeds mi->retry_count then connection is not re-established and function signals error.

Unless suppress_warnings is true, a warning is written to the server error log when reconnecting. The warning message, the messages used to report errors, and the thread stages, are taken from rm. In case mi->retry_count is exceeded, no messages are added to the log.

Parameters
[in]thdThread context.
[in]mysqlMySQL connection.
[in]miMaster connection information.
[in,out]retry_countNumber of attempts to reconnect.
[in]suppress_warningstrue when a normal net read timeout has caused to reconnecting.
[in]messagesError/warning messages and thread stage information. See class Reconnect_messages.
Return values
0OK.
1There was an error.

◆ update_change_replication_source_options()

static bool update_change_replication_source_options ( THD thd,
LEX_MASTER_INFO lex_mi,
Master_info mi,
bool  have_both_receive_execute_option,
bool  have_execute_option,
bool  have_receive_option 
)
static

This method aggregates the the instantiation of options for the command CHANGE REPLICATION SOURCE.

Parameters
thdPointer to THD object for the client thread executing the statement.
lex_mistructure that holds all change replication source options given on the command
miPointer to Master_info object belonging to the replica channel to be configured
have_both_receive_execute_optionthe command will change options that affect both the applier and receiver
have_execute_optionthe command will change applier related options
have_receive_optionthe command will change receiver related options
Returns
returns true if an error occurred, false otherwise

◆ validate_change_replication_source_options()

static std::pair< bool, bool > validate_change_replication_source_options ( THD thd,
const LEX_MASTER_INFO lex_mi,
Master_info mi,
int  thread_mask 
)
static

This method aggregates the validation checks made for the command CHANGE REPLICATION SOURCE.

Parameters
thdPointer to THD object for the client thread executing the statement.
lex_mistructure that holds all change replication source options given on the command
miPointer to Master_info object for the channel that holds the the configuration
thread_maskThe thread mask identifying which threads are running
Returns
A pair of booleans <return_value, remove_mta_info> return_value: true if an error occurred, false otherwise remove_mta_info: if true remove MTA worker info

Although this check is redone when the user is set, we do an early check here to avoid failures in the middle of configuration

◆ validate_gtid_option_restrictions()

static int validate_gtid_option_restrictions ( const LEX_MASTER_INFO lex_mi,
Master_info mi 
)
static

This function validates that change replication source options are valid according to the current GTID_MODE.

This method assumes it will only be called when GTID_MODE != ON

Parameters
lex_mistructure that holds all change replication source options
miPointer to Master_info object belonging to the replica channel to be configured
Returns
false if the configuration is valid true some configuration option is invalid with GTID_MODE

◆ wait_for_relay_log_space()

static bool wait_for_relay_log_space ( Relay_log_info rli)
static

◆ write_ignored_events_info_to_relay_log()

static int write_ignored_events_info_to_relay_log ( THD thd,
Master_info mi 
)
static

◆ write_rotate_to_master_pos_into_relay_log()

static int write_rotate_to_master_pos_into_relay_log ( THD thd,
Master_info mi,
bool  force_flush_mi_info 
)
static

Variable Documentation

◆ all_slave_memory

PSI_memory_info all_slave_memory[]
static
Initial value:
"Relay_log_info::mta_coor", 0, 0,
#define PSI_DOCUMENT_ME
Definition: component_common.h:28
static PSI_memory_key key_memory_rli_mta_coor
Definition: rpl_replica.cc:395

◆ all_slave_threads

PSI_thread_info all_slave_threads[]
static
Initial value:
= {
{&key_thread_replica_io, "replica_io", "rpl_rca_io", PSI_FLAG_THREAD_SYSTEM,
{&key_thread_replica_sql, "replica_sql", "rpl_rca_sql",
{&key_thread_replica_worker, "replica_worker", "rpl_rca_wkr",
{&key_thread_replica_monitor_io, "replica_monitor", "rpl_rca_mon",
#define PSI_FLAG_SINGLETON
Singleton flag.
Definition: component_common.h:34
static PSI_thread_key key_thread_replica_io
Definition: rpl_replica.cc:397
static PSI_thread_key key_thread_replica_monitor_io
Definition: rpl_replica.cc:398
static PSI_thread_key key_thread_replica_worker
Definition: rpl_replica.cc:398
static PSI_thread_key key_thread_replica_sql
Definition: rpl_replica.cc:397
#define PSI_FLAG_THREAD_SYSTEM
System thread flag.
Definition: psi_bits.h:123

◆ binlog_checksum_typelib

TYPELIB binlog_checksum_typelib
extern

◆ key_memory_Incident_log_event_message

PSI_memory_key key_memory_Incident_log_event_message
extern

◆ key_memory_rli_mta_coor

PSI_memory_key key_memory_rli_mta_coor
static

◆ key_memory_Rows_query_log_event_rows_query

PSI_memory_key key_memory_Rows_query_log_event_rows_query
extern

◆ key_thread_replica_io

PSI_thread_key key_thread_replica_io
static

◆ key_thread_replica_monitor_io

PSI_thread_key key_thread_replica_monitor_io
static

◆ key_thread_replica_sql

PSI_thread_key key_thread_replica_sql
static

◆ key_thread_replica_worker

PSI_thread_key key_thread_replica_worker
static

◆ master_retry_count

ulong master_retry_count
extern

◆ mts_coordinator_basic_nap

const ulong mts_coordinator_basic_nap = 5

◆ mts_slave_worker_queue_len_max

const ulong mts_slave_worker_queue_len_max = 16384

◆ mts_worker_underrun_level

const ulong mts_worker_underrun_level = 10

◆ opt_binlog_index_name

char* opt_binlog_index_name
extern

◆ opt_log_replica_updates

bool opt_log_replica_updates
extern

◆ opt_relay_logname

char* opt_relay_logname
extern

◆ opt_relay_logname_supplied

bool opt_relay_logname_supplied
extern

◆ opt_relaylog_index_name

char * opt_relaylog_index_name

◆ opt_relaylog_index_name_supplied

bool opt_relaylog_index_name_supplied
extern

◆ opt_replica_skip_errors

char* opt_replica_skip_errors
extern

◆ opt_skip_replica_start

bool opt_skip_replica_start
extern

If set, slave is not autostarted.

◆ reconnect_messages_after_failed_dump

Reconnect_messages reconnect_messages_after_failed_dump
static
Initial value:
{
"Replica I/O thread killed while retrying source dump",
"failed dump request, reconnecting to try again, log '%s' at position "
"%s",
"COM_BINLOG_DUMP"}
PSI_stage_info stage_replica_waiting_to_reconnect_after_failed_binlog_dump_request
Definition: mysqld.cc:13961
PSI_stage_info stage_replica_reconnecting_after_failed_binlog_dump_request
Definition: mysqld.cc:13956

◆ reconnect_messages_after_failed_event_read

Reconnect_messages reconnect_messages_after_failed_event_read
static
Initial value:
{
"Replica I/O thread killed while waiting to reconnect after a failed read",
"Replica I/O thread: Failed reading log event, reconnecting to retry, "
"log '%s' at position %s",
""}
PSI_stage_info stage_replica_waiting_to_reconnect_after_failed_event_read
Definition: mysqld.cc:13962
PSI_stage_info stage_replica_reconnecting_after_failed_event_read
Definition: mysqld.cc:13957

◆ reconnect_messages_after_failed_registration

Reconnect_messages reconnect_messages_after_failed_registration
static
Initial value:
{
"Replica I/O thread killed while waiting to reconnect after a failed "
"registration on source",
"failed registering on source, reconnecting to try again, "
"log '%s' at position %s",
"COM_REGISTER_REPLICA"}
PSI_stage_info stage_replica_waiting_to_reconnect_after_failed_registration_on_source
Definition: mysqld.cc:13963
PSI_stage_info stage_replica_reconnecting_after_failed_registration_on_source
Definition: mysqld.cc:13958

◆ relay_log_basename [1/2]

const char* relay_log_basename = nullptr

◆ relay_log_basename [2/2]

const char* relay_log_basename
extern

◆ relay_log_index [1/2]

const char* relay_log_index = nullptr

◆ relay_log_index [2/2]

const char* relay_log_index
extern

◆ relay_log_space_limit [1/2]

ulonglong relay_log_space_limit = 0

◆ relay_log_space_limit [2/2]

ulonglong relay_log_space_limit
extern

◆ replica_load_tmpdir [1/2]

char* replica_load_tmpdir = nullptr

◆ replica_load_tmpdir [2/2]

char* replica_load_tmpdir
extern

◆ replicate_same_server_id [1/2]

bool replicate_same_server_id

◆ replicate_same_server_id [2/2]

bool replicate_same_server_id
extern

◆ report_host

char* report_host
extern

◆ report_password

char * report_password

◆ report_port

uint report_port
extern

◆ report_user

char* report_user
extern

◆ RPL_MASTER_INFO

thread_local Master_info* RPL_MASTER_INFO = nullptr
static

◆ SEQ_MAX_TIMESTAMP

const int64 SEQ_MAX_TIMESTAMP = LLONG_MAX

Maximum value of binlog logical timestamp.

◆ server_id

MYSQL_PLUGIN_IMPORT ulong server_id

◆ server_id_supplied

bool server_id_supplied
extern

◆ server_version

◆ slave_error_mask [1/2]

MY_BITMAP slave_error_mask

◆ slave_error_mask [2/2]

MY_BITMAP slave_error_mask
extern

◆ slave_skip_error_names [1/2]

char slave_skip_error_names[]
extern

◆ slave_skip_error_names [2/2]

char slave_skip_error_names[SHOW_VAR_FUNC_BUFF_SIZE]

◆ sql_replica_skip_counter

uint sql_replica_skip_counter

a copy of active_mi->rli->slave_skip_counter, for showing in SHOW GLOBAL VARIABLES, INFORMATION_SCHEMA.GLOBAL_VARIABLES and @sql_replica_skip_counter without taking all the mutexes needed to access active_mi->rli->slave_skip_counter properly.

◆ use_slave_mask [1/2]

bool use_slave_mask = false

◆ use_slave_mask [2/2]

bool use_slave_mask
extern