MySQL 9.0.0
Source Code Documentation
Commit_stage_manager Class Reference

Class for maintaining the commit stages for binary log group commit. More...

#include <rpl_commit_stage_manager.h>

Classes

class  Mutex_queue
 

Public Types

enum  StageID {
  BINLOG_FLUSH_STAGE , SYNC_STAGE , COMMIT_STAGE , AFTER_COMMIT_STAGE ,
  COMMIT_ORDER_FLUSH_STAGE , STAGE_COUNTER
}
 Constants for queues for different stages. More...
 

Public Member Functions

void init (PSI_mutex_key key_LOCK_flush_queue, PSI_mutex_key key_LOCK_sync_queue, PSI_mutex_key key_LOCK_commit_queue, PSI_mutex_key key_LOCK_after_commit_queue, PSI_mutex_key key_LOCK_done, PSI_mutex_key key_LOCK_wait_for_group_turn, PSI_cond_key key_COND_done, PSI_cond_key key_COND_flush_queue, PSI_cond_key key_COND_wait_for_group_turn)
 Initializes m_stage_cond_binlog, m_stage_cond_commit_order, m_stage_cond_leader condition variables and m_lock_done mutex. More...
 
void deinit ()
 Deinitializes m_stage_cond_binlog, m_stage_cond_commit_order, m_stage_cond_leader condition variables and m_lock_done mutex. More...
 
void wait_for_ticket_turn (THD *thd, bool update_ticket_manager=true)
 Waits for the THD session parameter underlying BGC ticket to become active. More...
 
bool append_to (StageID stage, THD *thd)
 Appends the given THD session object to the given stage queue. More...
 
bool enroll_for (StageID stage, THD *first, mysql_mutex_t *stage_mutex, mysql_mutex_t *enter_mutex)
 Enroll a set of sessions for a stage. More...
 
std::pair< bool, THD * > pop_front (StageID stage)
 Remove first member from the queue for given stage. More...
 
void clear_preempt_status (THD *head)
 The method ensures the follower's execution path can be preempted by the leader's thread. More...
 
THDfetch_queue_acquire_lock (StageID stage)
 Fetch the entire queue and empty it. More...
 
THDfetch_queue_skip_acquire_lock (StageID stage)
 Fetch the entire queue and empty it. More...
 
void wait_count_or_timeout (ulong count, long usec, StageID stage)
 Introduces a wait operation on the executing thread. More...
 
void signal_done (THD *queue, StageID stage=BINLOG_FLUSH_STAGE)
 The function is called after follower thread are processed by leader, to unblock follower threads. More...
 
void signal_end_of_ticket (bool force=false)
 Signals threads waiting on their BGC ticket turn. More...
 
void update_session_ticket_state (THD *thd)
 Updates the THD session object underlying BGC context. More...
 
void update_ticket_manager (std::uint64_t sessions_count, const binlog::BgcTicket &session_ticket)
 Adds the given session count to the total of processed sessions in the ticket manager active window, ends the active window if possible and notifies other threads that are waiting for a given ticket to have an active processing window. More...
 
void finish_session_ticket (THD *thd)
 Waits for the session's ticket, if needed, and resets the session's ticket context. More...
 
void process_final_stage_for_ordered_commit_group (THD *first)
 This function gets called after transactions are flushed to the engine i.e. More...
 
void lock_queue (StageID stage)
 Wrapper on Mutex_queue lock(), acquires lock on stage queue. More...
 
void unlock_queue (StageID stage)
 Wrapper on Mutex_queue unlock(), releases lock on stage queue. More...
 

Static Public Member Functions

static Commit_stage_managerget_instance ()
 Fetch Commit_stage_manager class instance. More...
 
static void disable_manual_session_tickets ()
 Disables the ability for session BGC tickets to be set manually. More...
 
static void enable_manual_session_tickets ()
 Enables the ability for session BGC tickets to be set manually. More...
 

Private Member Functions

 Commit_stage_manager ()
 
 Commit_stage_manager (const Commit_stage_manager &)=delete
 
const Commit_stage_manageroperator= (const Commit_stage_manager &)=delete
 

Private Attributes

bool m_is_initialized
 check if Commit_stage_manager variables already initialized. More...
 
Mutex_queue m_queue [STAGE_COUNTER]
 Queues for sessions. More...
 
mysql_cond_t m_stage_cond_leader
 The binlog leader waits on this condition variable till it is indicated to wake up. More...
 
mysql_cond_t m_stage_cond_binlog
 Condition variable to indicate that the binlog threads can wake up and continue. More...
 
mysql_cond_t m_stage_cond_commit_order
 Condition variable to indicate that the flush to storage engine is done and commit order threads can again wake up and continue. More...
 
mysql_mutex_t m_lock_done
 Mutex used for the condition variable above. More...
 
mysql_mutex_t m_queue_lock [STAGE_COUNTER - 1]
 Mutex used for the stage level locks. More...
 
THDleader_thd
 Save pointer to leader thread which is used later to awake leader. More...
 
bool leader_await_preempt_status
 Flag is set by Leader when it starts waiting for follower's all-clear. More...
 
mysql_cond_t m_cond_preempt
 Condition variable to indicate a follower started waiting for commit. More...
 
mysql_cond_t m_cond_wait_for_ticket_turn
 Condition variable to wait for a given ticket to become active. More...
 
mysql_mutex_t m_lock_wait_for_ticket_turn
 Mutex to protect the wait for a given ticket to become active. More...
 

Detailed Description

Class for maintaining the commit stages for binary log group commit.

Member Enumeration Documentation

◆ StageID

Constants for queues for different stages.

Enumerator
BINLOG_FLUSH_STAGE 
SYNC_STAGE 
COMMIT_STAGE 
AFTER_COMMIT_STAGE 
COMMIT_ORDER_FLUSH_STAGE 
STAGE_COUNTER 

Constructor & Destructor Documentation

◆ Commit_stage_manager() [1/2]

Commit_stage_manager::Commit_stage_manager ( )
inlineprivate

◆ Commit_stage_manager() [2/2]

Commit_stage_manager::Commit_stage_manager ( const Commit_stage_manager )
privatedelete

Member Function Documentation

◆ append_to()

bool Commit_stage_manager::append_to ( StageID  stage,
THD thd 
)

Appends the given THD session object to the given stage queue.

It verifies that the given session's ticket is the active ticket, if not, waits on m_cond_wait_for_ticket_turn condition variable until it is.

Parameters
stageThe stage to add the THD parameter to.
thdThe THD session object to queue.
Returns
True if the session is a group leader, false otherwise.

◆ clear_preempt_status()

void Commit_stage_manager::clear_preempt_status ( THD head)

The method ensures the follower's execution path can be preempted by the leader's thread.

Preempt status of head follower is checked to engange the leader into waiting when set.

Parameters
headTHD* of a follower thread

◆ deinit()

void Commit_stage_manager::deinit ( )

Deinitializes m_stage_cond_binlog, m_stage_cond_commit_order, m_stage_cond_leader condition variables and m_lock_done mutex.

◆ disable_manual_session_tickets()

void Commit_stage_manager::disable_manual_session_tickets ( )
static

Disables the ability for session BGC tickets to be set manually.

◆ enable_manual_session_tickets()

void Commit_stage_manager::enable_manual_session_tickets ( )
static

Enables the ability for session BGC tickets to be set manually.

◆ enroll_for()

bool Commit_stage_manager::enroll_for ( StageID  stage,
THD first,
mysql_mutex_t stage_mutex,
mysql_mutex_t enter_mutex 
)

Enroll a set of sessions for a stage.

This will queue the session thread for writing and flushing.

If the thread being queued is assigned as stage leader, it will return immediately.

If wait_if_follower is true the thread is not the stage leader, the thread will be wait for the queue to be processed by the leader before it returns. In DBUG-ON version the follower marks is preempt status as ready.

The session threads entering this function acquires mutexes, and few of them are not released while exiting based on thread and stage type.

  • A binlog leader (returning true when stage!=COMMIT_ORDER_FLUSH_STAGE) will acquire the stage mutex in this function and not release it.
  • A commit order leader of the flush stage (returning true when stage==COMMIT_ORDER_FLUSH_STAGE) will acquire both the stage mutex and the flush queue mutex in this function, and not release them.
  • A follower (returning false) will release any mutexes it takes, before returning from the function.
Parameters
[in]stageStage identifier for the queue to append to.
[in]firstQueue to append.
[in]stage_mutexPointer to the currently held stage mutex, or nullptr if we're not in a stage, that will be released when changing stage.
[in]enter_mutexPointer to the mutex that will be taken when changing stage.
Return values
trueThread is stage leader.
falseThread was not stage leader and processing has been done.

◆ fetch_queue_acquire_lock()

THD * Commit_stage_manager::fetch_queue_acquire_lock ( StageID  stage)

Fetch the entire queue and empty it.

It acquires queue lock before fetching and emptying the queue threads.

Parameters
[in]stageStage identifier for the queue to append to.
Returns
Pointer to the first session of the queue.

◆ fetch_queue_skip_acquire_lock()

THD * Commit_stage_manager::fetch_queue_skip_acquire_lock ( StageID  stage)

Fetch the entire queue and empty it.

The caller must acquire queue lock before calling this function.

Parameters
[in]stageStage identifier for the queue to append to.
Returns
Pointer to the first session of the queue.

◆ finish_session_ticket()

void Commit_stage_manager::finish_session_ticket ( THD thd)

Waits for the session's ticket, if needed, and resets the session's ticket context.

Parameters
thdThe THD sessions object to finish the ticket's related work.

◆ get_instance()

Commit_stage_manager & Commit_stage_manager::get_instance ( )
static

Fetch Commit_stage_manager class instance.

Returns
Reference to the Commit_stage_manager class instance.

◆ init()

void Commit_stage_manager::init ( PSI_mutex_key  key_LOCK_flush_queue,
PSI_mutex_key  key_LOCK_sync_queue,
PSI_mutex_key  key_LOCK_commit_queue,
PSI_mutex_key  key_LOCK_after_commit_queue,
PSI_mutex_key  key_LOCK_done,
PSI_mutex_key  key_LOCK_wait_for_group_turn,
PSI_cond_key  key_COND_done,
PSI_cond_key  key_COND_flush_queue,
PSI_cond_key  key_COND_wait_for_group_turn 
)

Initializes m_stage_cond_binlog, m_stage_cond_commit_order, m_stage_cond_leader condition variables and m_lock_done mutex.

The binlog follower threads blocks on m_stage_cond_binlog condition variable till signalled to wake up from leader thread. And similarly commit order follower threads blocks on m_stage_cond_commit_order condition variable till signalled to wake up from leader thread.

The first binlog thread supposed to be leader finds that commit order queue is not empty then it blocks on m_stage_cond_leader till commit order leader signals it to awake and become new leader.

m_lock_done mutex is shared by all three stages.

Parameters
key_LOCK_flush_queuemutex instrumentation key
key_LOCK_sync_queuemutex instrumentation key
key_LOCK_commit_queuemutex instrumentation key
key_LOCK_after_commit_queuemutex instrumentation key
key_LOCK_donemutex instrumentation key
key_LOCK_wait_for_group_turnmutex instrumentation key
key_COND_donecond instrumentation key
key_COND_flush_queuecond instrumentation key
key_COND_wait_for_group_turncond instrumentation key

reuse key_COND_done 'cos a new PSI object would be wasteful in !NDEBUG

Initialize mutex for flush, sync, commit and after commit stage queue. The binlog flush stage and commit order flush stage share same mutex.

◆ lock_queue()

void Commit_stage_manager::lock_queue ( StageID  stage)
inline

Wrapper on Mutex_queue lock(), acquires lock on stage queue.

Parameters
[in]stageStage identifier for the queue to append to.

◆ operator=()

const Commit_stage_manager & Commit_stage_manager::operator= ( const Commit_stage_manager )
privatedelete

◆ pop_front()

std::pair< bool, THD * > Commit_stage_manager::pop_front ( StageID  stage)
inline

Remove first member from the queue for given stage.

Return values
Returnsstd::pair<bool, THD *> object. The first boolean value of pair if true determines queue is not empty, and false determines queue is empty. The second value returns the first removed member.

◆ process_final_stage_for_ordered_commit_group()

void Commit_stage_manager::process_final_stage_for_ordered_commit_group ( THD first)

This function gets called after transactions are flushed to the engine i.e.

after calling ha_flush_logs, to unblock commit order thread list which are not needed to wait for other stages.

Parameters
firstthe thread list which needs to ne unblocked

◆ signal_done()

void Commit_stage_manager::signal_done ( THD queue,
StageID  stage = BINLOG_FLUSH_STAGE 
)

The function is called after follower thread are processed by leader, to unblock follower threads.

Parameters
queuethe thread list which needs to ne unblocked
stageStage identifier current thread belong to.

◆ signal_end_of_ticket()

void Commit_stage_manager::signal_end_of_ticket ( bool  force = false)

Signals threads waiting on their BGC ticket turn.

Parameters
forceWhether or not to force the signaling, despit the state of the ticket manager.

◆ unlock_queue()

void Commit_stage_manager::unlock_queue ( StageID  stage)
inline

Wrapper on Mutex_queue unlock(), releases lock on stage queue.

Parameters
[in]stageStage identifier for the queue to append to.

◆ update_session_ticket_state()

void Commit_stage_manager::update_session_ticket_state ( THD thd)

Updates the THD session object underlying BGC context.

Parameters
thdThe THD object to update the BGC context for.

◆ update_ticket_manager()

void Commit_stage_manager::update_ticket_manager ( std::uint64_t  sessions_count,
const binlog::BgcTicket session_ticket 
)

Adds the given session count to the total of processed sessions in the ticket manager active window, ends the active window if possible and notifies other threads that are waiting for a given ticket to have an active processing window.

Parameters
sessions_countThe number of sessions to add to the ticket manager processed sessions count.
session_ticketThe session ticket (used for validations).

◆ wait_count_or_timeout()

void Commit_stage_manager::wait_count_or_timeout ( ulong  count,
long  usec,
StageID  stage 
)

Introduces a wait operation on the executing thread.

The waiting is done until the timeout elapses or count is reached (whichever comes first).

If count == 0, then the session will wait until the timeout elapses. If timeout == 0, then there is no waiting.

Parameters
usecthe number of microseconds to wait.
countwait for as many as count to join the queue the session is waiting on
stagewhich stage queue size to compare count against.

◆ wait_for_ticket_turn()

void Commit_stage_manager::wait_for_ticket_turn ( THD thd,
bool  update_ticket_manager = true 
)

Waits for the THD session parameter underlying BGC ticket to become active.

Parameters
thdThe THD session that holds the ticket to wait for.
update_ticket_managerIndicates whether to mark ticket as consumed by the session (add session to processed sessions) after the ticket is opened for processing.

Member Data Documentation

◆ leader_await_preempt_status

bool Commit_stage_manager::leader_await_preempt_status
private

Flag is set by Leader when it starts waiting for follower's all-clear.

◆ leader_thd

THD* Commit_stage_manager::leader_thd
private

Save pointer to leader thread which is used later to awake leader.

◆ m_cond_preempt

mysql_cond_t Commit_stage_manager::m_cond_preempt
private

Condition variable to indicate a follower started waiting for commit.

◆ m_cond_wait_for_ticket_turn

mysql_cond_t Commit_stage_manager::m_cond_wait_for_ticket_turn
private

Condition variable to wait for a given ticket to become active.

◆ m_is_initialized

bool Commit_stage_manager::m_is_initialized
private

check if Commit_stage_manager variables already initialized.

◆ m_lock_done

mysql_mutex_t Commit_stage_manager::m_lock_done
private

Mutex used for the condition variable above.

◆ m_lock_wait_for_ticket_turn

mysql_mutex_t Commit_stage_manager::m_lock_wait_for_ticket_turn
private

Mutex to protect the wait for a given ticket to become active.

◆ m_queue

Mutex_queue Commit_stage_manager::m_queue[STAGE_COUNTER]
private

Queues for sessions.

We need five queues:

  • Binlog flush queue: transactions that are going to be flushed to the engine and written to the binary log.
  • Commit order flush queue: transactions that are not going to write the binlog at all, but participate in the beginning of the group commit, up to and including the engine flush.
  • Sync queue: transactions that are going to be synced to disk
  • Commit queue: transactions that are going to to be committed (when binlog_order_commit=1).
  • After commit queue: transactions for which after commit hook is to be executed.

◆ m_queue_lock

mysql_mutex_t Commit_stage_manager::m_queue_lock[STAGE_COUNTER - 1]
private

Mutex used for the stage level locks.

◆ m_stage_cond_binlog

mysql_cond_t Commit_stage_manager::m_stage_cond_binlog
private

Condition variable to indicate that the binlog threads can wake up and continue.

◆ m_stage_cond_commit_order

mysql_cond_t Commit_stage_manager::m_stage_cond_commit_order
private

Condition variable to indicate that the flush to storage engine is done and commit order threads can again wake up and continue.

◆ m_stage_cond_leader

mysql_cond_t Commit_stage_manager::m_stage_cond_leader
private

The binlog leader waits on this condition variable till it is indicated to wake up.

If binlog flush queue gets first thread in the queue but by then commit order flush queue has already elected leader. The the first thread of binlog queue waits on this condition variable and get signalled to wake up from commit order flush queue leader later.


The documentation for this class was generated from the following files: