122                const char *assign_gtids_to_anonymous_transactions_value);
 
  305#ifdef HAVE_PSI_INTERFACE 
  387                          ulonglong immediate_ts_arg, 
bool skipped = 
false) {
 
  447                          const char *transient_err_message_arg,
 
  448                          ulong trans_retries_arg) {
 
  450        transient_errno_arg, transient_err_message_arg, trans_retries_arg);
 
  518      char const *param_privilege_checks_username,
 
  519      char const *param_privilege_checks_hostname);
 
  544      char const *param_privilege_checks_username,
 
  545      char const *param_privilege_checks_hostname);
 
  556      char const *param_privilege_checks_username,
 
  557      char const *param_privilege_checks_hostname);
 
  566  std::pair<const char *, const char *>
 
  591                                    char const *channel_name = 
nullptr,
 
  592                                    char const *user_name = 
nullptr,
 
  593                                    char const *host_name = 
nullptr) 
const;
 
  890                          const char *buff_coord) 
const;
 
 1002      down_cast<Until_position *>(
until_option)->notify_log_name_change();
 
 1022      down_cast<Until_position *>(
until_option)->notify_log_name_change();
 
 1037                              bool force = 
false);
 
 1054                        bool update_THD_status = 
true);
 
 1068                        bool update_THD_status = 
true);
 
 1082                        bool update_THD_status = 
true);
 
 1093                      TABLE **conv_table_var)
 const {
 
 1094    assert(tabledef_var && conv_table_var);
 
 1097      if (ptr->table == table_arg) {
 
 1098        *tabledef_var = &
static_cast<RPL_Table_ref *
>(ptr)->m_tabledef;
 
 1099        *conv_table_var = 
static_cast<RPL_Table_ref *
>(ptr)->m_conv_table;
 
 1100        DBUG_PRINT(
"debug", (
"Fetching table data for table %s.%s:" 
 1101                             " tabledef: %p, conv_table: %p",
 
 1103                             *tabledef_var, *conv_table_var));
 
 1294  std::atomic<longlong>
 
 1495    DBUG_PRINT(
"info", (
"is_in_stmt()=%d", ret));
 
 1509    DBUG_PRINT(
"info", (
"is_in_trx_or_stmt()=%d", ret));
 
 1540    DBUG_PRINT(
"info", (
"is_in_group()=%d", ret));
 
 1568  int rli_init_info(
bool skip_received_gtid_set_and_relaylog_recovery = 
false);
 
 1704                 uint param_id, 
const char *param_channel, 
bool is_rli_fake);
 
 1758                                       ulong current_version);
 
 2232  if (!rli) 
return false;
 
 2242                      ->has_ddl_committed) &&
 
 2439      std::vector<std::tuple<Access_bitmask, TABLE const *, Rows_log_event *>>
 
 2440          &extra_privileges) 
const;
 
 2454      std::initializer_list<std::string_view> extra_privileges) 
const;
 
 2468  bool has_access(std::initializer_list<Access_bitmask> extra_privileges) 
const;
 
 2521                                std::vector<std::string> &columns) 
const;
 
Contains the classes representing events occurring in the replication stream.
 
Utility class to allow RAII pattern with Security_context class.
Definition: rpl_rli.h:2389
 
Applier_security_context_guard(Applier_security_context_guard &&)=delete
 
void extract_columns_to_check(TABLE const *table, Rows_log_event *event, std::vector< std::string > &columns) const
Definition: rpl_rli.cc:3535
 
virtual ~Applier_security_context_guard()
Destructor that restores the backed up security context, if needed.
Definition: rpl_rli.cc:3452
 
Applier_security_context_guard(Relay_log_info const *rli, THD const *thd)
If needed, backs up the current thd security context and replaces it with a security context for PRIV...
Definition: rpl_rli.cc:3401
 
bool skip_priv_checks() const
Returns whether or not privilege checks may be skipped within the current context.
Definition: rpl_rli.cc:3460
 
Security_context m_applier_security_ctx
Applier security context based on PRIVILEGE_CHECK_USER user.
Definition: rpl_rli.h:2510
 
bool m_privilege_checks_none
Flag that states if privilege check should be skipped.
Definition: rpl_rli.h:2516
 
Applier_security_context_guard & operator=(const Applier_security_context_guard &)=delete
 
std::string get_username() const
Returns the username for the user for which the security context was initialized.
Definition: rpl_rli.cc:3523
 
THD const  * m_thd
The THD object for which the security context will be initialized.
Definition: rpl_rli.h:2508
 
Security_context * m_previous
Backed up security context.
Definition: rpl_rli.h:2514
 
Security_context * m_current
Currently in use security context.
Definition: rpl_rli.h:2512
 
bool m_logged_in_acl_user
Flag that states if there is a logged user.
Definition: rpl_rli.h:2518
 
bool has_access(std::vector< std::tuple< Access_bitmask, TABLE const *, Rows_log_event * > > &extra_privileges) const
Checks if the PRIVILEGE_CHECKS_USER user has access to the privilieges passed on by extra_privileges ...
Definition: rpl_rli.cc:3488
 
std::string get_hostname() const
Returns the hostname for the user for which the security context was initialized.
Definition: rpl_rli.cc:3529
 
Applier_security_context_guard & operator=(Applier_security_context_guard &&)=delete
 
Relay_log_info const  * m_target
The Relay_log_info object holding the info required to initialize the context.
Definition: rpl_rli.h:2504
 
Applier_security_context_guard(const Applier_security_context_guard &)=delete
 
This class is used to store the type and value for Assign_gtids_to_anonymous_transactions parameter o...
Definition: rpl_rli.h:93
 
bool set_info(enum_type assign_gtids_to_anonymous_transactions_type, const char *assign_gtids_to_anonymous_transactions_value)
Definition: rpl_rli.cc:3334
 
rpl_sidno m_sidno
Definition: rpl_rli.h:130
 
enum_type get_type() const
Definition: rpl_rli.cc:3330
 
enum_type m_type
This stores the type of Assign_gtids_to_anonymous_transactions info.
Definition: rpl_rli.h:126
 
Assign_gtids_to_anonymous_transactions_info()
The default constructor initializes parameters to their default value.
Definition: rpl_rli.h:110
 
std::string m_value
Stores the UUID in case the m_type is not OFF.
Definition: rpl_rli.h:128
 
enum_type
This accepted value of the type of the Assign_gtids_to_anonymous_transactions info OFF : Anonymous gt...
Definition: rpl_rli.h:106
 
rpl_sidno get_sidno() const
Definition: rpl_rli.h:114
 
std::string get_value() const
Definition: rpl_rli.cc:3325
 
This has the functionality of mysql_rwlock_t, with two differences:
Definition: rpl_gtid.h:324
 
void assert_some_lock() const
Assert that some thread holds either the read or the write lock.
Definition: rpl_gtid.h:571
 
On a replica and only on a replica, this class is responsible for committing the applied transactions...
Definition: rpl_replica_commit_order_manager.h:197
 
Definition: rpl_utility.h:524
 
void rewind()
Definition: rpl_utility.cc:1301
 
This is a subclass if Gtid_event and Log_event.
Definition: log_event.h:3939
 
rpl_gno get_gno() const override
Return the GNO for this GTID.
Definition: log_event.h:4108
 
rpl_sidno get_sidno(bool need_lock)
Return the SIDNO relative to the global tsid_map for this GTID.
Definition: log_event.cc:13534
 
enum_gtid_type get_type() const
Return the gtid type for this Gtid_log_event: this can be either ANONYMOUS_GTID, AUTOMATIC_GTID,...
Definition: log_event.h:4072
 
Stores information to monitor a transaction during the different replication stages.
Definition: rpl_gtid.h:1412
 
void clear()
Clear all monitoring information.
Definition: rpl_gtid_misc.cc:496
 
void finish()
Sets the final information, copy processing info to last_processed and clears processing info.
Definition: rpl_gtid_misc.cc:563
 
void start(Gtid gtid_arg, ulonglong original_ts_arg, ulonglong immediate_ts_arg, bool skipped_arg=false)
Sets the initial monitoring information.
Definition: rpl_gtid_misc.cc:523
 
bool is_processing_trx_set()
Returns true if the processing_trx is set, false otherwise.
Definition: rpl_gtid_misc.cc:593
 
void store_transient_error(uint transient_errno_arg, const char *transient_err_message_arg, ulong trans_retries_arg)
Stores the information about the last transient error in the current transaction, namely: the error n...
Definition: rpl_gtid_misc.cc:611
 
void clear_processing_trx()
Clear only the processing_trx monitoring info.
Definition: rpl_gtid_misc.cc:503
 
Represents a set of GTIDs.
Definition: rpl_gtid.h:1556
 
enum_return_status ensure_sidno(rpl_sidno sidno)
Allocates space for all sidnos up to the given sidno in the array of intervals.
Definition: rpl_gtid_set.cc:145
 
void _add_gtid(rpl_sidno sidno, rpl_gno gno)
Adds the given GTID to this Gtid_set.
Definition: rpl_gtid.h:1626
 
Tsid_map * get_tsid_map() const
Return the Tsid_map associated with this Gtid_set.
Definition: rpl_gtid.h:2021
 
This is the abstract base class for binary log events.
Definition: log_event.h:538
 
virtual mysql::binlog::event::Log_event_type get_type_code() const
Definition: log_event.h:797
 
Utility class to allow RAII pattern with MDL_request and MDL_context classes.
Definition: rpl_rli.h:2285
 
bool lock(MDL_key::enum_mdl_namespace namespace_arg, enum_mdl_type mdl_type_arg, bool blocking=false)
Uses the target THD object MDL context to acquire the lock identified by namespace_arg with MDL type ...
Definition: rpl_rli.cc:3373
 
bool is_locked()
Returns whether or not the lock as been acquired within this object life-cycle.
Definition: rpl_rli.cc:3399
 
THD * m_target
The THD object holding the MDL context used for acquiring/releasing.
Definition: rpl_rli.h:2350
 
MDL_lock_guard(THD *target)
Constructor that initializes the object and the target THD object but doesn't try to acquire any lock...
Definition: rpl_rli.cc:3363
 
MDL_request m_request
The MDL request holding the MDL ticket issued upon acquisition.
Definition: rpl_rli.h:2352
 
virtual ~MDL_lock_guard()
Destructor that unlocks all acquired locks.
Definition: rpl_rli.cc:3392
 
A pending metadata lock request.
Definition: mdl.h:802
 
Definition: rpl_mta_submode.h:55
 
bool empty() const
Definition: prealloced_array.h:226
 
size_t size() const
Definition: prealloced_array.h:227
 
A Query event is written to the binary log whenever the database is modified on the master,...
Definition: log_event.h:1285
 
RAII class to control the slave applier execution context binding with a being handled event.
Definition: rpl_rli.h:2267
 
Relay_log_info * m_rli
Definition: rpl_rli.h:2268
 
void set_current_event(Log_event *ev)
Definition: rpl_rli.h:2275
 
~RLI_current_event_raii()
Definition: rpl_rli.h:2276
 
RLI_current_event_raii(Relay_log_info *rli_arg, Log_event *ev)
Definition: rpl_rli.h:2271
 
Definition: rpl_rli.h:203
 
bool curr_group_seen_begin
Definition: rpl_rli.h:1215
 
uint checkpoint_group
Definition: rpl_rli.h:1245
 
static constexpr int RLI_FLUSH_IGNORE_SYNC_OPT
Ignore server sync options and flush.
Definition: rpl_rli.h:1574
 
enum_return_check check_if_info_was_cleared(const enum_return_check &previous_result) const
Checks if the underlying Rpl_info handler holds information for the fields to be kept between slave r...
Definition: rpl_rli.cc:1992
 
Tsid_map * get_tsid_map()
Definition: rpl_rli.h:840
 
enum_require_table_primary_key m_require_table_primary_key_check
Identifies what is the slave policy on primary keys in tables.
Definition: rpl_rli.h:826
 
Prealloced_array< Slave_job_item, 8 > curr_group_da
Definition: rpl_rli.h:1212
 
enum_priv_checks_status check_applier_acl_user(char const *param_privilege_checks_username, char const *param_privilege_checks_hostname)
Checks the existence of user provided as part of the PRIVILEGE_CHECKS_USER option.
Definition: rpl_rli.cc:3088
 
Format_description_log_event * get_rli_description_event() const
Return the current Format_description_log_event.
Definition: rpl_rli.h:1749
 
ulonglong group_source_log_start_pos
Definition: rpl_rli.h:728
 
ulonglong mts_events_assigned
Definition: rpl_rli.h:1283
 
static const int APPLIER_METADATA_LINES_WITH_ID
Definition: rpl_rli.h:1883
 
char cached_charset[6]
Definition: rpl_rli.h:961
 
std::atomic< int32 > atomic_channel_open_temp_tables
Definition: rpl_rli.h:353
 
TABLE * save_temporary_tables
Definition: rpl_rli.h:347
 
std::tuple< ulonglong, ulonglong > get_group_source_log_start_end_pos() const
Get event group positions in source binary log on a replica which is processed by a worker in MTA or ...
Definition: rpl_rli.cc:2942
 
int stmt_done(my_off_t event_log_pos)
Helper function to do after statement completion.
Definition: rpl_rli.cc:1238
 
ulong wq_size_waits_cnt
Definition: rpl_rli.h:1287
 
mysql_cond_t log_space_cond
Definition: rpl_rli.h:938
 
ulonglong get_group_master_log_pos_info() const
Definition: rpl_rli.h:1619
 
bool reset_group_relay_log_pos(const char **errmsg)
Reset group_relay_log_name and group_relay_log_pos to the start of the first relay log file.
Definition: rpl_rli.cc:521
 
void set_event_start_pos(my_off_t pos)
Definition: rpl_rli.h:1654
 
bool is_in_group() const
A group is defined as the entire range of events that constitute a transaction or auto-committed stat...
Definition: rpl_rli.h:1538
 
bool cached_charset_compare(char *charset) const
Definition: rpl_rli.cc:1228
 
int count_relay_log_space()
Definition: rpl_rli.cc:500
 
int flush_current_log()
Definition: rpl_rli.cc:1891
 
ulonglong max_updated_index
Definition: rpl_rli.h:1241
 
Rows_query_log_event * rows_query_ev
Definition: rpl_rli.h:1090
 
bool is_long_find_row_note_printed()
Definition: rpl_rli.h:1735
 
void notify_relay_log_change()
Invalidates cached until_log_name and event_relay_log_name comparison result.
Definition: rpl_rli.h:1000
 
bool set_info_search_keys(Rpl_info_handler *to) override
To search in the slave repositories, each slave info object (mi, rli or worker) should use a primary ...
Definition: rpl_rli.cc:2328
 
bool is_in_trx_or_stmt() const
Auxiliary function used by is_in_group.
Definition: rpl_rli.h:1507
 
std::atomic< ulong > pending_jobs
Definition: rpl_rli.h:1191
 
void set_filter(Rpl_filter *channel_filter)
Set replication filter for the channel.
Definition: rpl_rli.h:1820
 
time_t mta_coordinator_has_waited_stat
Last moment in time the MTA printed a coordinator waited stats.
Definition: rpl_rli.h:1314
 
bool curr_group_seen_gtid
Definition: rpl_rli.h:1214
 
Gtid_set * gtid_set
Definition: rpl_rli.h:753
 
ulong mts_wq_no_underrun_cnt
Definition: rpl_rli.h:1293
 
const char * get_group_master_log_name() const
Definition: rpl_rli.h:1609
 
MY_BITMAP recovery_groups
Definition: rpl_rli.h:1246
 
table_mapping m_table_map
Definition: rpl_rli.h:1088
 
ulong mts_slave_worker_queue_len_max
Definition: rpl_rli.h:1195
 
mysql_cond_t logical_clock_cond
Definition: rpl_rli.h:318
 
bool is_in_stmt() const
Auxiliary function used by is_in_group.
Definition: rpl_rli.h:1493
 
ulonglong mts_groups_assigned
Definition: rpl_rli.h:1284
 
bool reinit_sql_thread_io_cache(const char *log, bool need_data_lock)
 
void close_temporary_tables()
Definition: rpl_rli.cc:1010
 
ulong retried_trans
Definition: rpl_rli.h:970
 
struct timespec last_clock
Identifies the last time a checkpoint routine has been executed.
Definition: rpl_rli.h:993
 
void detach_engine_ha_data(THD *thd)
Detaches the engine ha_data from THD.
Definition: rpl_rli.cc:2794
 
mysql_mutex_t pending_jobs_lock
Definition: rpl_rli.h:1192
 
bool inited_hash_workers
Definition: rpl_rli.h:1162
 
void set_event_relay_log_pos(ulonglong log_pos)
Definition: rpl_rli.h:1657
 
static const int APPLIER_METADATA_LINES_WITH_REQUIRE_ROW_FORMAT
Definition: rpl_rli.h:1917
 
ulonglong group_relay_log_pos
Definition: rpl_rli.h:700
 
uchar slave_version_split[3]
Definition: rpl_rli.h:1759
 
std::atomic< uint32 > slave_skip_counter
Definition: rpl_rli.h:934
 
ulong mts_coordinator_basic_nap
Definition: rpl_rli.h:1234
 
bool group_source_log_seen_start_pos
Event group beginning event has been seen.
Definition: rpl_rli.h:724
 
size_t slave_patternload_file_size
Definition: rpl_rli.h:988
 
char ign_master_log_name_end[FN_REFLEN]
Definition: rpl_rli.h:980
 
bool recovery_groups_inited
Definition: rpl_rli.h:1247
 
ulonglong event_relay_log_pos
Definition: rpl_rli.h:702
 
void end_info()
Definition: rpl_rli.cc:1871
 
bool rli_fake
Definition: rpl_rli.h:761
 
ulong opt_replica_parallel_workers
Definition: rpl_rli.h:1236
 
void set_allow_drop_write_set(bool does_not_require_ws)
Definition: rpl_rli.h:2051
 
std::atomic< longlong > mts_total_wait_overlap
Definition: rpl_rli.h:1295
 
time_t get_sql_delay()
Definition: rpl_rli.h:1689
 
void clear_gtid_monitoring_info()
Clears the Gtid_monitoring_info fields.
Definition: rpl_rli.h:435
 
char group_master_log_name[FN_REFLEN]
Definition: rpl_rli.h:749
 
void clear_mts_recovery_groups()
Definition: rpl_rli.h:1375
 
Prealloced_array< db_worker_hash_entry *, 4 > curr_group_assigned_parts
Definition: rpl_rli.h:1210
 
const char * get_for_channel_str(bool upper_case=false) const override
Definition: rpl_rli.cc:2686
 
ulonglong ign_master_log_pos_end
Definition: rpl_rli.h:981
 
void cleanup_after_query()
Definition: rpl_rli.h:1433
 
Commit_order_manager * get_commit_order_manager()
Definition: rpl_rli.h:1772
 
ulong trans_retries
Definition: rpl_rli.h:970
 
void init_workers(ulong)
The method to invoke at slave threads start.
Definition: rpl_rli.cc:271
 
~Relay_log_info() override
Definition: rpl_rli.cc:296
 
uint32 m_flags
Definition: rpl_rli.h:1862
 
bool relay_log_sanitized
Flag that ensures the relay log is sanitized only once.
Definition: rpl_rli.h:766
 
void clear_tables_to_lock()
Definition: rpl_rli.cc:1373
 
const char * get_until_log_name()
Definition: rpl_rli.cc:2703
 
void post_rollback()
Execute actions at replicated atomic DLL post rollback time.
Definition: rpl_rli.h:2120
 
static const int APPLIER_METADATA_LINES_WITH_ASSIGN_GTIDS_TO_ANONYMOUS_TRANSACTIONS_TYPE
Definition: rpl_rli.h:1931
 
my_off_t event_start_pos
Definition: rpl_rli.h:736
 
bool m_is_engine_ha_data_detached
Definition: rpl_rli.h:2020
 
static size_t get_number_info_rli_fields()
Definition: rpl_rli.cc:2066
 
bool reported_unsafe_warning
Definition: rpl_rli.h:1712
 
static const int APPLIER_METADATA_LINES_WITH_PRIV_CHECKS_USERNAME
Definition: rpl_rli.h:1895
 
std::atomic< ulong > mts_wq_overrun_cnt
Definition: rpl_rli.h:1286
 
void set_require_table_primary_key_check(enum_require_table_primary_key require_pk)
Sets the field that tells what is the slave policy concerning primary keys on replicated tables.
Definition: rpl_rli.cc:3311
 
RPL_Table_ref * tables_to_lock
Definition: rpl_rli.h:1086
 
uint64 original_commit_timestamp
Definition: rpl_rli.h:1131
 
mysql_cond_t pending_jobs_cond
Definition: rpl_rli.h:1193
 
void set_commit_order_manager(Commit_order_manager *mngr)
Definition: rpl_rli.h:1774
 
std::vector< Slave_worker * > workers_copy_pfs
Definition: rpl_rli.h:1178
 
void set_thd_tx_priority(int priority)
Definition: rpl_rli.h:2039
 
void fill_coord_err_buf(loglevel level, int err_code, const char *buff_coord) const
Update the error number, message and timestamp fields.
Definition: rpl_rli.cc:565
 
enum_return_status add_gtid_set(const Gtid_set *gtid_set)
Adds a GTID set to received GTID set.
Definition: rpl_rli.cc:2693
 
bool replicate_same_server_id
Definition: rpl_rli.h:326
 
void add_logged_gtid(rpl_sidno sidno, rpl_gno gno)
Definition: rpl_rli.h:844
 
bool is_time_for_mta_checkpoint()
Check if it is time to compute MTS checkpoint.
Definition: rpl_rli.cc:2957
 
void set_flag(enum_state_flag flag)
Set the value of a replication state flag.
Definition: rpl_rli.h:1461
 
std::string m_privilege_checks_hostname
The host name part of the user passed on to PRIVILEGE_CHECKS_USER.
Definition: rpl_rli.h:798
 
mysql_mutex_t slave_worker_hash_lock
Definition: rpl_rli.h:1164
 
ulong exit_counter
Definition: rpl_rli.h:1240
 
void reset_notified_relay_log_change()
While a group is executed by a Worker the relay log can change.
Definition: rpl_rli.cc:356
 
static void set_nullable_fields(MY_BITMAP *nullable_fields)
Sets bits for columns that are allowed to be NULL.
Definition: rpl_rli.cc:2070
 
bool m_is_applier_source_position_info_invalid
Are positions invalid.
Definition: rpl_rli.h:835
 
bool is_until_satisfied_all_transactions_read_from_relay_log()
Definition: rpl_rli.h:2071
 
void set_master_info(Master_info *info)
Definition: rpl_rli.cc:1900
 
bool gtid_retrieved_initialized
Definition: rpl_rli.h:763
 
ulonglong stats_exec_time
Definition: rpl_rli.h:1307
 
bool force_flush_postponed_due_to_split_trans
Definition: rpl_rli.h:1770
 
volatile ulong mts_wq_underrun_w_id
Definition: rpl_rli.h:1220
 
void clear_processing_trx()
Clears the processing_trx structure fields.
Definition: rpl_rli.h:430
 
void deinit_workers()
The method to invoke at slave threads stop.
Definition: rpl_rli.cc:294
 
void clear_privilege_checks_user()
Clears the info related to the data initialized from PRIVILEGE_CHECKS_USER.
Definition: rpl_rli.cc:2994
 
my_off_t get_event_start_pos()
Definition: rpl_rli.h:1655
 
bool get_ignore_write_set_memory_limit()
Definition: rpl_rli.h:2047
 
virtual void post_commit(bool on_rollback)
Cleanup of any side effect that pre_commit() inflicts, including restore of the last executed group c...
Definition: rpl_rli.cc:2866
 
std::atomic< ulonglong > log_space_limit
Definition: rpl_rli.h:905
 
void start_sql_delay(time_t delay_end)
Indicate that a delay starts.
Definition: rpl_rli.cc:2078
 
void cached_charset_invalidate()
Last charset (6 bytes) seen by slave SQL thread is cached here; it helps the thread save 3 get_charse...
Definition: rpl_rli.cc:1221
 
uint tables_to_lock_count
Definition: rpl_rli.h:1087
 
int get_thd_tx_priority()
Definition: rpl_rli.h:2041
 
bool mts_recovery_group_seen_begin
Definition: rpl_rli.h:1250
 
bool long_find_row_note_printed
Definition: rpl_rli.h:1963
 
Deferred_log_events * deferred_events
Definition: rpl_rli.h:1139
 
Rpl_filter * rpl_filter
Definition: rpl_rli.h:255
 
ulong replica_parallel_workers
Definition: rpl_rli.h:1238
 
enum Relay_log_info::@160 commit_timestamps_status
the status of the commit timestamps for the relay log
 
Until_option * until_option
Definition: rpl_rli.h:2008
 
ulonglong mts_total_wait_worker_avail
Definition: rpl_rli.h:1300
 
int flush_info(const int flush_flags)
Stores the file and position where the execute-slave thread are in the relay log:
Definition: rpl_rli.cc:1956
 
bool is_relay_log_recovery
Definition: rpl_rli.h:338
 
Relay_log_info(bool is_slave_recovery, PSI_mutex_key *param_key_info_run_lock, PSI_mutex_key *param_key_info_data_lock, PSI_mutex_key *param_key_info_sleep_lock, PSI_mutex_key *param_key_info_thd_lock, PSI_mutex_key *param_key_info_data_cond, PSI_mutex_key *param_key_info_start_cond, PSI_mutex_key *param_key_info_stop_cond, PSI_mutex_key *param_key_info_sleep_cond, uint param_id, const char *param_channel, bool is_rli_fake)
Definition: rpl_rli.cc:113
 
bool workers_array_initialized
Definition: rpl_rli.h:1189
 
bool write_info(Rpl_info_handler *to) override
Definition: rpl_rli.cc:2336
 
time_t sql_delay_end
During a delay, specifies the point in time when the delay ends.
Definition: rpl_rli.h:1860
 
void set_ignore_write_set_memory_limit(bool ignore_limit)
Definition: rpl_rli.h:2043
 
bool mts_wq_oversize
Definition: rpl_rli.h:1198
 
bool m_privilege_checks_user_corrupted
Tells whether or not the internal data regarding PRIVILEGE_CHECKS_USER is corrupted.
Definition: rpl_rli.h:805
 
enum_state_flag
Flags for the state of the replication.
Definition: rpl_rli.h:259
 
@ IN_STMT
The replication thread is inside a statement.
Definition: rpl_rli.h:261
 
@ STATE_FLAGS_COUNT
Flag counter.
Definition: rpl_rli.h:264
 
ulong mts_recovery_group_cnt
Definition: rpl_rli.h:1248
 
time_t row_stmt_start_timestamp
Definition: rpl_rli.h:1962
 
static constexpr int RLI_FLUSH_IGNORE_GTID_ONLY
Flush disresgarding the value of GTID_ONLY.
Definition: rpl_rli.h:1576
 
void set_group_master_log_name(const char *log_file_name)
Definition: rpl_rli.h:1623
 
void clear_flag(enum_state_flag flag)
Clear the value of a replication state flag.
Definition: rpl_rli.h:1477
 
ulong mts_wq_overfill_cnt
Definition: rpl_rli.h:1301
 
bool belongs_to_client()
Definition: rpl_rli.h:300
 
void set_future_event_relay_log_pos(ulonglong log_pos)
Definition: rpl_rli.h:1605
 
bool curr_group_isolated
Definition: rpl_rli.h:1216
 
void set_long_find_row_note_printed()
Definition: rpl_rli.h:1729
 
void set_privilege_checks_user_corrupted(bool is_corrupted)
Sets the flag that tells whether or not the data regarding the PRIVILEGE_CHECKS_USER is corrupted.
Definition: rpl_rli.cc:3001
 
char group_relay_log_name[FN_REFLEN]
Event group means a group of events of a transaction.
Definition: rpl_rli.h:699
 
time_t get_row_stmt_start_timestamp()
Definition: rpl_rli.h:1719
 
ulonglong get_future_event_relay_log_pos()
Definition: rpl_rli.h:1602
 
time_t get_sql_delay_end()
Definition: rpl_rli.h:1691
 
long mts_online_stat_curr
Definition: rpl_rli.h:1281
 
void cleanup_context(THD *, bool)
Definition: rpl_rli.cc:1285
 
char new_group_master_log_name[FN_REFLEN]
Storage for holding newly computed values for the last executed event group coordinates while the cur...
Definition: rpl_rli.h:1322
 
Slave_worker * get_worker(size_t n)
Definition: rpl_rli.h:1339
 
mysql_mutex_t log_space_lock
Definition: rpl_rli.h:937
 
ulong adapt_to_master_version(Format_description_log_event *fdle)
adaptation for the slave applier to specific master versions.
Definition: rpl_rli.cc:2539
 
static const int APPLIER_METADATA_LINES_WITH_DELAY
Definition: rpl_rli.h:1872
 
bool is_engine_ha_data_detached()
Checks whether engine ha data is detached from THD.
Definition: rpl_rli.h:2110
 
bool is_mts_recovery() const
returns true if there is any gap-group of events to execute at slave starting phase.
Definition: rpl_rli.h:1373
 
void notify_group_master_log_name_update()
The same as notify_group_relay_log_name_update but for group_master_log_name.
Definition: rpl_rli.h:1020
 
enum_require_row_status
Definition: rpl_rli.h:245
 
@ PRIV_CHECKS_USER_NOT_NULL
Value for privilege_checks_user is not empty.
 
@ SUCCESS
Function ended successfully.
 
Assign_gtids_to_anonymous_transactions_info m_assign_gtids_to_anonymous_transactions_info
Stores the information related to the ASSIGN_GTIDS_TO_ANONYMOUS_TRANSACTIONS parameter of CHANGE REPL...
Definition: rpl_rli.h:291
 
virtual Relay_log_info * get_c_rli()
Get coordinator's RLI.
Definition: rpl_rli.h:1813
 
void clear_sql_delay()
Reset the delay.
Definition: rpl_rli.h:927
 
Log_event * current_event
Reference to being applied event.
Definition: rpl_rli.h:2027
 
std::atomic< long > mts_wq_excess_cnt
Definition: rpl_rli.h:1231
 
time_t mts_last_online_stat
Definition: rpl_rli.h:1312
 
my_off_t get_until_log_pos()
Definition: rpl_rli.cc:2712
 
Mts_submode * current_mts_submode
Definition: rpl_rli.h:1361
 
time_t set_row_stmt_start_timestamp()
Definition: rpl_rli.h:1721
 
Slave_committed_queue * gaq
Definition: rpl_rli.h:1205
 
bool is_relay_log_truncated()
Definition: rpl_rli.h:838
 
bool m_ignore_write_set_memory_limit
If the SQL thread should or not ignore the set limit for write set collection.
Definition: rpl_rli.h:1999
 
my_off_t new_group_master_log_pos
Definition: rpl_rli.h:1323
 
static const int APPLIER_METADATA_LINES_WITH_PRIV_CHECKS_HOSTNAME
Definition: rpl_rli.h:1907
 
bool clear_info()
Clears from this Relay_log_info object all attribute values that are not to be kept.
Definition: rpl_rli.cc:2020
 
void clear_until_option()
Definition: rpl_rli.h:1797
 
ulonglong stats_read_time
Definition: rpl_rli.h:1308
 
bool is_privilege_checks_user_null() const
Returns whether or not there is no user configured for PRIVILEGE_CHECKS_USER.
Definition: rpl_rli.cc:2983
 
void set_sql_delay(time_t _sql_delay)
Definition: rpl_rli.h:1690
 
Checkable_rwlock * get_tsid_lock()
Definition: rpl_rli.h:842
 
std::atomic< ulonglong > log_space_total
Definition: rpl_rli.h:905
 
static const int PRIV_CHECKS_HOSTNAME_LENGTH
Definition: rpl_rli.h:1912
 
bool is_applier_source_position_info_invalid() const
Returns if the applier positions are marked as being invalid or not.
Definition: rpl_rli.cc:3321
 
bool is_group_master_log_pos_invalid
Flag that the group_master_log_pos is invalid.
Definition: rpl_rli.h:900
 
void reset_row_stmt_start_timestamp()
Definition: rpl_rli.h:1727
 
MYSQL_BIN_LOG relay_log
Definition: rpl_rli.h:332
 
bool m_require_row_format
Tells if the slave is only accepting events logged with row based logging.
Definition: rpl_rli.h:814
 
void reset_notified_checkpoint(ulong count, time_t new_ts, bool update_timestamp=false)
While a group is executed by a Worker the relay log can change.
Definition: rpl_rli.cc:383
 
int rli_init_info(bool skip_received_gtid_set_and_relaylog_recovery=false)
Initialize the relay log info.
Definition: rpl_rli.cc:1502
 
bool m_allow_drop_write_set
Even if a component says all transactions require write sets, this variable says the SQL thread trans...
Definition: rpl_rli.h:2005
 
uint rli_checkpoint_seqno
Definition: rpl_rli.h:1243
 
static const int APPLIER_METADATA_LINES_WITH_WORKERS
Definition: rpl_rli.h:1877
 
bool is_until_satisfied_at_start_slave()
Definition: rpl_rli.h:2059
 
std::pair< const char *, const char * > print_applier_security_context_user_host() const
Returns a printable representation of the username and hostname currently being used in the applier s...
Definition: rpl_rli.cc:3114
 
void set_group_master_log_pos(ulonglong log_pos)
Definition: rpl_rli.h:1627
 
volatile my_off_t group_master_log_pos
Definition: rpl_rli.h:750
 
long mts_worker_underrun_level
Definition: rpl_rli.h:1232
 
bool is_until_satisfied_before_dispatching_event(const Log_event *ev)
Definition: rpl_rli.h:2063
 
ulonglong get_group_relay_log_pos()
Definition: rpl_rli.h:1634
 
void set_group_relay_log_name(const char *log_file_name, size_t len)
Definition: rpl_rli.h:1639
 
enum Relay_log_info::@162 mts_group_status
 
int inc_group_relay_log_pos(ulonglong log_pos, bool need_data_lock, bool force=false)
Last executed event group coordinates are updated and optionally forcibly flushed to a repository.
Definition: rpl_rli.cc:938
 
@ UNTIL_SQL_VIEW_ID
Definition: rpl_rli.h:957
 
@ UNTIL_MASTER_POS
Definition: rpl_rli.h:952
 
@ UNTIL_NONE
Definition: rpl_rli.h:951
 
@ UNTIL_SQL_BEFORE_GTIDS
Definition: rpl_rli.h:954
 
@ UNTIL_SQL_AFTER_GTIDS
Definition: rpl_rli.h:955
 
@ UNTIL_SQL_AFTER_MTS_GAPS
Definition: rpl_rli.h:956
 
@ UNTIL_DONE
Definition: rpl_rli.h:958
 
@ UNTIL_RELAY_POS
Definition: rpl_rli.h:953
 
bool is_group_relay_log_name_invalid(const char **errmsg)
Check if group_relay_log_name is in index file.
Definition: rpl_rli.cc:535
 
Relay_log_info & operator=(const Relay_log_info &info)
 
const char * get_rpl_log_name() const
Definition: rpl_rli.h:1660
 
void set_until_option(Until_option *option)
Definition: rpl_rli.h:1791
 
void finished_processing()
When the processing of a transaction is completed, that timestamp is recorded, the information is cop...
Definition: rpl_rli.h:417
 
enum_priv_checks_status set_privilege_checks_user(char const *param_privilege_checks_username, char const *param_privilege_checks_hostname)
Initializes data related to PRIVILEGE_CHECKS_USER, specifically the user name and the user hostname.
Definition: rpl_rli.cc:3007
 
int purge_relay_logs(THD *thd, const char **errmsg, bool delete_only=false)
Purges relay logs.
Definition: rpl_rli.cc:1042
 
bool is_privilege_checks_user_corrupted() const
Returns whether or not the internal data regarding PRIVILEGE_CHECKS_USER is corrupted.
Definition: rpl_rli.cc:2990
 
int init_until_option(THD *thd, const LEX_SOURCE_INFO *master_param)
Initialize until option object when starting slave.
Definition: rpl_rli.cc:2721
 
bool is_until_satisfied_after_dispatching_event()
Definition: rpl_rli.h:2067
 
enum_priv_checks_status initialize_security_context(THD *thd)
Initializes the security context associated with the PRIVILEGE_CHECKS_USER user that is to be used by...
Definition: rpl_rli.cc:3256
 
@ COMMIT_TS_FOUND
Definition: rpl_rli.h:365
 
@ COMMIT_TS_NOT_FOUND
Definition: rpl_rli.h:363
 
@ COMMIT_TS_UNKNOWN
Definition: rpl_rli.h:361
 
void set_group_relay_log_pos(ulonglong log_pos)
Definition: rpl_rli.h:1642
 
const char * get_group_relay_log_name()
Definition: rpl_rli.h:1633
 
enum_priv_checks_status check_privilege_checks_user()
Checks the validity and integrity of the data related to PRIVILEGE_CHECKS_USER, specifically the user...
Definition: rpl_rli.cc:3037
 
const char * get_event_relay_log_name()
Definition: rpl_rli.h:1646
 
virtual int set_rli_description_event(Format_description_log_event *fdle)
Delete the existing event and set a new one.
Definition: rpl_rli.cc:2419
 
Gtid_monitoring_info * gtid_monitoring_info
Stores information on the last processed transaction or the transaction that is currently being proce...
Definition: rpl_rli.h:781
 
std::atomic< ulong > abort_pos_wait
Definition: rpl_rli.h:936
 
const char * add_channel_to_relay_log_name(char *buff, uint buff_size, const char *base_name)
sets the suffix required for relay log names in multisource replication.
Definition: rpl_rli.cc:1189
 
size_t get_worker_count()
Definition: rpl_rli.h:1328
 
ulonglong get_group_master_log_pos() const
Definition: rpl_rli.h:1616
 
const char * get_group_master_log_name_info() const
Definition: rpl_rli.h:1612
 
void report_privilege_check_error(enum loglevel level, enum_priv_checks_status status_code, bool to_client, char const *channel_name=nullptr, char const *user_name=nullptr, char const *host_name=nullptr) const
Outputs the error message associated with applier thread user privilege checks error error_code.
Definition: rpl_rli.cc:3132
 
enum_require_table_primary_key
Identifies what is the replica policy on primary keys in tables.
Definition: rpl_rli.h:270
 
@ PK_CHECK_ON
The replica enforces tables to have primary keys for a given channel.
Definition: rpl_rli.h:279
 
@ PK_CHECK_STREAM
The replica sets the value of sql_require_primary_key according to the source replicated value.
Definition: rpl_rli.h:277
 
@ PK_CHECK_OFF
The replica does not enforce any policy around primary keys.
Definition: rpl_rli.h:281
 
@ PK_CHECK_GENERATE
The replica generates GIPKs for incoming keyless tables.
Definition: rpl_rli.h:283
 
@ PK_CHECK_NONE
No policy, used on PFS.
Definition: rpl_rli.h:272
 
void started_processing(Gtid gtid_arg, ulonglong original_ts_arg, ulonglong immediate_ts_arg, bool skipped=false)
Stores the details of the transaction which has just started processing.
Definition: rpl_rli.h:386
 
bool pre_commit()
The method implements a pre-commit hook to add up a new statement typically to a DDL transaction to u...
Definition: rpl_rli.h:2132
 
void retried_processing(uint transient_errno_arg, const char *transient_err_message_arg, ulong trans_retries_arg)
When a transaction is retried, the error number and message, and total number of retries are stored.
Definition: rpl_rli.h:446
 
static constexpr int RLI_FLUSH_NO_OPTION
No flush options given to relay log flush.
Definition: rpl_rli.h:1572
 
bool sql_thread_kill_accepted
Definition: rpl_rli.h:1717
 
static const int PRIV_CHECKS_USERNAME_LENGTH
Definition: rpl_rli.h:1900
 
void slave_close_thread_tables(THD *)
Definition: rpl_rli.cc:1414
 
enum_priv_checks_status
Set of possible return values for the member methods related to PRIVILEGE_CHECKS_USER management.
Definition: rpl_rli.h:211
 
@ USERNAME_NULL_HOSTNAME_NOT_NULL
Value for the username part of the user is NULL but the value for the hostname is not NULL.
 
@ USER_DOES_NOT_HAVE_PRIVILEGES
Provided user doesn't have the necessary privileges to execute the needed operations.
 
@ USER_ANONYMOUS
Value for user is anonymous (''@'...')
 
@ USERNAME_TOO_LONG
Value for the username part of the user is larger than 32 characters.
 
@ HOSTNAME_TOO_LONG
Value for the hostname part of the user is larger than 255 characters.
 
@ LOAD_DATA_EVENT_NOT_ALLOWED
Provided user doesn't have FILE privileges during the execution of a LOAD DATAevent.
 
@ USER_DATA_CORRUPTED
Values provided for the internal variables are corrupted.
 
@ USER_DOES_NOT_EXIST
Provided user doesn't exists.
 
@ SUCCESS
Function ended successfully.
 
@ HOSTNAME_SYNTAX_ERROR
Value for the hostname part of the user has illegal characters.
 
bool ddl_not_atomic
Raised when slave applies and writes to its binary log statement which is not atomic DDL and has no X...
Definition: rpl_rli.h:2037
 
bool get_flag(enum_state_flag flag)
Get the value of a replication state flag.
Definition: rpl_rli.h:1470
 
static const int APPLIER_METADATA_LINES_WITH_REQUIRE_TABLE_PRIMARY_KEY_CHECK
Definition: rpl_rli.h:1923
 
static const int MAXIMUM_APPLIER_METADATA_LINES
Definition: rpl_rli.h:1949
 
void inc_event_relay_log_pos()
Definition: rpl_rli.h:1025
 
bool read_info(Rpl_info_handler *from) override
Definition: rpl_rli.cc:2084
 
void started_processing(Gtid_log_event *gtid_log_ev_arg)
Stores the details of the transaction which has just started processing.
Definition: rpl_rli.h:400
 
struct timespec stats_begin
Definition: rpl_rli.h:1310
 
mysql_mutex_t exit_count_lock
Definition: rpl_rli.h:1194
 
void notify_relay_log_truncated()
Receiver thread notifies that it truncated some data from relay log.
Definition: rpl_rli.cc:2946
 
enum Relay_log_info::@161 until_condition
 
time_t last_event_start_time
Definition: rpl_rli.h:1128
 
ulonglong get_event_relay_log_pos()
Definition: rpl_rli.h:1647
 
ulong adapt_to_master_version_updown(ulong master_version, ulong current_version)
The method compares two supplied versions and carries out down- or up- grade customization of executi...
Definition: rpl_rli.cc:2619
 
std::atomic_bool is_receiver_waiting_for_rl_space
Definition: rpl_rli.h:910
 
enum_priv_checks_status initialize_applier_security_context()
Initializes the security context associated with the PRIVILEGE_CHECKS_USER user that is to be used by...
Definition: rpl_rli.cc:3292
 
ulonglong mts_pending_jobs_size
Definition: rpl_rli.h:1196
 
Format_description_log_event * rli_description_event
Definition: rpl_rli.h:1825
 
bool mts_finalize_recovery()
Called when gaps execution is ended so it is crash-safe to reset the last session Workers info.
Definition: rpl_rli.cc:440
 
void set_group_relay_log_name(const char *log_file_name)
Definition: rpl_rli.h:1635
 
int thd_tx_priority
Definition: rpl_rli.h:1993
 
ulonglong group_source_log_end_pos
Definition: rpl_rli.h:732
 
my_off_t new_group_relay_log_pos
Definition: rpl_rli.h:1325
 
void clear_relay_log_truncated()
Applier clears the flag after it handled the situation.
Definition: rpl_rli.cc:2952
 
Replication_transaction_boundary_parser transaction_parser
Definition: rpl_rli.h:650
 
char slave_patternload_file[FN_REFLEN]
Definition: rpl_rli.h:987
 
ulong mts_recovery_index
Definition: rpl_rli.h:1249
 
void unset_long_find_row_note_printed()
Definition: rpl_rli.h:1731
 
mysql_mutex_t mts_temp_table_LOCK
Definition: rpl_rli.h:312
 
bool deferred_events_collecting
Definition: rpl_rli.h:1145
 
bool m_relay_log_truncated
It will be set to true when receiver truncated relay log for some reason.
Definition: rpl_rli.h:788
 
bool is_mts_in_group()
returns true if Coordinator is scheduling events belonging to the same group and has not reached yet ...
Definition: rpl_rli.h:1398
 
time_t last_master_timestamp
Definition: rpl_rli.h:921
 
bool is_parallel_exec() const
returns true if events are to be executed in parallel
Definition: rpl_rli.h:1386
 
int wait_for_gtid_set(THD *thd, const char *gtid, double timeout, bool update_THD_status=true)
Wait for a GTID set to be executed.
Definition: rpl_rli.cc:779
 
malloc_unordered_map< std::string, unique_ptr_with_deleter< db_worker_hash_entry > > mapping_db_to_worker
Definition: rpl_rli.h:1161
 
std::string coordinator_log_after_purge
Definition: rpl_rli.h:919
 
static const int APPLIER_METADATA_LINES_WITH_ASSIGN_GTIDS_TO_ANONYMOUS_TRANSACTIONS_VALUE
Definition: rpl_rli.h:1939
 
bool error_on_rli_init_info
Definition: rpl_rli.h:458
 
std::string get_privilege_checks_username() const
Retrieves the username part of the PRIVILEGE_CHECKS_USER option of CHANGE MASTER TO statement.
Definition: rpl_rli.cc:2975
 
Master_info * mi
Definition: rpl_rli.h:350
 
std::string m_privilege_checks_username
The user name part of the user passed on to PRIVILEGE_CHECKS_USER.
Definition: rpl_rli.h:793
 
@ MTS_KILLED_GROUP
Definition: rpl_rli.h:1275
 
@ MTS_NOT_IN_GROUP
Definition: rpl_rli.h:1271
 
@ MTS_IN_GROUP
Definition: rpl_rli.h:1273
 
@ MTS_END_GROUP
Definition: rpl_rli.h:1274
 
bool is_processing_trx()
Definition: rpl_rli.h:422
 
bool is_row_format_required() const
Returns whether the slave is running in row mode only.
Definition: rpl_rli.cc:3297
 
const Gtid_set * get_gtid_set() const
Definition: rpl_rli.h:860
 
void cleanup_after_session()
Definition: rpl_rli.h:1437
 
PSI_mutex_key m_key_mta_temp_table_LOCK
Definition: rpl_rli.h:306
 
virtual bool commit_positions()
The method implements updating a slave info table.
Definition: rpl_rli.cc:2815
 
char event_relay_log_name[FN_REFLEN]
Definition: rpl_rli.h:701
 
enum_require_table_primary_key get_require_table_primary_key_check() const
Returns what is the slave policy concerning primary keys on replicated tables.
Definition: rpl_rli.cc:3307
 
Slave_worker_array workers
Definition: rpl_rli.h:1156
 
bool get_allow_drop_write_set()
Definition: rpl_rli.h:2055
 
ulonglong future_event_relay_log_pos
Definition: rpl_rli.h:703
 
Commit_order_manager * commit_order_mngr
Definition: rpl_rli.h:1833
 
struct timespec ts_exec[2]
Definition: rpl_rli.h:1309
 
std::string get_privilege_checks_hostname() const
Retrieves the host part of the PRIVILEGE_CHECKS_USER option of CHANGE MASTER TO statement.
Definition: rpl_rli.cc:2979
 
void set_group_source_log_start_end_pos(const Log_event *ev)
Process an event and based on its type () set group beginning and end.
Definition: rpl_rli.cc:2919
 
Gtid_monitoring_info * get_gtid_monitoring_info()
Definition: rpl_rli.h:371
 
char new_group_relay_log_name[FN_REFLEN]
Definition: rpl_rli.h:1324
 
void set_event_relay_log_name(const char *log_file_name)
Definition: rpl_rli.h:1648
 
void set_applier_source_position_info_invalid(bool invalid)
Marks the applier position information as being invalid or not.
Definition: rpl_rli.cc:3317
 
int wait_for_pos(THD *thd, String *log_name, longlong log_pos, double timeout)
Waits until the SQL thread reaches (has executed up to) the log/position or timed out.
Definition: rpl_rli.cc:601
 
void set_require_row_format(bool require_row)
Sets the flag that tells whether or not the slave is running in row mode only.
Definition: rpl_rli.cc:3301
 
void reattach_engine_ha_data(THD *thd)
Reattaches the engine ha_data to THD.
Definition: rpl_rli.cc:2804
 
bool mts_end_group_sets_max_dbs
Definition: rpl_rli.h:1217
 
time_t sql_delay
Delay slave SQL thread by this amount of seconds.
Definition: rpl_rli.h:1850
 
mysql_cond_t slave_worker_hash_cond
Definition: rpl_rli.h:1165
 
Relay_log_info(const Relay_log_info &info)
 
Slave_worker * last_assigned_worker
Definition: rpl_rli.h:1200
 
ulong recovery_parallel_workers
Definition: rpl_rli.h:1242
 
ulonglong mts_pending_jobs_size_max
Definition: rpl_rli.h:1197
 
bool get_table_data(TABLE *table_arg, table_def **tabledef_var, TABLE **conv_table_var) const
Definition: rpl_rli.h:1092
 
enum_mts_parallel_type channel_mts_submode
Definition: rpl_rli.h:1359
 
static const int APPLIER_METADATA_LINES_WITH_CHANNEL
Definition: rpl_rli.h:1888
 
mysql_mutex_t mts_gaq_LOCK
Definition: rpl_rli.h:317
 
This is the class for verifying transaction boundaries in a replication event stream.
Definition: rpl_trx_boundary_parser.h:51
 
Common base class for all row-containing log events.
Definition: log_event.h:2764
 
It is used to record the original query for the rows events in RBR.
Definition: log_event.h:3756
 
Rpl_filter.
Definition: rpl_filter.h:214
 
Definition: rpl_info_factory.h:41
 
Definition: rpl_info_handler.h:58
 
Definition: rpl_info.h:43
 
mysql_mutex_t data_lock
Definition: rpl_info.h:58
 
bool is_transactional() const
Definition: rpl_info.h:107
 
THD * info_thd
Definition: rpl_info.h:78
 
A set of THD members describing the current authenticated user.
Definition: sql_security_ctx.h:54
 
Group Assigned Queue whose first element identifies first gap in committed sequence.
Definition: rpl_rli_pdb.h:351
 
Definition: rpl_rli_pdb.h:498
 
Using this class is fraught with peril, and you need to be very careful when doing so.
Definition: sql_string.h:167
 
For each client connection we create a separate thread with THD serving as a thread/connection descri...
Definition: sql_lexer_thd.h:36
 
Gtid owned_gtid
The GTID of the currently owned transaction.
Definition: sql_class.h:3804
 
bool slave_thread
Definition: sql_class.h:2774
 
Relay_log_info * rli_slave
Definition: sql_class.h:1088
 
enum enum_thread_type system_thread
Definition: sql_class.h:2567
 
System_variables variables
Definition: sql_lexer_thd.h:64
 
bool is_commit_in_middle_of_statement
Definition: sql_class.h:3962
 
bool is_operating_substatement_implicitly
Definition: sql_class.h:2390
 
Table_ref * next_global
Definition: table.h:3564
 
Represents a bidirectional map between TSID and SIDNO.
Definition: rpl_gtid.h:749
 
Checkable_rwlock * get_tsid_lock() const
Return the tsid_lock.
Definition: rpl_gtid.h:872
 
This is the abstract base class for slave start until options.
Definition: rpl_replica_until_options.h:56
 
bool is_satisfied_after_dispatching_event()
check if the until option is satisfied after applied or dispatched a event.
Definition: rpl_replica_until_options.h:97
 
bool is_satisfied_before_dispatching_event(const Log_event *ev)
check if the until option is satisfied before applying or dispatching a event.
Definition: rpl_replica_until_options.h:83
 
bool is_satisfied_at_start_slave()
Check if the until option is already satisfied at coordinator starting.
Definition: rpl_replica_until_options.h:67
 
bool is_satisfied_all_transactions_read_from_relay_log()
check if the until option is waiting for more transactions to be read from the relay log.
Definition: rpl_replica_until_options.h:111
 
std::unordered_map, but with my_malloc, so that you can track the memory used using PSI memory keys.
Definition: map_helpers.h:157
 
uint64_t immediate_commit_timestamp
Timestamp when the transaction was committed on the nearest source.
Definition: control_events.h:1036
 
uint64_t original_commit_timestamp
Timestamp when the transaction was committed on the originating source.
Definition: control_events.h:1034
 
uint16_t error_code
Definition: statement_events.h:560
 
A table definition from the master.
Definition: rpl_utility.h:248
 
Maps table id's (integers) to table pointers.
Definition: rpl_tblmap.h:51
 
#define mysql_mutex_lock(M)
Definition: mysql_mutex.h:50
 
#define mysql_mutex_unlock(M)
Definition: mysql_mutex.h:57
 
#define base_name(A)
Definition: my_sys.h:715
 
uint sql_replica_skip_counter
a copy of active_mi->rli->slave_skip_counter, for showing in SHOW GLOBAL VARIABLES,...
Definition: rpl_replica.cc:8786
 
bool is_atomic_ddl(THD *thd, bool using_trans_arg)
The function lists all DDL instances that are supported for crash-recovery (WL9175).
Definition: log_event.cc:3703
 
unsigned int PSI_mutex_key
Instrumented mutex key.
Definition: psi_mutex_bits.h:52
 
static int flag
Definition: hp_test1.cc:40
 
std::string log_file_name(const Log_files_context &ctx, Log_file_id file_id)
Provides name of the log file with the given file id, e.g.
Definition: log0files_io.cc:716
 
Binary log event definitions.
 
std::unique_ptr< T, void(*)(T *)> unique_ptr_with_deleter
std::unique_ptr, but with a custom delete function.
Definition: map_helpers.h:89
 
void bitmap_free(MY_BITMAP *map)
Definition: my_bitmap.cc:158
 
#define DBUG_PRINT(keyword, arglist)
Definition: my_dbug.h:181
 
Some integer typedefs for easier portability.
 
unsigned long long int ulonglong
Definition: my_inttypes.h:56
 
ulonglong my_off_t
Definition: my_inttypes.h:72
 
unsigned char uchar
Definition: my_inttypes.h:52
 
long long int longlong
Definition: my_inttypes.h:55
 
uint64_t uint64
Definition: my_inttypes.h:69
 
uint32_t uint32
Definition: my_inttypes.h:67
 
Common #defines and includes for file and socket I/O.
 
#define FN_REFLEN
Definition: my_io.h:83
 
Definition of the global "loglevel" enumeration.
 
loglevel
Definition: my_loglevel.h:41
 
Defines various enable/disable and HAVE_ macros related to the performance schema instrumentation sys...
 
#define HAVE_PSI_INTERFACE
Definition: my_psi_config.h:39
 
Common header for many mysys elements.
 
static int count
Definition: myisam_ftdump.cc:45
 
Instrumentation helpers for conditions.
 
ABI for instrumented mutexes.
 
static PFS_engine_table_share_proxy table
Definition: pfs.cc:61
 
const std::string charset("charset")
 
static bool timeout(bool(*wait_condition)())
Timeout function.
Definition: log0meb.cc:498
 
@ QUERY_EVENT
Definition: binlog_event.h:300
 
std::vector< T, ut::allocator< T > > vector
Specialization of vector which uses allocator.
Definition: ut0new.h:2879
 
Instrumentation helpers for mutexes.
 
PSI_memory_key key_memory_db_worker_hash_entry
Definition: psi_memory_key.cc:108
 
Instrumentation helpers for mutexes.
 
#define OPTION_BEGIN
Definition: query_options.h:75
 
required uint32 status
Definition: replication_asynchronous_connection_failover.proto:61
 
required uint32 priority
Definition: replication_group_member_actions.proto:35
 
required string event
Definition: replication_group_member_actions.proto:32
 
mysql::gtid::gno_t rpl_gno
GNO, the second (numeric) component of a GTID, is an alias of mysql::gtid::gno_t.
Definition: rpl_gtid.h:112
 
enum_return_status
Generic return type for many functions that can succeed or fail.
Definition: rpl_gtid.h:137
 
@ ASSIGNED_GTID
Specifies that the transaction has been assigned a GTID (UUID:NUMBER).
Definition: rpl_gtid.h:3893
 
cs::index::rpl_sidno rpl_sidno
Type of SIDNO (source ID number, first component of GTID)
Definition: rpl_gtid.h:108
 
enum_return_check
Definition: rpl_info_handler.h:51
 
enum_mts_parallel_type
Definition: rpl_mta_submode.h:47
 
bool operator!(Relay_log_info::enum_priv_checks_status status)
Negation operator for enum_priv_checks_status, to facilitate validation against SUCCESS.
Definition: rpl_rli.cc:2967
 
Prealloced_array< Slave_worker *, 4 > Slave_worker_array
Definition: rpl_rli.h:80
 
struct slave_job_item Slave_job_item
 
bool is_mts_worker(const THD *thd)
Definition: rpl_rli.h:2178
 
bool is_committed_ddl(Log_event *ev)
Checks whether the supplied event encodes a (2pc-aware) DDL that has been already committed.
Definition: rpl_rli.h:2194
 
bool is_atomic_ddl_commit_on_slave(THD *thd)
Checks whether the transaction identified by the argument is executed by a slave applier thread is an...
Definition: rpl_rli.h:2226
 
bool mysql_show_relaylog_events(THD *thd)
Execute a SHOW RELAYLOG EVENTS statement.
Definition: rpl_rli.cc:1457
 
bool is_mts_db_partitioned(Relay_log_info *rli)
Auxiliary function to check if we have a db partitioned MTS.
Definition: rpl_rli.cc:2682
 
Replication transaction boundary parser.
 
enum_mdl_type
Type of metadata lock request.
Definition: sql_lexer_yacc_state.h:106
 
char * strmake(char *dst, const char *src, size_t length)
Definition: strmake.cc:42
 
TODO: Move this structure to mysql/binlog/event/control_events.h when we start using C++11.
Definition: rpl_gtid.h:1100
 
rpl_sidno sidno
SIDNO of this Gtid.
Definition: rpl_gtid.h:1104
 
Structure to hold parameters for CHANGE REPLICATION SOURCE, START REPLICA, and STOP REPLICA.
Definition: sql_lex.h:353
 
enum_mdl_namespace
Object namespaces.
Definition: mdl.h:400
 
const char * str
Definition: mysql_lex_string.h:41
 
Definition: my_bitmap.h:43
 
Extend the normal Table_ref with a few new fields needed by the slave thread, but nowhere else.
Definition: rpl_utility.h:512
 
LEX_CSTRING table_name
Definition: table.h:781
 
LEX_CSTRING db
Definition: table.h:780
 
TABLE_SHARE * s
Definition: table.h:1408
 
Legends running throughout the module:
Definition: rpl_rli_pdb.h:74
 
An instrumented cond structure.
Definition: mysql_cond_bits.h:50
 
An instrumented mutex structure.
Definition: mysql_mutex_bits.h:50
 
Log_event * data
Definition: rpl_rli.h:83
 
my_off_t relay_pos
Definition: rpl_rli.h:84
 
char event_relay_log_name[FN_REFLEN+1]
Definition: rpl_rli.h:85
 
static bool update_timestamp(THD *thd, set_var *var)
Definition: sys_vars.cc:5380
 
@ SYSTEM_THREAD_SLAVE_WORKER
Definition: thread_type.h:42
 
@ SYSTEM_THREAD_SLAVE_SQL
Definition: thread_type.h:37
 
Include file for Sun RPC to compile out of the box.
 
int n
Definition: xcom_base.cc:509