Commit 5d2619b6 authored by Marko Mäkelä's avatar Marko Mäkelä

MDEV-19584 Allocate recv_sys statically

There is only one InnoDB crash recovery subsystem.
Allocating recv_sys statically removes one level of pointer indirection
and makes code more readable, and removes the awkward initialization of
recv_sys->dblwr.

recv_sys_t::create(): Replaces recv_sys_init().

recv_sys_t::debug_free(): Replaces recv_sys_debug_free().

recv_sys_t::close(): Replaces recv_sys_close().

recv_sys_t::add(): Replaces recv_add_to_hash_table().

recv_sys_t::empty(): Replaces recv_sys_empty_hash().
parent 592fe954
...@@ -2667,7 +2667,7 @@ static lsn_t xtrabackup_copy_log(lsn_t start_lsn, lsn_t end_lsn, bool last) ...@@ -2667,7 +2667,7 @@ static lsn_t xtrabackup_copy_log(lsn_t start_lsn, lsn_t end_lsn, bool last)
log_block, log_block,
scanned_lsn + data_len); scanned_lsn + data_len);
recv_sys->scanned_lsn = scanned_lsn + data_len; recv_sys.scanned_lsn = scanned_lsn + data_len;
if (data_len == OS_FILE_LOG_BLOCK_SIZE) { if (data_len == OS_FILE_LOG_BLOCK_SIZE) {
/* We got a full log block. */ /* We got a full log block. */
...@@ -2719,13 +2719,13 @@ static lsn_t xtrabackup_copy_log(lsn_t start_lsn, lsn_t end_lsn, bool last) ...@@ -2719,13 +2719,13 @@ static lsn_t xtrabackup_copy_log(lsn_t start_lsn, lsn_t end_lsn, bool last)
static bool xtrabackup_copy_logfile(bool last = false) static bool xtrabackup_copy_logfile(bool last = false)
{ {
ut_a(dst_log_file != NULL); ut_a(dst_log_file != NULL);
ut_ad(recv_sys != NULL); ut_ad(recv_sys.is_initialised());
lsn_t start_lsn; lsn_t start_lsn;
lsn_t end_lsn; lsn_t end_lsn;
recv_sys->parse_start_lsn = log_copy_scanned_lsn; recv_sys.parse_start_lsn = log_copy_scanned_lsn;
recv_sys->scanned_lsn = log_copy_scanned_lsn; recv_sys.scanned_lsn = log_copy_scanned_lsn;
start_lsn = ut_uint64_align_down(log_copy_scanned_lsn, start_lsn = ut_uint64_align_down(log_copy_scanned_lsn,
OS_FILE_LOG_BLOCK_SIZE); OS_FILE_LOG_BLOCK_SIZE);
...@@ -2748,15 +2748,15 @@ static bool xtrabackup_copy_logfile(bool last = false) ...@@ -2748,15 +2748,15 @@ static bool xtrabackup_copy_logfile(bool last = false)
if (lsn == start_lsn) { if (lsn == start_lsn) {
start_lsn = 0; start_lsn = 0;
} else { } else {
mutex_enter(&recv_sys->mutex); mutex_enter(&recv_sys.mutex);
start_lsn = xtrabackup_copy_log(start_lsn, lsn, last); start_lsn = xtrabackup_copy_log(start_lsn, lsn, last);
mutex_exit(&recv_sys->mutex); mutex_exit(&recv_sys.mutex);
} }
log_mutex_exit(); log_mutex_exit();
if (!start_lsn) { if (!start_lsn) {
msg(recv_sys->found_corrupt_log msg(recv_sys.found_corrupt_log
? "xtrabackup_copy_logfile() failed: corrupt log." ? "xtrabackup_copy_logfile() failed: corrupt log."
: "xtrabackup_copy_logfile() failed."); : "xtrabackup_copy_logfile() failed.");
return true; return true;
...@@ -4071,7 +4071,7 @@ static bool xtrabackup_backup_func() ...@@ -4071,7 +4071,7 @@ static bool xtrabackup_backup_func()
ut_crc32_init(); ut_crc32_init();
crc_init(); crc_init();
recv_sys_init(); recv_sys.create();
#ifdef WITH_INNODB_DISALLOW_WRITES #ifdef WITH_INNODB_DISALLOW_WRITES
srv_allow_writes_event = os_event_create(0); srv_allow_writes_event = os_event_create(0);
...@@ -4231,7 +4231,7 @@ static bool xtrabackup_backup_func() ...@@ -4231,7 +4231,7 @@ static bool xtrabackup_backup_func()
/* copy log file by current position */ /* copy log file by current position */
log_copy_scanned_lsn = checkpoint_lsn_start; log_copy_scanned_lsn = checkpoint_lsn_start;
recv_sys->recovered_lsn = log_copy_scanned_lsn; recv_sys.recovered_lsn = log_copy_scanned_lsn;
log_optimized_ddl_op = backup_optimized_ddl_op; log_optimized_ddl_op = backup_optimized_ddl_op;
if (xtrabackup_copy_logfile()) if (xtrabackup_copy_logfile())
...@@ -5471,7 +5471,7 @@ static bool xtrabackup_prepare_func(char** argv) ...@@ -5471,7 +5471,7 @@ static bool xtrabackup_prepare_func(char** argv)
sync_check_init(); sync_check_init();
ut_d(sync_check_enable()); ut_d(sync_check_enable());
ut_crc32_init(); ut_crc32_init();
recv_sys_init(); recv_sys.create();
log_sys.create(); log_sys.create();
recv_recovery_on = true; recv_recovery_on = true;
......
...@@ -1253,10 +1253,10 @@ buf_madvise_do_dump() ...@@ -1253,10 +1253,10 @@ buf_madvise_do_dump()
srv_log_buffer_size * 2, srv_log_buffer_size * 2,
MADV_DODUMP); MADV_DODUMP);
} }
/* mirrors recv_sys_init() */ /* mirrors recv_sys_t::create() */
if (recv_sys->buf) if (recv_sys.buf)
{ {
ret+= madvise(recv_sys->buf, recv_sys->len, MADV_DODUMP); ret+= madvise(recv_sys.buf, recv_sys.len, MADV_DODUMP);
} }
buf_pool_mutex_enter_all(); buf_pool_mutex_enter_all();
...@@ -1770,7 +1770,7 @@ buf_chunk_not_freed( ...@@ -1770,7 +1770,7 @@ buf_chunk_not_freed(
== block->page.newest_modification); == block->page.newest_modification);
ut_ad(block->page.oldest_modification == 0 ut_ad(block->page.oldest_modification == 0
|| block->page.oldest_modification || block->page.oldest_modification
== recv_sys->recovered_lsn == recv_sys.recovered_lsn
|| srv_force_recovery || srv_force_recovery
== SRV_FORCE_NO_LOG_REDO); == SRV_FORCE_NO_LOG_REDO);
ut_ad(block->page.buf_fix_count == 0); ut_ad(block->page.buf_fix_count == 0);
...@@ -5571,9 +5571,9 @@ buf_page_create( ...@@ -5571,9 +5571,9 @@ buf_page_create(
mtr); mtr);
} }
mutex_exit(&recv_sys->mutex); mutex_exit(&recv_sys.mutex);
block = buf_page_get_with_no_latch(page_id, zip_size, mtr); block = buf_page_get_with_no_latch(page_id, zip_size, mtr);
mutex_enter(&recv_sys->mutex); mutex_enter(&recv_sys.mutex);
return block; return block;
} }
......
...@@ -358,7 +358,7 @@ buf_dblwr_init_or_load_pages( ...@@ -358,7 +358,7 @@ buf_dblwr_init_or_load_pages(
byte* doublewrite; byte* doublewrite;
byte* unaligned_read_buf; byte* unaligned_read_buf;
ibool reset_space_ids = FALSE; ibool reset_space_ids = FALSE;
recv_dblwr_t& recv_dblwr = recv_sys->dblwr; recv_dblwr_t& recv_dblwr = recv_sys.dblwr;
/* We do the file i/o past the buffer pool */ /* We do the file i/o past the buffer pool */
...@@ -523,7 +523,7 @@ buf_dblwr_process() ...@@ -523,7 +523,7 @@ buf_dblwr_process()
ulint page_no_dblwr = 0; ulint page_no_dblwr = 0;
byte* read_buf; byte* read_buf;
byte* unaligned_read_buf; byte* unaligned_read_buf;
recv_dblwr_t& recv_dblwr = recv_sys->dblwr; recv_dblwr_t& recv_dblwr = recv_sys.dblwr;
if (!buf_dblwr) { if (!buf_dblwr) {
return; return;
......
...@@ -3034,7 +3034,7 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*) ...@@ -3034,7 +3034,7 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*)
" See the man page of setpriority()."; " See the man page of setpriority().";
} }
/* Signal that setpriority() has been attempted. */ /* Signal that setpriority() has been attempted. */
os_event_set(recv_sys->flush_end); os_event_set(recv_sys.flush_end);
#endif /* UNIV_LINUX */ #endif /* UNIV_LINUX */
do { do {
...@@ -3042,13 +3042,13 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*) ...@@ -3042,13 +3042,13 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*)
ulint n_flushed_lru = 0; ulint n_flushed_lru = 0;
ulint n_flushed_list = 0; ulint n_flushed_list = 0;
os_event_wait(recv_sys->flush_start); os_event_wait(recv_sys.flush_start);
if (!recv_writer_thread_active) { if (!recv_writer_thread_active) {
break; break;
} }
switch (recv_sys->flush_type) { switch (recv_sys.flush_type) {
case BUF_FLUSH_LRU: case BUF_FLUSH_LRU:
/* Flush pages from end of LRU if required */ /* Flush pages from end of LRU if required */
pc_request(0, LSN_MAX); pc_request(0, LSN_MAX);
...@@ -3069,8 +3069,8 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*) ...@@ -3069,8 +3069,8 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*)
ut_ad(0); ut_ad(0);
} }
os_event_reset(recv_sys->flush_start); os_event_reset(recv_sys.flush_start);
os_event_set(recv_sys->flush_end); os_event_set(recv_sys.flush_end);
} while (recv_writer_thread_active); } while (recv_writer_thread_active);
os_event_wait(buf_flush_event); os_event_wait(buf_flush_event);
......
...@@ -4413,7 +4413,7 @@ fil_aio_wait( ...@@ -4413,7 +4413,7 @@ fil_aio_wait(
ut_ad(type.is_read()); ut_ad(type.is_read());
if (recv_recovery_is_on() && !srv_force_recovery) { if (recv_recovery_is_on() && !srv_force_recovery) {
recv_sys->found_corrupt_fs = true; recv_sys.found_corrupt_fs = true;
} }
if (fil_space_t* space = fil_space_acquire_for_io(space_id)) { if (fil_space_t* space = fil_space_acquire_for_io(space_id)) {
......
...@@ -777,7 +777,7 @@ Datafile::restore_from_doublewrite() ...@@ -777,7 +777,7 @@ Datafile::restore_from_doublewrite()
} }
/* Find if double write buffer contains page_no of given space id. */ /* Find if double write buffer contains page_no of given space id. */
const byte* page = recv_sys->dblwr.find_page(m_space_id, 0); const byte* page = recv_sys.dblwr.find_page(m_space_id, 0);
const page_id_t page_id(m_space_id, 0); const page_id_t page_id(m_space_id, 0);
if (page == NULL) { if (page == NULL) {
......
...@@ -69,17 +69,6 @@ Initiates the rollback of active transactions. */ ...@@ -69,17 +69,6 @@ Initiates the rollback of active transactions. */
void void
recv_recovery_rollback_active(void); recv_recovery_rollback_active(void);
/*===============================*/ /*===============================*/
/** Clean up after recv_sys_init() */
void
recv_sys_close();
/** Initialize the redo log recovery subsystem. */
void
recv_sys_init();
/********************************************************//**
Frees the recovery system. */
void
recv_sys_debug_free(void);
/*=====================*/
/********************************************************//** /********************************************************//**
Reset the state of the recovery system variables. */ Reset the state of the recovery system variables. */
...@@ -105,7 +94,7 @@ enum store_t { ...@@ -105,7 +94,7 @@ enum store_t {
/** Adds data from a new log block to the parsing buffer of recv_sys if /** Adds data from a new log block to the parsing buffer of recv_sys if
recv_sys->parse_start_lsn is non-zero. recv_sys.parse_start_lsn is non-zero.
@param[in] log_block log block to add @param[in] log_block log block to add
@param[in] scanned_lsn lsn of how far we were able to find @param[in] scanned_lsn lsn of how far we were able to find
data in this log block data in this log block
...@@ -201,14 +190,11 @@ struct recv_sys_t{ ...@@ -201,14 +190,11 @@ struct recv_sys_t{
buf_flush_t flush_type;/*!< type of the flush request. buf_flush_t flush_type;/*!< type of the flush request.
BUF_FLUSH_LRU: flush end of LRU, keeping free blocks. BUF_FLUSH_LRU: flush end of LRU, keeping free blocks.
BUF_FLUSH_LIST: flush all of blocks. */ BUF_FLUSH_LIST: flush all of blocks. */
ibool apply_log_recs; /** whether recv_recover_page(), invoked from buf_page_io_complete(),
/*!< this is TRUE when log rec application to should apply log records*/
pages is allowed; this flag tells the bool apply_log_recs;
i/o-handler if it should do log record /** whether recv_apply_hashed_log_recs() is running */
application */ bool apply_batch_on;
ibool apply_batch_on;
/*!< this is TRUE when a log rec application
batch is running */
byte* buf; /*!< buffer for parsing log records */ byte* buf; /*!< buffer for parsing log records */
size_t buf_size; /*!< size of buf */ size_t buf_size; /*!< size of buf */
ulint len; /*!< amount of data in buf */ ulint len; /*!< amount of data in buf */
...@@ -262,6 +248,32 @@ struct recv_sys_t{ ...@@ -262,6 +248,32 @@ struct recv_sys_t{
/** Lastly added LSN to the hash table of log records. */ /** Lastly added LSN to the hash table of log records. */
lsn_t last_stored_lsn; lsn_t last_stored_lsn;
/** Initialize the redo log recovery subsystem. */
void create();
/** Free most recovery data structures. */
void debug_free();
/** Clean up after create() */
void close();
bool is_initialised() const { return buf_size != 0; }
/** Store a redo log record for applying.
@param type record type
@param space tablespace identifier
@param page_no page number
@param body record body
@param rec_end end of record
@param lsn start LSN of the mini-transaction
@param end_lsn end LSN of the mini-transaction */
inline void add(mlog_id_t type, ulint space, ulint page_no,
byte* body, byte* rec_end, lsn_t lsn,
lsn_t end_lsn);
/** Empty a fully processed set of stored redo log records. */
inline void empty();
/** Determine whether redo log recovery progress should be reported. /** Determine whether redo log recovery progress should be reported.
@param[in] time the current time @param[in] time the current time
@return whether progress should be reported @return whether progress should be reported
...@@ -278,7 +290,7 @@ struct recv_sys_t{ ...@@ -278,7 +290,7 @@ struct recv_sys_t{
}; };
/** The recovery system */ /** The recovery system */
extern recv_sys_t* recv_sys; extern recv_sys_t recv_sys;
/** TRUE when applying redo log records during crash recovery; FALSE /** TRUE when applying redo log records during crash recovery; FALSE
otherwise. Note that this is FALSE while a background thread is otherwise. Note that this is FALSE while a background thread is
......
...@@ -1609,11 +1609,11 @@ logs_empty_and_mark_files_at_shutdown(void) ...@@ -1609,11 +1609,11 @@ logs_empty_and_mark_files_at_shutdown(void)
} else { } else {
ut_ad(!srv_dict_stats_thread_active); ut_ad(!srv_dict_stats_thread_active);
} }
if (recv_sys && recv_sys->flush_start) { if (recv_sys.flush_start) {
/* This is in case recv_writer_thread was never /* This is in case recv_writer_thread was never
started, or buf_flush_page_cleaner_coordinator started, or buf_flush_page_cleaner_coordinator
failed to notice its termination. */ failed to notice its termination. */
os_event_set(recv_sys->flush_start); os_event_set(recv_sys.flush_start);
} }
} }
#define COUNT_INTERVAL 600U #define COUNT_INTERVAL 600U
...@@ -1951,7 +1951,7 @@ void log_t::close() ...@@ -1951,7 +1951,7 @@ void log_t::close()
if (!srv_read_only_mode && srv_scrub_log) if (!srv_read_only_mode && srv_scrub_log)
os_event_destroy(log_scrub_event); os_event_destroy(log_scrub_event);
recv_sys_close(); recv_sys.close();
} }
/******************************************************//** /******************************************************//**
......
This diff is collapsed.
...@@ -100,7 +100,7 @@ mlog_parse_initial_log_record( ...@@ -100,7 +100,7 @@ mlog_parse_initial_log_record(
*type = mlog_id_t(*ptr & ~MLOG_SINGLE_REC_FLAG); *type = mlog_id_t(*ptr & ~MLOG_SINGLE_REC_FLAG);
if (UNIV_UNLIKELY(*type > MLOG_BIGGEST_TYPE if (UNIV_UNLIKELY(*type > MLOG_BIGGEST_TYPE
&& !EXTRA_CHECK_MLOG_NUMBER(*type))) { && !EXTRA_CHECK_MLOG_NUMBER(*type))) {
recv_sys->found_corrupt_log = true; recv_sys.found_corrupt_log = true;
return NULL; return NULL;
} }
...@@ -234,7 +234,7 @@ mlog_parse_nbytes( ...@@ -234,7 +234,7 @@ mlog_parse_nbytes(
break; break;
default: default:
corrupt: corrupt:
recv_sys->found_corrupt_log = true; recv_sys.found_corrupt_log = true;
ptr = NULL; ptr = NULL;
} }
...@@ -401,7 +401,7 @@ mlog_parse_string( ...@@ -401,7 +401,7 @@ mlog_parse_string(
ptr += 2; ptr += 2;
if (offset >= srv_page_size || len + offset > srv_page_size) { if (offset >= srv_page_size || len + offset > srv_page_size) {
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
...@@ -641,7 +641,7 @@ mlog_parse_index( ...@@ -641,7 +641,7 @@ mlog_parse_index(
n_core_fields = mach_read_from_2(ptr); n_core_fields = mach_read_from_2(ptr);
if (!n_core_fields || n_core_fields > n) { if (!n_core_fields || n_core_fields > n) {
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
......
...@@ -1064,7 +1064,7 @@ page_cur_parse_insert_rec( ...@@ -1064,7 +1064,7 @@ page_cur_parse_insert_rec(
if (offset >= srv_page_size) { if (offset >= srv_page_size) {
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
...@@ -1078,7 +1078,7 @@ page_cur_parse_insert_rec( ...@@ -1078,7 +1078,7 @@ page_cur_parse_insert_rec(
} }
if (end_seg_len >= srv_page_size << 1) { if (end_seg_len >= srv_page_size << 1) {
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
...@@ -2255,7 +2255,7 @@ page_cur_parse_delete_rec( ...@@ -2255,7 +2255,7 @@ page_cur_parse_delete_rec(
ptr += 2; ptr += 2;
if (UNIV_UNLIKELY(offset >= srv_page_size)) { if (UNIV_UNLIKELY(offset >= srv_page_size)) {
recv_sys->found_corrupt_log = true; recv_sys.found_corrupt_log = true;
return NULL; return NULL;
} }
......
...@@ -3841,7 +3841,7 @@ page_zip_parse_write_blob_ptr( ...@@ -3841,7 +3841,7 @@ page_zip_parse_write_blob_ptr(
|| offset >= srv_page_size || offset >= srv_page_size
|| z_offset >= srv_page_size) { || z_offset >= srv_page_size) {
corrupt: corrupt:
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
...@@ -3983,7 +3983,7 @@ page_zip_parse_write_node_ptr( ...@@ -3983,7 +3983,7 @@ page_zip_parse_write_node_ptr(
|| offset >= srv_page_size || offset >= srv_page_size
|| z_offset >= srv_page_size) { || z_offset >= srv_page_size) {
corrupt: corrupt:
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
...@@ -4198,7 +4198,7 @@ page_zip_parse_write_trx_id( ...@@ -4198,7 +4198,7 @@ page_zip_parse_write_trx_id(
|| offset >= srv_page_size || offset >= srv_page_size
|| z_offset >= srv_page_size) { || z_offset >= srv_page_size) {
corrupt: corrupt:
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
...@@ -4619,7 +4619,7 @@ page_zip_parse_write_header( ...@@ -4619,7 +4619,7 @@ page_zip_parse_write_header(
if (len == 0 || offset + len >= PAGE_DATA) { if (len == 0 || offset + len >= PAGE_DATA) {
corrupt: corrupt:
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
...@@ -4897,7 +4897,7 @@ byte* page_zip_parse_compress(const byte* ptr, const byte* end_ptr, ...@@ -4897,7 +4897,7 @@ byte* page_zip_parse_compress(const byte* ptr, const byte* end_ptr,
if (!page_zip || page_zip_get_size(page_zip) < size if (!page_zip || page_zip_get_size(page_zip) < size
|| block->page.id.page_no() < 3) { || block->page.id.page_no() < 3) {
corrupt: corrupt:
recv_sys->found_corrupt_log = TRUE; recv_sys.found_corrupt_log = TRUE;
return(NULL); return(NULL);
} }
......
...@@ -1107,11 +1107,11 @@ srv_shutdown_all_bg_threads() ...@@ -1107,11 +1107,11 @@ srv_shutdown_all_bg_threads()
ut_ad(!srv_read_only_mode); ut_ad(!srv_read_only_mode);
/* e. Exit the i/o threads */ /* e. Exit the i/o threads */
if (recv_sys->flush_start != NULL) { if (recv_sys.flush_start != NULL) {
os_event_set(recv_sys->flush_start); os_event_set(recv_sys.flush_start);
} }
if (recv_sys->flush_end != NULL) { if (recv_sys.flush_end != NULL) {
os_event_set(recv_sys->flush_end); os_event_set(recv_sys.flush_end);
} }
os_event_set(buf_flush_event); os_event_set(buf_flush_event);
...@@ -1532,7 +1532,7 @@ dberr_t srv_start(bool create_new_db) ...@@ -1532,7 +1532,7 @@ dberr_t srv_start(bool create_new_db)
#endif /* UNIV_DEBUG */ #endif /* UNIV_DEBUG */
log_sys.create(); log_sys.create();
recv_sys_init(); recv_sys.create();
lock_sys.create(srv_lock_table_size); lock_sys.create(srv_lock_table_size);
/* Create i/o-handler threads: */ /* Create i/o-handler threads: */
...@@ -1560,7 +1560,7 @@ dberr_t srv_start(bool create_new_db) ...@@ -1560,7 +1560,7 @@ dberr_t srv_start(bool create_new_db)
#ifdef UNIV_LINUX #ifdef UNIV_LINUX
/* Wait for the setpriority() call to finish. */ /* Wait for the setpriority() call to finish. */
os_event_wait(recv_sys->flush_end); os_event_wait(recv_sys.flush_end);
#endif /* UNIV_LINUX */ #endif /* UNIV_LINUX */
srv_start_state_set(SRV_START_STATE_IO); srv_start_state_set(SRV_START_STATE_IO);
} }
...@@ -1574,7 +1574,7 @@ dberr_t srv_start(bool create_new_db) ...@@ -1574,7 +1574,7 @@ dberr_t srv_start(bool create_new_db)
if (err != DB_SUCCESS) { if (err != DB_SUCCESS) {
return(srv_init_abort(DB_ERROR)); return(srv_init_abort(DB_ERROR));
} }
recv_sys_debug_free(); recv_sys.debug_free();
} }
/* Open or create the data files. */ /* Open or create the data files. */
...@@ -1867,7 +1867,7 @@ dberr_t srv_start(bool create_new_db) ...@@ -1867,7 +1867,7 @@ dberr_t srv_start(bool create_new_db)
err = recv_recovery_from_checkpoint_start(flushed_lsn); err = recv_recovery_from_checkpoint_start(flushed_lsn);
recv_sys->dblwr.pages.clear(); recv_sys.dblwr.pages.clear();
if (err != DB_SUCCESS) { if (err != DB_SUCCESS) {
return(srv_init_abort(err)); return(srv_init_abort(err));
...@@ -1899,8 +1899,8 @@ dberr_t srv_start(bool create_new_db) ...@@ -1899,8 +1899,8 @@ dberr_t srv_start(bool create_new_db)
recv_apply_hashed_log_recs(true); recv_apply_hashed_log_recs(true);
if (recv_sys->found_corrupt_log if (recv_sys.found_corrupt_log
|| recv_sys->found_corrupt_fs) { || recv_sys.found_corrupt_fs) {
return(srv_init_abort(DB_CORRUPTION)); return(srv_init_abort(DB_CORRUPTION));
} }
...@@ -2520,7 +2520,7 @@ void innodb_shutdown() ...@@ -2520,7 +2520,7 @@ void innodb_shutdown()
/* 4. Free all allocated memory */ /* 4. Free all allocated memory */
pars_lexer_close(); pars_lexer_close();
recv_sys_close(); recv_sys.close();
ut_ad(buf_pool_ptr || !srv_was_started); ut_ad(buf_pool_ptr || !srv_was_started);
if (buf_pool_ptr) { if (buf_pool_ptr) {
......
...@@ -727,9 +727,9 @@ static my_bool trx_roll_count_callback(rw_trx_hash_element_t *element, ...@@ -727,9 +727,9 @@ static my_bool trx_roll_count_callback(rw_trx_hash_element_t *element,
void trx_roll_report_progress() void trx_roll_report_progress()
{ {
ib_time_t time = ut_time(); ib_time_t time = ut_time();
mutex_enter(&recv_sys->mutex); mutex_enter(&recv_sys.mutex);
bool report = recv_sys->report(time); bool report = recv_sys.report(time);
mutex_exit(&recv_sys->mutex); mutex_exit(&recv_sys.mutex);
if (report) { if (report) {
trx_roll_count_callback_arg arg; trx_roll_count_callback_arg arg;
......
...@@ -388,7 +388,7 @@ trx_undo_parse_page_init(const byte* ptr, const byte* end_ptr, page_t* page) ...@@ -388,7 +388,7 @@ trx_undo_parse_page_init(const byte* ptr, const byte* end_ptr, page_t* page)
const ulint type = *ptr++; const ulint type = *ptr++;
if (type > TRX_UNDO_UPDATE) { if (type > TRX_UNDO_UPDATE) {
recv_sys->found_corrupt_log = true; recv_sys.found_corrupt_log = true;
} else if (page) { } else if (page) {
/* Starting with MDEV-12288 in MariaDB 10.3.1, we use /* Starting with MDEV-12288 in MariaDB 10.3.1, we use
type=0 for the combined insert/update undo log type=0 for the combined insert/update undo log
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment