MySQL 9.0.0
Source Code Documentation
Binary Log

Namespaces

namespace  anonymous_namespace{binlog.cc}
 

Classes

class  MYSQL_BIN_LOG::Binlog_ofile
 Logical binlog file which wraps and hides the detail of lower layer storage implementation. More...
 
class  Thd_backup_and_restore
 Helper class to switch to a new thread and then go back to the previous one, when the object is destroyed using RAII. More...
 
class  binlog_cache_data
 Caches for non-transactional and transactional data before writing it to the binary log. More...
 
class  binlog_stmt_cache_data
 
class  binlog_trx_cache_data
 
class  binlog_cache_mngr
 
class  Binlog_event_writer
 Auxiliary class to copy serialized events to the binary log and correct some of the fields that are not known until just before writing the event. More...
 
class  Binlog_cache_compressor
 Controls the execution flow when we compress the transaction cache into memory and write back the compressed data to the transaction cache. More...
 

Macros

#define MY_OFF_T_UNDEF   (~(my_off_t)0UL)
 
#define LIMIT_UNSAFE_WARNING_ACTIVATION_TIMEOUT   50
 
#define LIMIT_UNSAFE_WARNING_ACTIVATION_THRESHOLD_COUNT   50
 

Enumerations

enum  enum_read_gtids_from_binlog_status {
  GOT_GTIDS , GOT_PREVIOUS_GTIDS , NO_GTIDS , ERROR ,
  TRUNCATED
}
 

Functions

static int binlog_init (void *p)
 
static int binlog_start_trans_and_stmt (THD *thd, Log_event *start_event)
 Function to start a statement and optionally a transaction for the binary log. More...
 
static int binlog_close_connection (handlerton *hton, THD *thd)
 
static int binlog_savepoint_set (handlerton *hton, THD *thd, void *sv)
 
static int binlog_savepoint_rollback (handlerton *hton, THD *thd, void *sv)
 
static bool binlog_savepoint_rollback_can_release_mdl (handlerton *, THD *thd)
 Check whether binlog state allows to safely release MDL locks after rollback to savepoint. More...
 
static int binlog_commit (handlerton *, THD *, bool)
 This function is called once after each statement. More...
 
static int binlog_rollback (handlerton *, THD *thd, bool all)
 This function is called when a transaction or a statement is rolled back. More...
 
static int binlog_prepare (handlerton *hton, THD *thd, bool all)
 
static int binlog_set_prepared_in_tc (handlerton *hton, THD *thd)
 
static void binlog_prepare_row_images (const THD *thd, TABLE *table)
 
static bool is_loggable_xa_prepare (THD *thd)
 Predicate function yields true when XA transaction is being logged having a proper state ready for prepare or commit in one phase. More...
 
static std::pair< bool, int > check_purge_conditions (const MYSQL_BIN_LOG &log)
 Checks whether purge conditions are met to be able to run purge for binary log files. More...
 
static time_t calculate_auto_purge_lower_time_bound ()
 This function abstracts the calculation of the binary log files retention lower bound. More...
 
static bool check_auto_purge_conditions ()
 Checks if automatic purge conditions are met and therefore the purge is allowed to be done. More...
 
static binlog_cache_mngrthd_get_cache_mngr (const THD *thd)
 
void check_binlog_cache_size (THD *thd)
 Checks if the BINLOG_CACHE_SIZE's value is greater than MAX_BINLOG_CACHE_SIZE. More...
 
void check_binlog_stmt_cache_size (THD *thd)
 Checks if the BINLOG_STMT_CACHE_SIZE's value is greater than MAX_BINLOG_STMT_CACHE_SIZE. More...
 
bool binlog_enabled ()
 Check whether binlog_hton has valid slot and enabled. More...
 
static void binlog_trans_log_savepos (THD *thd, my_off_t *pos)
 
static int binlog_dummy_recover (handlerton *, XA_recover_txn *, uint, MEM_ROOT *)
 
static int binlog_deinit (void *)
 
enum xa_option_words get_xa_opt (THD *thd)
 
static uint purge_log_get_error_code (int res)
 purge logs, master and slave sides both, related error code converter. More...
 
static bool purge_error_message (THD *thd, int res)
 
bool is_transaction_empty (THD *thd)
 Check if the the transaction is empty. More...
 
int check_trx_rw_engines (THD *thd, Transaction_ctx::enum_trx_scope trx_scope)
 Check if the transaction has no rw flag set for any of the storage engines. More...
 
bool is_empty_transaction_in_binlog_cache (const THD *thd)
 Check if at least one of transacaction and statement binlog caches contains an empty transaction, other one is empty or contains an empty transaction, which has two binlog events "BEGIN" and "COMMIT". More...
 
bool trans_has_updated_trans_table (const THD *thd)
 This function checks if a transactional table was updated by the current transaction. More...
 
bool stmt_has_updated_trans_table (Ha_trx_info_list const &ha_list)
 This function checks if a transactional table was updated by the current statement. More...
 
bool ending_trans (THD *thd, const bool all)
 This function checks if a transaction, either a multi-statement or a single statement transaction is about to commit or not. More...
 
bool ending_single_stmt_trans (THD *thd, const bool all)
 This function checks if a single statement transaction is about to commit or not. More...
 
bool trans_cannot_safely_rollback (const THD *thd)
 This function checks if a transaction cannot be rolled back safely. More...
 
bool stmt_cannot_safely_rollback (const THD *thd)
 This function checks if current statement cannot be rollded back safely. More...
 
bool purge_source_logs_to_file (THD *thd, const char *to_log)
 Execute a PURGE BINARY LOGS TO <log> command. More...
 
bool purge_source_logs_before_date (THD *thd, time_t purge_time)
 Execute a PURGE BINARY LOGS BEFORE <date> command. More...
 
int query_error_code (const THD *thd, bool not_killed)
 
int log_loaded_block (IO_CACHE *file)
 Load data's io cache specific hook to be executed before a chunk of data is being read into the cache's buffer The function instantiates and writes into the binlog replication events along LOAD DATA processing. More...
 
template<class BINLOG_FILE_READER >
bool show_binlog_events (THD *thd, MYSQL_BIN_LOG *binary_log)
 
bool mysql_show_binlog_events (THD *thd)
 Execute a SHOW BINLOG EVENTS statement. More...
 
static bool is_number (const char *str, ulong *res, bool allow_wildcards)
 Check if a string is a valid number. More...
 
static int find_uniq_filename (char *name, uint32 new_index_number)
 Find a unique filename for 'filename. More...
 
static bool read_gtids_and_update_trx_parser_from_relaylog (const char *filename, Gtid_set *retrieved_gtids, bool verify_checksum, mysql::binlog::event::Transaction_boundary_parser *trx_parser, Gtid_monitoring_info *partial_trx)
 Add the GTIDs from the given relaylog file and also update the IO thread transaction parser. More...
 
static enum_read_gtids_from_binlog_status read_gtids_from_binlog (const char *filename, Gtid_set *all_gtids, Gtid_set *prev_gtids, Gtid *first_gtid, Tsid_map *tsid_map, bool verify_checksum, bool is_relay_log)
 Reads GTIDs from the given binlog file. More...
 
static int call_after_sync_hook (THD *queue_head)
 Auxiliary function used in ordered_commit. More...
 
static void register_binlog_handler (THD *thd, bool trx)
 
static bool has_write_table_with_auto_increment (Table_ref *tables)
 
static bool has_write_table_with_auto_increment_and_query_block (Table_ref *tables)
 
static bool has_write_table_auto_increment_not_first_in_pk (Table_ref *tables)
 
static bool has_nondeterministic_default (const TABLE *table)
 Checks if a table has a column with a non-deterministic DEFAULT expression. More...
 
static bool has_write_table_with_nondeterministic_default (const Table_ref *tables)
 Checks if a Table_ref contains a table that has been opened for writing, and that has a column with a non-deterministic DEFAULT expression. More...
 
static bool has_acl_table_read (THD *thd, const Table_ref *tl_list)
 Checks if we have reads from ACL tables in table list. More...
 
static bool fulltext_unsafe_set (TABLE_SHARE *s)
 
const char * get_locked_tables_mode_name (enum_locked_tables_mode locked_tables_mode)
 Getter for the enum enum_locked_tables_mode. More...
 
static bool handle_gtid_consistency_violation (THD *thd, int error_code, int log_error_code)
 Given that a possible violation of gtid consistency has happened, checks if gtid-inconsistencies are forbidden by the current value of ENFORCE_GTID_CONSISTENCY and GTID_MODE. More...
 
static const char * show_query_type (THD::enum_binlog_query_type qtype)
 
static void reset_binlog_unsafe_suppression ()
 Auxiliary function to reset the limit unsafety warning suppression. More...
 
static void print_unsafe_warning_to_log (int unsafe_type, char *buf, const char *query)
 Auxiliary function to print warning in the error log. More...
 
static void do_unsafe_limit_checkout (char *buf, int unsafe_type, const char *query)
 Auxiliary function to check if the warning for limit unsafety should be thrown or suppressed. More...
 
int binlog_stmt_cache_data::finalize (THD *thd)
 
int binlog_cache_data::write_event (Log_event *event)
 
void binlog_cache_data::set_incident (std::string_view incident_message="Non-transactional changes were not written to the binlog.")
 
bool binlog_cache_data::has_incident (void) const
 
bool MYSQL_BIN_LOG::assign_automatic_gtids_to_flush_group (THD *first_seen)
 Assign automatic generated GTIDs for all commit group threads in the flush stage having gtid_next.type == AUTOMATIC_GTID. More...
 
bool MYSQL_BIN_LOG::write_transaction (THD *thd, binlog_cache_data *cache_data, Binlog_event_writer *writer, bool parallelization_barrier)
 Write the Gtid_log_event to the binary log (prior to writing the statement or transaction cache). More...
 
int MYSQL_BIN_LOG::gtid_end_transaction (THD *thd)
 If the thread owns a GTID, this function generates an empty transaction and releases ownership of the GTID. More...
 
std::pair< std::list< std::string >, mysql::utils::ErrorMYSQL_BIN_LOG::get_filename_list ()
 
bool MYSQL_BIN_LOG::reencrypt_logs ()
 Re-encrypt previous existent binary/relay logs as below. More...
 
bool binlog_cache_data::compress (THD *thd)
 Compress the current transaction "in-place", if possible. More...
 
int binlog_cache_data::finalize (THD *thd, Log_event *end_event)
 This function finalizes the cache preparing for commit or rollback. More...
 
int binlog_cache_data::finalize (THD *thd, Log_event *end_event, XID_STATE *xs)
 The method writes XA END query to XA-prepared transaction's cache and calls the "basic" finalize(). More...
 
int binlog_cache_mngr::handle_deferred_cache_write_incident (THD *thd)
 Handle previous failure writing cache, aka. More...
 
int binlog_cache_data::flush (THD *thd, my_off_t *bytes, bool *wrote_xid, bool parallelization_barrier)
 Flush caches to the binary log. More...
 
void binlog_trx_cache_data::truncate (THD *thd, bool all)
 This function truncates the transactional cache upon committing or rolling back either a transaction or a statement. More...
 
int MYSQL_BIN_LOG::write_xa_to_cache (THD *thd)
 Logging XA commit/rollback of a prepared transaction. More...
 
bool MYSQL_BIN_LOG::write_incident_commit (THD *thd, std::string_view incident_message)
 Write an incident and call commit. More...
 
int MYSQL_BIN_LOG::rollback (THD *thd, bool all) override
 Write a rollback record of the transaction to the binary log. More...
 
bool MYSQL_BIN_LOG::is_current_stmt_binlog_enabled_and_caches_empty (const THD *thd) const
 Checks whether binlog caches are disabled (binlog does not cache data) or empty in case binloggging is enabled in the current call to this function. More...
 
 MYSQL_BIN_LOG::MYSQL_BIN_LOG (uint *sync_period, bool relay_log=false)
 
 MYSQL_BIN_LOG::~MYSQL_BIN_LOG () override
 
void MYSQL_BIN_LOG::cleanup ()
 
void MYSQL_BIN_LOG::init_pthread_objects ()
 
int MYSQL_BIN_LOG::generate_new_name (char *new_name, const char *log_name, uint32 new_index_number=0)
 
const char * MYSQL_BIN_LOG::generate_name (const char *log_name, const char *suffix, char *buff)
 
bool MYSQL_BIN_LOG::init_and_set_log_file_name (const char *log_name, const char *new_name, uint32 new_index_number)
 
bool MYSQL_BIN_LOG::open (PSI_file_key log_file_key, const char *log_name, const char *new_name, uint32 new_index_number)
 Open the logfile and init IO_CACHE. More...
 
bool MYSQL_BIN_LOG::open_index_file (const char *index_file_name_arg, const char *log_name, bool need_lock_index)
 
bool MYSQL_BIN_LOG::find_first_log (std::string &binlog_file_name, std::string &errmsg)
 Find the oldest binary log referenced by the index file. More...
 
bool MYSQL_BIN_LOG::find_first_log_not_in_gtid_set (char *binlog_file_name, const Gtid_set *gtid_set, Gtid *first_gtid, std::string &errmsg)
 Find the oldest binary log that contains any GTID that is not in the given gtid set. More...
 
bool MYSQL_BIN_LOG::init_gtid_sets (Gtid_set *all_gtids, Gtid_set *lost_gtids, bool verify_checksum, bool need_lock, mysql::binlog::event::Transaction_boundary_parser *trx_parser, Gtid_monitoring_info *partial_trx, bool is_server_starting=false)
 Reads the set of all GTIDs in the binary/relay log, and the set of all lost GTIDs in the binary log, and stores each set in respective argument. More...
 
bool MYSQL_BIN_LOG::open_binlog (const char *log_name, const char *new_name, ulong max_size_arg, bool null_created_arg, bool need_lock_index, bool need_tsid_lock, Format_description_log_event *extra_description_event, uint32 new_index_number=0)
 Create a new binary log. More...
 
int MYSQL_BIN_LOG::get_current_log (Log_info *linfo, bool need_lock_log=true)
 
int MYSQL_BIN_LOG::raw_get_current_log (Log_info *linfo)
 
bool MYSQL_BIN_LOG::check_write_error (const THD *thd)
 
void MYSQL_BIN_LOG::report_cache_write_error (THD *thd, bool is_transactional)
 
int MYSQL_BIN_LOG::find_log_pos (Log_info *linfo, const char *log_name, bool need_lock_index)
 
int MYSQL_BIN_LOG::find_next_log (Log_info *linfo, bool need_lock_index)
 
int MYSQL_BIN_LOG::find_next_relay_log (char log_name[FN_REFLEN+1])
 
std::pair< int, std::list< std::string > > MYSQL_BIN_LOG::get_log_index (bool need_lock_index=true)
 
bool MYSQL_BIN_LOG::reset_logs (THD *thd, bool delete_only=false)
 Removes files, as part of a RESET BINARY LOGS AND GTIDS or RESET REPLICA statement, by deleting all logs referred to in the index file and the index file. More...
 
int MYSQL_BIN_LOG::remove_logs_outside_range_from_index (const std::string &first, const std::string &last)
 
int MYSQL_BIN_LOG::purge_logs (const char *to_log, bool included, bool need_lock_index, bool need_update_threads, ulonglong *decrease_log_space, bool auto_purge)
 Remove all logs before the given log from disk and from the index file. More...
 
int MYSQL_BIN_LOG::purge_index_entry (THD *thd, ulonglong *decrease_log_space, bool need_lock_index)
 
int MYSQL_BIN_LOG::purge_logs_before_date (time_t purge_time, bool auto_purge)
 Remove all logs before the given file date from disk and from the index file. More...
 
void MYSQL_BIN_LOG::make_log_name (char *buf, const char *log_ident)
 Create a new log file name. More...
 
bool MYSQL_BIN_LOG::is_active (const char *log_file_name) const
 Check if we are writing/reading to the given log file. More...
 
void MYSQL_BIN_LOG::inc_prep_xids (THD *thd)
 Increment the prepared XID counter. More...
 
void MYSQL_BIN_LOG::dec_prep_xids (THD *thd)
 Decrement the prepared XID counter. More...
 
int MYSQL_BIN_LOG::new_file (Format_description_log_event *extra_description_event)
 
int MYSQL_BIN_LOG::new_file_without_locking (Format_description_log_event *extra_description_event)
 
int MYSQL_BIN_LOG::new_file_impl (bool need_lock, Format_description_log_event *extra_description_event)
 Start writing to a new log file or reopen the old file. More...
 
bool MYSQL_BIN_LOG::after_write_to_relay_log (Master_info *mi)
 Called after an event has been written to the relay log by the IO thread. More...
 
bool MYSQL_BIN_LOG::truncate_update_log_file (const char *log_name, my_off_t valid_pos, my_off_t binlog_size, bool update)
 Truncte log file and clear LOG_EVENT_BINLOG_IN_USE_F when update is set. More...
 
bool MYSQL_BIN_LOG::write_event (Log_event *ev, Master_info *mi)
 
bool MYSQL_BIN_LOG::write_buffer (const char *buf, uint len, Master_info *mi)
 
bool MYSQL_BIN_LOG::flush ()
 
bool MYSQL_BIN_LOG::flush_and_sync (const bool force=false)
 Flush binlog cache and synchronize to disk. More...
 
void MYSQL_BIN_LOG::start_union_events (THD *thd, query_id_t query_id_param)
 
void MYSQL_BIN_LOG::stop_union_events (THD *thd)
 
bool MYSQL_BIN_LOG::is_query_in_union (THD *thd, query_id_t query_id_param)
 
void MYSQL_BIN_LOG::update_thd_next_event_pos (THD *thd)
 
int MYSQL_BIN_LOG::flush_and_set_pending_rows_event (THD *thd, Rows_log_event *event, bool is_transactional)
 
bool MYSQL_BIN_LOG::write_event (Log_event *event_info)
 Write an event to the binary log cache. More...
 
int MYSQL_BIN_LOG::rotate (bool force_rotate, bool *check_purge)
 The method executes rotation when LOCK_log is already acquired by the caller. More...
 
void MYSQL_BIN_LOG::auto_purge_at_server_startup ()
 This member function is to be called at server startup. More...
 
void MYSQL_BIN_LOG::auto_purge ()
 This function runs automatic purge if the conditions to meet automatic purge are met. More...
 
int MYSQL_BIN_LOG::rotate_and_purge (THD *thd, bool force_rotate)
 Execute a FLUSH LOGS statement. More...
 
uint MYSQL_BIN_LOG::next_file_id ()
 
int MYSQL_BIN_LOG::get_gtid_executed (Tsid_map *tsid_map, Gtid_set *gtid_set)
 Deep copy global_tsid_map and gtid_executed. More...
 
void MYSQL_BIN_LOG::register_log_info (Log_info *log_info)
 Register Log_info so that log_in_use and adjust_linfo_offsets can operate on all logs. More...
 
void MYSQL_BIN_LOG::unregister_log_info (Log_info *log_info)
 Unregister Log_info when it is no longer needed. More...
 
int MYSQL_BIN_LOG::log_in_use (const char *log_name)
 Check if any threads use log name. More...
 
void MYSQL_BIN_LOG::adjust_linfo_offsets (my_off_t purge_offset)
 Adjust the position pointer in the binary log file for all running replicas. More...
 
bool MYSQL_BIN_LOG::do_write_cache (Binlog_cache_storage *cache, class Binlog_event_writer *writer)
 Write the contents of the given IO_CACHE to the binary log. More...
 
bool MYSQL_BIN_LOG::write_stmt_directly (THD *thd, const char *stmt, size_t stmt_len, enum enum_sql_command sql_command)
 Write a dml into statement cache and then flush it into binlog. More...
 
bool MYSQL_BIN_LOG::write_event_to_binlog (Log_event *ev)
 
bool MYSQL_BIN_LOG::write_event_to_binlog_and_sync (Log_event *ev)
 
bool MYSQL_BIN_LOG::write_cache (THD *thd, class binlog_cache_data *cache_data, class Binlog_event_writer *writer)
 Write the contents of the statement or transaction cache to the binary log. More...
 
void MYSQL_BIN_LOG::report_binlog_write_error ()
 
int MYSQL_BIN_LOG::wait_for_update ()
 Wait until we get a signal that the binary log has been updated. More...
 
int MYSQL_BIN_LOG::wait_for_update (const std::chrono::nanoseconds &timeout)
 Wait until we get a signal that the binary log has been updated. More...
 
void MYSQL_BIN_LOG::close (uint exiting, bool need_lock_log, bool need_lock_index)
 Close the log file. More...
 
void MYSQL_BIN_LOG::harvest_bytes_written (Relay_log_info *rli, bool need_log_space_lock)
 Adds bytes written in the current relay log into the variable handling the total number of bytes acquired by the replica. More...
 
void MYSQL_BIN_LOG::set_max_size (ulong max_size_arg)
 
bool MYSQL_BIN_LOG::read_binlog_in_use_flag (Binlog_file_reader &binlog_file_reader)
 Read binary log stream header and Format_desc event from binlog_file_reader. More...
 
int MYSQL_BIN_LOG::open_binlog (const char *opt_name)
 
bool MYSQL_BIN_LOG::truncate_relaylog_file (Master_info *mi, my_off_t valid_pos)
 Truncate the active relay log file in the specified position. More...
 
void MYSQL_BIN_LOG::close () override
 This is called on shutdown, after ha_panic. More...
 
int MYSQL_BIN_LOG::prepare (THD *thd, bool all) override
 Log a prepare record of the transaction to the storage engines. More...
 
enum_result MYSQL_BIN_LOG::commit (THD *thd, bool all) override
 Commit the transaction in the transaction coordinator. More...
 
std::pair< int, my_off_tMYSQL_BIN_LOG::flush_thread_caches (THD *thd)
 Flush caches for session. More...
 
void MYSQL_BIN_LOG::init_thd_variables (THD *thd, bool all, bool skip_commit)
 Set thread variables used while flushing a transaction. More...
 
THDMYSQL_BIN_LOG::fetch_and_process_flush_stage_queue (const bool check_and_skip_flush_logs=false)
 Fetch and empty BINLOG_FLUSH_STAGE and COMMIT_ORDER_FLUSH_STAGE flush queues and flush transactions to the disk, and unblock threads executing slave preserve commit order. More...
 
int MYSQL_BIN_LOG::process_flush_stage_queue (my_off_t *total_bytes_var, THD **out_queue_var)
 Execute the flush stage. More...
 
void MYSQL_BIN_LOG::process_commit_stage_queue (THD *thd, THD *queue)
 Commit a sequence of sessions. More...
 
void MYSQL_BIN_LOG::process_after_commit_stage_queue (THD *thd, THD *first)
 Process after commit for a sequence of sessions. More...
 
bool MYSQL_BIN_LOG::change_stage (THD *thd, Commit_stage_manager::StageID stage, THD *queue, mysql_mutex_t *leave_mutex, mysql_mutex_t *enter_mutex)
 Enter a stage of the ordered commit procedure. More...
 
int MYSQL_BIN_LOG::flush_cache_to_file (my_off_t *flush_end_pos)
 Flush the I/O cache to file. More...
 
std::pair< bool, bool > MYSQL_BIN_LOG::sync_binlog_file (bool force)
 Call fsync() to sync the file to disk. More...
 
int MYSQL_BIN_LOG::finish_commit (THD *thd)
 Helper function executed when leaving ordered_commit. More...
 
void MYSQL_BIN_LOG::handle_binlog_flush_or_sync_error (THD *thd, bool need_lock_log, const char *message)
 Helper function to handle flush or sync stage errors. More...
 
int MYSQL_BIN_LOG::ordered_commit (THD *thd, bool all, bool skip_commit=false)
 Flush and commit the transaction. More...
 
void MYSQL_BIN_LOG::report_missing_purged_gtids (const Gtid_set *slave_executed_gtid_set, std::string &errmsg)
 Function to report the missing GTIDs. More...
 
void MYSQL_BIN_LOG::report_missing_gtids (const Gtid_set *previous_gtid_set, const Gtid_set *slave_executed_gtid_set, std::string &errmsg)
 Function to report the missing GTIDs. More...
 
void MYSQL_BIN_LOG::signal_update ()
 Notifies waiting threads that binary log has been updated. More...
 
void MYSQL_BIN_LOG::update_binlog_end_pos (bool need_lock=true)
 
void MYSQL_BIN_LOG::update_binlog_end_pos (const char *file, my_off_t pos)
 
bool THD::is_binlog_cache_empty (bool is_transactional) const
 Return true if the statement/transaction cache is currently empty, false otherwise. More...
 
int THD::binlog_setup_trx_data ()
 
int THD::binlog_write_table_map (TABLE *table, bool is_transactional, bool binlog_rows_query)
 This function writes a table map to the binary log. More...
 
Rows_log_eventTHD::binlog_get_pending_rows_event (bool is_transactional) const
 This function retrieves a pending row event from a cache which is specified through the parameter is_transactional. More...
 
void THD::add_to_binlog_accessed_dbs (const char *db)
 
void THD::check_and_emit_warning_for_non_composable_engines (Table_ref *table_ref)
 Iterates over the table and call check_and_registered_engine and emits error for non-composable engines. More...
 
int THD::decide_logging_format (Table_ref *tables)
 Decide on logging format to use for the statement and issue errors or warnings as needed. More...
 
bool THD::is_ddl_gtid_compatible ()
 
bool THD::is_dml_gtid_compatible (bool some_transactional_table, bool some_non_transactional_table, bool non_transactional_tables_are_tmp)
 is_dml_gtid_compatible() and is_ddl_gtid_compatible() check if the statement that is about to be processed will safely get a GTID. More...
 
template<class RowsEventT >
Rows_log_eventTHD::binlog_prepare_pending_rows_event (TABLE *table, uint32 serv_id, size_t needed, bool is_transactional, const unsigned char *extra_row_info, uint32 source_part_id=INT_MAX)
 
int THD::binlog_write_row (TABLE *table, bool is_transactional, const uchar *new_data, const unsigned char *extra_row_info)
 
int THD::binlog_update_row (TABLE *table, bool is_transactional, const uchar *old_data, const uchar *new_data, const uchar *extra_row_info)
 
int THD::binlog_delete_row (TABLE *table, bool is_transactional, const uchar *old_data, const unsigned char *extra_row_info)
 
int THD::binlog_flush_pending_rows_event (bool stmt_end, bool is_transactional)
 
void THD::issue_unsafe_warnings ()
 Auxiliary method used by binlog_query() to raise warnings. More...
 
int THD::binlog_query (enum_binlog_query_type qtype, const char *query, size_t query_len, bool is_trans, bool direct, bool suppress_use, int errcode)
 Log the current query. More...
 

Variables

static ulonglong limit_unsafe_suppression_start_time = 0
 
static bool unsafe_warning_suppression_is_activated = false
 
static int limit_unsafe_warning_count = 0
 
static handlertonbinlog_hton
 
bool opt_binlog_order_commits = true
 
const char * log_bin_index = nullptr
 
const char * log_bin_basename = nullptr
 
ulong rpl_read_size
 
MYSQL_BIN_LOG mysql_bin_logsync_binlog_period
 
static const char * g_stage_name []
 Names for the stages. More...
 
struct st_mysql_storage_engine binlog_storage_engine
 

Detailed Description

Macro Definition Documentation

◆ LIMIT_UNSAFE_WARNING_ACTIVATION_THRESHOLD_COUNT

#define LIMIT_UNSAFE_WARNING_ACTIVATION_THRESHOLD_COUNT   50

◆ LIMIT_UNSAFE_WARNING_ACTIVATION_TIMEOUT

#define LIMIT_UNSAFE_WARNING_ACTIVATION_TIMEOUT   50

◆ MY_OFF_T_UNDEF

#define MY_OFF_T_UNDEF   (~(my_off_t)0UL)

Enumeration Type Documentation

◆ enum_read_gtids_from_binlog_status

Enumerator
GOT_GTIDS 
GOT_PREVIOUS_GTIDS 
NO_GTIDS 
ERROR 
TRUNCATED 

Function Documentation

◆ MYSQL_BIN_LOG()

MYSQL_BIN_LOG::MYSQL_BIN_LOG ( uint *  sync_period,
bool  relay_log = false 
)

◆ ~MYSQL_BIN_LOG()

MYSQL_BIN_LOG::~MYSQL_BIN_LOG ( )
override

◆ add_to_binlog_accessed_dbs()

void THD::add_to_binlog_accessed_dbs ( const char *  db_param)
Parameters
db_paramdb name c-string to be inserted into alphabetically sorted THD::binlog_accessed_db_names list.

Note, that space for both the data and the node struct are allocated in THD::main_mem_root. The list lasts for the top-level query time and is reset in THD::cleanup_after_query().

◆ adjust_linfo_offsets()

void MYSQL_BIN_LOG::adjust_linfo_offsets ( my_off_t  purge_offset)

Adjust the position pointer in the binary log file for all running replicas.

SYNOPSIS adjust_linfo_offsets() purge_offset Number of bytes removed from start of log index file NOTES

  • This is called when doing a PURGE when we delete lines from the index log file. This method expects the LOCK_index to be taken so there are no concurrent edits against linfo objects being iterated. REQUIREMENTS
  • Before calling this function, we have to ensure that no threads are using any binary log file before purge_offset. TODO
  • Inform the replica threads that they should sync the position in the binary log file with flush_relay_log_info. Now they sync is done for next read.

◆ after_write_to_relay_log()

bool MYSQL_BIN_LOG::after_write_to_relay_log ( Master_info mi)
private

Called after an event has been written to the relay log by the IO thread.

This flushes and possibly syncs the file (according to the sync options), rotates the file if it has grown over the limit, and finally calls signal_update().

Note
The caller must hold LOCK_log before invoking this function.
Parameters
miMaster_info for the IO thread.
Return values
falsesuccess
trueerror

◆ assign_automatic_gtids_to_flush_group()

bool MYSQL_BIN_LOG::assign_automatic_gtids_to_flush_group ( THD first_seen)

Assign automatic generated GTIDs for all commit group threads in the flush stage having gtid_next.type == AUTOMATIC_GTID.

Parameters
first_seenThe first thread seen entering the flush stage.
Returns
Returns false if succeeds, otherwise true is returned.

◆ auto_purge()

void MYSQL_BIN_LOG::auto_purge ( )

This function runs automatic purge if the conditions to meet automatic purge are met.

The method executes logs purging routine.

Such conditions are: log is open, instance is not locked for backup and automatic purge is enabled.

If all conditions are met, purge is done according to the configuration of the purge window.

◆ auto_purge_at_server_startup()

void MYSQL_BIN_LOG::auto_purge_at_server_startup ( )

This member function is to be called at server startup.

It checks if purge can be done and does if it can.

◆ binlog_close_connection()

static int binlog_close_connection ( handlerton hton,
THD thd 
)
static

◆ binlog_commit()

static int binlog_commit ( handlerton ,
THD ,
bool   
)
static

This function is called once after each statement.

See also
MYSQL_BIN_LOG::commit
handlerton::commit

◆ binlog_deinit()

static int binlog_deinit ( void *  )
static

◆ binlog_delete_row()

int THD::binlog_delete_row ( TABLE table,
bool  is_transactional,
const uchar old_data,
const unsigned char *  extra_row_info 
)

Save a reference to the original read and write set bitmaps. We will need this to restore the bitmaps at the end.

This will remove spurious fields required during execution but not needed for binlogging. This is done according to the: binlog-row-image option.

◆ binlog_dummy_recover()

static int binlog_dummy_recover ( handlerton ,
XA_recover_txn ,
uint  ,
MEM_ROOT  
)
static

◆ binlog_enabled()

bool binlog_enabled ( )

Check whether binlog_hton has valid slot and enabled.

◆ binlog_flush_pending_rows_event()

int THD::binlog_flush_pending_rows_event ( bool  stmt_end,
bool  is_transactional 
)

◆ binlog_get_pending_rows_event()

Rows_log_event * THD::binlog_get_pending_rows_event ( bool  is_transactional) const

This function retrieves a pending row event from a cache which is specified through the parameter is_transactional.

Respectively, when it is true, the pending event is returned from the transactional cache. Otherwise from the non-transactional cache.

Parameters
is_transactionaltrue indicates a transactional cache, otherwise false a non-transactional.
Returns
The row event if any.

◆ binlog_init()

static int binlog_init ( void *  p)
static

◆ binlog_prepare()

static int binlog_prepare ( handlerton hton,
THD thd,
bool  all 
)
static

◆ binlog_prepare_pending_rows_event()

template<class RowsEventT >
Rows_log_event * THD::binlog_prepare_pending_rows_event ( TABLE table,
uint32  serv_id,
size_t  needed,
bool  is_transactional,
const unsigned char *  extra_row_info,
uint32  source_part_id = INT_MAX 
)

◆ binlog_prepare_row_images()

void binlog_prepare_row_images ( const THD thd,
TABLE table 
)
static

Remove from read_set spurious columns. The write_set has been handled before in table->mark_columns_needed_for_update.

if there is a primary key in the table (ie, user declared PK or a non-null unique index) and we dont want to ship the entire image, and the handler involved supports this.

Just to be sure that tmp_set is currently not in use as the read_set already.

NOBLOB: Remove unnecessary BLOB fields from read_set (the ones that are not part of PK).

◆ binlog_query()

int THD::binlog_query ( THD::enum_binlog_query_type  qtype,
const char *  query_arg,
size_t  query_len,
bool  is_trans,
bool  direct,
bool  suppress_use,
int  errcode 
)

Log the current query.

The query will be logged in either row format or statement format depending on the value of current_stmt_binlog_format_row field and the value of the qtype parameter.

This function must be called:

  • After the all calls to ha_*_row() functions have been issued.
  • After any writes to system tables. Rationale: if system tables were written after a call to this function, and the master crashes after the call to this function and before writing the system tables, then the master and slave get out of sync.
  • Before tables are unlocked and closed.
See also
decide_logging_format
Return values
0Success
nonzeroIf there is a failure when writing the query (e.g., write failure), then the error code is returned.

◆ binlog_rollback()

static int binlog_rollback ( handlerton hton,
THD thd,
bool  all 
)
static

This function is called when a transaction or a statement is rolled back.

◆ binlog_savepoint_rollback()

static int binlog_savepoint_rollback ( handlerton hton,
THD thd,
void *  sv 
)
static

◆ binlog_savepoint_rollback_can_release_mdl()

static bool binlog_savepoint_rollback_can_release_mdl ( handlerton ,
THD thd 
)
static

Check whether binlog state allows to safely release MDL locks after rollback to savepoint.

Parameters
thdThe client thread that executes the transaction.
Returns
true - It is safe to release MDL locks. false - If it is not.

If we have not updated any non-transactional tables rollback to savepoint will simply truncate binlog cache starting from SAVEPOINT command. So it should be safe to release MDL acquired after SAVEPOINT command in this case.

◆ binlog_savepoint_set()

static int binlog_savepoint_set ( handlerton ,
THD thd,
void *  sv 
)
static
Note
How do we handle this (unlikely but legal) case:
  [transaction] + [update to non-trans table] + [rollback to savepoint] ?
The problem occurs when a savepoint is before the update to the non-transactional table. Then when there's a rollback to the savepoint, if we simply truncate the binlog cache, we lose the part of the binlog cache where the update is. If we want to not lose it, we need to write the SAVEPOINT command and the ROLLBACK TO SAVEPOINT command to the binlog cache. The latter is easy: it's just write at the end of the binlog cache, but the former should be inserted to the place where the user called SAVEPOINT. The solution is that when the user calls SAVEPOINT, we write it to the binlog cache (so no need to later insert it). As transactions are never intermixed in the binary log (i.e. they are serialized), we won't have conflicts with savepoint names when using mysqlbinlog or in the slave SQL thread. Then when ROLLBACK TO SAVEPOINT is called, if we updated some non-transactional table, we don't truncate the binlog cache but instead write ROLLBACK TO SAVEPOINT to it; otherwise we truncate the binlog cache (which will chop the SAVEPOINT command from the binlog cache, which is good as in that case there is no need to have it in the binlog).

◆ binlog_set_prepared_in_tc()

static int binlog_set_prepared_in_tc ( handlerton hton,
THD thd 
)
static

◆ binlog_setup_trx_data()

int THD::binlog_setup_trx_data ( )

◆ binlog_start_trans_and_stmt()

static int binlog_start_trans_and_stmt ( THD thd,
Log_event start_event 
)
static

Function to start a statement and optionally a transaction for the binary log.

This function does three things:

  • Starts a transaction if not in autocommit mode or if a BEGIN statement has been seen.
  • Start a statement transaction to allow us to truncate the cache.
  • Save the current binlog position so that we can roll back the statement by truncating the cache.

    We only update the saved position if the old one was undefined, the reason is that there are some cases (e.g., for CREATE-SELECT) where the position is saved twice (e.g., both in Query_result_create::prepare() and THD::binlog_write_table_map()), but we should use the first. This means that calls to this function can be used to start the statement before the first table map event, to include some extra events.

Note however that IMMEDIATE_LOGGING implies that the statement is written without BEGIN/COMMIT.

Parameters
thdThread variable
start_eventThe first event requested to be written into the binary log

◆ binlog_trans_log_savepos()

static void binlog_trans_log_savepos ( THD thd,
my_off_t pos 
)
static

◆ binlog_update_row()

int THD::binlog_update_row ( TABLE table,
bool  is_transactional,
const uchar old_data,
const uchar new_data,
const uchar extra_row_info 
)

Save a reference to the original read and write set bitmaps. We will need this to restore the bitmaps at the end.

This will remove spurious fields required during execution but not needed for binlogging. This is done according to the: binlog-row-image option.

◆ binlog_write_row()

int THD::binlog_write_row ( TABLE table,
bool  is_transactional,
const uchar new_data,
const unsigned char *  extra_row_info 
)

◆ binlog_write_table_map()

int THD::binlog_write_table_map ( TABLE table,
bool  is_transactional,
bool  binlog_rows_query 
)

This function writes a table map to the binary log.

Note that in order to keep the signature uniform with related methods, we use a redundant parameter to indicate whether a transactional table was changed or not. Sometimes it will write a Rows_query_log_event into binary log before the table map too.

Parameters
tablea pointer to the table.
is_transactionaltrue indicates a transactional table, otherwise false a non-transactional.
binlog_rows_querytrue indicates a Rows_query log event will be binlogged before table map, otherwise false indicates it will not be binlogged.
Returns
nonzero if an error pops up when writing the table map event or the Rows_query log event.

◆ calculate_auto_purge_lower_time_bound()

static time_t calculate_auto_purge_lower_time_bound ( )
static

This function abstracts the calculation of the binary log files retention lower bound.

It is just a function that makes it easier to handle the fact that there are two mutually exclusive variables that control the purge period and one of them is deprecated.

NOTE: This function and part of the purge validation functions should really move to a retention policy class that abstracts the retention policy altogether and its controls. Perhaps we can do that once a refactoring is done to also include retention based on storage space occupied. Then we can use the same retention abstraction for binary and relay logs and possibly extend the options to retain (binary) log files not only based on time, but also on space used.

Returns
time_t the time after which log files are considered expired.

◆ call_after_sync_hook()

static int call_after_sync_hook ( THD queue_head)
inlinestatic

Auxiliary function used in ordered_commit.

◆ change_stage()

bool MYSQL_BIN_LOG::change_stage ( THD thd,
Commit_stage_manager::StageID  stage,
THD queue,
mysql_mutex_t leave_mutex,
mysql_mutex_t enter_mutex 
)
private

Enter a stage of the ordered commit procedure.

Entering is stage is done by:

  • Atomically entering a queue of THD objects (which is just one for the first phase).
  • If the queue was empty, the thread is the leader for that stage and it should process the entire queue for that stage.
  • If the queue was not empty, the thread is a follower and can go waiting for the commit to finish.

The function will lock the stage mutex if the calling thread was designated leader for the phase.

Parameters
[in]thdSession structure
[in]stageThe stage to enter
[in]queueThread queue for the stage
[in]leave_mutexMutex that will be released when changing stage
[in]enter_mutexMutex that will be taken when changing stage
Return values
trueIn case this thread did not become leader, the function returns true after the leader has completed the commit on its behalf, so the thread should continue doing the thread-local processing after the commit (i.e. call finish_commit).
falseThe thread is the leader for the stage and should do the processing.

◆ check_and_emit_warning_for_non_composable_engines()

void THD::check_and_emit_warning_for_non_composable_engines ( Table_ref table_ref)

Iterates over the table and call check_and_registered_engine and emits error for non-composable engines.

Emit a deprecation warning when a transaction or statement writes to tables of multiple engines, if at least one of those engines is non-transactional or non-composable (see below for definition of composable).

Parameters
[in]table_refTables involved in the query

InnoDB and Blackhole are composable storage engines, and the others are not.

We use the term composable storage engine. This refers to engines that are not just transactional (when used alone), but even transactional when used together with other composable storage engines. While all of InnoDB, Blackhole, and NDB are transactional, only InnoDB and Blackhole are composable; NDB is non-composable due to the way it writes the binary log. All non-transactional engines are non-composable.

◆ check_auto_purge_conditions()

static bool check_auto_purge_conditions ( )
static

Checks if automatic purge conditions are met and therefore the purge is allowed to be done.

If not met returns true. Otherwise, false.

Returns
false if the check is successful. True otherwise.

◆ check_binlog_cache_size()

void check_binlog_cache_size ( THD thd)

Checks if the BINLOG_CACHE_SIZE's value is greater than MAX_BINLOG_CACHE_SIZE.

If this happens, the BINLOG_CACHE_SIZE is set to MAX_BINLOG_CACHE_SIZE.

◆ check_binlog_stmt_cache_size()

void check_binlog_stmt_cache_size ( THD thd)

Checks if the BINLOG_STMT_CACHE_SIZE's value is greater than MAX_BINLOG_STMT_CACHE_SIZE.

If this happens, the BINLOG_STMT_CACHE_SIZE is set to MAX_BINLOG_STMT_CACHE_SIZE.

◆ check_purge_conditions()

static std::pair< bool, int > check_purge_conditions ( const MYSQL_BIN_LOG log)
static

Checks whether purge conditions are met to be able to run purge for binary log files.

This function checks whether the binary log is open, if the instance is not locked for backup.

Parameters
logThe reference to the binary log.
Returns
std::pair<bool, int> the first element states whether there is a purge condition violation. The second element states what is the associated error code, if any.

◆ check_trx_rw_engines()

int check_trx_rw_engines ( THD thd,
Transaction_ctx::enum_trx_scope  trx_scope 
)

Check if the transaction has no rw flag set for any of the storage engines.

Parameters
thdThe client thread that executed the current statement.
trx_scopeThe transaction scope to look into.
Return values
thenumber of engines which have actual changes.

◆ check_write_error()

bool MYSQL_BIN_LOG::check_write_error ( const THD thd)

◆ cleanup()

void MYSQL_BIN_LOG::cleanup ( )

◆ close() [1/2]

void MYSQL_BIN_LOG::close ( )
overridevirtual

This is called on shutdown, after ha_panic.

Implements TC_LOG.

◆ close() [2/2]

void MYSQL_BIN_LOG::close ( uint  exiting,
bool  need_lock_log,
bool  need_lock_index 
)

Close the log file.

Parameters
exitingBitmask for one or more of the following bits:
  • LOG_CLOSE_INDEX : if we should close the index file
  • LOG_CLOSE_TO_BE_OPENED : if we intend to call open at once after close.
  • LOG_CLOSE_STOP_EVENT : write a 'stop' event to the log
need_lock_logIf true, this function acquires LOCK_log; otherwise the caller should already have acquired it.
need_lock_indexIf true, this function acquires LOCK_index; otherwise the caller should already have acquired it.
Note
One can do an open on the object at once after doing a close. The internal structures are not freed until cleanup() is called

TODO(WL#7546): Change the implementation to Stop_event after write() is moved into libbinlogevents

◆ commit()

TC_LOG::enum_result MYSQL_BIN_LOG::commit ( THD thd,
bool  all 
)
overridevirtual

Commit the transaction in the transaction coordinator.

This function will commit the sessions transaction in the binary log and in the storage engines (by calling ha_commit_low). If the transaction was successfully logged (or not successfully unlogged) but the commit in the engines did not succeed, there is a risk of inconsistency between the engines and the binary log.

For binary log group commit, the commit is separated into three parts:

  1. First part consists of filling the necessary caches and finalizing them (if they need to be finalized). After this, nothing is added to any of the caches.
  2. Second part execute an ordered flush and commit. This will be done using the group commit functionality in ordered_commit.
  3. Third part checks any errors resulting from the ordered commit and handles them appropriately.
Return values
RESULT_SUCCESSsuccess
RESULT_ABORTEDerror, transaction was neither logged nor committed
RESULT_INCONSISTENTerror, transaction was logged but not committed

Implements TC_LOG.

◆ compress()

bool binlog_cache_data::compress ( THD thd)
protected

Compress the current transaction "in-place", if possible.

This attempts to compress the transaction if it satisfies the necessary pre-conditions. Otherwise it does nothing.

Return values
trueError: the cache has been corrupted and the transaction must be aborted.
falseSuccess: the transaction was either compressed successfully, or compression was not attempted, or compression failed and left the uncompressed transaction intact.

◆ dec_prep_xids()

void MYSQL_BIN_LOG::dec_prep_xids ( THD thd)
private

Decrement the prepared XID counter.

Signal m_prep_xids_cond if the counter reaches zero.

◆ decide_logging_format()

int THD::decide_logging_format ( Table_ref tables)

Decide on logging format to use for the statement and issue errors or warnings as needed.

The decision depends on the following parameters:

  • The logging mode, i.e., the value of binlog_format. Can be statement, mixed, or row.
  • The type of statement. There are three types of statements: "normal" safe statements; unsafe statements; and row injections. An unsafe statement is one that, if logged in statement format, might produce different results when replayed on the slave (e.g., queries with a LIMIT clause). A row injection is either a BINLOG statement, or a row event executed by the slave's SQL thread.
  • The capabilities of tables modified by the statement. The capabilities vector* for a table is a set of flags associated with the table. Currently, it only includes two flags: row capability flag and statement capability flag.

    The row capability flag is set if and only if the engine can handle row-based logging. The statement capability flag is set if and only if the table can handle statement-based logging.

Decision table for logging format

The following table summarizes how the format and generated warning/error depends on the tables' capabilities, the statement type, and the current binlog_format.

Row capable N NNNNNNNNN YYYYYYYYY YYYYYYYYY Statement capable N YYYYYYYYY NNNNNNNNN YYYYYYYYY

Statement type * SSSUUUIII SSSUUUIII SSSUUUIII

binlog_format * SMRSMRSMR SMRSMRSMR SMRSMRSMR

Logged format - SS-S--— -RR-RR-RR SRRSRR-RR Warning/Error 1 –2732444 5–5–6– —7–6–

Legend

Row capable: N - Some table not row-capable, Y - All tables row-capable Stmt capable: N - Some table not stmt-capable, Y - All tables stmt-capable Statement type: (S)afe, (U)nsafe, or Row (I)njection binlog_format: (S)TATEMENT, (M)IXED, or (R)OW Logged format: (S)tatement or (R)ow Warning/Error: Warnings and error messages are as follows:

  1. Error: Cannot execute statement: binlogging impossible since both row-incapable engines and statement-incapable engines are involved.
  2. Error: Cannot execute statement: binlogging impossible since BINLOG_FORMAT = ROW and at least one table uses a storage engine limited to statement-logging.
  3. Error: Cannot execute statement: binlogging of unsafe statement is impossible when storage engine is limited to statement-logging and BINLOG_FORMAT = MIXED.
  4. Error: Cannot execute row injection: binlogging impossible since at least one table uses a storage engine limited to statement-logging.
  5. Error: Cannot execute statement: binlogging impossible since BINLOG_FORMAT = STATEMENT and at least one table uses a storage engine limited to row-logging.
  6. Error: Cannot execute row injection: binlogging impossible since BINLOG_FORMAT = STATEMENT.
  7. Warning: Unsafe statement binlogged in statement format since BINLOG_FORMAT = STATEMENT.

In addition, we can produce the following error (not depending on the variables of the decision diagram):

  1. Error: Cannot execute statement: binlogging impossible since more than one engine is involved and at least one engine is self-logging.
  2. Error: Do not allow users to modify a gtid_executed table explicitly by a XA transaction.

For each error case above, the statement is prevented from being logged, we report an error, and roll back the statement. For warnings, we set the thd->binlog_flags variable: the warning will be printed only if the statement is successfully logged.

See also
THD::binlog_query
Parameters
[in]tablesTables involved in the query
Return values
0No error; statement can be logged.
-1One of the error conditions above applies (1, 2, 4, 5, 6 or 9).

The number of tables used in the current statement, that should be replicated.

The number of tables written to in the current statement, that should not be replicated. A table should not be replicated when it is considered 'local' to a MySQL instance. Currently, these tables are:

  • mysql.slow_log
  • mysql.general_log
  • mysql.slave_relay_log_info
  • mysql.slave_master_info
  • mysql.slave_worker_info
  • performance_schema.*
  • TODO: information_schema.* In practice, from this list, only performance_schema.* tables are written to by user queries.

Indicate whether we already reported a warning on modifying gtid_executed table.

◆ do_unsafe_limit_checkout()

static void do_unsafe_limit_checkout ( char *  buf,
int  unsafe_type,
const char *  query 
)
static

Auxiliary function to check if the warning for limit unsafety should be thrown or suppressed.

Details of the implementation can be found in the comments inline.

Parameters
bufBuffer to hold the warning message text
unsafe_typeThe type of unsafety.
queryThe actual query statement.

TODO: Remove this function and implement a general service for all warnings that would prevent flooding the error log. => switch to log_throttle class?

◆ do_write_cache()

bool MYSQL_BIN_LOG::do_write_cache ( Binlog_cache_storage cache,
class Binlog_event_writer writer 
)
private

Write the contents of the given IO_CACHE to the binary log.

The cache will be reset as a READ_CACHE to be able to read the contents from it.

The data will be post-processed: see class Binlog_event_writer for details.

Parameters
cacheEvents will be read from this IO_CACHE.
writerEvents will be written to this Binlog_event_writer.
Return values
trueIO error.
falseSuccess.
See also
MYSQL_BIN_LOG::write_cache

◆ ending_single_stmt_trans()

bool ending_single_stmt_trans ( THD thd,
const bool  all 
)

This function checks if a single statement transaction is about to commit or not.

Parameters
thdThe client thread that executed the current statement.
allCommitting a transaction (i.e. true) or a statement (i.e. false).
Returns
true if committing a single statement transaction, otherwise false.

◆ ending_trans()

bool ending_trans ( THD thd,
const bool  all 
)

This function checks if a transaction, either a multi-statement or a single statement transaction is about to commit or not.

Parameters
thdThe client thread that executed the current statement.
allCommitting a transaction (i.e. true) or a statement (i.e. false).
Returns
true if committing a transaction, otherwise false.

◆ fetch_and_process_flush_stage_queue()

THD * MYSQL_BIN_LOG::fetch_and_process_flush_stage_queue ( const bool  check_and_skip_flush_logs = false)
private

Fetch and empty BINLOG_FLUSH_STAGE and COMMIT_ORDER_FLUSH_STAGE flush queues and flush transactions to the disk, and unblock threads executing slave preserve commit order.

Parameters
[in]check_and_skip_flush_logsif false then flush prepared records of transactions to the log of storage engine. if true then flush prepared records of transactions to the log of storage engine only if COMMIT_ORDER_FLUSH_STAGE queue is non-empty.
Returns
Pointer to the first session of the BINLOG_FLUSH_STAGE stage queue.

◆ finalize() [1/3]

int binlog_stmt_cache_data::finalize ( THD thd)

◆ finalize() [2/3]

int binlog_cache_data::finalize ( THD thd,
Log_event end_event 
)

This function finalizes the cache preparing for commit or rollback.

The function just writes all the necessary events to the cache but does not flush the data to the binary log file. That is the role of the binlog_cache_data::flush function.

See also
binlog_cache_data::flush
Parameters
thdThe thread whose transaction should be flushed
end_eventThe end event either commit/rollback
Returns
nonzero if an error pops up when flushing the cache.

◆ finalize() [3/3]

int binlog_cache_data::finalize ( THD thd,
Log_event end_event,
XID_STATE xs 
)

The method writes XA END query to XA-prepared transaction's cache and calls the "basic" finalize().

Returns
error code, 0 success

◆ find_first_log()

bool MYSQL_BIN_LOG::find_first_log ( std::string &  binlog_file_name,
std::string &  errmsg 
)

Find the oldest binary log referenced by the index file.

Parameters
[out]binlog_file_namethe file name of oldest log found
[out]errmsgthe error message outputted, which is left untouched if the function returns false
Returns
false on success, true on error.

◆ find_first_log_not_in_gtid_set()

bool MYSQL_BIN_LOG::find_first_log_not_in_gtid_set ( char *  binlog_file_name,
const Gtid_set gtid_set,
Gtid first_gtid,
std::string &  errmsg 
)

Find the oldest binary log that contains any GTID that is not in the given gtid set.

Parameters
[out]binlog_file_namethe file name of oldest binary log found
[in]gtid_setthe given gtid set
[out]first_gtidthe first GTID information from the binary log file returned at binlog_file_name
[out]errmsgthe error message outputted, which is left untouched if the function returns false
Returns
false on success, true on error.

◆ find_log_pos()

int MYSQL_BIN_LOG::find_log_pos ( Log_info linfo,
const char *  log_name,
bool  need_lock_index 
)

◆ find_next_log()

int MYSQL_BIN_LOG::find_next_log ( Log_info linfo,
bool  need_lock_index 
)

◆ find_next_relay_log()

int MYSQL_BIN_LOG::find_next_relay_log ( char  log_name[FN_REFLEN+1])

◆ find_uniq_filename()

static int find_uniq_filename ( char *  name,
uint32  new_index_number 
)
static

Find a unique filename for 'filename.

#'.

Set '#' to the highest existing log file extension plus one.

This function will return nonzero if: (i) the generated name exceeds FN_REFLEN; (ii) if the number of extensions is exhausted; or (iii) some other error happened while examining the filesystem.

Returns
nonzero if not possible to get unique filename.

◆ finish_commit()

int MYSQL_BIN_LOG::finish_commit ( THD thd)
private

Helper function executed when leaving ordered_commit.

This function contain the necessary code for fetching the error code, doing post-commit checks, and wrapping up the commit if necessary.

It is typically called when enter_stage indicates that the thread should bail out, and also when the ultimate leader thread finishes executing ordered_commit.

It is typically used in this manner:

if (enter_stage(thd, Thread_queue::BINLOG_FLUSH_STAGE, thd, &LOCK_log))
return finish_commit(thd);
mysql_mutex_t LOCK_log
Definition: binlog.h:115
int finish_commit(THD *thd)
Helper function executed when leaving ordered_commit.
Definition: binlog.cc:7712
Returns
Error code if the session commit failed, or zero on success.

◆ flush() [1/2]

bool MYSQL_BIN_LOG::flush ( )

◆ flush() [2/2]

int binlog_cache_data::flush ( THD thd,
my_off_t bytes_written,
bool *  wrote_xid,
bool  parallelization_barrier 
)

Flush caches to the binary log.

If the cache is finalized, the cache will be flushed to the binary log file. If the cache is not finalized, nothing will be done.

If flushing fails for any reason, an error will be reported and the cache will be reset. Flushing can fail in the following circumstances:

  • It was not possible to write the cache to the file. In this case, it does not make sense to keep the cache.
See also
binlog_cache_data::finalize

◆ flush_and_set_pending_rows_event()

int MYSQL_BIN_LOG::flush_and_set_pending_rows_event ( THD thd,
Rows_log_event event,
bool  is_transactional 
)

◆ flush_and_sync()

bool MYSQL_BIN_LOG::flush_and_sync ( const bool  force = false)

Flush binlog cache and synchronize to disk.

This function flushes events in binlog cache to binary log file, it will do synchronizing according to the setting of system variable 'sync_binlog'. If file is synchronized, synced will be set to 1, otherwise 0.

Parameters
[in]forceif true, ignores the 'sync_binlog' and synchronizes the file.
Return values
0Success
otherFailure

◆ flush_cache_to_file()

int MYSQL_BIN_LOG::flush_cache_to_file ( my_off_t end_pos_var)
private

Flush the I/O cache to file.

Flush the binary log to the binlog file if any byte where written and signal that the binary log file has been updated if the flush succeeds.

◆ flush_thread_caches()

std::pair< int, my_off_t > MYSQL_BIN_LOG::flush_thread_caches ( THD thd)
private

Flush caches for session.

Note
set_trans_pos is called with a pointer to the file name that the binary log currently use and a rotation will change the contents of the variable.

The position is used when calling the after_flush, after_commit, and after_rollback hooks, but these have been placed so that they occur before a rotation is executed.

It is the responsibility of any plugin that use this position to copy it if they need it after the hook has returned.

The current "global" transaction_counter is stepped and its new value is assigned to the transaction.

◆ fulltext_unsafe_set()

static bool fulltext_unsafe_set ( TABLE_SHARE s)
inlinestatic

◆ generate_name()

const char * MYSQL_BIN_LOG::generate_name ( const char *  log_name,
const char *  suffix,
char *  buff 
)

◆ generate_new_name()

int MYSQL_BIN_LOG::generate_new_name ( char *  new_name,
const char *  log_name,
uint32  new_index_number = 0 
)
private

◆ get_current_log()

int MYSQL_BIN_LOG::get_current_log ( Log_info linfo,
bool  need_lock_log = true 
)

◆ get_filename_list()

std::pair< std::list< std::string >, mysql::utils::Error > MYSQL_BIN_LOG::get_filename_list ( )
See also
Binlog_index_monitor::get_filename_list

◆ get_gtid_executed()

int MYSQL_BIN_LOG::get_gtid_executed ( Tsid_map tsid_map,
Gtid_set gtid_set 
)

Deep copy global_tsid_map and gtid_executed.

Both operations are done under LOCK_commit and global_tsid_lock protection.

Parameters
[out]tsid_mapThe Tsid_map to which global_tsid_map will be copied.
[out]gtid_setThe Gtid_set to which gtid_executed will be copied.
Returns
the operation status
Return values
0OK
!=0Error

◆ get_locked_tables_mode_name()

const char * get_locked_tables_mode_name ( enum_locked_tables_mode  locked_tables_mode)

Getter for the enum enum_locked_tables_mode.

Parameters
locked_tables_modeenum for types of locked tables mode
Returns
The string representation of that enum value

◆ get_log_index()

std::pair< int, std::list< std::string > > MYSQL_BIN_LOG::get_log_index ( bool  need_lock_index = true)
See also
Binlog_index_monitor::get_log_index

◆ get_xa_opt()

enum xa_option_words get_xa_opt ( THD thd)
inline

◆ gtid_end_transaction()

int MYSQL_BIN_LOG::gtid_end_transaction ( THD thd)

If the thread owns a GTID, this function generates an empty transaction and releases ownership of the GTID.

  • If the binary log is disabled for this thread, the GTID is inserted directly into the mysql.gtid_executed table and the GTID is included in @global.gtid_executed. (This only happens for DDL, since DML will save the GTID into table and release ownership inside ha_commit_trans.)
  • If the binary log is enabled for this thread, an empty transaction consisting of GTID, BEGIN, COMMIT is written to the binary log, the GTID is included in @global.gtid_executed, and the GTID is added to the mysql.gtid_executed table on the next binlog rotation.

This function must be called by any committing statement (COMMIT, implicitly committing statements, or Xid_log_event), after the statement has completed execution, regardless of whether the statement updated the database.

This logic ensures that an empty transaction is generated for the following cases:

  • Explicit empty transaction: SET GTID_NEXT = 'UUID:NUMBER'; BEGIN; COMMIT;
  • Transaction or DDL that gets completely filtered out in the slave thread.
Parameters
thdThe committing thread
Return values
0Success
nonzeroError

◆ handle_binlog_flush_or_sync_error()

void MYSQL_BIN_LOG::handle_binlog_flush_or_sync_error ( THD thd,
bool  need_lock_log,
const char *  message 
)
private

Helper function to handle flush or sync stage errors.

If binlog_error_action= ABORT_SERVER, server will be aborted after reporting the error to the client. If binlog_error_action= IGNORE_ERROR, binlog will be closed for the reset of the life time of the server. close() call is protected with LOCK_log to avoid any parallel operations on binary log.

Parameters
thdThread object that faced flush/sync error
need_lock_log> Indicates true if LOCk_log is needed before closing binlog (happens when we are handling sync error) > Indicates false if LOCK_log is already acquired by the thread (happens when we are handling flush error)
messageMessage stating the reason of the failure

◆ handle_deferred_cache_write_incident()

int binlog_cache_mngr::handle_deferred_cache_write_incident ( THD thd)

Handle previous failure writing cache, aka.

incident.

Failure writing to the cache in a statement which can't be rolled-back is handled by setting the 'incident' flag on the cache. In this way the error is deferred and handled later by calling this function which replaces the incomplete cache content with an Incident_log_event.

It's also possible for any session to mark the cache as having an incident and thus trigger this functionality. The main user of this is the NDB binlog which need to write incident(s) when the NDB connection state changes or when other fatal error occurs, @see BINLOG::write_incident_commit.

Parameters
thdThread variable
Return values
0Success.
non-zeroError occured writing to cache
Note
This function handles previous error while writing to the cache by attempting to write (something hopefully smaller) to the cache and thus it may also fail, but at least an attempt has been made to get something sensible into the cache. Returning error invokes binlog_error_action.

Construct the Incident_log_event before reset()

◆ handle_gtid_consistency_violation()

static bool handle_gtid_consistency_violation ( THD thd,
int  error_code,
int  log_error_code 
)
static

Given that a possible violation of gtid consistency has happened, checks if gtid-inconsistencies are forbidden by the current value of ENFORCE_GTID_CONSISTENCY and GTID_MODE.

If forbidden, generates error or warning accordingly.

Parameters
thdThe thread that has issued the GTID-violating statement.
error_codeThe error code to use, if error or warning is to be generated.
log_error_codeThe error code to use, if error message is to be logged.
Return values
falseError was generated.
trueNo error was generated (possibly a warning was generated).

◆ harvest_bytes_written()

void MYSQL_BIN_LOG::harvest_bytes_written ( Relay_log_info rli,
bool  need_log_space_lock 
)

Adds bytes written in the current relay log into the variable handling the total number of bytes acquired by the replica.

Resets the counter of bytes written. If requested by caller, acquires relay log space lock

Parameters
rliPointer to the applier metadata object
need_log_space_lockInformation on whether to acquire the lock protecting data responsible for keeping the relay log space at bay

◆ has_acl_table_read()

static bool has_acl_table_read ( THD thd,
const Table_ref tl_list 
)
static

Checks if we have reads from ACL tables in table list.

Parameters
thdCurrent thread
tl_listTable_ref used by current command.
Returns
true, if we statement is unsafe, otherwise false.

◆ has_incident()

bool binlog_cache_data::has_incident ( void  ) const
See also
handle_deferred_cache_write_incident

◆ has_nondeterministic_default()

static bool has_nondeterministic_default ( const TABLE table)
static

Checks if a table has a column with a non-deterministic DEFAULT expression.

◆ has_write_table_auto_increment_not_first_in_pk()

static bool has_write_table_auto_increment_not_first_in_pk ( Table_ref tables)
static

◆ has_write_table_with_auto_increment()

static bool has_write_table_with_auto_increment ( Table_ref tables)
static

◆ has_write_table_with_auto_increment_and_query_block()

static bool has_write_table_with_auto_increment_and_query_block ( Table_ref tables)
static

◆ has_write_table_with_nondeterministic_default()

static bool has_write_table_with_nondeterministic_default ( const Table_ref tables)
static

Checks if a Table_ref contains a table that has been opened for writing, and that has a column with a non-deterministic DEFAULT expression.

◆ inc_prep_xids()

void MYSQL_BIN_LOG::inc_prep_xids ( THD thd)
private

Increment the prepared XID counter.

◆ init_and_set_log_file_name()

bool MYSQL_BIN_LOG::init_and_set_log_file_name ( const char *  log_name,
const char *  new_name,
uint32  new_index_number 
)
private

◆ init_gtid_sets()

bool MYSQL_BIN_LOG::init_gtid_sets ( Gtid_set all_gtids,
Gtid_set lost_gtids,
bool  verify_checksum,
bool  need_lock,
mysql::binlog::event::Transaction_boundary_parser trx_parser,
Gtid_monitoring_info partial_trx,
bool  is_server_starting = false 
)

Reads the set of all GTIDs in the binary/relay log, and the set of all lost GTIDs in the binary log, and stores each set in respective argument.

Parameters
all_gtidsWill be filled with all GTIDs in this binary/relay log.
lost_gtidsWill be filled with all GTIDs in the Previous_gtids_log_event of the first binary log that has a Previous_gtids_log_event. This is requested to binary logs but not to relay logs.
verify_checksumIf true, checksums will be checked.
need_lockIf true, LOCK_log, LOCK_index, and global_tsid_lock->wrlock are acquired; otherwise they are asserted to be taken already.
[out]trx_parserThis will be used to return the actual relaylog transaction parser state because of the possibility of partial transactions.
[out]partial_trxIf a transaction was left incomplete on the relaylog, its GTID information should be returned to be used in the case of the rest of the transaction be added to the relaylog.
is_server_startingTrue if the server is starting.
Returns
false on success, true on error.

◆ init_pthread_objects()

void MYSQL_BIN_LOG::init_pthread_objects ( )

◆ init_thd_variables()

void MYSQL_BIN_LOG::init_thd_variables ( THD thd,
bool  all,
bool  skip_commit 
)
private

Set thread variables used while flushing a transaction.

Parameters
[in]thdthread whose variables need to be set
[in]allThis is true if this is a real transaction commit, and false otherwise.
[in]skip_commitThis is true if the call to ha_commit_low should be skipped (it is handled by the caller somehow) and false otherwise (the normal case).

◆ is_active()

bool MYSQL_BIN_LOG::is_active ( const char *  log_file_name) const

Check if we are writing/reading to the given log file.

◆ is_binlog_cache_empty()

bool THD::is_binlog_cache_empty ( bool  is_transactional) const

Return true if the statement/transaction cache is currently empty, false otherwise.

Parameters
is_transactionalif true, check the transaction cache. If false, check the statement cache.

◆ is_current_stmt_binlog_enabled_and_caches_empty()

bool MYSQL_BIN_LOG::is_current_stmt_binlog_enabled_and_caches_empty ( const THD thd) const

Checks whether binlog caches are disabled (binlog does not cache data) or empty in case binloggging is enabled in the current call to this function.

This function may be safely called in case binlogging is disabled.

Return values
truebinlog local caches are empty or disabled and binlogging is enabled
falsebinlog local caches are enabled and contain data or binlogging is disabled

◆ is_ddl_gtid_compatible()

bool THD::is_ddl_gtid_compatible ( )

◆ is_dml_gtid_compatible()

bool THD::is_dml_gtid_compatible ( bool  some_transactional_table,
bool  some_non_transactional_table,
bool  non_transactional_tables_are_tmp 
)

is_dml_gtid_compatible() and is_ddl_gtid_compatible() check if the statement that is about to be processed will safely get a GTID.

Currently, the following cases may lead to errors (e.g. duplicated GTIDs) and as such are forbidden:

  1. DML statements that mix non-transactional updates with transactional updates.
  2. Transactions that use non-transactional tables after having used transactional tables.
  3. CREATE...SELECT statement;
  4. CREATE TEMPORARY TABLE or DROP TEMPORARY TABLE within a transaction

The first two conditions have to be checked in decide_logging_format, because that's where we know if the table is transactional or not. These are implemented in is_dml_gtid_compatible().

The third and fourth conditions have to be checked in mysql_execute_command because (1) that prevents implicit commit from being executed if the statement fails; (2) DROP TEMPORARY TABLE does not invoke decide_logging_format. These are implemented in is_ddl_gtid_compatible().

In the cases where GTID violations generate errors, is_ddl_gtid_compatible() needs to be called before the implicit pre-commit, so that there is no implicit commit if the statement fails.

In the cases where GTID violations do not generate errors, is_ddl_gtid_compatible() needs to be called after the implicit pre-commit, because in these cases the function will increase the global counter automatic_gtid_violating_transaction_count or anonymous_gtid_violating_transaction_count. If there is an ongoing transaction, the implicit commit will commit the transaction, which will call update_gtids_impl, which should decrease the counters depending on whether the old was violating GTID-consistency or not. Thus, we should increase the counters only after the old transaction is committed.

Parameters
some_transactional_tabletrue if the statement updates some transactional table; false otherwise.
some_non_transactional_tabletrue if the statement updates some non-transactional table; false otherwise.
non_transactional_tables_are_tmptrue if all updated non-transactional tables are temporary.
Return values
trueif the statement is compatible;
falseif the statement is not compatible.

◆ is_empty_transaction_in_binlog_cache()

bool is_empty_transaction_in_binlog_cache ( const THD thd)

Check if at least one of transacaction and statement binlog caches contains an empty transaction, other one is empty or contains an empty transaction, which has two binlog events "BEGIN" and "COMMIT".

Parameters
thdThe client thread that executed the current statement.
Return values
trueAt least one of transacaction and statement binlog caches contains an empty transaction, other one is empty or contains an empty transaction.
falseOtherwise.

◆ is_loggable_xa_prepare()

bool is_loggable_xa_prepare ( THD thd)
inlinestatic

Predicate function yields true when XA transaction is being logged having a proper state ready for prepare or commit in one phase.

Parameters
thdTHD pointer of running transaction
Returns
true When the being prepared transaction should be binlogged, false otherwise.

◆ is_number()

static bool is_number ( const char *  str,
ulong *  res,
bool  allow_wildcards 
)
static

Check if a string is a valid number.

Parameters
strString to test
resStore value here
allow_wildcardsSet to 1 if we should ignore '' and '_'
Note
For the moment the allow_wildcards argument is not used Should be moved to some other file.
Return values
1String is a number
0String is not a number

◆ is_query_in_union()

bool MYSQL_BIN_LOG::is_query_in_union ( THD thd,
query_id_t  query_id_param 
)

◆ is_transaction_empty()

bool is_transaction_empty ( THD thd)

Check if the the transaction is empty.

Parameters
thdThe client thread that executed the current statement.
Return values
trueNo changes found in any storage engine
falseOtherwise.

◆ issue_unsafe_warnings()

void THD::issue_unsafe_warnings ( )

Auxiliary method used by binlog_query() to raise warnings.

The type of warning and the type of unsafeness is stored in THD::binlog_unsafe_warning_flags.

◆ log_in_use()

int MYSQL_BIN_LOG::log_in_use ( const char *  log_name)

Check if any threads use log name.

Note
This method expects the LOCK_index to be taken so there are no concurrent edits against linfo objects being iterated
Parameters
log_namename of a log which is checked for usage

◆ log_loaded_block()

int log_loaded_block ( IO_CACHE file)

Load data's io cache specific hook to be executed before a chunk of data is being read into the cache's buffer The function instantiates and writes into the binlog replication events along LOAD DATA processing.

Parameters
filepointer to io-cache
Return values
0success
1failure

◆ make_log_name()

void MYSQL_BIN_LOG::make_log_name ( char *  buf,
const char *  log_ident 
)

Create a new log file name.

Parameters
[out]bufBuffer allocated with at least FN_REFLEN bytes where new name is stored.
log_identIdentity of the binary/relay log.
Note
If file name will be longer then FN_REFLEN it will be truncated

◆ mysql_show_binlog_events()

bool mysql_show_binlog_events ( THD thd)

Execute a SHOW BINLOG EVENTS statement.

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

◆ new_file()

int MYSQL_BIN_LOG::new_file ( Format_description_log_event extra_description_event)

◆ new_file_impl()

int MYSQL_BIN_LOG::new_file_impl ( bool  need_lock_log,
Format_description_log_event extra_description_event 
)
private

Start writing to a new log file or reopen the old file.

Parameters
need_lock_logIf true, this function acquires LOCK_log; otherwise the caller should already have acquired it.
extra_description_eventThe master's FDE to be written by the I/O thread while creating a new relay log file. This should be NULL for binary log files.
Return values
0success
nonzero- error
Note
The new file name is stored last in the index file

◆ new_file_without_locking()

int MYSQL_BIN_LOG::new_file_without_locking ( Format_description_log_event extra_description_event)

◆ next_file_id()

uint MYSQL_BIN_LOG::next_file_id ( )

◆ open()

bool MYSQL_BIN_LOG::open ( PSI_file_key  log_file_key,
const char *  log_name,
const char *  new_name,
uint32  new_index_number 
)
private

Open the logfile and init IO_CACHE.

Parameters
log_file_keyThe file instrumentation key for this file
log_nameThe name of the log to open
new_nameThe new name for the logfile. NULL forces generate_new_name() to be called.
new_index_numberThe binary log file index number to start from after the RESET BINARY LOGS AND GTIDS command is called.
Returns
true if error, false otherwise.

◆ open_binlog() [1/2]

bool MYSQL_BIN_LOG::open_binlog ( const char *  log_name,
const char *  new_name,
ulong  max_size_arg,
bool  null_created_arg,
bool  need_lock_index,
bool  need_tsid_lock,
Format_description_log_event extra_description_event,
uint32  new_index_number = 0 
)

Create a new binary log.

Open a (new) binlog file.

Parameters
log_nameName of binlog
new_nameName of binlog, too. todo: what's the difference between new_name and log_name?
max_size_argThe size at which this binlog will be rotated.
null_created_argIf false, and a Format_description_log_event is written, then the Format_description_log_event will have the timestamp 0. Otherwise, it the timestamp will be the time when the event was written to the log.
need_lock_indexIf true, LOCK_index is acquired; otherwise LOCK_index must be taken by the caller.
need_tsid_lockIf true, the read lock on global_tsid_lock will be acquired. Otherwise, the caller must hold the read lock on global_tsid_lock.
extra_description_eventThe master's FDE to be written by the I/O thread while creating a new relay log file. This should be NULL for binary log files.
new_index_numberThe binary log file index number to start from after the RESET BINARY LOGS AND GTIDS command is called.
  • Open the log file and the index file. Register the new file name in it
  • When calling this when the file is in use, you must have a locks on LOCK_log and LOCK_index.
Return values
0ok
1error

Perhaps we might need the code below in MYSQL_BIN_LOG::cleanup for "real life" purposes as well?

◆ open_binlog() [2/2]

int MYSQL_BIN_LOG::open_binlog ( const char *  opt_name)

◆ open_index_file()

bool MYSQL_BIN_LOG::open_index_file ( const char *  index_file_name_arg,
const char *  log_name,
bool  need_lock_index 
)

◆ ordered_commit()

int MYSQL_BIN_LOG::ordered_commit ( THD thd,
bool  all,
bool  skip_commit = false 
)
private

Flush and commit the transaction.

This will execute an ordered flush and commit of all outstanding transactions and is the main function for the binary log group commit logic. The function performs the ordered commit in four stages.

Pre-condition: transactions should have called ha_prepare_low, using HA_IGNORE_DURABILITY, before entering here.

Stage#0 implements replica-preserve-commit-order for applier threads that write the binary log. i.e. it forces threads to enter the queue in the correct commit order.

The stage#1 flushes the caches to the binary log and under LOCK_log and marks all threads that were flushed as not pending.

The stage#2 syncs the binary log for all transactions in the group.

The stage#3 executes under LOCK_commit and commits all transactions in order.

There are three queues of THD objects: one for each stage. The Commit_order_manager maintains it own queue and its own order for the commit. So Stage#0 doesn't maintain separate StageID.

When a transaction enters a stage, it adds itself to a queue. If the queue was empty so that this becomes the first transaction in the queue, the thread is the leader of the queue. Otherwise it is a follower. The leader will do all work for all threads in the queue, and the followers will wait until the last stage is finished.

Stage 0 (SLAVE COMMIT ORDER):

  1. If replica-preserve-commit-order and is slave applier worker thread, then waits until its turn to commit i.e. till it is on the top of the queue.
  2. When it reaches top of the queue, it signals next worker in the commit order queue to awake.

Stage 1 (FLUSH):

  1. Sync the engines (ha_flush_logs), since they prepared using non-durable settings (HA_IGNORE_DURABILITY).
  2. Generate GTIDs for all transactions in the queue.
  3. Write the session caches for all transactions in the queue to the binary log.
  4. Increment the counter of prepared XIDs.

Stage 2 (SYNC):

  1. If it is time to sync, based on the sync_binlog option, sync the binlog.
  2. If sync_binlog==1, signal dump threads that they can read up to the position after the last transaction in the queue

Stage 3 (COMMIT): This is performed by each thread separately, if binlog_order_commits=0. Otherwise by the leader does it for all threads.

  1. Call the after_sync hook.
  2. update the max_committed counter in the dependency_tracker
  3. call ha_commit_low
  4. Call the after_commit hook
  5. Update gtids
  6. Decrement the counter of prepared transactions

If the binary log needs to be rotated, it is done after this. During rotation, it takes a lock that prevents new commit groups from executing the flush stage, and waits until the counter of prepared transactions becomes 0, before it creates the new file.

Parameters
[in]thdSession to commit transaction for
[in]allThis is true if this is a real transaction commit, and false otherwise.
[in]skip_commitThis is true if the call to ha_commit_low should be skipped and false otherwise (the normal case).

After commit stage

◆ prepare()

int MYSQL_BIN_LOG::prepare ( THD thd,
bool  all 
)
overridevirtual

Log a prepare record of the transaction to the storage engines.

Parameters
thdSession to log transaction record for.
alltrue if an explicit commit or an implicit commit for a statement, false if an internal commit of the statement.
Returns
Error code on failure, zero on success.

Implements TC_LOG.

◆ print_unsafe_warning_to_log()

static void print_unsafe_warning_to_log ( int  unsafe_type,
char *  buf,
const char *  query 
)
static

Auxiliary function to print warning in the error log.

◆ process_after_commit_stage_queue()

void MYSQL_BIN_LOG::process_after_commit_stage_queue ( THD thd,
THD first 
)
private

Process after commit for a sequence of sessions.

Parameters
thdThe "master" thread
firstFirst thread in the queue of threads to commit

◆ process_commit_stage_queue()

void MYSQL_BIN_LOG::process_commit_stage_queue ( THD thd,
THD first 
)
private

Commit a sequence of sessions.

This function commit an entire queue of sessions starting with the session in first. If there were an error in the flushing part of the ordered commit, the error code is passed in and all the threads are marked accordingly (but not committed).

It will also add the GTIDs of the transactions to gtid_executed.

See also
MYSQL_BIN_LOG::ordered_commit
Parameters
thdThe "master" thread
firstFirst thread in the queue of threads to commit

◆ process_flush_stage_queue()

int MYSQL_BIN_LOG::process_flush_stage_queue ( my_off_t total_bytes_var,
THD **  out_queue_var 
)
private

Execute the flush stage.

Parameters
[out]total_bytes_varPointer to variable that will be set to total number of bytes flushed, or NULL.
[out]out_queue_varPointer to the sessions queue in flush stage.
Returns
Error code on error, zero on success

◆ purge_error_message()

static bool purge_error_message ( THD thd,
int  res 
)
static

◆ purge_index_entry()

int MYSQL_BIN_LOG::purge_index_entry ( THD thd,
ulonglong decrease_log_space,
bool  need_lock_index 
)

◆ purge_log_get_error_code()

static uint purge_log_get_error_code ( int  res)
static

purge logs, master and slave sides both, related error code converter.

Called from purge_error_message(), MYSQL_BIN_LOG::reset_logs()

Parameters
resan error code as used by purging routines
Returns
the user level error code ER_*

◆ purge_logs()

int MYSQL_BIN_LOG::purge_logs ( const char *  to_log,
bool  included,
bool  need_lock_index,
bool  need_update_threads,
ulonglong decrease_log_space,
bool  auto_purge 
)

Remove all logs before the given log from disk and from the index file.

Parameters
to_logDelete all log file name before this file.
includedIf true, to_log is deleted too.
need_lock_indexSet to true, if the lock_index of the binary log shall be acquired, false if the called is already the owner of the lock_index.
need_update_threadsIf we want to update the log coordinates of all threads. False for relay logs, true otherwise.
decrease_log_spaceIf not null, decrement this variable of the amount of log space freed
auto_purgeTrue if this is an automatic purge.
Note
If any of the logs before the deleted one is in use, only purge logs up to this one.
Return values
0ok
LOG_INFO_EOFto_log not found
LOG_INFO_EMFILEtoo many files opened
LOG_INFO_FATALif any other than ENOENT error from mysql_file_stat() or mysql_file_delete()

◆ purge_logs_before_date()

int MYSQL_BIN_LOG::purge_logs_before_date ( time_t  purge_time,
bool  auto_purge 
)

Remove all logs before the given file date from disk and from the index file.

Parameters
purge_timeDelete all log files before given date.
auto_purgeTrue if this is an automatic purge.
Note
If any of the logs before the deleted one is in use, only purge logs up to this one.
Return values
0ok
LOG_INFO_PURGE_NO_ROTATEBinary file that can't be rotated LOG_INFO_FATAL if any other than ENOENT error from mysql_file_stat() or mysql_file_delete()

◆ purge_source_logs_before_date()

bool purge_source_logs_before_date ( THD thd,
time_t  purge_time 
)

Execute a PURGE BINARY LOGS BEFORE <date> command.

Parameters
thdPointer to THD object for the client thread executing the statement.
purge_timeDate before which logs should be purged.
Return values
falsesuccess
truefailure

◆ purge_source_logs_to_file()

bool purge_source_logs_to_file ( THD thd,
const char *  to_log 
)

Execute a PURGE BINARY LOGS TO <log> command.

Purges the binary log files up to the file name passed as a paramenter.

Parameters
thdPointer to THD object for the client thread executing the statement.
to_logName of the last log to purge.
Return values
falsesuccess
truefailure

◆ query_error_code()

int query_error_code ( const THD thd,
bool  not_killed 
)

◆ raw_get_current_log()

int MYSQL_BIN_LOG::raw_get_current_log ( Log_info linfo)

◆ read_binlog_in_use_flag()

bool MYSQL_BIN_LOG::read_binlog_in_use_flag ( Binlog_file_reader binlog_file_reader)
private

Read binary log stream header and Format_desc event from binlog_file_reader.

Check for LOG_EVENT_BINLOG_IN_USE_F flag.

Parameters
[in]binlog_file_readera Binlog_file_reader
Returns
true - LOG_EVENT_BINLOG_IN_USE_F is set false - LOG_EVENT_BINLOG_IN_USE_F is not set or an error occurred while reading log events

◆ read_gtids_and_update_trx_parser_from_relaylog()

static bool read_gtids_and_update_trx_parser_from_relaylog ( const char *  filename,
Gtid_set retrieved_gtids,
bool  verify_checksum,
mysql::binlog::event::Transaction_boundary_parser trx_parser,
Gtid_monitoring_info partial_trx 
)
static

Add the GTIDs from the given relaylog file and also update the IO thread transaction parser.

Parameters
filenameRelaylog file to read from.
retrieved_gtidsGtid_set to store the GTIDs found on the relaylog file.
verify_checksumSet to true to verify event checksums.
trx_parserThe transaction boundary parser to be used in order to only add a GTID to the gtid_set after ensuring the transaction is fully stored on the relay log.
partial_trxThe trx_monitoring_info of the last incomplete transaction found in the relay log.
Return values
falseThe file was successfully read and all GTIDs from Previous_gtids and Gtid_log_event from complete transactions were added to the retrieved_set.
trueThere was an error during the procedure.

◆ read_gtids_from_binlog()

static enum_read_gtids_from_binlog_status read_gtids_from_binlog ( const char *  filename,
Gtid_set all_gtids,
Gtid_set prev_gtids,
Gtid first_gtid,
Tsid_map tsid_map,
bool  verify_checksum,
bool  is_relay_log 
)
static

Reads GTIDs from the given binlog file.

Parameters
filenameFile to read from.
all_gtidsIf not NULL, then the GTIDs from the Previous_gtids_log_event and from all Gtid_log_events are stored in this object.
prev_gtidsIf not NULL, then the GTIDs from the Previous_gtids_log_events are stored in this object.
first_gtidIf not NULL, then the first GTID information from the file will be stored in this object.
tsid_mapThe tsid_map object to use in the rpl_sidno generation of the Gtid_log_event. If lock is needed in the tsid_map, the caller must hold it.
verify_checksumSet to true to verify event checksums.
is_relay_logSet to true, if filename is a Relay Log, false if it is a Binary Log.
Return values
GOT_GTIDSThe file was successfully read and it contains both Gtid_log_events and Previous_gtids_log_events. This is only possible if either all_gtids or first_gtid are not null.
GOT_PREVIOUS_GTIDSThe file was successfully read and it contains Previous_gtids_log_events but no Gtid_log_events. For binary logs, if no all_gtids and no first_gtid are specified, this function will be done right after reading the PREVIOUS_GTIDS regardless of the rest of the content of the binary log file.
NO_GTIDSThe file was successfully read and it does not contain GTID events.
ERROROut of memory, or IO error, or malformed event structure, or the file is malformed (e.g., contains Gtid_log_events but no Previous_gtids_log_event).
TRUNCATEDThe file was truncated before the end of the first Previous_gtids_log_event.

◆ reencrypt_logs()

bool MYSQL_BIN_LOG::reencrypt_logs ( )

Re-encrypt previous existent binary/relay logs as below.

Starting from the next to last entry on the index file, iterating down to the first one:

  • If the file is encrypted, re-encrypt it. Otherwise, skip it.
  • If failed to open the file, report an error.
Return values
FalseSuccess
TrueError

◆ register_binlog_handler()

static void register_binlog_handler ( THD thd,
bool  trx 
)
static

◆ register_log_info()

void MYSQL_BIN_LOG::register_log_info ( Log_info log_info)

Register Log_info so that log_in_use and adjust_linfo_offsets can operate on all logs.

Note that register_log_info, unregister_log_info, log_in_use, adjust_linfo_offsets are is used on global mysql_bin_log object.

Parameters
log_infopointer to Log_info which is registred

◆ remove_logs_outside_range_from_index()

int MYSQL_BIN_LOG::remove_logs_outside_range_from_index ( const std::string &  first,
const std::string &  last 
)
See also
Binlog_index_monitor::remove_logs_outside_range_from_index

◆ report_binlog_write_error()

void MYSQL_BIN_LOG::report_binlog_write_error ( )
private

◆ report_cache_write_error()

void MYSQL_BIN_LOG::report_cache_write_error ( THD thd,
bool  is_transactional 
)

◆ report_missing_gtids()

void MYSQL_BIN_LOG::report_missing_gtids ( const Gtid_set previous_gtid_set,
const Gtid_set slave_executed_gtid_set,
std::string &  errmsg 
)

Function to report the missing GTIDs.

This function logs the missing transactions on master to its error log as a warning. If the missing GTIDs are too long to print in a message, it suggests the steps to extract the missing transactions.

This function also informs slave about the GTID set sent by the slave, transactions missing on the master and few suggestions to recover from the error. This message shall be wrapped by ER_SOURCE_FATAL_ERROR_READING_BINLOG on slave and will be logged as an error.

This function will be called from find_first_log_not_in_gtid_set() function.

Parameters
previous_gtid_setPrevious GTID set found
slave_executed_gtid_setGTID set executed by slave
errmsgPointer to the error message

◆ report_missing_purged_gtids()

void MYSQL_BIN_LOG::report_missing_purged_gtids ( const Gtid_set slave_executed_gtid_set,
std::string &  errmsg 
)

Function to report the missing GTIDs.

This function logs the missing transactions on master to its error log as a warning. If the missing GTIDs are too long to print in a message, it suggests the steps to extract the missing transactions.

This function also informs slave about the GTID set sent by the slave, transactions missing on the master and few suggestions to recover from the error. This message shall be wrapped by ER_SOURCE_FATAL_ERROR_READING_BINLOG on slave and will be logged as an error.

This function will be called from mysql_binlog_send() function.

Parameters
slave_executed_gtid_setGTID set executed by slave
errmsgPointer to the error message

◆ reset_binlog_unsafe_suppression()

static void reset_binlog_unsafe_suppression ( )
static

Auxiliary function to reset the limit unsafety warning suppression.

◆ reset_logs()

bool MYSQL_BIN_LOG::reset_logs ( THD thd,
bool  delete_only = false 
)

Removes files, as part of a RESET BINARY LOGS AND GTIDS or RESET REPLICA statement, by deleting all logs referred to in the index file and the index file.

Then, it creates a new index file and a new log file.

The new index file will only contain the new log file.

Parameters
thdThread
delete_onlyIf true, do not create a new index file and a new log file.
Note
If not called from slave thread, write start event to new log
Return values
0ok
1error

◆ rollback()

int MYSQL_BIN_LOG::rollback ( THD thd,
bool  all 
)
overridevirtual

Write a rollback record of the transaction to the binary log.

For binary log group commit, the rollback is separated into three parts:

  1. First part consists of filling the necessary caches and finalizing them (if they need to be finalized). After a cache is finalized, nothing can be added to the cache.
  2. Second part execute an ordered flush and commit. This will be done using the group commit functionality in ordered_commit.

    Since we roll back the transaction early, we call ordered_commit with the skip_commit flag set. The ha_commit_low call inside ordered_commit will then not be called.

  3. Third part checks any errors resulting from the flush and handles them appropriately.
See also
MYSQL_BIN_LOG::ordered_commit
ha_commit_low
ha_rollback_low
Parameters
thdSession to commit
allThis is true if this is a real transaction rollback, and false otherwise.
Returns
Error code, or zero if there were no error.

Implements TC_LOG.

◆ rotate()

int MYSQL_BIN_LOG::rotate ( bool  force_rotate,
bool *  check_purge 
)

The method executes rotation when LOCK_log is already acquired by the caller.

Parameters
force_rotatecaller can request the log rotation
check_purgeis set to true if rotation took place
Note
If rotation fails, for instance the server was unable to create a new log file, we still try to write an incident event to the current log.
The caller must hold LOCK_log when invoking this function.
Return values
nonzero- error in rotating routine.

◆ rotate_and_purge()

int MYSQL_BIN_LOG::rotate_and_purge ( THD thd,
bool  force_rotate 
)

Execute a FLUSH LOGS statement.

The method is a shortcut of rotate() and purge(). LOCK_log is acquired prior to rotate and is released after it.

Parameters
thdCurrent session.
force_rotatecaller can request the log rotation
Return values
nonzero- error in rotating routine.

◆ set_incident()

void binlog_cache_data::set_incident ( std::string_view  incident_message = "Non-transactional changes were not written to the binlog.")
See also
handle_deferred_cache_write_incident

◆ set_max_size()

void MYSQL_BIN_LOG::set_max_size ( ulong  max_size_arg)

◆ show_binlog_events()

template<class BINLOG_FILE_READER >
bool show_binlog_events ( THD thd,
MYSQL_BIN_LOG binary_log 
)

◆ show_query_type()

static const char * show_query_type ( THD::enum_binlog_query_type  qtype)
static

◆ signal_update()

void MYSQL_BIN_LOG::signal_update ( )
protected

Notifies waiting threads that binary log has been updated.

◆ start_union_events()

void MYSQL_BIN_LOG::start_union_events ( THD thd,
query_id_t  query_id_param 
)

◆ stmt_cannot_safely_rollback()

bool stmt_cannot_safely_rollback ( const THD thd)

This function checks if current statement cannot be rollded back safely.

Parameters
thdThe client thread that executed the current statement.
Returns
true if cannot be safely rolled back, false otherwise.

◆ stmt_has_updated_trans_table()

bool stmt_has_updated_trans_table ( Ha_trx_info_list const &  ha_list)

This function checks if a transactional table was updated by the current statement.

Parameters
ha_listRegistered storage engine handler list.
Returns
true if a transactional table was updated, false otherwise.

◆ stop_union_events()

void MYSQL_BIN_LOG::stop_union_events ( THD thd)

◆ sync_binlog_file()

std::pair< bool, bool > MYSQL_BIN_LOG::sync_binlog_file ( bool  force)
private

Call fsync() to sync the file to disk.

◆ thd_get_cache_mngr()

static binlog_cache_mngr * thd_get_cache_mngr ( const THD thd)
static

◆ trans_cannot_safely_rollback()

bool trans_cannot_safely_rollback ( const THD thd)

This function checks if a transaction cannot be rolled back safely.

Parameters
thdThe client thread that executed the current statement.
Returns
true if cannot be safely rolled back, false otherwise.

◆ trans_has_updated_trans_table()

bool trans_has_updated_trans_table ( const THD thd)

This function checks if a transactional table was updated by the current transaction.

Parameters
thdThe client thread that executed the current statement.
Returns
true if a transactional table was updated, false otherwise.

◆ truncate()

void binlog_trx_cache_data::truncate ( THD thd,
bool  all 
)

This function truncates the transactional cache upon committing or rolling back either a transaction or a statement.

Parameters
thdThe thread whose transaction should be flushed
alltrue means truncate the transaction, otherwise the statement must be truncated.

◆ truncate_relaylog_file()

bool MYSQL_BIN_LOG::truncate_relaylog_file ( Master_info mi,
my_off_t  truncate_pos 
)

Truncate the active relay log file in the specified position.

Parameters
miMaster_info of the channel going to truncate the relay log file.
truncate_posThe position to truncate the active relay log file.
Returns
False on success and true on failure.

◆ truncate_update_log_file()

bool MYSQL_BIN_LOG::truncate_update_log_file ( const char *  log_name,
my_off_t  valid_pos,
my_off_t  binlog_size,
bool  update 
)

Truncte log file and clear LOG_EVENT_BINLOG_IN_USE_F when update is set.

Parameters
[in]log_namename of the log file to be trunacted
[in]valid_posposition at which to truncate the log file
[in]binlog_sizelength of the log file before truncated
[in]updateshould the LOG_EVENT_BINLOG_IN_USE_F flag be cleared true - set LOG_EVENT_BINLOG_IN_USE_F to 0 false - do not modify LOG_EVENT_BINLOG_IN_USE_F flag
Returns
true - sucess, false - failed

◆ unregister_log_info()

void MYSQL_BIN_LOG::unregister_log_info ( Log_info log_info)

Unregister Log_info when it is no longer needed.

Parameters
log_infopointer to Log_info which is registred

◆ update_binlog_end_pos() [1/2]

void MYSQL_BIN_LOG::update_binlog_end_pos ( bool  need_lock = true)

◆ update_binlog_end_pos() [2/2]

void MYSQL_BIN_LOG::update_binlog_end_pos ( const char *  file,
my_off_t  pos 
)
inline

◆ update_thd_next_event_pos()

void MYSQL_BIN_LOG::update_thd_next_event_pos ( THD thd)

◆ wait_for_update() [1/2]

int MYSQL_BIN_LOG::wait_for_update ( )

Wait until we get a signal that the binary log has been updated.

Return values
0success
Note
LOCK_binlog_end_pos must be owned before calling this function, may be temporarily released while the thread is waiting and is reacquired before returning from the function

◆ wait_for_update() [2/2]

int MYSQL_BIN_LOG::wait_for_update ( const std::chrono::nanoseconds &  timeout)

Wait until we get a signal that the binary log has been updated.

NOTES

Parameters
[in]timeouta pointer to a timespec; NULL means to wait w/o timeout.
Return values
0if got signalled on update
non-0if wait timeout elapsed
Note
LOCK_binlog_end_pos must be owned before calling this function, may be temporarily released while the thread is waiting and is reacquired before returning from the function

◆ write_buffer()

bool MYSQL_BIN_LOG::write_buffer ( const char *  buf,
uint  len,
Master_info mi 
)

◆ write_cache()

bool MYSQL_BIN_LOG::write_cache ( THD thd,
class binlog_cache_data cache_data,
class Binlog_event_writer writer 
)

Write the contents of the statement or transaction cache to the binary log.

Comparison with do_write_cache:

  • do_write_cache is a lower-level function that only performs the actual write.
  • write_cache is a higher-level function that calls do_write_cache and additionally performs some maintenance tasks, including:
    • report any errors that occurred
    • update gtid_state
    • update thd.binlog_next_event_pos
Parameters
thdThread variable
cache_dataEvents will be read from the IO_CACHE of this cache_data object.
writerEvents will be written to this Binlog_event_writer.
Return values
trueIO error.
falseSuccess.
Note
We only come here if there is something in the cache.
Whatever is in the cache is always a complete transaction.
'cache' needs to be reinitialized after this functions returns.

◆ write_event() [1/3]

bool MYSQL_BIN_LOG::write_event ( Log_event ev,
Master_info mi 
)

◆ write_event() [2/3]

int binlog_cache_data::write_event ( Log_event event)

◆ write_event() [3/3]

bool MYSQL_BIN_LOG::write_event ( Log_event event_info)

Write an event to the binary log cache.

◆ write_event_to_binlog()

bool MYSQL_BIN_LOG::write_event_to_binlog ( Log_event ev)
inline

◆ write_event_to_binlog_and_sync()

bool MYSQL_BIN_LOG::write_event_to_binlog_and_sync ( Log_event ev)

◆ write_incident_commit()

bool MYSQL_BIN_LOG::write_incident_commit ( THD thd,
std::string_view  incident_message 
)

Write an incident and call commit.

The incident is written by marking the transaction cache as having an incident and then commit it to binlog. During commit, the flush stage will write and Incident_log_event and preceding Gtid_log_event to give the incident a proper GTID. The incident will also cause the binlog to be rotated and check if some purge is applicable.

Parameters
[in]thdthe THD object of current thread.
[in]incident_messageA message describing the incident.
Returns
Returns false if succeeds, otherwise true is returned.

◆ write_stmt_directly()

bool MYSQL_BIN_LOG::write_stmt_directly ( THD thd,
const char *  stmt,
size_t  stmt_len,
enum enum_sql_command  sql_command 
)

Write a dml into statement cache and then flush it into binlog.

It writes Gtid_log_event and BEGIN, COMMIT automatically.

It is aimed to handle cases of "background" logging where a statement is logged indirectly, like "TRUNCATE TABLE a_memory_table". So don't use it on any normal statement.

Parameters
[in]thdthe THD object of current thread.
[in]stmtthe DML statement.
[in]stmt_lenthe length of the DML statement.
[in]sql_commandthe type of SQL command.
Returns
Returns false if succeeds, otherwise true is returned.

◆ write_transaction()

bool MYSQL_BIN_LOG::write_transaction ( THD thd,
binlog_cache_data cache_data,
Binlog_event_writer writer,
bool  parallelization_barrier 
)

Write the Gtid_log_event to the binary log (prior to writing the statement or transaction cache).

Parameters
thdThread that is committing.
cache_dataThe cache that is flushing.
writerThe event will be written to this Binlog_event_writer object.
parallelization_barrierThe transaction is a parallelization_barrier and the dependency tracker should mark subsequent transactions to depend on it.
Return values
falseSuccess.
trueError.

◆ write_xa_to_cache()

int MYSQL_BIN_LOG::write_xa_to_cache ( THD thd)

Logging XA commit/rollback of a prepared transaction.

It fills in the appropriate event in the statement cache whenever xid state is marked with is_binlogged() flag that indicates the prepared part of the transaction must've been logged.

About early returns from the function:

  • ONE_PHASE option to XA-COMMIT is handled to skip writing XA-commit event now.
  • check is for the read-only XA that is not to be logged.
Parameters
thdTHD handle
Returns
error code, 0 success

Variable Documentation

◆ binlog_hton

handlerton* binlog_hton
static

◆ binlog_storage_engine

struct st_mysql_storage_engine binlog_storage_engine
Initial value:
= {
#define MYSQL_HANDLERTON_INTERFACE_VERSION
Definition: plugin.h:704

◆ g_stage_name

const char* g_stage_name[]
static
Initial value:
= {"FLUSH", "SYNC", "COMMIT", "AFTER_COMMIT",
"COMMIT_ORDER_FLUSH"}

Names for the stages.

◆ limit_unsafe_suppression_start_time

ulonglong limit_unsafe_suppression_start_time = 0
static

◆ limit_unsafe_warning_count

int limit_unsafe_warning_count = 0
static

◆ log_bin_basename

const char* log_bin_basename = nullptr

◆ log_bin_index

const char* log_bin_index = nullptr

◆ opt_binlog_order_commits

bool opt_binlog_order_commits = true

◆ rpl_read_size

ulong rpl_read_size

◆ sync_binlog_period

MYSQL_BIN_LOG mysql_bin_log& sync_binlog_period

◆ unsafe_warning_suppression_is_activated

bool unsafe_warning_suppression_is_activated = false
static