|  | MySQL 8.0.43
    Source Code Documentation | 
Dirty page archive system. More...
#include <arch0arch.h>
| Classes | |
| class | Recovery | 
| Recovery system data structure for the archiver.  More... | |
| Public Member Functions | |
| Arch_Page_Sys () | |
| Constructor: Initialize elements and create mutex.  More... | |
| ~Arch_Page_Sys () | |
| Destructor: Free memory buffer and mutexes.  More... | |
| int | start (Arch_Group **group, lsn_t *start_lsn, Arch_Page_Pos *start_pos, bool is_durable, bool restart, bool recovery) | 
| Start dirty page ID archiving.  More... | |
| int | stop (Arch_Group *group, lsn_t *stop_lsn, Arch_Page_Pos *stop_pos, bool is_durable) | 
| Stop dirty page ID archiving.  More... | |
| int | recovery_load_and_start (const Arch_Recv_Group_Info &info) | 
| Start dirty page ID archiving during recovery.  More... | |
| void | release (Arch_Group *group, bool is_durable, Arch_Page_Pos start_pos) | 
| Release the current group from client.  More... | |
| void | track_page (buf_page_t *bpage, lsn_t track_lsn, lsn_t frame_lsn, bool force) | 
| Check and add page ID to archived data.  More... | |
| void | flush_at_checkpoint (lsn_t checkpoint_lsn) | 
| Flush all the unflushed inactive blocks and flush the active block if required.  More... | |
| bool | archive (bool *wait) | 
| Archive dirty page IDs in current group.  More... | |
| void | arch_mutex_enter () | 
| Acquire dirty page ID archiver mutex.  More... | |
| void | arch_mutex_exit () | 
| Release page ID archiver mutex.  More... | |
| void | arch_oper_mutex_enter () | 
| Acquire dirty page ID archive operation mutex.  More... | |
| void | arch_oper_mutex_exit () | 
| Release page ID archiver operatiion mutex.  More... | |
| bool | save_reset_point (bool is_durable) | 
| bool | wait_for_reset_info_flush (uint64_t request_block) | 
| Wait for reset info to be flushed to disk.  More... | |
| int | fetch_group_within_lsn_range (lsn_t &start_id, lsn_t &stop_id, Arch_Group **group) | 
| Get the group which has tracked pages between the start_id and stop_id.  More... | |
| uint | purge (lsn_t *purge_lsn) | 
| Purge the archived files until the specified purge LSN.  More... | |
| void | update_stop_info (Arch_Block *cur_blk) | 
| Update the stop point in all the required structures.  More... | |
| void | get_status (std::vector< std::pair< lsn_t, bool > > &status) | 
| Fetch the status of the page tracking system.  More... | |
| bool | get_num_pages (Arch_Page_Pos start_pos, Arch_Page_Pos stop_pos, uint64_t &num_pages) | 
| Given start and stop position find number of pages tracked between them.  More... | |
| int | get_num_pages (lsn_t &start_id, lsn_t &stop_id, uint64_t *num_pages) | 
| Get approximate number of tracked pages between two given LSN values.  More... | |
| bool | get_pages (Arch_Group *group, Arch_Page_Pos *read_pos, uint read_len, byte *read_buff) | 
| Get page IDs from a specific position.  More... | |
| int | get_pages (MYSQL_THD thd, Page_Track_Callback cbk_func, void *cbk_ctx, lsn_t &start_id, lsn_t &stop_id, byte *buf, uint buf_len) | 
| Get archived page Ids between two given LSN values.  More... | |
| void | post_recovery_init () | 
| Set the latest stop LSN to the checkpoint LSN at the time it's called.  More... | |
| dberr_t | recover () | 
| Recover the archiver system at the time of startup.  More... | |
| void | print () | 
| Print information related to the archiver for debugging purposes.  More... | |
| void | set_read_only_mode () | 
| Set the state of the archiver system to read only.  More... | |
| bool | is_init () const | 
| Check if archiver system is in initial state.  More... | |
| bool | is_active () const | 
| Check if archiver system is active.  More... | |
| bool | is_abort () const | 
| ib_mutex_t * | get_mutex () | 
| Get the mutex protecting concurrent start, stop operations required for initialising group during recovery.  More... | |
| ib_mutex_t * | get_oper_mutex () | 
| Page_Arch_Client_Ctx * | get_sys_client () const | 
| Fetch the system client context.  More... | |
| lsn_t | get_latest_stop_lsn () const | 
| Arch_Page_Sys (Arch_Page_Sys const &)=delete | |
| Disable copy construction.  More... | |
| Arch_Page_Sys & | operator= (Arch_Page_Sys const &)=delete | 
| Disable assignment.  More... | |
| Private Member Functions | |
| bool | wait_idle () | 
| Wait for archive system to come out of ARCH_STATE_PREPARE_IDLE.  More... | |
| bool | is_gap_small () | 
| Check if the gap from last reset is short.  More... | |
| void | set_tracking_buf_pool (lsn_t tracking_lsn) | 
| Enable tracking pages in all buffer pools.  More... | |
| void | track_initial_pages () | 
| Track pages for which IO is already started.  More... | |
| dberr_t | flush_blocks (bool *wait) | 
| Flush the blocks to disk.  More... | |
| dberr_t | flush_inactive_blocks (Arch_Page_Pos &cur_pos, Arch_Page_Pos end_pos) | 
| Flush all the blocks which are ready to be flushed but not flushed.  More... | |
| dberr_t | flush_active_block (Arch_Page_Pos cur_pos, bool partial_reset_block_flush) | 
| Do a partial flush of the current active block.  More... | |
| Private Attributes | |
| ib_mutex_t | m_mutex | 
| Mutex protecting concurrent start, stop operations.  More... | |
| Arch_State | m_state {ARCH_STATE_INIT} | 
| Archiver system state.  More... | |
| Arch_Grp_List | m_group_list {} | 
| List of log archive groups.  More... | |
| Arch_Page_Pos | m_last_pos {} | 
| Position where last client started archiving.  More... | |
| lsn_t | m_last_lsn {LSN_MAX} | 
| LSN when last client started archiving.  More... | |
| lsn_t | m_latest_stop_lsn {LSN_MAX} | 
| Latest LSN until where the tracked pages have been flushed.  More... | |
| lsn_t | m_latest_purged_lsn {LSN_MAX} | 
| LSN until where the groups are purged.  More... | |
| ib_mutex_t | m_oper_mutex | 
| Mutex protecting concurrent operation on data.  More... | |
| Arch_Group * | m_current_group {nullptr} | 
| Current archive group.  More... | |
| ArchPageData | m_data {} | 
| In memory data buffer.  More... | |
| Arch_Page_Pos | m_write_pos {} | 
| Position to add new page ID.  More... | |
| Arch_Page_Pos | m_reset_pos {} | 
| Position to add new reset element.  More... | |
| Arch_Page_Pos | m_request_flush_pos {} | 
| Position set to explicitly request the flush archiver to flush until this position.  More... | |
| uint64_t | m_request_blk_num_with_lsn {std::numeric_limits<uint64_t>::max()} | 
| Block number set to explicitly request the flush archiver to partially flush the current active block with reset LSN.  More... | |
| uint64_t | m_flush_blk_num_with_lsn {std::numeric_limits<uint64_t>::max()} | 
| Block number set once the flush archiver partially flushes the current active block with reset LSN.  More... | |
| Arch_Page_Pos | m_flush_pos {} | 
| Position for start flushing.  More... | |
| uint | m_last_reset_file_index {0} | 
| The index of the file the last reset belonged to.  More... | |
| Page_Arch_Client_Ctx * | m_ctx | 
| System client.  More... | |
Dirty page archive system.
| Arch_Page_Sys::Arch_Page_Sys | ( | ) | 
Constructor: Initialize elements and create mutex.
| Arch_Page_Sys::~Arch_Page_Sys | ( | ) | 
Destructor: Free memory buffer and mutexes.
| 
 | delete | 
Disable copy construction.
| 
 | inline | 
Acquire dirty page ID archiver mutex.
It synchronizes concurrent start and stop operations by multiple clients.
| 
 | inline | 
Release page ID archiver mutex.
| 
 | inline | 
Acquire dirty page ID archive operation mutex.
It synchronizes concurrent page ID write to memory buffer.
| 
 | inline | 
Release page ID archiver operatiion mutex.
| bool Arch_Page_Sys::archive | ( | bool * | wait | ) | 
Archive dirty page IDs in current group.
This interface is for archiver background task to flush page archive data to disk by calling it repeatedly over time.
| [out] | wait | true, if no more data to archive | 
| int Arch_Page_Sys::fetch_group_within_lsn_range | ( | lsn_t & | start_id, | 
| lsn_t & | stop_id, | ||
| Arch_Group ** | group | ||
| ) | 
Get the group which has tracked pages between the start_id and stop_id.
| [in,out] | start_id | start LSN from which tracked pages are required; updated to the actual start LSN used for the search | 
| [in,out] | stop_id | stop_lsn until when tracked pages are required; updated to the actual stop LSN used for the search | 
| [out] | group | group which has the required tracked pages, else nullptr. | 
| 
 | private | 
Do a partial flush of the current active block.
| [in] | cur_pos | position of block which needs to be flushed | 
| [in] | partial_reset_block_flush | true if reset block needs to be flushed | 
| void Arch_Page_Sys::flush_at_checkpoint | ( | lsn_t | checkpoint_lsn | ) | 
Flush all the unflushed inactive blocks and flush the active block if required.
| [in] | checkpoint_lsn | next checkpoint LSN | 
| 
 | private | 
Flush the blocks to disk.
| [out] | wait | true, if no more data to archive | 
| 
 | private | 
Flush all the blocks which are ready to be flushed but not flushed.
| [out] | cur_pos | position of block which needs to be flushed | 
| [in] | end_pos | position of block until which the blocks need to be flushed | 
| 
 | inline | 
| 
 | inline | 
Get the mutex protecting concurrent start, stop operations required for initialising group during recovery.
| bool Arch_Page_Sys::get_num_pages | ( | Arch_Page_Pos | start_pos, | 
| Arch_Page_Pos | stop_pos, | ||
| uint64_t & | num_pages | ||
| ) | 
Given start and stop position find number of pages tracked between them.
| [in] | start_pos | start position | 
| [in] | stop_pos | stop position | 
| [out] | num_pages | number of pages tracked between start and stop position | 
Get approximate number of tracked pages between two given LSN values.
| [in,out] | start_id | fetch archived page Ids from this LSN | 
| [in,out] | stop_id | fetch archived page Ids until this LSN | 
| [out] | num_pages | number of pages tracked between specified LSN range | 
| 
 | inline | 
| bool Arch_Page_Sys::get_pages | ( | Arch_Group * | group, | 
| Arch_Page_Pos * | read_pos, | ||
| uint | read_len, | ||
| byte * | read_buff | ||
| ) | 
Get page IDs from a specific position.
Caller must ensure that read_len doesn't exceed the block.
| [in] | group | group whose pages we're interested in | 
| [in] | read_pos | position in archived data | 
| [in] | read_len | amount of data to read | 
| [out] | read_buff | buffer to return the page IDs. | 
| int Arch_Page_Sys::get_pages | ( | MYSQL_THD | thd, | 
| Page_Track_Callback | cbk_func, | ||
| void * | cbk_ctx, | ||
| lsn_t & | start_id, | ||
| lsn_t & | stop_id, | ||
| byte * | buf, | ||
| uint | buf_len | ||
| ) | 
Get archived page Ids between two given LSN values.
Attempt to read blocks directly from in memory buffer. If overwritten, copy from archived files.
| [in] | thd | thread handle | 
| [in] | cbk_func | called repeatedly with page ID buffer | 
| [in] | cbk_ctx | callback function context | 
| [in,out] | start_id | fetch archived page Ids from this LSN | 
| [in,out] | stop_id | fetch archived page Ids until this LSN | 
| [in] | buf | buffer to fill page IDs | 
| [in] | buf_len | buffer length in bytes | 
| 
 | inline | 
Fetch the status of the page tracking system.
| [out] | status | vector of a pair of (ID, bool) where ID is the start/stop point and bool is true if the ID is a start point else false | 
| 
 | inline | 
Fetch the system client context.
| 
 | inline | 
| 
 | inline | 
Check if archiver system is active.
| 
 | private | 
Check if the gap from last reset is short.
If not many page IDs are added till last reset, we avoid taking a new reset point
| 
 | inline | 
Check if archiver system is in initial state.
| 
 | delete | 
Disable assignment.
| void Arch_Page_Sys::post_recovery_init | ( | ) | 
Set the latest stop LSN to the checkpoint LSN at the time it's called.
| void Arch_Page_Sys::print | ( | ) | 
Print information related to the archiver for debugging purposes.
Purge the archived files until the specified purge LSN.
| [in] | purge_lsn | purge lsn until where files needs to be purged | 
| 0 | if purge was successful | 
| dberr_t Arch_Page_Sys::recover | ( | ) | 
Recover the archiver system at the time of startup.
Recover information related to all the durable groups and start archiving if any group was active at the time of crash/shutdown.
| int Arch_Page_Sys::recovery_load_and_start | ( | const Arch_Recv_Group_Info & | info | ) | 
Start dirty page ID archiving during recovery.
| [in,out] | info | information related to a group required for recovery | 
| void Arch_Page_Sys::release | ( | Arch_Group * | group, | 
| bool | is_durable, | ||
| Arch_Page_Pos | start_pos | ||
| ) | 
Release the current group from client.
| [in] | group | group the client is attached to | 
| [in] | is_durable | if client needs durable archiving | 
| [in] | start_pos | start position when the client calling the release was started | 
| bool Arch_Page_Sys::save_reset_point | ( | bool | is_durable | ) | 
| 
 | inline | 
Set the state of the archiver system to read only.
| 
 | private | 
Enable tracking pages in all buffer pools.
| [in] | tracking_lsn | track pages from this LSN | 
| int Arch_Page_Sys::start | ( | Arch_Group ** | group, | 
| lsn_t * | start_lsn, | ||
| Arch_Page_Pos * | start_pos, | ||
| bool | is_durable, | ||
| bool | restart, | ||
| bool | recovery | ||
| ) | 
Start dirty page ID archiving.
If archiving is already in progress, the client is attached to current group.
| [out] | group | page archive group the client gets attached to | 
| [out] | start_lsn | start lsn for client in archived data | 
| [out] | start_pos | start position for client in archived data | 
| [in] | is_durable | true if client needs durable archiving | 
| [in] | restart | true if client is already attached to current group | 
| [in] | recovery | true if archiving is being started during recovery | 
| int Arch_Page_Sys::stop | ( | Arch_Group * | group, | 
| lsn_t * | stop_lsn, | ||
| Arch_Page_Pos * | stop_pos, | ||
| bool | is_durable | ||
| ) | 
Stop dirty page ID archiving.
If other clients are there, the client is detached from the current group.
| [in] | group | page archive group the client is attached to | 
| [out] | stop_lsn | stop lsn for client | 
| [out] | stop_pos | stop position in archived data | 
| [in] | is_durable | true if client needs durable archiving | 
| 
 | private | 
Track pages for which IO is already started.
We read the io_fix flag without holding buf_page_get_mutex(bpage), but we hold flush_state_mutex which is also taken when transitioning:
| void Arch_Page_Sys::track_page | ( | buf_page_t * | bpage, | 
| lsn_t | track_lsn, | ||
| lsn_t | frame_lsn, | ||
| bool | force | ||
| ) | 
Check and add page ID to archived data.
Check for duplicate page.
| [in] | bpage | page to track | 
| [in] | track_lsn | LSN when tracking started | 
| [in] | frame_lsn | current LSN of the page | 
| [in] | force | if true, add page ID without check | 
| void Arch_Page_Sys::update_stop_info | ( | Arch_Block * | cur_blk | ) | 
Update the stop point in all the required structures.
| [in] | cur_blk | block which needs to be updated with the stop info | 
| bool Arch_Page_Sys::wait_for_reset_info_flush | ( | uint64_t | request_block | ) | 
Wait for reset info to be flushed to disk.
| [in] | request_block | block number until which blocks need to be flushed | 
| 
 | private | 
Wait for archive system to come out of ARCH_STATE_PREPARE_IDLE.
If the system is preparing to idle, start needs to wait for it to come to idle state.
| 
 | private | 
System client.
| 
 | private | 
Current archive group.
| 
 | private | 
In memory data buffer.
| 
 | private | 
Block number set once the flush archiver partially flushes the current active block with reset LSN.
| 
 | private | 
Position for start flushing.
| 
 | private | 
List of log archive groups.
| 
 | private | 
Position where last client started archiving.
| 
 | private | 
The index of the file the last reset belonged to.
 
Latest LSN until where the tracked pages have been flushed.
| 
 | private | 
Mutex protecting concurrent start, stop operations.
| 
 | private | 
Mutex protecting concurrent operation on data.
| 
 | private | 
Block number set to explicitly request the flush archiver to partially flush the current active block with reset LSN.
| 
 | private | 
Position set to explicitly request the flush archiver to flush until this position.
| 
 | private | 
Position to add new reset element.
| 
 | private | 
Archiver system state.
| 
 | private | 
Position to add new page ID.