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)
log_block,
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) {
/* 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)
static bool xtrabackup_copy_logfile(bool last = false)
{
ut_a(dst_log_file != NULL);
ut_ad(recv_sys != NULL);
ut_ad(recv_sys.is_initialised());
lsn_t start_lsn;
lsn_t end_lsn;
recv_sys->parse_start_lsn = log_copy_scanned_lsn;
recv_sys->scanned_lsn = log_copy_scanned_lsn;
recv_sys.parse_start_lsn = log_copy_scanned_lsn;
recv_sys.scanned_lsn = log_copy_scanned_lsn;
start_lsn = ut_uint64_align_down(log_copy_scanned_lsn,
OS_FILE_LOG_BLOCK_SIZE);
......@@ -2748,15 +2748,15 @@ static bool xtrabackup_copy_logfile(bool last = false)
if (lsn == start_lsn) {
start_lsn = 0;
} else {
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
start_lsn = xtrabackup_copy_log(start_lsn, lsn, last);
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
}
log_mutex_exit();
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.");
return true;
......@@ -4071,7 +4071,7 @@ static bool xtrabackup_backup_func()
ut_crc32_init();
crc_init();
recv_sys_init();
recv_sys.create();
#ifdef WITH_INNODB_DISALLOW_WRITES
srv_allow_writes_event = os_event_create(0);
......@@ -4231,7 +4231,7 @@ static bool xtrabackup_backup_func()
/* copy log file by current position */
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;
if (xtrabackup_copy_logfile())
......@@ -5471,7 +5471,7 @@ static bool xtrabackup_prepare_func(char** argv)
sync_check_init();
ut_d(sync_check_enable());
ut_crc32_init();
recv_sys_init();
recv_sys.create();
log_sys.create();
recv_recovery_on = true;
......
......@@ -1253,10 +1253,10 @@ buf_madvise_do_dump()
srv_log_buffer_size * 2,
MADV_DODUMP);
}
/* mirrors recv_sys_init() */
if (recv_sys->buf)
/* mirrors recv_sys_t::create() */
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();
......@@ -1770,7 +1770,7 @@ buf_chunk_not_freed(
== block->page.newest_modification);
ut_ad(block->page.oldest_modification == 0
|| block->page.oldest_modification
== recv_sys->recovered_lsn
== recv_sys.recovered_lsn
|| srv_force_recovery
== SRV_FORCE_NO_LOG_REDO);
ut_ad(block->page.buf_fix_count == 0);
......@@ -5571,9 +5571,9 @@ buf_page_create(
mtr);
}
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
block = buf_page_get_with_no_latch(page_id, zip_size, mtr);
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
return block;
}
......
......@@ -358,7 +358,7 @@ buf_dblwr_init_or_load_pages(
byte* doublewrite;
byte* unaligned_read_buf;
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 */
......@@ -523,7 +523,7 @@ buf_dblwr_process()
ulint page_no_dblwr = 0;
byte* 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) {
return;
......
......@@ -3034,7 +3034,7 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*)
" See the man page of setpriority().";
}
/* Signal that setpriority() has been attempted. */
os_event_set(recv_sys->flush_end);
os_event_set(recv_sys.flush_end);
#endif /* UNIV_LINUX */
do {
......@@ -3042,13 +3042,13 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*)
ulint n_flushed_lru = 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) {
break;
}
switch (recv_sys->flush_type) {
switch (recv_sys.flush_type) {
case BUF_FLUSH_LRU:
/* Flush pages from end of LRU if required */
pc_request(0, LSN_MAX);
......@@ -3069,8 +3069,8 @@ DECLARE_THREAD(buf_flush_page_cleaner_coordinator)(void*)
ut_ad(0);
}
os_event_reset(recv_sys->flush_start);
os_event_set(recv_sys->flush_end);
os_event_reset(recv_sys.flush_start);
os_event_set(recv_sys.flush_end);
} while (recv_writer_thread_active);
os_event_wait(buf_flush_event);
......
......@@ -4413,7 +4413,7 @@ fil_aio_wait(
ut_ad(type.is_read());
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)) {
......
......@@ -777,7 +777,7 @@ Datafile::restore_from_doublewrite()
}
/* 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);
if (page == NULL) {
......
......@@ -69,17 +69,6 @@ Initiates the rollback of active transactions. */
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. */
......@@ -105,7 +94,7 @@ enum store_t {
/** 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] scanned_lsn lsn of how far we were able to find
data in this log block
......@@ -201,14 +190,11 @@ struct recv_sys_t{
buf_flush_t flush_type;/*!< type of the flush request.
BUF_FLUSH_LRU: flush end of LRU, keeping free blocks.
BUF_FLUSH_LIST: flush all of blocks. */
ibool apply_log_recs;
/*!< this is TRUE when log rec application to
pages is allowed; this flag tells the
i/o-handler if it should do log record
application */
ibool apply_batch_on;
/*!< this is TRUE when a log rec application
batch is running */
/** whether recv_recover_page(), invoked from buf_page_io_complete(),
should apply log records*/
bool apply_log_recs;
/** whether recv_apply_hashed_log_recs() is running */
bool apply_batch_on;
byte* buf; /*!< buffer for parsing log records */
size_t buf_size; /*!< size of buf */
ulint len; /*!< amount of data in buf */
......@@ -262,6 +248,32 @@ struct recv_sys_t{
/** Lastly added LSN to the hash table of log records. */
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.
@param[in] time the current time
@return whether progress should be reported
......@@ -278,7 +290,7 @@ struct recv_sys_t{
};
/** 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
otherwise. Note that this is FALSE while a background thread is
......
......@@ -1609,11 +1609,11 @@ logs_empty_and_mark_files_at_shutdown(void)
} else {
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
started, or buf_flush_page_cleaner_coordinator
failed to notice its termination. */
os_event_set(recv_sys->flush_start);
os_event_set(recv_sys.flush_start);
}
}
#define COUNT_INTERVAL 600U
......@@ -1951,7 +1951,7 @@ void log_t::close()
if (!srv_read_only_mode && srv_scrub_log)
os_event_destroy(log_scrub_event);
recv_sys_close();
recv_sys.close();
}
/******************************************************//**
......
......@@ -66,7 +66,7 @@ this must be less than srv_page_size as it is stored in the buffer pool */
#define RECV_READ_AHEAD_AREA 32
/** The recovery system */
recv_sys_t* recv_sys;
recv_sys_t recv_sys;
/** TRUE when applying redo log records during crash recovery; FALSE
otherwise. Note that this is FALSE while a background thread is
rolling back incomplete transactions. */
......@@ -253,7 +253,7 @@ class mlog_init_t
ut_allocator<std::pair<const page_id_t, init> > >
map;
/** Map of page initialization operations.
FIXME: Merge this to recv_sys->addr_hash! */
FIXME: Merge this to recv_sys.addr_hash! */
map inits;
public:
/** Record that a page will be initialized by the redo log.
......@@ -262,7 +262,7 @@ class mlog_init_t
@param[in] lsn log sequence number */
void add(ulint space, ulint page_no, lsn_t lsn)
{
ut_ad(mutex_own(&recv_sys->mutex));
ut_ad(mutex_own(&recv_sys.mutex));
const init init = { lsn, false };
std::pair<map::iterator, bool> p = inits.insert(
map::value_type(page_id_t(space, page_no), init));
......@@ -277,17 +277,17 @@ class mlog_init_t
@param[in] page_id page id
@param[in,out] init initialize log or load log
@return the latest page initialization;
not valid after releasing recv_sys->mutex. */
not valid after releasing recv_sys.mutex. */
init& last(page_id_t page_id)
{
ut_ad(mutex_own(&recv_sys->mutex));
ut_ad(mutex_own(&recv_sys.mutex));
return inits.find(page_id)->second;
}
/** At the end of each recovery batch, reset the 'created' flags. */
void reset()
{
ut_ad(mutex_own(&recv_sys->mutex));
ut_ad(mutex_own(&recv_sys.mutex));
ut_ad(recv_no_ibuf_operations);
for (map::iterator i= inits.begin(); i != inits.end(); i++) {
i->second.created = false;
......@@ -308,7 +308,7 @@ class mlog_init_t
@param[in,out] mtr dummy mini-transaction */
void ibuf_merge(mtr_t& mtr)
{
ut_ad(mutex_own(&recv_sys->mutex));
ut_ad(mutex_own(&recv_sys.mutex));
ut_ad(!recv_no_ibuf_operations);
mtr.start();
......@@ -321,13 +321,13 @@ class mlog_init_t
i->first, 0, RW_X_LATCH, NULL,
BUF_GET_IF_IN_POOL, __FILE__, __LINE__,
&mtr, NULL)) {
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
ibuf_merge_or_delete_for_page(
block, i->first,
block->zip_size(), true);
mtr.commit();
mtr.start();
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
}
}
......@@ -351,10 +351,10 @@ static void recv_addr_trim(ulint space_id, unsigned pages, lsn_t lsn)
DBUG_LOG("ib_log",
"discarding log beyond end of tablespace "
<< page_id_t(space_id, pages) << " before LSN " << lsn);
ut_ad(mutex_own(&recv_sys->mutex));
for (ulint i = recv_sys->addr_hash->n_cells; i--; ) {
ut_ad(mutex_own(&recv_sys.mutex));
for (ulint i = recv_sys.addr_hash->n_cells; i--; ) {
hash_cell_t* const cell = hash_get_nth_cell(
recv_sys->addr_hash, i);
recv_sys.addr_hash, i);
for (recv_addr_t* addr = static_cast<recv_addr_t*>(cell->node),
*next;
addr; addr = next) {
......@@ -463,7 +463,7 @@ fil_name_process(
<< " has been found in two places: '"
<< f.name << "' and '" << name << "'."
" You must delete one of them.";
recv_sys->found_corrupt_fs = true;
recv_sys.found_corrupt_fs = true;
}
break;
......@@ -486,7 +486,7 @@ fil_name_process(
forcing recovery. */
ib::info()
<< "At LSN: " << recv_sys->recovered_lsn
<< "At LSN: " << recv_sys.recovered_lsn
<< ": unable to open file " << name
<< " for tablespace " << space_id;
}
......@@ -515,7 +515,7 @@ fil_name_process(
" disk is broken, and you cannot"
" remove the .ibd file, you can set"
" --innodb_force_recovery.";
recv_sys->found_corrupt_fs = true;
recv_sys.found_corrupt_fs = true;
break;
}
......@@ -596,7 +596,7 @@ fil_name_parse(
case MLOG_FILE_NAME:
if (corrupt) {
ib::error() << "MLOG_FILE_NAME incorrect:" << ptr;
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
break;
}
......@@ -606,7 +606,7 @@ fil_name_parse(
case MLOG_FILE_DELETE:
if (corrupt) {
ib::error() << "MLOG_FILE_DELETE incorrect:" << ptr;
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
break;
}
......@@ -619,11 +619,11 @@ fil_name_parse(
== SRV_UNDO_TABLESPACE_SIZE_IN_PAGES);
ut_a(srv_is_undo_tablespace(space_id));
compile_time_assert(
UT_ARR_SIZE(recv_sys->truncated_undo_spaces)
UT_ARR_SIZE(recv_sys.truncated_undo_spaces)
== TRX_SYS_MAX_UNDO_SPACES);
recv_sys_t::trunc& t = recv_sys->truncated_undo_spaces[
recv_sys_t::trunc& t = recv_sys.truncated_undo_spaces[
space_id - srv_undo_space_id_start];
t.lsn = recv_sys->recovered_lsn;
t.lsn = recv_sys.recovered_lsn;
t.pages = uint32_t(first_page_no);
} else if (log_file_op) {
log_file_op(space_id,
......@@ -634,7 +634,7 @@ fil_name_parse(
case MLOG_FILE_RENAME2:
if (corrupt) {
ib::error() << "MLOG_FILE_RENAME2 incorrect:" << ptr;
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
}
/* The new name follows the old name. */
......@@ -676,7 +676,7 @@ fil_name_parse(
if (corrupt) {
ib::error() << "MLOG_FILE_RENAME2 new_name incorrect:" << ptr
<< " new_name: " << new_name;
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
break;
}
......@@ -699,47 +699,48 @@ fil_name_parse(
space_id, first_page_no,
reinterpret_cast<const char*>(ptr),
reinterpret_cast<const char*>(new_name))) {
recv_sys->found_corrupt_fs = true;
recv_sys.found_corrupt_fs = true;
}
}
return(end_ptr);
}
/** Clean up after recv_sys_init() */
void
recv_sys_close()
/** Clean up after recv_sys_t::create() */
void recv_sys_t::close()
{
if (recv_sys != NULL) {
recv_sys->dblwr.pages.clear();
ut_ad(this == &recv_sys);
ut_ad(!recv_writer_thread_active);
if (recv_sys->addr_hash != NULL) {
hash_table_free(recv_sys->addr_hash);
}
if (is_initialised()) {
dblwr.pages.clear();
if (recv_sys->heap != NULL) {
mem_heap_free(recv_sys->heap);
if (addr_hash) {
hash_table_free(addr_hash);
addr_hash = NULL;
}
if (recv_sys->flush_start != NULL) {
os_event_destroy(recv_sys->flush_start);
if (heap) {
mem_heap_free(heap);
heap = NULL;
}
if (recv_sys->flush_end != NULL) {
os_event_destroy(recv_sys->flush_end);
if (flush_start) {
os_event_destroy(flush_start);
}
if (recv_sys->buf != NULL) {
ut_free_dodump(recv_sys->buf, recv_sys->buf_size);
if (flush_end) {
os_event_destroy(flush_end);
}
ut_ad(!recv_writer_thread_active);
mutex_free(&recv_sys->writer_mutex);
mutex_free(&recv_sys->mutex);
if (buf) {
ut_free_dodump(buf, buf_size);
buf = NULL;
}
ut_free(recv_sys);
recv_sys = NULL;
buf_size = 0;
mutex_free(&writer_mutex);
mutex_free(&mutex);
}
recv_spaces.clear();
......@@ -794,20 +795,20 @@ DECLARE_THREAD(recv_writer_thread)(
int64_t sig_count = os_event_reset(buf_flush_event);
os_event_wait_time_low(buf_flush_event, 100000, sig_count);
mutex_enter(&recv_sys->writer_mutex);
mutex_enter(&recv_sys.writer_mutex);
if (!recv_recovery_is_on()) {
mutex_exit(&recv_sys->writer_mutex);
mutex_exit(&recv_sys.writer_mutex);
break;
}
/* Flush pages from end of LRU if required */
os_event_reset(recv_sys->flush_end);
recv_sys->flush_type = BUF_FLUSH_LRU;
os_event_set(recv_sys->flush_start);
os_event_wait(recv_sys->flush_end);
os_event_reset(recv_sys.flush_end);
recv_sys.flush_type = BUF_FLUSH_LRU;
os_event_set(recv_sys.flush_start);
os_event_wait(recv_sys.flush_end);
mutex_exit(&recv_sys->writer_mutex);
mutex_exit(&recv_sys.writer_mutex);
}
recv_writer_thread_active = false;
......@@ -822,23 +823,26 @@ DECLARE_THREAD(recv_writer_thread)(
}
/** Initialize the redo log recovery subsystem. */
void
recv_sys_init()
void recv_sys_t::create()
{
ut_ad(recv_sys == NULL);
ut_ad(this == &recv_sys);
ut_ad(!is_initialised());
ut_ad(!flush_start);
ut_ad(!flush_end);
mutex_create(LATCH_ID_RECV_SYS, &mutex);
mutex_create(LATCH_ID_RECV_WRITER, &writer_mutex);
recv_sys = static_cast<recv_sys_t*>(ut_zalloc_nokey(sizeof(*recv_sys)));
mutex_create(LATCH_ID_RECV_SYS, &recv_sys->mutex);
mutex_create(LATCH_ID_RECV_WRITER, &recv_sys->writer_mutex);
recv_sys->heap = mem_heap_create_typed(256, MEM_HEAP_FOR_RECV_SYS);
heap = mem_heap_create_typed(256, MEM_HEAP_FOR_RECV_SYS);
if (!srv_read_only_mode) {
recv_sys->flush_start = os_event_create(0);
recv_sys->flush_end = os_event_create(0);
flush_start = os_event_create(0);
flush_end = os_event_create(0);
}
flush_type = BUF_FLUSH_LRU;
apply_log_recs = false;
apply_batch_on = false;
ulint size = buf_pool_get_curr_size();
/* Set appropriate value of recv_n_pool_free_frames. */
if (size >= 10 << 20) {
......@@ -846,58 +850,63 @@ recv_sys_init()
recv_n_pool_free_frames = 512;
}
recv_sys->buf = static_cast<byte*>(
ut_malloc_dontdump(RECV_PARSING_BUF_SIZE));
recv_sys->buf_size = RECV_PARSING_BUF_SIZE;
recv_sys->addr_hash = hash_create(size / 512);
recv_sys->progress_time = ut_time();
buf = static_cast<byte*>(ut_malloc_dontdump(RECV_PARSING_BUF_SIZE));
buf_size = RECV_PARSING_BUF_SIZE;
len = 0;
parse_start_lsn = 0;
scanned_lsn = 0;
scanned_checkpoint_no = 0;
recovered_offset = 0;
recovered_lsn = 0;
found_corrupt_log = false;
found_corrupt_fs = false;
mlog_checkpoint_lsn = 0;
addr_hash = hash_create(size / 512);
n_addrs = 0;
progress_time = ut_time();
recv_max_page_lsn = 0;
/* Call the constructor for recv_sys_t::dblwr member */
new (&recv_sys->dblwr) recv_dblwr_t();
memset(truncated_undo_spaces, 0, sizeof truncated_undo_spaces);
last_stored_lsn = 0;
}
/** Empty a fully processed hash table. */
static
void
recv_sys_empty_hash()
/** Empty a fully processed set of stored redo log records. */
inline void recv_sys_t::empty()
{
ut_ad(mutex_own(&(recv_sys->mutex)));
ut_a(recv_sys->n_addrs == 0);
ut_ad(mutex_own(&mutex));
ut_a(n_addrs == 0);
hash_table_free(recv_sys->addr_hash);
mem_heap_empty(recv_sys->heap);
hash_table_free(addr_hash);
mem_heap_empty(heap);
recv_sys->addr_hash = hash_create(buf_pool_get_curr_size() / 512);
addr_hash = hash_create(buf_pool_get_curr_size() / 512);
}
/********************************************************//**
Frees the recovery system. */
void
recv_sys_debug_free(void)
/*=====================*/
/** Free most recovery data structures. */
void recv_sys_t::debug_free()
{
mutex_enter(&(recv_sys->mutex));
ut_ad(this == &recv_sys);
ut_ad(is_initialised());
mutex_enter(&mutex);
hash_table_free(recv_sys->addr_hash);
mem_heap_free(recv_sys->heap);
ut_free_dodump(recv_sys->buf, recv_sys->buf_size);
hash_table_free(addr_hash);
mem_heap_free(heap);
ut_free_dodump(buf, buf_size);
recv_sys->buf_size = 0;
recv_sys->buf = NULL;
recv_sys->heap = NULL;
recv_sys->addr_hash = NULL;
buf = NULL;
heap = NULL;
addr_hash = NULL;
/* wake page cleaner up to progress */
if (!srv_read_only_mode) {
ut_ad(!recv_recovery_is_on());
ut_ad(!recv_writer_thread_active);
os_event_reset(buf_flush_event);
os_event_set(recv_sys->flush_start);
os_event_set(flush_start);
}
mutex_exit(&(recv_sys->mutex));
mutex_exit(&mutex);
}
/** Read a log segment to log_sys.buf.
......@@ -992,12 +1001,12 @@ bool log_t::files::read_log_seg(lsn_t* start_lsn, lsn_t end_lsn)
if (dl < LOG_BLOCK_HDR_SIZE
|| (dl != OS_FILE_LOG_BLOCK_SIZE
&& dl > log_sys.trailer_offset())) {
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
goto fail;
}
}
if (recv_sys->report(ut_time())) {
if (recv_sys.report(ut_time())) {
ib::info() << "Read redo log up to LSN=" << *start_lsn;
service_manager_extend_timeout(INNODB_EXTEND_TIMEOUT_INTERVAL,
"Read redo log up to LSN=" LSN_PF,
......@@ -1022,7 +1031,7 @@ static
void
recv_synchronize_groups()
{
const lsn_t recovered_lsn = recv_sys->recovered_lsn;
const lsn_t recovered_lsn = recv_sys.recovered_lsn;
/* Read the last recovered log block to the recovery system buffer:
the block is always incomplete */
......@@ -1181,9 +1190,9 @@ static dberr_t recv_log_format_0_recover(lsn_t lsn, bool crypt)
/* Mark the redo log for upgrading. */
srv_log_file_size = 0;
recv_sys->parse_start_lsn = recv_sys->recovered_lsn
= recv_sys->scanned_lsn
= recv_sys->mlog_checkpoint_lsn = lsn;
recv_sys.parse_start_lsn = recv_sys.recovered_lsn
= recv_sys.scanned_lsn
= recv_sys.mlog_checkpoint_lsn = lsn;
log_sys.last_checkpoint_lsn = log_sys.next_checkpoint_lsn
= log_sys.lsn = log_sys.write_lsn
= log_sys.current_flush_lsn = log_sys.flushed_to_disk_lsn
......@@ -1330,7 +1339,7 @@ recv_parse_or_apply_log_rec_body(
mtr_t* mtr)
{
ut_ad(!block == !mtr);
ut_ad(!apply || recv_sys->mlog_checkpoint_lsn != 0);
ut_ad(!apply || recv_sys.mlog_checkpoint_lsn);
switch (type) {
case MLOG_FILE_NAME:
......@@ -1350,7 +1359,7 @@ recv_parse_or_apply_log_rec_body(
case MLOG_TRUNCATE:
ib::error() << "Cannot crash-upgrade from "
"old-style TRUNCATE TABLE";
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
return NULL;
default:
break;
......@@ -1372,7 +1381,7 @@ recv_parse_or_apply_log_rec_body(
} else if (apply
&& !is_predefined_tablespace(space_id)
&& recv_spaces.find(space_id) == recv_spaces.end()) {
if (recv_sys->recovered_lsn < recv_sys->mlog_checkpoint_lsn) {
if (recv_sys.recovered_lsn < recv_sys.mlog_checkpoint_lsn) {
/* We have not seen all records between the
checkpoint and MLOG_CHECKPOINT. There should be
a MLOG_FILE_DELETE for this tablespace later. */
......@@ -1385,8 +1394,8 @@ recv_parse_or_apply_log_rec_body(
ib::error() << "Missing MLOG_FILE_NAME or MLOG_FILE_DELETE"
" for redo log record " << type << " (page "
<< space_id << ":" << page_no << ") at "
<< recv_sys->recovered_lsn << ".";
recv_sys->found_corrupt_log = true;
<< recv_sys.recovered_lsn << ".";
recv_sys.found_corrupt_log = true;
return(NULL);
} else {
parse_log:
......@@ -1727,7 +1736,7 @@ recv_parse_or_apply_log_rec_body(
ptr = const_cast<byte*>(fil_parse_write_crypt_data(ptr, end_ptr, &err));
if (err != DB_SUCCESS) {
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
}
break;
default:
......@@ -1735,7 +1744,7 @@ recv_parse_or_apply_log_rec_body(
ib::error() << "Incorrect log record type "
<< ib::hex(unsigned(type));
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
}
if (index) {
......@@ -1773,7 +1782,7 @@ recv_hash(
ulint space, /*!< in: space */
ulint page_no)/*!< in: page number */
{
return(hash_calc_hash(recv_fold(space, page_no), recv_sys->addr_hash));
return(hash_calc_hash(recv_fold(space, page_no), recv_sys.addr_hash));
}
/*********************************************************************//**
......@@ -1786,12 +1795,12 @@ recv_get_fil_addr_struct(
ulint space, /*!< in: space id */
ulint page_no)/*!< in: page number */
{
ut_ad(mutex_own(&recv_sys->mutex));
ut_ad(mutex_own(&recv_sys.mutex));
recv_addr_t* recv_addr;
for (recv_addr = static_cast<recv_addr_t*>(
HASH_GET_FIRST(recv_sys->addr_hash,
HASH_GET_FIRST(recv_sys.addr_hash,
recv_hash(space, page_no)));
recv_addr != 0;
recv_addr = static_cast<recv_addr_t*>(
......@@ -1807,26 +1816,18 @@ recv_get_fil_addr_struct(
return(NULL);
}
/*******************************************************************//**
Adds a new log record to the hash table of log records. */
static
void
recv_add_to_hash_table(
/*===================*/
mlog_id_t type, /*!< in: log record type */
ulint space, /*!< in: space id */
ulint page_no, /*!< in: page number */
byte* body, /*!< in: log record body */
byte* rec_end, /*!< in: log record end */
lsn_t start_lsn, /*!< in: start lsn of the mtr */
lsn_t end_lsn) /*!< in: end lsn of the mtr */
/** 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 recv_sys_t::add(mlog_id_t type, ulint space, ulint page_no,
byte* body, byte* rec_end, lsn_t lsn,
lsn_t end_lsn)
{
recv_t* recv;
ulint len;
recv_data_t* recv_data;
recv_data_t** prev_field;
recv_addr_t* recv_addr;
ut_ad(type != MLOG_FILE_DELETE);
ut_ad(type != MLOG_FILE_CREATE2);
ut_ad(type != MLOG_FILE_RENAME2);
......@@ -1836,21 +1837,18 @@ recv_add_to_hash_table(
ut_ad(type != MLOG_INDEX_LOAD);
ut_ad(type != MLOG_TRUNCATE);
len = ulint(rec_end - body);
recv = static_cast<recv_t*>(
mem_heap_alloc(recv_sys->heap, sizeof(recv_t)));
recv_t* recv= static_cast<recv_t*>(mem_heap_alloc(heap, sizeof *recv));
recv->type = type;
recv->len = ulint(rec_end - body);
recv->start_lsn = start_lsn;
recv->start_lsn = lsn;
recv->end_lsn = end_lsn;
recv_addr = recv_get_fil_addr_struct(space, page_no);
recv_addr_t* recv_addr = recv_get_fil_addr_struct(space, page_no);
if (recv_addr == NULL) {
recv_addr = static_cast<recv_addr_t*>(
mem_heap_alloc(recv_sys->heap, sizeof(recv_addr_t)));
mem_heap_alloc(heap, sizeof(recv_addr_t)));
recv_addr->space = space;
recv_addr->page_no = page_no;
......@@ -1858,9 +1856,9 @@ recv_add_to_hash_table(
UT_LIST_INIT(recv_addr->rec_list, &recv_t::rec_list);
HASH_INSERT(recv_addr_t, addr_hash, recv_sys->addr_hash,
HASH_INSERT(recv_addr_t, addr_hash, addr_hash,
recv_fold(space, page_no), recv_addr);
recv_sys->n_addrs++;
n_addrs++;
}
switch (type) {
......@@ -1871,38 +1869,36 @@ recv_add_to_hash_table(
ut_ad(recv_addr->state == RECV_NOT_PROCESSED
|| recv_addr->state == RECV_WILL_NOT_READ);
recv_addr->state = RECV_WILL_NOT_READ;
mlog_init.add(space, page_no, start_lsn);
mlog_init.add(space, page_no, lsn);
default:
break;
}
UT_LIST_ADD_LAST(recv_addr->rec_list, recv);
prev_field = &(recv->data);
recv_data_t** prev_field = &recv->data;
/* Store the log record body in chunks of less than srv_page_size:
recv_sys->heap grows into the buffer pool, and bigger chunks could not
heap grows into the buffer pool, and bigger chunks could not
be allocated */
while (rec_end > body) {
ulint rec_len = ulint(rec_end - body);
len = ulint(rec_end - body);
if (len > RECV_DATA_BLOCK_SIZE) {
len = RECV_DATA_BLOCK_SIZE;
if (rec_len > RECV_DATA_BLOCK_SIZE) {
rec_len = RECV_DATA_BLOCK_SIZE;
}
recv_data = static_cast<recv_data_t*>(
mem_heap_alloc(recv_sys->heap,
sizeof(recv_data_t) + len));
recv_data_t* recv_data = static_cast<recv_data_t*>(
mem_heap_alloc(heap, sizeof(recv_data_t) + rec_len));
*prev_field = recv_data;
memcpy(recv_data + 1, body, len);
memcpy(recv_data + 1, body, rec_len);
prev_field = &(recv_data->next);
prev_field = &recv_data->next;
body += len;
body += rec_len;
}
*prev_field = NULL;
......@@ -1953,8 +1949,8 @@ static void recv_recover_page(buf_block_t* block, mtr_t& mtr,
page_t* page;
page_zip_des_t* page_zip;
ut_ad(mutex_own(&recv_sys->mutex));
ut_ad(recv_sys->apply_log_recs);
ut_ad(mutex_own(&recv_sys.mutex));
ut_ad(recv_sys.apply_log_recs);
ut_ad(recv_needed_recovery);
ut_ad(recv_addr->state != RECV_BEING_PROCESSED);
ut_ad(recv_addr->state != RECV_PROCESSED);
......@@ -1969,7 +1965,7 @@ static void recv_recover_page(buf_block_t* block, mtr_t& mtr,
DBUG_LOG("ib_log", "Applying log to page " << block->page.id);
recv_addr->state = RECV_BEING_PROCESSED;
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
page = block->frame;
page_zip = buf_block_get_page_zip(block);
......@@ -2085,7 +2081,7 @@ static void recv_recover_page(buf_block_t* block, mtr_t& mtr,
ib_time_t time = ut_time();
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
if (recv_max_page_lsn < page_lsn) {
recv_max_page_lsn = page_lsn;
......@@ -2094,9 +2090,9 @@ static void recv_recover_page(buf_block_t* block, mtr_t& mtr,
ut_ad(recv_addr->state == RECV_BEING_PROCESSED);
recv_addr->state = RECV_PROCESSED;
ut_a(recv_sys->n_addrs > 0);
if (ulint n = --recv_sys->n_addrs) {
if (recv_sys->report(time)) {
ut_a(recv_sys.n_addrs > 0);
if (ulint n = --recv_sys.n_addrs) {
if (recv_sys.report(time)) {
ib::info() << "To recover: " << n << " pages from log";
service_manager_extend_timeout(
INNODB_EXTEND_TIMEOUT_INTERVAL, "To recover: " ULINTPF " pages from log", n);
......@@ -2126,8 +2122,8 @@ void recv_recover_page(buf_page_t* bpage)
__FILE__, __LINE__, &mtr);
ut_a(success);
mutex_enter(&recv_sys->mutex);
if (!recv_sys->apply_log_recs) {
mutex_enter(&recv_sys.mutex);
if (!recv_sys.apply_log_recs) {
} else if (recv_addr_t* recv_addr = recv_get_fil_addr_struct(
bpage->id.space(), bpage->id.page_no())) {
switch (recv_addr->state) {
......@@ -2142,7 +2138,7 @@ void recv_recover_page(buf_page_t* bpage)
mtr.commit();
func_exit:
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
ut_ad(mtr.has_committed());
}
......@@ -2168,10 +2164,10 @@ static void recv_read_in_area(const page_id_t page_id)
}
}
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
buf_read_recv_pages(FALSE, page_id.space(), page_nos,
ulint(p - page_nos));
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
}
/** Apply the hash table of stored log records to persistent data pages.
......@@ -2183,18 +2179,18 @@ void recv_apply_hashed_log_recs(bool last_batch)
|| srv_operation == SRV_OPERATION_RESTORE
|| srv_operation == SRV_OPERATION_RESTORE_EXPORT);
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
while (recv_sys->apply_batch_on) {
bool abort = recv_sys->found_corrupt_log;
mutex_exit(&recv_sys->mutex);
while (recv_sys.apply_batch_on) {
bool abort = recv_sys.found_corrupt_log;
mutex_exit(&recv_sys.mutex);
if (abort) {
return;
}
os_thread_sleep(500000);
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
}
ut_ad(!last_batch == log_mutex_own());
......@@ -2205,15 +2201,15 @@ void recv_apply_hashed_log_recs(bool last_batch)
ut_d(recv_no_log_write = recv_no_ibuf_operations);
if (ulint n = recv_sys->n_addrs) {
if (ulint n = recv_sys.n_addrs) {
if (!log_sys.log.subformat && !srv_force_recovery
&& srv_undo_tablespaces_open) {
ib::error() << "Recovery of separately logged"
" TRUNCATE operations is no longer supported."
" Set innodb_force_recovery=1"
" if no *trunc.log files exist";
recv_sys->found_corrupt_log = true;
mutex_exit(&recv_sys->mutex);
recv_sys.found_corrupt_log = true;
mutex_exit(&recv_sys.mutex);
return;
}
......@@ -2224,11 +2220,11 @@ void recv_apply_hashed_log_recs(bool last_batch)
sd_notifyf(0, "STATUS=%s" ULINTPF " pages from redo log",
msg, n);
}
recv_sys->apply_log_recs = TRUE;
recv_sys->apply_batch_on = TRUE;
recv_sys.apply_log_recs = true;
recv_sys.apply_batch_on = true;
for (ulint id = srv_undo_tablespaces_open; id--; ) {
recv_sys_t::trunc& t = recv_sys->truncated_undo_spaces[id];
recv_sys_t::trunc& t = recv_sys.truncated_undo_spaces[id];
if (t.lsn) {
recv_addr_trim(id + srv_undo_space_id_start, t.pages,
t.lsn);
......@@ -2237,16 +2233,16 @@ void recv_apply_hashed_log_recs(bool last_batch)
mtr_t mtr;
for (ulint i = 0; i < hash_get_n_cells(recv_sys->addr_hash); i++) {
for (ulint i = 0; i < hash_get_n_cells(recv_sys.addr_hash); i++) {
for (recv_addr_t* recv_addr = static_cast<recv_addr_t*>(
HASH_GET_FIRST(recv_sys->addr_hash, i));
HASH_GET_FIRST(recv_sys.addr_hash, i));
recv_addr;
recv_addr = static_cast<recv_addr_t*>(
HASH_GET_NEXT(addr_hash, recv_addr))) {
if (!UT_LIST_GET_LEN(recv_addr->rec_list)) {
ignore:
ut_a(recv_sys->n_addrs);
recv_sys->n_addrs--;
ut_a(recv_sys.n_addrs);
recv_sys.n_addrs--;
continue;
}
......@@ -2361,16 +2357,16 @@ void recv_apply_hashed_log_recs(bool last_batch)
/* Wait until all the pages have been processed */
while (recv_sys->n_addrs != 0) {
const bool abort = recv_sys->found_corrupt_log
|| recv_sys->found_corrupt_fs;
while (recv_sys.n_addrs != 0) {
const bool abort = recv_sys.found_corrupt_log
|| recv_sys.found_corrupt_fs;
if (recv_sys->found_corrupt_fs && !srv_force_recovery) {
if (recv_sys.found_corrupt_fs && !srv_force_recovery) {
ib::info() << "Set innodb_force_recovery=1"
" to ignore corrupted pages.";
}
mutex_exit(&(recv_sys->mutex));
mutex_exit(&(recv_sys.mutex));
if (abort) {
return;
......@@ -2378,47 +2374,47 @@ void recv_apply_hashed_log_recs(bool last_batch)
os_thread_sleep(500000);
mutex_enter(&(recv_sys->mutex));
mutex_enter(&(recv_sys.mutex));
}
if (!last_batch) {
/* Flush all the file pages to disk and invalidate them in
the buffer pool */
mutex_exit(&(recv_sys->mutex));
mutex_exit(&(recv_sys.mutex));
log_mutex_exit();
/* Stop the recv_writer thread from issuing any LRU
flush batches. */
mutex_enter(&recv_sys->writer_mutex);
mutex_enter(&recv_sys.writer_mutex);
/* Wait for any currently run batch to end. */
buf_flush_wait_LRU_batch_end();
os_event_reset(recv_sys->flush_end);
recv_sys->flush_type = BUF_FLUSH_LIST;
os_event_set(recv_sys->flush_start);
os_event_wait(recv_sys->flush_end);
os_event_reset(recv_sys.flush_end);
recv_sys.flush_type = BUF_FLUSH_LIST;
os_event_set(recv_sys.flush_start);
os_event_wait(recv_sys.flush_end);
buf_pool_invalidate();
/* Allow batches from recv_writer thread. */
mutex_exit(&recv_sys->writer_mutex);
mutex_exit(&recv_sys.writer_mutex);
log_mutex_enter();
mutex_enter(&(recv_sys->mutex));
mutex_enter(&(recv_sys.mutex));
mlog_init.reset();
} else if (!recv_no_ibuf_operations) {
/* We skipped this in buf_page_create(). */
mlog_init.ibuf_merge(mtr);
}
recv_sys->apply_log_recs = FALSE;
recv_sys->apply_batch_on = FALSE;
recv_sys.apply_log_recs = false;
recv_sys.apply_batch_on = false;
recv_sys_empty_hash();
recv_sys.empty();
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
}
/** Tries to parse a single log record.
......@@ -2464,7 +2460,7 @@ recv_parse_log_rec(
if (new_ptr != NULL) {
const lsn_t lsn = static_cast<lsn_t>(
*space) << 32 | *page_no;
ut_a(lsn == recv_sys->recovered_lsn);
ut_a(lsn == recv_sys.recovered_lsn);
}
*type = MLOG_LSN;
......@@ -2485,7 +2481,7 @@ recv_parse_log_rec(
case MLOG_CHECKPOINT | MLOG_SINGLE_REC_FLAG:
ib::error() << "Incorrect log record type "
<< ib::hex(unsigned(*ptr));
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
return(0);
}
......@@ -2515,7 +2511,7 @@ recv_parse_log_rec(
recv_spaces_t::iterator it = recv_spaces.find(*space);
ut_ad(!recv_sys->mlog_checkpoint_lsn
ut_ad(!recv_sys.mlog_checkpoint_lsn
|| *space == TRX_SYS_SPACE
|| srv_is_undo_tablespace(*space)
|| it != recv_spaces.end());
......@@ -2567,28 +2563,28 @@ recv_report_corrupt_log(
ib::error() <<
"############### CORRUPT LOG RECORD FOUND ##################";
const ulint ptr_offset = ulint(ptr - recv_sys->buf);
const ulint ptr_offset = ulint(ptr - recv_sys.buf);
ib::info() << "Log record type " << type << ", page " << space << ":"
<< page_no << ". Log parsing proceeded successfully up to "
<< recv_sys->recovered_lsn << ". Previous log record type "
<< recv_sys.recovered_lsn << ". Previous log record type "
<< recv_previous_parsed_rec_type << ", is multi "
<< recv_previous_parsed_rec_is_multi << " Recv offset "
<< ptr_offset << ", prev "
<< recv_previous_parsed_rec_offset;
ut_ad(ptr <= recv_sys->buf + recv_sys->len);
ut_ad(ptr <= recv_sys.buf + recv_sys.len);
const ulint limit = 100;
const ulint prev_offset = std::min(recv_previous_parsed_rec_offset,
ptr_offset);
const ulint before = std::min(prev_offset, limit);
const ulint after = std::min(recv_sys->len - ptr_offset, limit);
const ulint after = std::min(recv_sys.len - ptr_offset, limit);
ib::info() << "Hex dump starting " << before << " bytes before and"
" ending " << after << " bytes after the corrupted record:";
const byte* start = recv_sys->buf + prev_offset - before;
const byte* start = recv_sys.buf + prev_offset - before;
ut_print_buf(stderr, start, ulint(ptr - start) + after);
putc('\n', stderr);
......@@ -2644,12 +2640,12 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
byte* body;
ut_ad(log_mutex_own());
ut_ad(mutex_own(&recv_sys->mutex));
ut_ad(recv_sys->parse_start_lsn != 0);
ut_ad(mutex_own(&recv_sys.mutex));
ut_ad(recv_sys.parse_start_lsn != 0);
loop:
ptr = recv_sys->buf + recv_sys->recovered_offset;
ptr = recv_sys.buf + recv_sys.recovered_offset;
end_ptr = recv_sys->buf + recv_sys->len;
end_ptr = recv_sys.buf + recv_sys.len;
if (ptr == end_ptr) {
......@@ -2671,7 +2667,7 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
if (single_rec) {
/* The mtr did not modify multiple pages */
old_lsn = recv_sys->recovered_lsn;
old_lsn = recv_sys.recovered_lsn;
/* Try to parse a log record, fetching its type, space id,
page no, and a pointer to the body of the log record */
......@@ -2679,12 +2675,12 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
len = recv_parse_log_rec(&type, ptr, end_ptr, &space,
&page_no, apply, &body);
if (recv_sys->found_corrupt_log) {
if (recv_sys.found_corrupt_log) {
recv_report_corrupt_log(ptr, type, space, page_no);
return(true);
}
if (recv_sys->found_corrupt_fs) {
if (recv_sys.found_corrupt_fs) {
return(true);
}
......@@ -2694,7 +2690,7 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
new_recovered_lsn = recv_calc_lsn_on_data_add(old_lsn, len);
if (new_recovered_lsn > recv_sys->scanned_lsn) {
if (new_recovered_lsn > recv_sys.scanned_lsn) {
/* The log record filled a log block, and we require
that also the next log block should have been scanned
in */
......@@ -2703,11 +2699,11 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
}
recv_previous_parsed_rec_type = type;
recv_previous_parsed_rec_offset = recv_sys->recovered_offset;
recv_previous_parsed_rec_offset = recv_sys.recovered_offset;
recv_previous_parsed_rec_is_multi = 0;
recv_sys->recovered_offset += len;
recv_sys->recovered_lsn = new_recovered_lsn;
recv_sys.recovered_offset += len;
recv_sys.recovered_lsn = new_recovered_lsn;
switch (type) {
lsn_t lsn;
......@@ -2723,9 +2719,9 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
"MLOG_CHECKPOINT(" LSN_PF ") %s at "
LSN_PF "\n", lsn,
lsn != checkpoint_lsn ? "ignored"
: recv_sys->mlog_checkpoint_lsn
: recv_sys.mlog_checkpoint_lsn
? "reread" : "read",
recv_sys->recovered_lsn);
recv_sys.recovered_lsn);
}
DBUG_PRINT("ib_log",
......@@ -2733,18 +2729,18 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
LSN_PF,
lsn,
lsn != checkpoint_lsn ? "ignored"
: recv_sys->mlog_checkpoint_lsn
: recv_sys.mlog_checkpoint_lsn
? "reread" : "read",
recv_sys->recovered_lsn));
recv_sys.recovered_lsn));
if (lsn == checkpoint_lsn) {
if (recv_sys->mlog_checkpoint_lsn) {
ut_ad(recv_sys->mlog_checkpoint_lsn
<= recv_sys->recovered_lsn);
if (recv_sys.mlog_checkpoint_lsn) {
ut_ad(recv_sys.mlog_checkpoint_lsn
<= recv_sys.recovered_lsn);
break;
}
recv_sys->mlog_checkpoint_lsn
= recv_sys->recovered_lsn;
recv_sys.mlog_checkpoint_lsn
= recv_sys.recovered_lsn;
return(true);
}
break;
......@@ -2766,10 +2762,10 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
}
/* fall through */
case STORE_YES:
recv_add_to_hash_table(
recv_sys.add(
type, space, page_no, body,
ptr + len, old_lsn,
recv_sys->recovered_lsn);
recv_sys.recovered_lsn);
}
/* fall through */
case MLOG_INDEX_LOAD:
......@@ -2806,7 +2802,7 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
&type, ptr, end_ptr, &space, &page_no,
false, &body);
if (recv_sys->found_corrupt_log) {
if (recv_sys.found_corrupt_log) {
corrupted_log:
recv_report_corrupt_log(
ptr, type, space, page_no);
......@@ -2816,11 +2812,11 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
if (ptr == end_ptr) {
} else if (type == MLOG_CHECKPOINT
|| (*ptr & MLOG_SINGLE_REC_FLAG)) {
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
goto corrupted_log;
}
if (recv_sys->found_corrupt_fs) {
if (recv_sys.found_corrupt_fs) {
return(true);
}
......@@ -2830,7 +2826,7 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
recv_previous_parsed_rec_type = type;
recv_previous_parsed_rec_offset
= recv_sys->recovered_offset + total_len;
= recv_sys.recovered_offset + total_len;
recv_previous_parsed_rec_is_multi = 1;
/* MLOG_FILE_NAME redo log records doesn't make changes
......@@ -2843,10 +2839,10 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
if (only_mlog_file) {
new_recovered_lsn = recv_calc_lsn_on_data_add(
recv_sys->recovered_lsn, len);
recv_sys.recovered_lsn, len);
mlog_rec_len += len;
recv_sys->recovered_offset += len;
recv_sys->recovered_lsn = new_recovered_lsn;
recv_sys.recovered_offset += len;
recv_sys.recovered_lsn = new_recovered_lsn;
}
total_len += len;
......@@ -2860,7 +2856,7 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
": multi-log end"
" total_len " ULINTPF
" n=" ULINTPF,
recv_sys->recovered_lsn,
recv_sys.recovered_lsn,
total_len, n_recs));
total_len -= mlog_rec_len;
break;
......@@ -2870,14 +2866,14 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
("scan " LSN_PF ": multi-log rec %s"
" len " ULINTPF
" page " ULINTPF ":" ULINTPF,
recv_sys->recovered_lsn,
recv_sys.recovered_lsn,
get_mlog_string(type), len, space, page_no));
}
new_recovered_lsn = recv_calc_lsn_on_data_add(
recv_sys->recovered_lsn, total_len);
recv_sys.recovered_lsn, total_len);
if (new_recovered_lsn > recv_sys->scanned_lsn) {
if (new_recovered_lsn > recv_sys.scanned_lsn) {
/* The log record filled a log block, and we require
that also the next log block should have been scanned
in */
......@@ -2887,10 +2883,10 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
/* Add all the records to the hash table */
ptr = recv_sys->buf + recv_sys->recovered_offset;
ptr = recv_sys.buf + recv_sys.recovered_offset;
for (;;) {
old_lsn = recv_sys->recovered_lsn;
old_lsn = recv_sys.recovered_lsn;
/* This will apply MLOG_FILE_ records. We
had to skip them in the first scan, because we
did not know if the mini-transaction was
......@@ -2899,21 +2895,21 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
&type, ptr, end_ptr, &space, &page_no,
apply, &body);
if (recv_sys->found_corrupt_log
if (recv_sys.found_corrupt_log
&& !recv_report_corrupt_log(
ptr, type, space, page_no)) {
return(true);
}
if (recv_sys->found_corrupt_fs) {
if (recv_sys.found_corrupt_fs) {
return(true);
}
ut_a(len != 0);
ut_a(!(*ptr & MLOG_SINGLE_REC_FLAG));
recv_sys->recovered_offset += len;
recv_sys->recovered_lsn
recv_sys.recovered_offset += len;
recv_sys.recovered_lsn
= recv_calc_lsn_on_data_add(old_lsn, len);
switch (type) {
......@@ -2950,7 +2946,7 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
}
/* fall through */
case STORE_YES:
recv_add_to_hash_table(
recv_sys.add(
type, space, page_no,
body, ptr + len,
old_lsn,
......@@ -2966,7 +2962,7 @@ bool recv_parse_log_recs(lsn_t checkpoint_lsn, store_t store, bool apply)
}
/** 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] scanned_lsn lsn of how far we were able to find
data in this log block
......@@ -2978,9 +2974,9 @@ bool recv_sys_add_to_parsing_buf(const byte* log_block, lsn_t scanned_lsn)
ulint start_offset;
ulint end_offset;
ut_ad(scanned_lsn >= recv_sys->scanned_lsn);
ut_ad(scanned_lsn >= recv_sys.scanned_lsn);
if (!recv_sys->parse_start_lsn) {
if (!recv_sys.parse_start_lsn) {
/* Cannot start parsing yet because no start point for
it found */
......@@ -2989,18 +2985,18 @@ bool recv_sys_add_to_parsing_buf(const byte* log_block, lsn_t scanned_lsn)
data_len = log_block_get_data_len(log_block);
if (recv_sys->parse_start_lsn >= scanned_lsn) {
if (recv_sys.parse_start_lsn >= scanned_lsn) {
return(false);
} else if (recv_sys->scanned_lsn >= scanned_lsn) {
} else if (recv_sys.scanned_lsn >= scanned_lsn) {
return(false);
} else if (recv_sys->parse_start_lsn > recv_sys->scanned_lsn) {
more_len = (ulint) (scanned_lsn - recv_sys->parse_start_lsn);
} else if (recv_sys.parse_start_lsn > recv_sys.scanned_lsn) {
more_len = (ulint) (scanned_lsn - recv_sys.parse_start_lsn);
} else {
more_len = (ulint) (scanned_lsn - recv_sys->scanned_lsn);
more_len = (ulint) (scanned_lsn - recv_sys.scanned_lsn);
}
if (more_len == 0) {
......@@ -3021,12 +3017,12 @@ bool recv_sys_add_to_parsing_buf(const byte* log_block, lsn_t scanned_lsn)
ut_ad(start_offset <= end_offset);
if (start_offset < end_offset) {
ut_memcpy(recv_sys->buf + recv_sys->len,
ut_memcpy(recv_sys.buf + recv_sys.len,
log_block + start_offset, end_offset - start_offset);
recv_sys->len += end_offset - start_offset;
recv_sys.len += end_offset - start_offset;
ut_a(recv_sys->len <= RECV_PARSING_BUF_SIZE);
ut_a(recv_sys.len <= RECV_PARSING_BUF_SIZE);
}
return(true);
......@@ -3035,12 +3031,12 @@ bool recv_sys_add_to_parsing_buf(const byte* log_block, lsn_t scanned_lsn)
/** Moves the parsing buffer data left to the buffer start. */
void recv_sys_justify_left_parsing_buf()
{
ut_memmove(recv_sys->buf, recv_sys->buf + recv_sys->recovered_offset,
recv_sys->len - recv_sys->recovered_offset);
ut_memmove(recv_sys.buf, recv_sys.buf + recv_sys.recovered_offset,
recv_sys.len - recv_sys.recovered_offset);
recv_sys->len -= recv_sys->recovered_offset;
recv_sys.len -= recv_sys.recovered_offset;
recv_sys->recovered_offset = 0;
recv_sys.recovered_offset = 0;
}
/** Scan redo log from a buffer and stores new log data to the parsing buffer.
......@@ -3071,7 +3067,7 @@ recv_scan_log_recs(
bool finished = false;
ulint data_len;
bool more_data = false;
bool apply = recv_sys->mlog_checkpoint_lsn != 0;
bool apply = recv_sys.mlog_checkpoint_lsn != 0;
ulint recv_parsing_buf_size = RECV_PARSING_BUF_SIZE;
ut_ad(start_lsn % OS_FILE_LOG_BLOCK_SIZE == 0);
......@@ -3099,10 +3095,10 @@ recv_scan_log_recs(
data_len = log_block_get_data_len(log_block);
if (scanned_lsn + data_len > recv_sys->scanned_lsn
if (scanned_lsn + data_len > recv_sys.scanned_lsn
&& log_block_get_checkpoint_no(log_block)
< recv_sys->scanned_checkpoint_no
&& (recv_sys->scanned_checkpoint_no
< recv_sys.scanned_checkpoint_no
&& (recv_sys.scanned_checkpoint_no
- log_block_get_checkpoint_no(log_block)
> 0x80000000UL)) {
......@@ -3112,16 +3108,16 @@ recv_scan_log_recs(
break;
}
if (!recv_sys->parse_start_lsn
if (!recv_sys.parse_start_lsn
&& (log_block_get_first_rec_group(log_block) > 0)) {
/* We found a point from which to start the parsing
of log records */
recv_sys->parse_start_lsn = scanned_lsn
recv_sys.parse_start_lsn = scanned_lsn
+ log_block_get_first_rec_group(log_block);
recv_sys->scanned_lsn = recv_sys->parse_start_lsn;
recv_sys->recovered_lsn = recv_sys->parse_start_lsn;
recv_sys.scanned_lsn = recv_sys.parse_start_lsn;
recv_sys.recovered_lsn = recv_sys.parse_start_lsn;
}
scanned_lsn += data_len;
......@@ -3132,17 +3128,17 @@ recv_scan_log_recs(
&& checkpoint_lsn == mach_read_from_8(LOG_BLOCK_HDR_SIZE
+ 1 + log_block)) {
/* The redo log is logically empty. */
ut_ad(recv_sys->mlog_checkpoint_lsn == 0
|| recv_sys->mlog_checkpoint_lsn
ut_ad(recv_sys.mlog_checkpoint_lsn == 0
|| recv_sys.mlog_checkpoint_lsn
== checkpoint_lsn);
recv_sys->mlog_checkpoint_lsn = checkpoint_lsn;
recv_sys.mlog_checkpoint_lsn = checkpoint_lsn;
DBUG_PRINT("ib_log", ("found empty log; LSN=" LSN_PF,
scanned_lsn));
finished = true;
break;
}
if (scanned_lsn > recv_sys->scanned_lsn) {
if (scanned_lsn > recv_sys.scanned_lsn) {
ut_ad(!srv_log_files_created);
if (!recv_needed_recovery) {
recv_needed_recovery = true;
......@@ -3155,7 +3151,7 @@ recv_scan_log_recs(
ib::info() << "Starting crash recovery from"
" checkpoint LSN="
<< recv_sys->scanned_lsn;
<< recv_sys.scanned_lsn;
}
/* We were able to find more log data: add it to the
......@@ -3168,12 +3164,12 @@ recv_scan_log_recs(
= (70 * 1024);
);
if (recv_sys->len + 4 * OS_FILE_LOG_BLOCK_SIZE
if (recv_sys.len + 4 * OS_FILE_LOG_BLOCK_SIZE
>= recv_parsing_buf_size) {
ib::error() << "Log parsing buffer overflow."
" Recovery may have failed!";
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
if (!srv_force_recovery) {
ib::error()
......@@ -3181,13 +3177,13 @@ recv_scan_log_recs(
" to ignore this error.";
return(true);
}
} else if (!recv_sys->found_corrupt_log) {
} else if (!recv_sys.found_corrupt_log) {
more_data = recv_sys_add_to_parsing_buf(
log_block, scanned_lsn);
}
recv_sys->scanned_lsn = scanned_lsn;
recv_sys->scanned_checkpoint_no
recv_sys.scanned_lsn = scanned_lsn;
recv_sys.scanned_checkpoint_no
= log_block_get_checkpoint_no(log_block);
}
......@@ -3202,33 +3198,33 @@ recv_scan_log_recs(
*group_scanned_lsn = scanned_lsn;
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
if (more_data && !recv_sys->found_corrupt_log) {
if (more_data && !recv_sys.found_corrupt_log) {
/* Try to parse more log records */
if (recv_parse_log_recs(checkpoint_lsn,
*store_to_hash, apply)) {
ut_ad(recv_sys->found_corrupt_log
|| recv_sys->found_corrupt_fs
|| recv_sys->mlog_checkpoint_lsn
== recv_sys->recovered_lsn);
ut_ad(recv_sys.found_corrupt_log
|| recv_sys.found_corrupt_fs
|| recv_sys.mlog_checkpoint_lsn
== recv_sys.recovered_lsn);
finished = true;
goto func_exit;
}
if (*store_to_hash != STORE_NO
&& mem_heap_get_size(recv_sys->heap) > available_memory) {
&& mem_heap_get_size(recv_sys.heap) > available_memory) {
DBUG_PRINT("ib_log", ("Ran out of memory and last "
"stored lsn " LSN_PF,
recv_sys->recovered_lsn));
recv_sys.recovered_lsn));
recv_sys->last_stored_lsn = recv_sys->recovered_lsn;
recv_sys.last_stored_lsn = recv_sys.recovered_lsn;
*store_to_hash = STORE_NO;
}
if (recv_sys->recovered_offset > recv_parsing_buf_size / 4) {
if (recv_sys.recovered_offset > recv_parsing_buf_size / 4) {
/* Move parsing buffer data to the buffer start */
recv_sys_justify_left_parsing_buf();
......@@ -3236,7 +3232,7 @@ recv_scan_log_recs(
}
func_exit:
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
return(finished);
}
......@@ -3256,28 +3252,28 @@ recv_group_scan_log_recs(
bool last_phase)
{
DBUG_ENTER("recv_group_scan_log_recs");
DBUG_ASSERT(!last_phase || recv_sys->mlog_checkpoint_lsn > 0);
DBUG_ASSERT(!last_phase || recv_sys.mlog_checkpoint_lsn > 0);
mutex_enter(&recv_sys->mutex);
recv_sys->len = 0;
recv_sys->recovered_offset = 0;
recv_sys->n_addrs = 0;
recv_sys_empty_hash();
mutex_enter(&recv_sys.mutex);
recv_sys.len = 0;
recv_sys.recovered_offset = 0;
recv_sys.n_addrs = 0;
recv_sys.empty();
srv_start_lsn = *contiguous_lsn;
recv_sys->parse_start_lsn = *contiguous_lsn;
recv_sys->scanned_lsn = *contiguous_lsn;
recv_sys->recovered_lsn = *contiguous_lsn;
recv_sys->scanned_checkpoint_no = 0;
recv_sys.parse_start_lsn = *contiguous_lsn;
recv_sys.scanned_lsn = *contiguous_lsn;
recv_sys.recovered_lsn = *contiguous_lsn;
recv_sys.scanned_checkpoint_no = 0;
recv_previous_parsed_rec_type = MLOG_SINGLE_REC_FLAG;
recv_previous_parsed_rec_offset = 0;
recv_previous_parsed_rec_is_multi = 0;
ut_ad(recv_max_page_lsn == 0);
ut_ad(last_phase || !recv_writer_thread_active);
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
lsn_t start_lsn;
lsn_t end_lsn;
store_t store_to_hash = recv_sys->mlog_checkpoint_lsn == 0
store_t store_to_hash = recv_sys.mlog_checkpoint_lsn == 0
? STORE_NO : (last_phase ? STORE_IF_EXISTS : STORE_YES);
ulint available_mem = srv_page_size
* (buf_pool_get_n_pages()
......@@ -3307,7 +3303,7 @@ recv_group_scan_log_recs(
start_lsn, end_lsn,
contiguous_lsn, &log_sys.log.scanned_lsn));
if (recv_sys->found_corrupt_log || recv_sys->found_corrupt_fs) {
if (recv_sys.found_corrupt_log || recv_sys.found_corrupt_fs) {
DBUG_RETURN(false);
}
......@@ -3369,9 +3365,9 @@ recv_validate_tablespace(bool rescan, bool& missing_tablespace)
{
dberr_t err = DB_SUCCESS;
for (ulint h = 0; h < hash_get_n_cells(recv_sys->addr_hash); h++) {
for (ulint h = 0; h < hash_get_n_cells(recv_sys.addr_hash); h++) {
for (recv_addr_t* recv_addr = static_cast<recv_addr_t*>(
HASH_GET_FIRST(recv_sys->addr_hash, h));
HASH_GET_FIRST(recv_sys.addr_hash, h));
recv_addr != 0;
recv_addr = static_cast<recv_addr_t*>(
HASH_GET_NEXT(addr_hash, recv_addr))) {
......@@ -3404,7 +3400,7 @@ recv_validate_tablespace(bool rescan, bool& missing_tablespace)
return(err);
}
/* When rescan is not needed then recv_sys->addr_hash will have
/* When rescan is not needed then recv_sys.addr_hash will have
all space id belongs to redo log. If rescan is needed and
innodb_force_recovery > 0 then InnoDB can ignore missing tablespace. */
for (recv_spaces_t::iterator i = recv_spaces.begin();
......@@ -3472,7 +3468,7 @@ recv_init_crash_recovery_spaces(bool rescan, bool& missing_tablespace)
" or MLOG_FILE_DELETE"
" before MLOG_CHECKPOINT for tablespace "
<< i->first;
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
return(DB_CORRUPTION);
} else {
i->second.status = file_name_t::MISSING;
......@@ -3528,7 +3524,7 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
if (err != DB_SUCCESS) {
srv_start_lsn = recv_sys->recovered_lsn = log_sys.lsn;
srv_start_lsn = recv_sys.recovered_lsn = log_sys.lsn;
log_mutex_exit();
return(err);
}
......@@ -3543,14 +3539,14 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
/* Start reading the log from the checkpoint lsn. The variable
contiguous_lsn contains an lsn up to which the log is known to
be contiguously written. */
recv_sys->mlog_checkpoint_lsn = 0;
recv_sys.mlog_checkpoint_lsn = 0;
ut_ad(RECV_SCAN_SIZE <= srv_log_buffer_size);
const lsn_t end_lsn = mach_read_from_8(
buf + LOG_CHECKPOINT_END_LSN);
ut_ad(recv_sys->n_addrs == 0);
ut_ad(recv_sys.n_addrs == 0);
contiguous_lsn = checkpoint_lsn;
switch (log_sys.log.format) {
case 0:
......@@ -3565,7 +3561,7 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
contiguous_lsn = end_lsn;
break;
}
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
log_mutex_exit();
return(DB_ERROR);
}
......@@ -3573,21 +3569,21 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
/* Look for MLOG_CHECKPOINT. */
recv_group_scan_log_recs(checkpoint_lsn, &contiguous_lsn, false);
/* The first scan should not have stored or applied any records. */
ut_ad(recv_sys->n_addrs == 0);
ut_ad(!recv_sys->found_corrupt_fs);
ut_ad(recv_sys.n_addrs == 0);
ut_ad(!recv_sys.found_corrupt_fs);
if (srv_read_only_mode && recv_needed_recovery) {
log_mutex_exit();
return(DB_READ_ONLY);
}
if (recv_sys->found_corrupt_log && !srv_force_recovery) {
if (recv_sys.found_corrupt_log && !srv_force_recovery) {
log_mutex_exit();
ib::warn() << "Log scan aborted at LSN " << contiguous_lsn;
return(DB_ERROR);
}
if (recv_sys->mlog_checkpoint_lsn == 0) {
if (recv_sys.mlog_checkpoint_lsn == 0) {
lsn_t scan_lsn = log_sys.log.scanned_lsn;
if (!srv_read_only_mode && scan_lsn != checkpoint_lsn) {
log_mutex_exit();
......@@ -3608,8 +3604,8 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
rescan = recv_group_scan_log_recs(
checkpoint_lsn, &contiguous_lsn, false);
if ((recv_sys->found_corrupt_log && !srv_force_recovery)
|| recv_sys->found_corrupt_fs) {
if ((recv_sys.found_corrupt_log && !srv_force_recovery)
|| recv_sys.found_corrupt_fs) {
log_mutex_exit();
return(DB_ERROR);
}
......@@ -3620,7 +3616,7 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
user about recovery: */
if (flush_lsn == checkpoint_lsn + SIZE_OF_MLOG_CHECKPOINT
&& recv_sys->mlog_checkpoint_lsn == checkpoint_lsn) {
&& recv_sys.mlog_checkpoint_lsn == checkpoint_lsn) {
/* The redo log is logically empty. */
} else if (checkpoint_lsn != flush_lsn) {
ut_ad(!srv_log_files_created);
......@@ -3652,7 +3648,7 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
}
}
log_sys.lsn = recv_sys->recovered_lsn;
log_sys.lsn = recv_sys.recovered_lsn;
if (recv_needed_recovery) {
bool missing_tablespace = false;
......@@ -3675,17 +3671,17 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
DBUG_PRINT("ib_log", ("Rescan of redo log to validate "
"the missing tablespace. Scan "
"from last stored LSN " LSN_PF,
recv_sys->last_stored_lsn));
recv_sys.last_stored_lsn));
lsn_t recent_stored_lsn = recv_sys->last_stored_lsn;
lsn_t recent_stored_lsn = recv_sys.last_stored_lsn;
rescan = recv_group_scan_log_recs(
checkpoint_lsn, &recent_stored_lsn, false);
ut_ad(!recv_sys->found_corrupt_fs);
ut_ad(!recv_sys.found_corrupt_fs);
missing_tablespace = false;
err = recv_sys->found_corrupt_log
err = recv_sys.found_corrupt_log
? DB_ERROR
: recv_validate_tablespace(
rescan, missing_tablespace);
......@@ -3715,15 +3711,15 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
recv_group_scan_log_recs(
checkpoint_lsn, &contiguous_lsn, true);
if ((recv_sys->found_corrupt_log
if ((recv_sys.found_corrupt_log
&& !srv_force_recovery)
|| recv_sys->found_corrupt_fs) {
|| recv_sys.found_corrupt_fs) {
log_mutex_exit();
return(DB_ERROR);
}
}
} else {
ut_ad(!rescan || recv_sys->n_addrs == 0);
ut_ad(!rescan || recv_sys.n_addrs == 0);
}
if (log_sys.log.scanned_lsn < checkpoint_lsn
......@@ -3737,11 +3733,11 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
" database is now corrupt!";
}
if (recv_sys->recovered_lsn < checkpoint_lsn) {
if (recv_sys.recovered_lsn < checkpoint_lsn) {
log_mutex_exit();
ib::error() << "Recovered only to lsn:"
<< recv_sys->recovered_lsn << " checkpoint_lsn: " << checkpoint_lsn;
<< recv_sys.recovered_lsn << " checkpoint_lsn: " << checkpoint_lsn;
return(DB_ERROR);
}
......@@ -3752,9 +3748,9 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
recv_synchronize_groups();
if (!recv_needed_recovery) {
ut_a(checkpoint_lsn == recv_sys->recovered_lsn);
ut_a(checkpoint_lsn == recv_sys.recovered_lsn);
} else {
srv_start_lsn = recv_sys->recovered_lsn;
srv_start_lsn = recv_sys.recovered_lsn;
}
log_sys.buf_free = ulong(log_sys.lsn % OS_FILE_LOG_BLOCK_SIZE);
......@@ -3776,11 +3772,11 @@ recv_recovery_from_checkpoint_start(lsn_t flush_lsn)
log_sys.next_checkpoint_no = ++checkpoint_no;
mutex_enter(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
recv_sys->apply_log_recs = TRUE;
recv_sys.apply_log_recs = true;
mutex_exit(&recv_sys->mutex);
mutex_exit(&recv_sys.mutex);
log_mutex_exit();
......@@ -3801,7 +3797,7 @@ recv_recovery_from_checkpoint_finish(void)
required because it grabs various mutexes and we want to
ensure that when we enable sync_order_checks there is no
mutex currently held by any thread. */
mutex_enter(&recv_sys->writer_mutex);
mutex_enter(&recv_sys.writer_mutex);
/* Free the resources of the recovery system */
recv_recovery_on = false;
......@@ -3811,7 +3807,7 @@ recv_recovery_from_checkpoint_finish(void)
in progress batches to finish. */
buf_flush_wait_LRU_batch_end();
mutex_exit(&recv_sys->writer_mutex);
mutex_exit(&recv_sys.writer_mutex);
ulint count = 0;
while (recv_writer_thread_active) {
......@@ -3824,7 +3820,7 @@ recv_recovery_from_checkpoint_finish(void)
}
}
recv_sys_debug_free();
recv_sys.debug_free();
/* Free up the flush_rbt. */
buf_flush_free_flush_rbt();
......
......@@ -100,7 +100,7 @@ mlog_parse_initial_log_record(
*type = mlog_id_t(*ptr & ~MLOG_SINGLE_REC_FLAG);
if (UNIV_UNLIKELY(*type > MLOG_BIGGEST_TYPE
&& !EXTRA_CHECK_MLOG_NUMBER(*type))) {
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
return NULL;
}
......@@ -234,7 +234,7 @@ mlog_parse_nbytes(
break;
default:
corrupt:
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
ptr = NULL;
}
......@@ -401,7 +401,7 @@ mlog_parse_string(
ptr += 2;
if (offset >= srv_page_size || len + offset > srv_page_size) {
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......@@ -641,7 +641,7 @@ mlog_parse_index(
n_core_fields = mach_read_from_2(ptr);
if (!n_core_fields || n_core_fields > n) {
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......
......@@ -1064,7 +1064,7 @@ page_cur_parse_insert_rec(
if (offset >= srv_page_size) {
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......@@ -1078,7 +1078,7 @@ page_cur_parse_insert_rec(
}
if (end_seg_len >= srv_page_size << 1) {
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......@@ -2255,7 +2255,7 @@ page_cur_parse_delete_rec(
ptr += 2;
if (UNIV_UNLIKELY(offset >= srv_page_size)) {
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
return NULL;
}
......
......@@ -3841,7 +3841,7 @@ page_zip_parse_write_blob_ptr(
|| offset >= srv_page_size
|| z_offset >= srv_page_size) {
corrupt:
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......@@ -3983,7 +3983,7 @@ page_zip_parse_write_node_ptr(
|| offset >= srv_page_size
|| z_offset >= srv_page_size) {
corrupt:
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......@@ -4198,7 +4198,7 @@ page_zip_parse_write_trx_id(
|| offset >= srv_page_size
|| z_offset >= srv_page_size) {
corrupt:
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......@@ -4619,7 +4619,7 @@ page_zip_parse_write_header(
if (len == 0 || offset + len >= PAGE_DATA) {
corrupt:
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......@@ -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
|| block->page.id.page_no() < 3) {
corrupt:
recv_sys->found_corrupt_log = TRUE;
recv_sys.found_corrupt_log = TRUE;
return(NULL);
}
......
......@@ -1107,11 +1107,11 @@ srv_shutdown_all_bg_threads()
ut_ad(!srv_read_only_mode);
/* e. Exit the i/o threads */
if (recv_sys->flush_start != NULL) {
os_event_set(recv_sys->flush_start);
if (recv_sys.flush_start != NULL) {
os_event_set(recv_sys.flush_start);
}
if (recv_sys->flush_end != NULL) {
os_event_set(recv_sys->flush_end);
if (recv_sys.flush_end != NULL) {
os_event_set(recv_sys.flush_end);
}
os_event_set(buf_flush_event);
......@@ -1532,7 +1532,7 @@ dberr_t srv_start(bool create_new_db)
#endif /* UNIV_DEBUG */
log_sys.create();
recv_sys_init();
recv_sys.create();
lock_sys.create(srv_lock_table_size);
/* Create i/o-handler threads: */
......@@ -1560,7 +1560,7 @@ dberr_t srv_start(bool create_new_db)
#ifdef UNIV_LINUX
/* Wait for the setpriority() call to finish. */
os_event_wait(recv_sys->flush_end);
os_event_wait(recv_sys.flush_end);
#endif /* UNIV_LINUX */
srv_start_state_set(SRV_START_STATE_IO);
}
......@@ -1574,7 +1574,7 @@ dberr_t srv_start(bool create_new_db)
if (err != DB_SUCCESS) {
return(srv_init_abort(DB_ERROR));
}
recv_sys_debug_free();
recv_sys.debug_free();
}
/* Open or create the data files. */
......@@ -1867,7 +1867,7 @@ dberr_t srv_start(bool create_new_db)
err = recv_recovery_from_checkpoint_start(flushed_lsn);
recv_sys->dblwr.pages.clear();
recv_sys.dblwr.pages.clear();
if (err != DB_SUCCESS) {
return(srv_init_abort(err));
......@@ -1899,8 +1899,8 @@ dberr_t srv_start(bool create_new_db)
recv_apply_hashed_log_recs(true);
if (recv_sys->found_corrupt_log
|| recv_sys->found_corrupt_fs) {
if (recv_sys.found_corrupt_log
|| recv_sys.found_corrupt_fs) {
return(srv_init_abort(DB_CORRUPTION));
}
......@@ -2520,7 +2520,7 @@ void innodb_shutdown()
/* 4. Free all allocated memory */
pars_lexer_close();
recv_sys_close();
recv_sys.close();
ut_ad(buf_pool_ptr || !srv_was_started);
if (buf_pool_ptr) {
......
......@@ -727,9 +727,9 @@ static my_bool trx_roll_count_callback(rw_trx_hash_element_t *element,
void trx_roll_report_progress()
{
ib_time_t time = ut_time();
mutex_enter(&recv_sys->mutex);
bool report = recv_sys->report(time);
mutex_exit(&recv_sys->mutex);
mutex_enter(&recv_sys.mutex);
bool report = recv_sys.report(time);
mutex_exit(&recv_sys.mutex);
if (report) {
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)
const ulint type = *ptr++;
if (type > TRX_UNDO_UPDATE) {
recv_sys->found_corrupt_log = true;
recv_sys.found_corrupt_log = true;
} else if (page) {
/* Starting with MDEV-12288 in MariaDB 10.3.1, we use
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