MySQL 9.1.0
Source Code Documentation
|
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_mngr * | thd_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::Error > | MYSQL_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_t > | MYSQL_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... | |
THD * | MYSQL_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 () |
bool | THD::binlog_configure_trx_cache_size (ulong new_size) |
Configure size of binlog transaction cache. More... | |
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_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 . 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_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) |
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 handlerton * | binlog_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_log & | sync_binlog_period |
static const char * | g_stage_name [] |
Names for the stages. More... | |
struct st_mysql_storage_engine | binlog_storage_engine |
#define LIMIT_UNSAFE_WARNING_ACTIVATION_THRESHOLD_COUNT 50 |
#define LIMIT_UNSAFE_WARNING_ACTIVATION_TIMEOUT 50 |
#define MY_OFF_T_UNDEF (~(my_off_t)0UL) |
MYSQL_BIN_LOG::MYSQL_BIN_LOG | ( | uint * | sync_period, |
bool | relay_log = false |
||
) |
|
override |
void THD::add_to_binlog_accessed_dbs | ( | const char * | db_param | ) |
db_param | db 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()
.
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
|
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().
mi | Master_info for the IO thread. |
false | success |
true | error |
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.
first_seen | The first thread seen entering the flush stage. |
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.
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.
|
static |
|
static |
This function is called once after each statement.
bool THD::binlog_configure_trx_cache_size | ( | ulong | new_size | ) |
Configure size of binlog transaction cache.
Used to configure the size of an individual cache, normally to a value that differs from the default binlog_cache_size
which controls the size otherwise.
new_size | The new size of cache. Value exceeding max_binlog_cache_size will be clamped and warning logged. Value must be a multiple of IO_SIZE which is the block size for all binlog cache size related variables. |
|
static |
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.
|
static |
bool binlog_enabled | ( | ) |
Check whether binlog_hton has valid slot and enabled.
int THD::binlog_flush_pending_rows_event | ( | bool | stmt_end, |
bool | is_transactional | ||
) |
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.
is_transactional | true indicates a transactional cache, otherwise false a non-transactional. |
|
static |
|
static |
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 |
||
) |
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).
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:
0 | Success |
nonzero | If there is a failure when writing the query (e.g., write failure), then the error code is returned. |
|
static |
This function is called when a transaction or a statement is rolled back.
|
static |
|
static |
Check whether binlog state allows to safely release MDL locks after rollback to savepoint.
thd | The client thread that executes the transaction. |
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.
|
static |
[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).
|
static |
int THD::binlog_setup_trx_data | ( | ) |
Function to start a statement and optionally a transaction for the binary log.
This function does three things:
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.
thd | Thread variable |
start_event | The first event requested to be written into the binary log |
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.
int THD::binlog_write_row | ( | TABLE * | table, |
bool | is_transactional, | ||
const uchar * | new_data, | ||
const unsigned char * | extra_row_info | ||
) |
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.
table | a pointer to the table. |
is_transactional | true indicates a transactional table, otherwise false a non-transactional. |
binlog_rows_query | true indicates a Rows_query log event will be binlogged before table map, otherwise false indicates it will not be binlogged. |
|
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.
|
inlinestatic |
Auxiliary function used in ordered_commit.
|
private |
Enter a stage of the ordered commit procedure.
Entering is stage is done by:
The function will lock the stage mutex if the calling thread was designated leader for the phase.
[in] | thd | Session structure |
[in] | stage | The stage to enter |
[in] | queue | Thread queue for the stage |
[in] | leave_mutex | Mutex that will be released when changing stage |
[in] | enter_mutex | Mutex that will be taken when changing stage |
true | In 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). |
false | The thread is the leader for the stage and should do the processing. |
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).
[in] | table_ref | Tables 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.
|
static |
Checks if automatic purge conditions are met and therefore the purge is allowed to be done.
If not met returns true. Otherwise, false.
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.
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.
|
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.
log | The reference to the binary log. |
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.
thd | The client thread that executed the current statement. |
trx_scope | The transaction scope to look into. |
the | number of engines which have actual changes. |
bool MYSQL_BIN_LOG::check_write_error | ( | const THD * | thd | ) |
void MYSQL_BIN_LOG::cleanup | ( | ) |
|
overridevirtual |
This is called on shutdown, after ha_panic.
Implements TC_LOG.
void MYSQL_BIN_LOG::close | ( | uint | exiting, |
bool | need_lock_log, | ||
bool | need_lock_index | ||
) |
Close the log file.
exiting | Bitmask for one or more of the following bits:
|
need_lock_log | If true, this function acquires LOCK_log; otherwise the caller should already have acquired it. |
need_lock_index | If true, this function acquires LOCK_index; otherwise the caller should already have acquired it. |
TODO(WL#7546): Change the implementation to Stop_event after write() is moved into libbinlogevents
|
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:
RESULT_SUCCESS | success |
RESULT_ABORTED | error, transaction was neither logged nor committed |
RESULT_INCONSISTENT | error, transaction was logged but not committed |
Implements TC_LOG.
|
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.
true | Error: the cache has been corrupted and the transaction must be aborted. |
false | Success: the transaction was either compressed successfully, or compression was not attempted, or compression failed and left the uncompressed transaction intact. |
|
private |
Decrement the prepared XID counter.
Signal m_prep_xids_cond if the counter reaches zero.
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 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.
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–
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:
In addition, we can produce the following error (not depending on the variables of the decision diagram):
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.
[in] | tables | Tables involved in the query |
0 | No error; statement can be logged. |
-1 | One 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:
Indicate whether we already reported a warning on modifying gtid_executed table.
|
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.
buf | Buffer to hold the warning message text |
unsafe_type | The type of unsafety. |
query | The 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?
|
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.
cache | Events will be read from this IO_CACHE. |
writer | Events will be written to this Binlog_event_writer. |
true | IO error. |
false | Success. |
bool ending_single_stmt_trans | ( | THD * | thd, |
const bool | all | ||
) |
This function checks if a single statement transaction is about to commit or not.
thd | The client thread that executed the current statement. |
all | Committing a transaction (i.e. true) or a statement (i.e. false). |
true
if committing a single statement transaction, otherwise false
. 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.
thd | The client thread that executed the current statement. |
all | Committing a transaction (i.e. true) or a statement (i.e. false). |
true
if committing a transaction, otherwise 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.
[in] | check_and_skip_flush_logs | if 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. |
int binlog_stmt_cache_data::finalize | ( | THD * | thd | ) |
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.
thd | The thread whose transaction should be flushed |
end_event | The end event either commit/rollback |
The method writes XA END query to XA-prepared transaction's cache and calls the "basic" finalize().
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.
[out] | binlog_file_name | the file name of oldest log found |
[out] | errmsg | the error message outputted, which is left untouched if the function returns false |
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.
[out] | binlog_file_name | the file name of oldest binary log found |
[in] | gtid_set | the given gtid set |
[out] | first_gtid | the first GTID information from the binary log file returned at binlog_file_name |
[out] | errmsg | the error message outputted, which is left untouched if the function returns false |
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] | ) |
|
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.
|
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:
bool MYSQL_BIN_LOG::flush | ( | ) |
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:
int MYSQL_BIN_LOG::flush_and_set_pending_rows_event | ( | THD * | thd, |
Rows_log_event * | event, | ||
bool | is_transactional | ||
) |
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.
[in] | force | if true, ignores the 'sync_binlog' and synchronizes the file. |
0 | Success |
other | Failure |
|
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 caches for session.
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.
|
inlinestatic |
const char * MYSQL_BIN_LOG::generate_name | ( | const char * | log_name, |
const char * | suffix, | ||
char * | buff | ||
) |
|
private |
int MYSQL_BIN_LOG::get_current_log | ( | Log_info * | linfo, |
bool | need_lock_log = true |
||
) |
std::pair< std::list< std::string >, mysql::utils::Error > MYSQL_BIN_LOG::get_filename_list | ( | ) |
Deep copy global_tsid_map and gtid_executed.
Both operations are done under LOCK_commit and global_tsid_lock protection.
[out] | tsid_map | The Tsid_map to which global_tsid_map will be copied. |
[out] | gtid_set | The Gtid_set to which gtid_executed will be copied. |
0 | OK |
!=0 | Error |
const char * get_locked_tables_mode_name | ( | enum_locked_tables_mode | locked_tables_mode | ) |
Getter for the enum enum_locked_tables_mode.
locked_tables_mode | enum for types of locked tables mode |
std::pair< int, std::list< std::string > > MYSQL_BIN_LOG::get_log_index | ( | bool | need_lock_index = true | ) |
|
inline |
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.
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:
thd | The committing thread |
0 | Success |
nonzero | Error |
|
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.
thd | Thread 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) |
message | Message stating the reason of the failure |
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.
thd | Thread variable |
0 | Success. |
non-zero | Error occured writing to cache |
Construct the Incident_log_event before reset()
|
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.
thd | The thread that has issued the GTID-violating statement. |
error_code | The error code to use, if error or warning is to be generated. |
log_error_code | The error code to use, if error message is to be logged. |
false | Error was generated. |
true | No error was generated (possibly a warning was generated). |
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
rli | Pointer to the applier metadata object |
need_log_space_lock | Information on whether to acquire the lock protecting data responsible for keeping the relay log space at bay |
Checks if we have reads from ACL tables in table list.
thd | Current thread |
tl_list | Table_ref used by current command. |
bool binlog_cache_data::has_incident | ( | void | ) | const |
|
static |
Checks if a table has a column with a non-deterministic DEFAULT expression.
|
static |
|
static |
|
static |
|
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.
|
private |
Increment the prepared XID counter.
|
private |
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.
all_gtids | Will be filled with all GTIDs in this binary/relay log. | |
lost_gtids | Will 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_checksum | If true, checksums will be checked. | |
need_lock | If true, LOCK_log, LOCK_index, and global_tsid_lock->wrlock are acquired; otherwise they are asserted to be taken already. | |
[out] | trx_parser | This will be used to return the actual relaylog transaction parser state because of the possibility of partial transactions. |
[out] | partial_trx | If 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_starting | True if the server is starting. |
void MYSQL_BIN_LOG::init_pthread_objects | ( | ) |
|
private |
Set thread variables used while flushing a transaction.
[in] | thd | thread whose variables need to be set |
[in] | all | This is true if this is a real transaction commit, and false otherwise. |
[in] | skip_commit | This 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). |
bool MYSQL_BIN_LOG::is_active | ( | const char * | log_file_name | ) | const |
Check if we are writing/reading to the given log file.
bool THD::is_binlog_cache_empty | ( | bool | is_transactional | ) | const |
Return true if the statement/transaction cache is currently empty, false otherwise.
is_transactional | if true, check the transaction cache. If false, check the statement cache. |
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.
true | binlog local caches are empty or disabled and binlogging is enabled |
false | binlog local caches are enabled and contain data or binlogging is disabled |
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.
Currently, the following cases may lead to errors (e.g. duplicated GTIDs) and as such are forbidden:
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.
some_transactional_table | true if the statement updates some transactional table; false otherwise. |
some_non_transactional_table | true if the statement updates some non-transactional table; false otherwise. |
non_transactional_tables_are_tmp | true if all updated non-transactional tables are temporary. |
true | if the statement is compatible; |
false | if the statement is not compatible. |
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".
thd | The client thread that executed the current statement. |
true | At least one of transacaction and statement binlog caches contains an empty transaction, other one is empty or contains an empty transaction. |
false | Otherwise. |
|
inlinestatic |
Predicate function yields true when XA transaction is being logged having a proper state ready for prepare or commit in one phase.
thd | THD pointer of running transaction |
|
static |
Check if a string is a valid number.
str | String to test |
res | Store value here |
allow_wildcards | Set to 1 if we should ignore '' and '_' |
1 | String is a number |
0 | String is not a number |
bool MYSQL_BIN_LOG::is_query_in_union | ( | THD * | thd, |
query_id_t | query_id_param | ||
) |
bool is_transaction_empty | ( | THD * | thd | ) |
Check if the the transaction is empty.
thd | The client thread that executed the current statement. |
true | No changes found in any storage engine |
false | Otherwise. |
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.
int MYSQL_BIN_LOG::log_in_use | ( | const char * | log_name | ) |
Check if any threads use log name.
log_name | name of a log which is checked for usage |
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.
file | pointer to io-cache |
0 | success |
1 | failure |
void MYSQL_BIN_LOG::make_log_name | ( | char * | buf, |
const char * | log_ident | ||
) |
Create a new log file name.
[out] | buf | Buffer allocated with at least FN_REFLEN bytes where new name is stored. |
log_ident | Identity of the binary/relay log. |
bool mysql_show_binlog_events | ( | THD * | thd | ) |
Execute a SHOW BINLOG EVENTS statement.
thd | Pointer to THD object for the client thread executing the statement. |
false | success |
true | failure |
int MYSQL_BIN_LOG::new_file | ( | Format_description_log_event * | extra_description_event | ) |
|
private |
Start writing to a new log file or reopen the old file.
need_lock_log | If true, this function acquires LOCK_log; otherwise the caller should already have acquired it. |
extra_description_event | The 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. |
0 | success |
nonzero | - error |
int MYSQL_BIN_LOG::new_file_without_locking | ( | Format_description_log_event * | extra_description_event | ) |
uint MYSQL_BIN_LOG::next_file_id | ( | ) |
|
private |
Open the logfile and init IO_CACHE.
log_file_key | The file instrumentation key for this file |
log_name | The name of the log to open |
new_name | The new name for the logfile. NULL forces generate_new_name() to be called. |
new_index_number | The binary log file index number to start from after the RESET BINARY LOGS AND GTIDS command is called. |
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.
log_name | Name of binlog |
new_name | Name of binlog, too. todo: what's the difference between new_name and log_name? |
max_size_arg | The size at which this binlog will be rotated. |
null_created_arg | If 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_index | If true, LOCK_index is acquired; otherwise LOCK_index must be taken by the caller. |
need_tsid_lock | If 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_event | The 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_number | The binary log file index number to start from after the RESET BINARY LOGS AND GTIDS command is called. |
0 | ok |
1 | error |
Perhaps we might need the code below in MYSQL_BIN_LOG::cleanup for "real life" purposes as well?
int MYSQL_BIN_LOG::open_binlog | ( | const char * | opt_name | ) |
bool MYSQL_BIN_LOG::open_index_file | ( | const char * | index_file_name_arg, |
const char * | log_name, | ||
bool | need_lock_index | ||
) |
|
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):
Stage 1 (FLUSH):
Stage 2 (SYNC):
Stage 3 (COMMIT): This is performed by each thread separately, if binlog_order_commits=0. Otherwise by the leader does it for all threads.
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.
[in] | thd | Session to commit transaction for |
[in] | all | This is true if this is a real transaction commit, and false otherwise. |
[in] | skip_commit | This is true if the call to ha_commit_low should be skipped and false otherwise (the normal case). |
After commit stage
|
overridevirtual |
Log a prepare record of the transaction to the storage engines.
thd | Session to log transaction record for. |
all | true if an explicit commit or an implicit commit for a statement, false if an internal commit of the statement. |
Implements TC_LOG.
|
static |
Auxiliary function to print warning in the error log.
Process after commit for a sequence of sessions.
thd | The "master" thread |
first | First thread in the queue of threads to commit |
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.
thd | The "master" thread |
first | First thread in the queue of threads to commit |
|
private |
Execute the flush stage.
[out] | total_bytes_var | Pointer to variable that will be set to total number of bytes flushed, or NULL. |
[out] | out_queue_var | Pointer to the sessions queue in flush stage. |
|
static |
int MYSQL_BIN_LOG::purge_index_entry | ( | THD * | thd, |
ulonglong * | decrease_log_space, | ||
bool | need_lock_index | ||
) |
|
static |
purge logs, master and slave sides both, related error code converter.
Called from purge_error_message()
, MYSQL_BIN_LOG::reset_logs()
res | an error code as used by purging routines |
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.
to_log | Delete all log file name before this file. |
included | If true, to_log is deleted too. |
need_lock_index | Set 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_threads | If we want to update the log coordinates of all threads. False for relay logs, true otherwise. |
decrease_log_space | If not null, decrement this variable of the amount of log space freed |
auto_purge | True if this is an automatic purge. |
0 | ok |
LOG_INFO_EOF | to_log not found |
LOG_INFO_EMFILE | too many files opened |
LOG_INFO_FATAL | if any other than ENOENT error from mysql_file_stat() or mysql_file_delete() |
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.
purge_time | Delete all log files before given date. |
auto_purge | True if this is an automatic purge. |
0 | ok |
LOG_INFO_PURGE_NO_ROTATE | Binary file that can't be rotated LOG_INFO_FATAL if any other than ENOENT error from mysql_file_stat() or mysql_file_delete() |
bool purge_source_logs_before_date | ( | THD * | thd, |
time_t | purge_time | ||
) |
Execute a PURGE BINARY LOGS BEFORE <date> command.
thd | Pointer to THD object for the client thread executing the statement. |
purge_time | Date before which logs should be purged. |
false | success |
true | failure |
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.
thd | Pointer to THD object for the client thread executing the statement. |
to_log | Name of the last log to purge. |
false | success |
true | failure |
int query_error_code | ( | const THD * | thd, |
bool | not_killed | ||
) |
int MYSQL_BIN_LOG::raw_get_current_log | ( | Log_info * | linfo | ) |
|
private |
Read binary log stream header and Format_desc event from binlog_file_reader.
Check for LOG_EVENT_BINLOG_IN_USE_F flag.
[in] | binlog_file_reader | a Binlog_file_reader |
|
static |
Add the GTIDs from the given relaylog file and also update the IO thread transaction parser.
filename | Relaylog file to read from. |
retrieved_gtids | Gtid_set to store the GTIDs found on the relaylog file. |
verify_checksum | Set to true to verify event checksums. |
trx_parser | The 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_trx | The trx_monitoring_info of the last incomplete transaction found in the relay log. |
false | The file was successfully read and all GTIDs from Previous_gtids and Gtid_log_event from complete transactions were added to the retrieved_set. |
true | There was an error during the procedure. |
|
static |
Reads GTIDs from the given binlog file.
filename | File to read from. |
all_gtids | If not NULL, then the GTIDs from the Previous_gtids_log_event and from all Gtid_log_events are stored in this object. |
prev_gtids | If not NULL, then the GTIDs from the Previous_gtids_log_events are stored in this object. |
first_gtid | If not NULL, then the first GTID information from the file will be stored in this object. |
tsid_map | The 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_checksum | Set to true to verify event checksums. |
is_relay_log | Set to true, if filename is a Relay Log, false if it is a Binary Log. |
GOT_GTIDS | The 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_GTIDS | The 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_GTIDS | The file was successfully read and it does not contain GTID events. |
ERROR | Out 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). |
TRUNCATED | The file was truncated before the end of the first Previous_gtids_log_event. |
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:
False | Success |
True | Error |
|
static |
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.
log_info | pointer to Log_info which is registred |
int MYSQL_BIN_LOG::remove_logs_outside_range_from_index | ( | const std::string & | first, |
const std::string & | last | ||
) |
|
private |
void MYSQL_BIN_LOG::report_cache_write_error | ( | THD * | thd, |
bool | is_transactional | ||
) |
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.
previous_gtid_set | Previous GTID set found |
slave_executed_gtid_set | GTID set executed by slave |
errmsg | Pointer to the error message |
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.
slave_executed_gtid_set | GTID set executed by slave |
errmsg | Pointer to the error message |
|
static |
Auxiliary function to reset the limit unsafety warning suppression.
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.
thd | Thread |
delete_only | If true, do not create a new index file and a new log file. |
0 | ok |
1 | error |
|
overridevirtual |
Write a rollback record of the transaction to the binary log.
For binary log group commit, the rollback is separated into three parts:
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.
thd | Session to commit |
all | This is true if this is a real transaction rollback, and false otherwise. |
Implements TC_LOG.
int MYSQL_BIN_LOG::rotate | ( | bool | force_rotate, |
bool * | check_purge | ||
) |
The method executes rotation when LOCK_log is already acquired by the caller.
force_rotate | caller can request the log rotation |
check_purge | is set to true if rotation took place |
nonzero | - error in rotating routine. |
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.
thd | Current session. |
force_rotate | caller can request the log rotation |
nonzero | - error in rotating routine. |
void binlog_cache_data::set_incident | ( | std::string_view | incident_message = "Non-transactional changes were not written to the binlog." | ) |
void MYSQL_BIN_LOG::set_max_size | ( | ulong | max_size_arg | ) |
bool show_binlog_events | ( | THD * | thd, |
MYSQL_BIN_LOG * | binary_log | ||
) |
|
static |
|
protected |
Notifies waiting threads that binary log has been updated.
void MYSQL_BIN_LOG::start_union_events | ( | THD * | thd, |
query_id_t | query_id_param | ||
) |
bool stmt_cannot_safely_rollback | ( | const THD * | thd | ) |
This function checks if current statement cannot be rollded back safely.
thd | The client thread that executed the current statement. |
true
if cannot be safely rolled back, false
otherwise. 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.
ha_list | Registered storage engine handler list. |
true
if a transactional table was updated, false
otherwise. void MYSQL_BIN_LOG::stop_union_events | ( | THD * | thd | ) |
|
private |
Call fsync() to sync the file to disk.
|
static |
bool trans_cannot_safely_rollback | ( | const THD * | thd | ) |
This function checks if a transaction cannot be rolled back safely.
thd | The client thread that executed the current statement. |
true
if cannot be safely rolled back, false
otherwise. bool trans_has_updated_trans_table | ( | const THD * | thd | ) |
This function checks if a transactional table was updated by the current transaction.
thd | The client thread that executed the current statement. |
true
if a transactional table was updated, false
otherwise. 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.
thd | The thread whose transaction should be flushed |
all | true means truncate the transaction, otherwise the statement must be truncated. |
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.
mi | Master_info of the channel going to truncate the relay log file. |
truncate_pos | The position to truncate the active relay 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.
[in] | log_name | name of the log file to be trunacted |
[in] | valid_pos | position at which to truncate the log file |
[in] | binlog_size | length of the log file before truncated |
[in] | update | should 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 |
void MYSQL_BIN_LOG::unregister_log_info | ( | Log_info * | log_info | ) |
Unregister Log_info when it is no longer needed.
log_info | pointer to Log_info which is registred |
void MYSQL_BIN_LOG::update_binlog_end_pos | ( | bool | need_lock = true | ) |
|
inline |
void MYSQL_BIN_LOG::update_thd_next_event_pos | ( | THD * | thd | ) |
int MYSQL_BIN_LOG::wait_for_update | ( | ) |
Wait until we get a signal that the binary log has been updated.
0 | success |
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
[in] | timeout | a pointer to a timespec; NULL means to wait w/o timeout. |
0 | if got signalled on update |
non-0 | if wait timeout elapsed |
bool MYSQL_BIN_LOG::write_buffer | ( | const char * | buf, |
uint | len, | ||
Master_info * | mi | ||
) |
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:
thd | Thread variable |
cache_data | Events will be read from the IO_CACHE of this cache_data object. |
writer | Events will be written to this Binlog_event_writer. |
true | IO error. |
false | Success. |
bool MYSQL_BIN_LOG::write_event | ( | Log_event * | ev, |
Master_info * | mi | ||
) |
int binlog_cache_data::write_event | ( | Log_event * | event | ) |
bool MYSQL_BIN_LOG::write_event | ( | Log_event * | event_info | ) |
Write an event to the binary log cache.
|
inline |
bool MYSQL_BIN_LOG::write_event_to_binlog_and_sync | ( | Log_event * | ev | ) |
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.
[in] | thd | the THD object of current thread. |
[in] | incident_message | A message describing the incident. |
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.
[in] | thd | the THD object of current thread. |
[in] | stmt | the DML statement. |
[in] | stmt_len | the length of the DML statement. |
[in] | sql_command | the type of SQL command. |
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).
thd | Thread that is committing. |
cache_data | The cache that is flushing. |
writer | The event will be written to this Binlog_event_writer object. |
parallelization_barrier | The transaction is a parallelization_barrier and the dependency tracker should mark subsequent transactions to depend on it. |
false | Success. |
true | Error. |
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:
thd | THD handle |
|
static |
struct st_mysql_storage_engine binlog_storage_engine |
|
static |
Names for the stages.
|
static |
|
static |
const char* log_bin_basename = nullptr |
const char* log_bin_index = nullptr |
bool opt_binlog_order_commits = true |
ulong rpl_read_size |
MYSQL_BIN_LOG mysql_bin_log& sync_binlog_period |
|
static |