MySQL 9.1.0
Source Code Documentation
|
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 | REPLICA_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 |
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_info > | extract_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_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. 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, std::size_t queued_size) |
Checks whether relay log space will be exceeded after queueing additional 'queued_size' bytes. More... | |
static bool | exceeds_relay_log_limit (Relay_log_info *rli, std::size_t queued_size) |
Checks whether relay log space limit will be exceeded after queueing additional 'queued_size' bytes. More... | |
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 REPLICA command. More... | |
bool | stop_slave_cmd (THD *thd) |
Entry point for the STOP REPLICA command. More... | |
static enum_read_rotate_from_relay_log_status | read_rotate_from_relay_log (char *filename, char *source_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_and_relaylog_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 REPLICATION SOURCE TO SOURCE_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=true, bool need_lock=true, bool need_log_space_lock=true) |
Rotates the relay log. More... | |
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_REPLICA_CONNECTION *connection_param, LEX_SOURCE_INFO *master_param, int thread_mask_input, Master_info *mi, bool set_mts_settings) |
Executes a START REPLICA 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 REPLICA statement. More... | |
int | reset_slave (THD *thd) |
Execute a RESET REPLICA (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 REPLICA command. More... | |
static bool | have_change_replication_source_receive_option (const LEX_SOURCE_INFO *lex_mi) |
This function checks if the given CHANGE REPLICATION SOURCE command has any receive option being set or changed. More... | |
static bool | have_change_replication_source_execute_option (const LEX_SOURCE_INFO *lex_mi, bool *need_relay_log_purge) |
This function checks if the given CHANGE REPLICATION SOURCE command has any execute option being set or changed. More... | |
static bool | have_change_replication_source_applier_and_receive_option (const LEX_SOURCE_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_SOURCE_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_SOURCE_INFO *lex_mi, Master_info *mi) |
This function is called if the change replication source command had at least one receive option. More... | |
static bool | change_execute_options (LEX_SOURCE_INFO *lex_mi, Master_info *mi) |
This function is called if the change replication source command had at least one execute option. More... | |
static bool | change_applier_receiver_options (THD *thd, LEX_SOURCE_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_SOURCE_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_SOURCE_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_SOURCE_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_SOURCE_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_SOURCE_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_SOURCE_INFO *lex_mi, bool preserve_logs) |
Execute a CHANGE REPLICATION SOURCE 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_SOURCE_INFO *lex_mi) |
Method used to check if the user is trying to update any other option for the change replication source apart from the SOURCE_USER and SOURCE_PASSWORD. More... | |
static bool | is_invalid_change_master_for_group_replication_applier (const LEX_SOURCE_INFO *lex_mi) |
Method used to check if the user is trying to update any other option for the change replication source apart from the PRIVILEGE_CHECKS_USER. More... | |
bool | change_master_cmd (THD *thd) |
Entry point for the CHANGE REPLICATION SOURCE command. More... | |
void | enable_applier_metric_collection () |
Enables metric collection for replication structures It affects new and already created and running channels. More... | |
void | disable_applier_metric_collection () |
Disables metric collection for replication structures It affects new and already created and running channels. 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... | |
#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.
COND | Condition to check |
ERRNO | Error number to return in non-debug builds |
#define BINLOG_MAGIC "\xfe\x62\x69\x6e" |
#define BINLOG_MAGIC_SIZE 4 |
#define EXPECTED_OPTIONS ((1ULL << 14) | (1ULL << 26) | (1ULL << 27) | (1ULL << 19)) |
#define FLAGSTR | ( | V, | |
F | |||
) | ((V) & (F) ? #F " " : "") |
#define I_TYPE_OFFSET 0 |
#define I_VAL_OFFSET 1 |
#define LOG_EVENT_OFFSET 4 |
#define MAX_LOG_EVENT_HEADER |
#define MAX_SLAVE_ERROR 14000 |
#define MAX_SLAVE_RETRY_PAUSE 5 |
#define MTS_MAX_BITS_IN_GROUP ((1L << 19) - 8) /* 524280 */ |
#define MTS_MAX_WORKERS 1024 |
#define MTS_WORKER_UNDEF ((ulong)-1) |
#define MYSQL_SLAVE_NOT_RUN 0 |
#define MYSQL_SLAVE_RUN_CONNECT 2 |
#define MYSQL_SLAVE_RUN_NOT_CONNECT 1 |
#define NUM_LOAD_DELIM_STRS 5 |
#define OPTIONS_WRITTEN_TO_BIN_LOG |
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).
#define PREFIX_SQL_LOAD "SQL_LOAD-" |
#define REPLICA_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
#define REPLICA_NET_TIMEOUT 60 |
#define SL_MASTER_HOST_OFFSET 10 |
#define SL_MASTER_PORT_OFFSET 8 |
#define SL_MASTER_POS_OFFSET 0 |
#define SLAVE_FORCE_ALL 4 |
#define SLAVE_WAIT_GROUP_DONE 60 |
#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)
using ColumnViewPtr = std::unique_ptr<cs::util::ReplicatedColumnsView> |
using sql_mode_t = uint64_t |
enum enum_command_status |
enum QUEUE_EVENT_RESULT |
ReplicaInitializer::ReplicaInitializer | ( | bool | opt_initialize, |
bool | opt_skip_replica_start, | ||
Rpl_channel_filters & | filters, | ||
char ** | replica_skip_erors | ||
) |
Constructor, calls init_replica()
[in] | opt_initialize | Server option used to indicate whether mysqld has been started with –initialize |
[in] | opt_skip_replica_start | When true, skips the start of replication threads |
[in] | filters | Replication filters |
[in] | replica_skip_erors | TBD |
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
[in,out] | mi | When new {mi,rli} are created, the reference is stored in *mi |
[in] | channel | The channel on which the change master was introduced. |
void add_replica_skip_errors | ( | const char * | arg | ) |
|
static |
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.
|
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:
It also does the following maintenance:
ptr_ev | a pointer to a reference to the event to apply. |
thd | The 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). |
rli | The 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). |
ptr_ev
location to indicate the event is taken over by a Worker.SLAVE_APPLY_EVENT_AND_UPDATE_POS_OK | OK. |
SLAVE_APPLY_EVENT_AND_UPDATE_POS_APPLY_ERROR | Error calling ev->apply_event(). |
SLAVE_APPLY_EVENT_AND_UPDATE_POS_UPDATE_POS_ERROR | No error calling ev->apply_event(), but error calling ev->update_pos(). |
SLAVE_APPLY_EVENT_AND_UPDATE_POS_APPEND_JOB_ERROR | append_item_to_jobs() failed, thread was killed while waiting for successful enqueue on worker. |
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.
|
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()
lex_mi | structure that holds all change replication source options |
mi | Pointer to Master_info object belonging to the replica channel to be configured |
|
static |
This function is called if the change replication source 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()
lex_mi | structure that holds all change replication source options given on the change replication source command. |
mi | Pointer to Master_info object belonging to the replica channel that will be configured |
int change_master | ( | THD * | thd, |
Master_info * | mi, | ||
LEX_SOURCE_INFO * | lex_mi, | ||
bool | preserve_logs | ||
) |
Execute a CHANGE REPLICATION SOURCE statement.
Apart from changing the receive/execute configurations/positions, this function also does the following:
thd | Pointer to THD object for the client thread executing the statement. |
mi | Pointer to Master_info object belonging to the slave's IO thread. |
lex_mi | Lex 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_logs | If 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. |
0 | success |
!=0 | error |
Is there a an error during validation
bool change_master_cmd | ( | THD * | thd | ) |
Entry point for the CHANGE REPLICATION SOURCE command.
Function decides to create a new channel or create an existing one.
[in] | thd | the client thread that issued the command. |
true | fail |
false | success. |
|
static |
This function checks all possible cases in which compression algorithm, compression level can be configured for a channel.
lex_mi | pointer to structure holding all options specified as part of change replication source to statement |
mi | pointer to structure holding all options specified as part of change replication source to statement after performing necessary checks |
false | in case of success |
true | in case of failures |
|
static |
This function is called if the change replication source 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()
thd | Pointer to THD object for the client thread executing the statement. |
lex_mi | structure that holds all change replication source options given on the change replication source 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. |
mi | Pointer to Master_info object belonging to the replica channel to be configured |
0 | no error i.e., success. |
!=0 | error. |
|
static |
|
static |
Checks the current replica configuration when starting a replication thread If some incompatibility is found an error is thrown.
mi | pointer to the source info repository object |
thread_mask | what replication threads are running |
|
static |
(end of group Replication)
Checks the current replica configuration against the server GTID mode If some incompatibility is found a warning is logged.
|
static |
Check if there is any slave SQL config conflict.
[in] | rli | The slave's rli object. |
|
static |
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.
mi | the Master_info reference that holds both Master_info and Relay_log_info data. |
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 | ||
) |
|
static |
|
static |
Let the worker applying the current group to rollback and gracefully finish its work before.
rli | The slave's relay log info. |
ev | a pointer to the event on hold before applying this rollback procedure. |
false | The rollback succeeded. |
true | There was an error while injecting events. |
|
inlinestatic |
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.
void disable_applier_metric_collection | ( | ) |
Disables metric collection for replication structures It affects new and already created and running channels.
void enable_applier_metric_collection | ( | ) |
Enables metric collection for replication structures It affects new and already created and running channels.
void end_info | ( | Master_info * | mi | ) |
void end_slave | ( | ) |
int evaluate_inter_option_dependencies | ( | const LEX_SOURCE_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.
lex_mi | structure that holds all change replication source options given on the command |
mi | Pointer to Master_info object for the channel that holds the the configuration |
We first define the variables used and then we group the checks for readability
|
inlinestatic |
Checks whether relay log space limit will be exceeded after queueing additional 'queued_size' bytes.
rli | Pointer to connection metadata object for the considered channel |
queued_size | Number of bytes we want to queue |
true | Limit exceeded |
false | Current size + 'queued_size' is within limits |
|
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:
0 | The event was applied. |
1 | The 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.
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.
buf | The event info buffer |
length | The length of the buffer |
fd_event | The Format description event associated |
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.
log_event | the event to extract info from |
|
inlinestatic |
|
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 source_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.
rli | Relay_log_info object to read relay log files and to set group_master_log_name and group_master_log_pos. |
0 | Success - Rotate event was found |
1 | Failure - Found some events replicated but no rotate event was found |
2 | When no rotate event from master was found. This can happen when slave server was restarted immediately after executing CHANGE REPLICATION SOURCE |
Callback function for mysql_binlog_open().
Sets gtid data in the command packet.
rpl | Replication stream information. |
packet_gtid_set | Pointer to command packet where gtid data should be stored. |
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.
mi | the Master_info reference that holds both Master_info and Relay_log_info data. |
force | shall the method ignore the server settings that limit flushes to this repository |
need_lock | shall the method take the associated data lock and log lock if false ownership is asserted |
flush_relay_log | should the method also flush the relay log file |
skip_repo_persistence | if this method shall skip the repository flush This won't skip the relay log flush if flush_relay_log = true |
int flush_relay_logs | ( | Master_info * | mi, |
THD * | thd | ||
) |
flushes the relay logs of a replication channel.
[in] | mi | Master_info corresponding to the channel. |
[in] | thd | the client thread carrying the command. |
1 | fail |
0 | ok |
-1 | deferred flush |
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.
[in] | thd | the client thread carrying the command. |
true | fail |
false | success |
int ReplicaInitializer::get_initialization_code | ( | ) | const |
Gets initialization code set-up at replica initialization.
|
static |
Get master's uuid on connecting.
mysql | MYSQL to request uuid from master. |
mi | Master_info to set master_uuid |
|
static |
int get_rpl_part_id | ( | partition_info * | part_info | ) |
This method is used to extract the partition_id from a partitioned table.
part_info | an object of class partition_info it will be used to call the methods responsible for returning the value of partition_id |
The | return value is the partition_id. |
|
static |
Purge Group Replication channels relay logs after this server being a recipient of clone.
void * handle_slave_io | ( | void * | arg | ) |
Slave IO thread entry point.
arg | Pointer to Master_info struct that holds information for the IO thread. |
void * handle_slave_sql | ( | void * | arg | ) |
Slave SQL thread entry point.
arg | Pointer to Relay_log_info object that holds information for the SQL thread. |
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
|
static |
|
static |
This function checks if the given CHANGE REPLICATION SOURCE command has any option that affect both the receiver and the applier.
lex_mi | structure that holds all options given on the change replication source command. |
false | no option that affects both applier and receiver was found |
true | At least one option affects both the applier and receiver. |
|
static |
This function checks if the given CHANGE REPLICATION SOURCE command has any execute option being set or changed.
lex_mi | structure that holds all options given on the change replication source command. | |
[out] | need_relay_log_purge |
|
false | No change replication source execute option. |
true | At least one execute option was there. |
|
static |
This function checks if the given CHANGE REPLICATION SOURCE command has any receive option being set or changed.
lex_mi | structure that holds all options given on the change replication source command. |
false | No change replication source receive options were found. |
true | At least one receive option was found. |
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 | ||
) |
int ignored_error_code | ( | int | err_code | ) |
Ignore error code specified on command line.
int init_recovery | ( | Master_info * | mi | ) |
|
private |
Performs replica initialization, creates default replication channel and sets channel filters.
|
staticprivate |
Initializes replica PSI keys in case PSI interface is available.
|
static |
Init function to set up array for errors that should be skipped for slave.
int init_replica_thread | ( | THD * | thd, |
SLAVE_THD_TYPE | thd_type | ||
) |
|
inlinestatic |
|
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.
mi | Master_info object. | |
query | Query string. | |
allowed_error | Allowed error code, or 0 if no errors are allowed. | |
[out] | master_res | If 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_row | If this is not NULL and there is no error, then mysql_fetch_row() will be called and the result stored in this pointer. |
COMMAND_STATUS_OK | No error. |
COMMAND_STATUS_ALLOWED_ERROR | There was an error and the error code was 'allowed_error'. |
COMMAND_STATUS_ERROR | There was an error and the error code was not 'allowed_error'. |
int io_thread_init_commands | ( | MYSQL * | mysql, |
Master_info * | mi | ||
) |
Set user variables after connecting to the master.
mysql | MYSQL to request uuid from master. |
mi | Master_info to set master_uuid |
|
inline |
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.
thd | an Query-log-event creator thread handle |
using_trans | The caller must specify the value according to the following rules: true when
|
false
otherwise.
|
inline |
The function checks the argument event properties to deduce whether it represents an atomic DDL.
evt | a reference to Log_event |
|
inlinestatic |
Check if in multi-statement transaction mode.
thd | THD object |
true | Success |
false | Failure |
|
static |
Method used to check if the user is trying to update any other option for the change replication source apart from the PRIVILEGE_CHECKS_USER.
In case user tries to update any other parameter apart from this one, this method will return error.
lex_mi | structure that holds all change replication source options given on the change replication source command. |
true | - The CHANGE REPLICATION SOURCE is updating a unsupported parameter for the recovery channel. |
false | - Everything is fine. The CHANGE REPLICATION SOURCE can execute with the given option(s) for the recovery channel. |
|
static |
Method used to check if the user is trying to update any other option for the change replication source apart from the SOURCE_USER and SOURCE_PASSWORD.
In case user tries to update any other parameter apart from these two, this method will return error.
lex_mi | structure that holds all change replication source options given on the change replication source command. |
true | - The CHANGE REPLICATION SOURCE is updating a unsupported parameter for the recovery channel. |
false | - Everything is fine. The CHANGE REPLICATION SOURCE can execute with the given option(s) for the recovery channel. |
bool is_network_error | ( | uint | errorno | ) |
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.
base_value | the current variable value |
option_value | the configuration input value (UNCHANGED,ENABLED,DISABLE) |
|
inline |
Check if the given event is a session control event, one of User_var_event
, Intvar_event
or Rand_event
.
evt | The event to check. |
User_var_event
, Intvar_event
or Rand_event
, false otherwise. int load_mi_and_rli_from_repositories | ( | Master_info * | mi, |
bool | ignore_if_no_info, | ||
int | thread_mask, | ||
bool | skip_received_gtid_set_and_relaylog_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.
mi | The Master_info object to use. |
ignore_if_no_info | If 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_mask | Indicate which repositories will be initialized: if (thread_mask&REPLICA_IO)!=0, then mi->init_info is called; if (thread_mask&REPLICA_SQL)!=0, then mi->rli->init_info is called. |
force_load | repositories 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_and_relaylog_recovery | When true, skips the received GTID set and relay log recovery. |
0 | Success |
nonzero | Error |
|
static |
Log a warning in case GTID_ONLY or SOURCE AUTO POSITION are disabled and the server contains invalid positions.
thd | the associated thread object |
lex_mi | structure that holds all change replication source options given on the command |
mi | Pointer to Master_info object |
|
inlinestatic |
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.
rli | pointer to Relay-log-info of Coordinator |
force | if true then hang in a loop till some progress |
false | Success |
true | Error |
|
static |
Orders jobs by comparing relay log information.
bool mts_recovery_groups | ( | Relay_log_info * | rli | ) |
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 | ||
) |
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.
[in,out] | packet | Pointer 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_length | Pointer 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] | out | Pointer where the value will be stored. |
false | Success. |
true | Failure, i.e., reached end of buffer. |
bool net_request_file | ( | NET * | net, |
const char * | fname | ||
) |
|
private |
In case debug mode is on, prints channel information.
|
static |
Convert slave skip errors bitmap into a printable string.
const char * print_slave_db_safe | ( | const char * | db | ) |
|
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.
mi | master_info for the slave |
rev | The rotate log event read from the master |
0 | ok |
1 | error |
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.
mi | The Master_info object representing this connection. |
buf | Pointer to the event data. |
event_len | Length of event data. |
do_flush_mi | True to flush master info after successfully queuing the event. |
QUEUE_EVENT_OK | on success. |
QUEUE_EVENT_ERROR_QUEUING | if there was an error while queuing. |
QUEUE_EVENT_ERROR_FLUSHING_INFO | if there was an error while flushing master info. |
|
static |
Read one event from the master.
mysql | MySQL connection. |
rpl | Replication stream information. |
mi | Master connection information. |
suppress_warnings | true when a normal net read timeout has caused us to try a reconnect. We do not want to print anything to the error log in this case because this an abnormal event in an idle server. |
'packet_error' | Error. |
number | Length of packet. |
|
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.
filename | Relay log name which needs to be parsed. | |
[out] | source_log_file | Set the source_log_file to the log file name that is extracted from rotate event. The source_log_file should contain string of len FN_REFLEN. |
[out] | master_log_pos | Set the master_log_pos to the log position extracted from rotate event. |
FOUND_ROTATE | When rotate event is found in the relay log |
NOT_FOUND_ROTATE | When rotate event is not found in the relay log |
ERROR | On error |
|
static |
bool reencrypt_relay_logs | ( | ) |
Re-encrypt previous relay logs with current master key for all slave channels.
false | Success. |
true | Error. |
|
static |
int remove_info | ( | Master_info * | mi | ) |
|
static |
Processes the outcome of applying an event, logs it properly if it's an error and return the proper error code to trigger.
|
static |
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.
mi | the Master_info reference that holds both Master_info and Relay_log_info data. |
int reset_slave | ( | THD * | thd | ) |
Execute a RESET REPLICA (for all channels), used in Multisource replication.
If resetting of a particular channel fails, it exits out.
[in] | thd | THD object of the client. |
0 | success |
1 | error |
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 replica. 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.
thd | Pointer to THD object of the client thread executing the statement. |
mi | Pointer to Master_info object for the slave. |
reset_all | Do a full reset or only clean master info structures |
0 | success |
!=0 | error |
bool reset_slave_cmd | ( | THD * | thd | ) |
Entry function for RESET REPLICA command.
Function either resets the slave for all channels or for a single channel. When RESET REPLICA ALL is given, the slave_info_objects (mi, rli & workers) are destroyed.
[in] | thd | the client thread with the command. |
false | OK |
true | not OK |
int rotate_relay_log | ( | Master_info * | mi, |
bool | log_master_fd = true , |
||
bool | need_lock = true , |
||
bool | need_log_space_lock = true |
||
) |
Rotates the relay log.
Locking order: a) log_lock, log_space_lock b) log_lock, end_pos_lock
mi | Pointer to connection metadata object |
log_master_fd | Information on whether rotate came from:
|
need_lock | When true, we acquire relay log lock, otherwise the caller must hold it |
need_log_space_lock | When true, we acquire the log protecting data structures responsible for handling the relay_log_space_limit, otherwise the caller must hold it |
|
static |
|
static |
void set_replica_skip_errors | ( | char ** | replica_skip_errors_ptr | ) |
Change arg to the string with the nice, human-readable skip error values.
replica_skip_errors_ptr | The pointer to be changed |
void set_slave_thread_default_charset | ( | THD * | thd, |
Relay_log_info const * | rli | ||
) |
void set_slave_thread_options | ( | THD * | thd | ) |
Set slave thread default options.
[in] | thd | The thread |
|
static |
|
static |
|
static |
Set for the thread options about the memory and size limits when transactions collect write sets.
thd | Thread handler |
ignore_limit | if the memory limits should be ignored |
allow_drop_write_set | if this thread does not require WS to always be logged |
bool show_slave_status | ( | THD * | thd | ) |
Method to the show the replication status in all channels.
[in] | thd | the client thread |
0 | success |
1 | Error |
bool show_slave_status | ( | THD * | thd, |
Master_info * | mi | ||
) |
Execute a SHOW REPLICA STATUS statement.
thd | Pointer to THD object for the client thread executing the statement. |
mi | Pointer to Master_info object for the IO thread. |
false | success |
true | failure |
Currently, show replica status works for a channel too, in multisource replication. But using performance schema tables is better.
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.
[in] | thd | the client thread. |
false | ok |
true | not ok |
|
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
[in,out] | field_list | field_list to fill the metadata |
[in] | io_gtid_set_size | the size to be allocated to store the retrieved gtid set |
[in] | sql_gtid_set_size | the size to be allocated to store the executed gtid set |
|
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;
[in] | thd | client thread |
[in] | mi | the master info. In the case of multisource replication, this master info corresponds to a channel. |
[in] | io_gtid_set_buffer | buffer related to Retrieved GTID set for each channel. |
[in] | sql_gtid_set_buffer | buffer related to Executed GTID set for each channel. |
0 | success |
1 | Error |
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.
thd | THD handle |
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.
|
inlinestatic |
Sleep for a given amount of time or until killed.
thd | Thread context of the current thread. |
seconds | The number of seconds to sleep. |
func | Function object to check if the thread has been killed. |
info | The Rpl_info object associated with this sleep. |
True | if the thread has been killed, false otherwise. |
|
static |
Instantiation of a Slave_worker and forking out a single Worker thread.
rli | Coordinator's Relay_log_info pointer |
i | identifier of the Worker |
|
static |
Initialization of the central rli members for Coordinator's role, communication channels such as Assigned Partition Hash (APH), and starting the Worker pool.
rli | Pointer to Coordinator's Relay_log_info instance. | |
n | Number of configured Workers in the upcoming session. | |
[out] | mts_inited | If the initialization processed was started. |
gtid_monitoring_info must be cleared when MTS is enabled or workers_copy_pfs has elements
|
static |
|
static |
If this is a lagging slave (specified with CHANGE REPLICATION SOURCE TO SOURCE_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.
ev | Event that is about to be executed. |
thd | The sql thread's THD object. |
rli | The sql thread's Relay_log_info structure. |
0 | If the delay timed out and the event shall be executed. |
nonzero | If the delay was interrupted and the event shall be skipped. |
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.
thd | pointer to a THD instance |
rli | pointer to Relay_log_info instance |
|
private |
This function starts replication threads.
[in] | skip_replica_start | When true, skips the start of replication threads threads |
bool start_slave | ( | THD * | thd | ) |
Function to start a slave for all channels.
Used in Multisource replication.
[in] | thd | THD object of the client. |
false | success |
true | error |
bool start_slave | ( | THD * | thd, |
LEX_REPLICA_CONNECTION * | connection_param, | ||
LEX_SOURCE_INFO * | master_param, | ||
int | thread_mask_input, | ||
Master_info * | mi, | ||
bool | set_mts_settings | ||
) |
Executes a START REPLICA statement.
thd | Pointer to THD object for the client thread executing the statement. |
connection_param | Connection parameters for starting threads |
master_param | Master parameters used for starting threads |
thread_mask_input | The 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 |
mi | Pointer to Master_info object for the slave's IO thread. |
set_mts_settings | If true, the channel uses the server MTS configured settings when starting the applier thread. |
false | success |
true | error |
bool start_slave_cmd | ( | THD * | thd | ) |
Entry point to the START REPLICA command.
The function decides to start replication threads on several channels or a single given channel.
[in] | thd | the client thread carrying the command. |
false | ok |
true | not ok. |
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 | ||
) |
SQL AFTER MTS GAPS has no effect when GTID_MODE=ON and SOURCE_AUTO_POS=1 as no gaps information was collected.
|
private |
This function starts replication threads.
int stop_slave | ( | THD * | thd | ) |
Function to stop a slave for all channels.
Used in Multisource replication.
[in] | thd | THD object of the client. |
0 | success |
1 | error |
int stop_slave | ( | THD * | thd, |
Master_info * | mi, | ||
bool | net_report, | ||
bool | for_one_channel, | ||
bool * | push_temp_tables_warning | ||
) |
Execute a STOP REPLICA statement.
thd | Pointer to THD object for the client thread executing the statement. |
mi | Pointer to Master_info object for the slave's IO thread. |
net_report | If true, saves the exit status into Diagnostics_area. |
for_one_channel | If the method is being invoked only for one channel |
push_temp_tables_warning | If 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. |
0 | success |
1 | error |
bool stop_slave_cmd | ( | THD * | thd | ) |
Entry point for the STOP REPLICA command.
This function stops replication threads for all channels or a single channel based on the command options supplied.
[in] | thd | the client thread. |
false | ok |
true | not ok. |
char * str_to_hex | ( | char * | to, |
const char * | from, | ||
size_t | len | ||
) |
Transforms a string into "" or its expression in 0x... form.
|
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.
thd | Current session. |
term_lock | Associated lock to use when waiting for term_cond |
term_cond | Condition that is signalled when the thread has terminated |
slave_running | Pointer to predicate to check for slave thread termination |
stop_wait_timeout | A pointer to a variable that denotes the time the thread has to stop before we time out and throw an error. |
need_lock_term | If 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. |
force | Force 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. |
0 | All OK, 1 on "STOP REPLICA" command timeout, ER_REPLICA_CHANNEL_NOT_RUNNING otherwise. |
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.
mi | the master info repository |
thread_mask | the mask identifying which thread(s) to terminate |
stop_wait_timeout | the timeout after which the method returns and error |
need_lock_term | If 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. |
0 | OK |
ER_REPLICA_NOT_RUNNING | The slave is already stopped |
ER_STOP_REPLICA_SQL_THREAD_TIMEOUT | There was a timeout when stopping the SQL thread |
ER_STOP_REPLICA_IO_THREAD_TIMEOUT | There was a timeout when stopping the IO thread |
ER_ERROR_DURING_FLUSH_LOGS | There was an error while flushing the log/repositories |
|
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.
[in] | thd | Thread context. |
[in] | mysql | MySQL connection. |
[in] | mi | Master connection information. |
[in,out] | retry_count | Number of attempts to reconnect. |
[in] | suppress_warnings | true when a normal net read timeout has caused to reconnecting. |
[in] | messages | Error/warning messages and thread stage information. See class Reconnect_messages. |
0 | OK. |
1 | There was an error. |
|
static |
This method aggregates the the instantiation of options for the command CHANGE REPLICATION SOURCE.
thd | Pointer to THD object for the client thread executing the statement. |
lex_mi | structure that holds all change replication source options given on the command |
mi | Pointer to Master_info object belonging to the replica channel to be configured |
have_both_receive_execute_option | the command will change options that affect both the applier and receiver |
have_execute_option | the command will change applier related options |
have_receive_option | the command will change receiver related options |
|
static |
This method aggregates the validation checks made for the command CHANGE REPLICATION SOURCE.
thd | Pointer to THD object for the client thread executing the statement. |
lex_mi | structure that holds all change replication source options given on the command |
mi | Pointer to Master_info object for the channel that holds the the configuration |
thread_mask | The thread mask identifying which threads are running |
Although this check is redone when the user is set, we do an early check here to avoid failures in the middle of configuration
|
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
lex_mi | structure that holds all change replication source options |
mi | Pointer to Master_info object belonging to the replica channel to be configured |
|
inlinestatic |
Checks whether relay log space will be exceeded after queueing additional 'queued_size' bytes.
If yes, function will request relay log purge, rotate the relay log and wait for notification from coordinator
true | Event may be queued by the receiver |
false | Failed to reclaim required relay log space |
|
static |
|
static |
|
static |
|
static |
|
extern |
|
extern |
|
static |
|
extern |
|
static |
|
static |
|
static |
|
static |
const ulong mts_coordinator_basic_nap = 5 |
const ulong mts_slave_worker_queue_len_max = 16384 |
const ulong mts_worker_underrun_level = 10 |
|
extern |
|
extern |
|
extern |
|
extern |
|
extern |
char * opt_relaylog_index_name |
|
extern |
|
extern |
|
extern |
If set, slave is not autostarted.
|
static |
|
static |
|
static |
const char* relay_log_basename = nullptr |
|
extern |
const char* relay_log_index = nullptr |
|
extern |
ulonglong relay_log_space_limit = 0 |
|
extern |
char* replica_load_tmpdir = nullptr |
|
extern |
bool replicate_same_server_id |
|
extern |
|
extern |
char * report_password |
|
extern |
|
extern |
|
static |
const int64 SEQ_MAX_TIMESTAMP = LLONG_MAX |
Maximum value of binlog logical timestamp.
MYSQL_PLUGIN_IMPORT ulong server_id |
|
extern |
MYSQL_PLUGIN_IMPORT char server_version[SERVER_VERSION_LENGTH] |
MY_BITMAP slave_error_mask |
|
extern |
|
extern |
char slave_skip_error_names[SHOW_VAR_FUNC_BUFF_SIZE] |
|
extern |
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.
bool use_slave_mask = false |
|
extern |