diff options
Diffstat (limited to 'storage/innobase/trx')
-rw-r--r-- | storage/innobase/trx/trx0i_s.cc | 161 | ||||
-rw-r--r-- | storage/innobase/trx/trx0purge.cc | 262 | ||||
-rw-r--r-- | storage/innobase/trx/trx0rec.cc | 148 | ||||
-rw-r--r-- | storage/innobase/trx/trx0roll.cc | 155 | ||||
-rw-r--r-- | storage/innobase/trx/trx0rseg.cc | 186 | ||||
-rw-r--r-- | storage/innobase/trx/trx0sys.cc | 120 | ||||
-rw-r--r-- | storage/innobase/trx/trx0trx.cc | 683 | ||||
-rw-r--r-- | storage/innobase/trx/trx0undo.cc | 93 |
8 files changed, 794 insertions, 1014 deletions
diff --git a/storage/innobase/trx/trx0i_s.cc b/storage/innobase/trx/trx0i_s.cc index d043c3d86c0..1232ccc5ad9 100644 --- a/storage/innobase/trx/trx0i_s.cc +++ b/storage/innobase/trx/trx0i_s.cc @@ -41,8 +41,6 @@ Created July 17, 2007 Vasil Dimov #include "rem0rec.h" #include "row0row.h" #include "srv0srv.h" -#include "sync0rw.h" -#include "sync0sync.h" #include "trx0sys.h" #include "que0que.h" #include "trx0purge.h" @@ -139,8 +137,7 @@ struct i_s_table_cache_t { /** This structure describes the intermediate buffer */ struct trx_i_s_cache_t { - rw_lock_t rw_lock; /*!< read-write lock protecting - the rest of this structure */ + srw_lock rw_lock; /*!< read-write lock protecting this */ Atomic_relaxed<ulonglong> last_read; /*!< last time the cache was read; measured in nanoseconds */ @@ -159,7 +156,7 @@ struct trx_i_s_cache_t { ha_storage_t* storage; /*!< storage for external volatile data that may become unavailable when we release - lock_sys.mutex */ + lock_sys.latch */ ulint mem_allocd; /*!< the amount of memory allocated with mem_alloc*() */ bool is_truncated; /*!< this is true if the memory @@ -180,7 +177,7 @@ trx_i_s_cache_t* trx_i_s_cache = &trx_i_s_cache_static; @retval 0xFFFF for table locks */ static uint16_t wait_lock_get_heap_no(const lock_t *lock) { - return lock_get_type(lock) == LOCK_REC + return !lock->is_table() ? static_cast<uint16_t>(lock_rec_find_set_bit(lock)) : uint16_t{0xFFFF}; } @@ -425,23 +422,30 @@ fill_trx_row( { const char* s; - ut_ad(lock_mutex_own()); + lock_sys.assert_locked(); - row->trx_id = trx_get_id_for_print(trx); + const lock_t* wait_lock = trx->lock.wait_lock; + + row->trx_id = trx->id; row->trx_started = trx->start_time; - row->trx_state = trx_get_que_state_str(trx); + if (trx->in_rollback) { + row->trx_state = "ROLLING BACK"; + } else if (trx->state == TRX_STATE_COMMITTED_IN_MEMORY) { + row->trx_state = "COMMITTING"; + } else if (wait_lock) { + row->trx_state = "LOCK WAIT"; + } else { + row->trx_state = "RUNNING"; + } + row->requested_lock_row = requested_lock_row; ut_ad(requested_lock_row == NULL || i_s_locks_row_validate(requested_lock_row)); - if (trx->lock.wait_lock != NULL) { + ut_ad(!wait_lock == !requested_lock_row); - ut_a(requested_lock_row != NULL); - row->trx_wait_started = trx->lock.wait_started; - } else { - ut_a(requested_lock_row == NULL); - row->trx_wait_started = 0; - } + const my_hrtime_t suspend_time= trx->lock.suspend_time; + row->trx_wait_started = wait_lock ? hrtime_to_time(suspend_time) : 0; row->trx_weight = static_cast<uintmax_t>(TRX_WEIGHT(trx)); @@ -482,15 +486,14 @@ thd_done: row->trx_tables_locked = lock_number_of_tables_locked(&trx->lock); - /* These are protected by both trx->mutex or lock_sys.mutex, - or just lock_sys.mutex. For reading, it suffices to hold - lock_sys.mutex. */ + /* These are protected by lock_sys.latch (which we are holding) + and sometimes also trx->mutex. */ row->trx_lock_structs = UT_LIST_GET_LEN(trx->lock.trx_locks); row->trx_lock_memory_bytes = mem_heap_get_size(trx->lock.lock_heap); - row->trx_rows_locked = lock_number_of_rows_locked(&trx->lock); + row->trx_rows_locked = trx->lock.n_rec_locks; row->trx_rows_modified = trx->undo_no; @@ -596,7 +599,7 @@ fill_lock_data( trx_i_s_cache_t* cache) /*!< in/out: cache where to store volatile data */ { - ut_a(lock_get_type(lock) == LOCK_REC); + ut_a(!lock->is_table()); switch (heap_no) { case PAGE_HEAP_NO_INFIMUM: @@ -615,7 +618,6 @@ fill_lock_data( const buf_block_t* block; const page_t* page; const rec_t* rec; - const dict_index_t* index; ulint n_fields; mem_heap_t* heap; rec_offs offsets_onstack[REC_OFFS_NORMAL_SIZE]; @@ -644,7 +646,8 @@ fill_lock_data( rec = page_find_rec_with_heap_no(page, heap_no); - index = lock_rec_get_index(lock); + const dict_index_t* index = lock->index; + ut_ad(index->is_primary() || !dict_index_is_online_ddl(index)); n_fields = dict_index_get_n_unique(index); @@ -687,6 +690,15 @@ fill_lock_data( return(TRUE); } +/** @return the table of a lock */ +static const dict_table_t *lock_get_table(const lock_t &lock) +{ + if (lock.is_table()) + return lock.un_member.tab_lock.table; + ut_ad(lock.index->is_primary() || !dict_index_is_online_ddl(lock.index)); + return lock.index->table; +} + /*******************************************************************//** Fills i_s_locks_row_t object. Returns its first argument. If memory can not be allocated then FALSE is returned. @@ -701,12 +713,9 @@ static bool fill_locks_row( volatile strings */ { row->lock_trx_id = lock->trx->id; - const auto lock_type = lock_get_type(lock); - ut_ad(lock_type == LOCK_REC || lock_type == LOCK_TABLE); - - const bool is_gap_lock = lock_type == LOCK_REC - && (lock->type_mode & LOCK_GAP); - switch (lock->type_mode & LOCK_MODE_MASK) { + const bool is_gap_lock = lock->is_gap(); + ut_ad(!is_gap_lock || !lock->is_table()); + switch (lock->mode()) { case LOCK_S: row->lock_mode = uint8_t(1 + is_gap_lock); break; @@ -727,8 +736,10 @@ static bool fill_locks_row( row->lock_mode = 0; } + const dict_table_t* table= lock_get_table(*lock); + row->lock_table = ha_storage_put_str_memlim( - cache->storage, lock_get_table_name(lock).m_name, + cache->storage, table->name.m_name, MAX_ALLOWED_FOR_STORAGE(cache)); /* memory could not be allocated */ @@ -737,9 +748,9 @@ static bool fill_locks_row( return false; } - if (lock_type == LOCK_REC) { + if (!lock->is_table()) { row->lock_index = ha_storage_put_str_memlim( - cache->storage, lock_rec_get_index_name(lock), + cache->storage, lock->index->name, MAX_ALLOWED_FOR_STORAGE(cache)); /* memory could not be allocated */ @@ -765,7 +776,7 @@ static bool fill_locks_row( row->lock_data = NULL; } - row->lock_table_id = lock_get_table_id(lock); + row->lock_table_id = table->id; row->hash_chain.value = row; ut_ad(i_s_locks_row_validate(row)); @@ -820,26 +831,19 @@ fold_lock( #else ulint ret; - switch (lock_get_type(lock)) { - case LOCK_REC: + if (!lock->is_table()) { ut_a(heap_no != 0xFFFF); ret = ut_fold_ulint_pair((ulint) lock->trx->id, lock->un_member.rec_lock.page_id. fold()); ret = ut_fold_ulint_pair(ret, heap_no); - - break; - case LOCK_TABLE: + } else { /* this check is actually not necessary for continuing correct operation, but something must have gone wrong if it fails. */ ut_a(heap_no == 0xFFFF); - ret = (ulint) lock_get_table_id(lock); - - break; - default: - ut_error; + ret = (ulint) lock_get_table(*lock)->id; } return(ret); @@ -863,26 +867,20 @@ locks_row_eq_lock( #ifdef TEST_NO_LOCKS_ROW_IS_EVER_EQUAL_TO_LOCK_T return(0); #else - switch (lock_get_type(lock)) { - case LOCK_REC: + if (!lock->is_table()) { ut_a(heap_no != 0xFFFF); return(row->lock_trx_id == lock->trx->id && row->lock_page == lock->un_member.rec_lock.page_id && row->lock_rec == heap_no); - - case LOCK_TABLE: + } else { /* this check is actually not necessary for continuing correct operation, but something must have gone wrong if it fails. */ ut_a(heap_no == 0xFFFF); return(row->lock_trx_id == lock->trx->id - && row->lock_table_id == lock_get_table_id(lock)); - - default: - ut_error; - return(FALSE); + && row->lock_table_id == lock_get_table(*lock)->id); } #endif } @@ -1049,25 +1047,22 @@ add_trx_relevant_locks_to_cache( requested lock row, or NULL or undefined */ { - ut_ad(lock_mutex_own()); + lock_sys.assert_locked(); /* If transaction is waiting we add the wait lock and all locks from another transactions that are blocking the wait lock. */ - if (trx->lock.que_state == TRX_QUE_LOCK_WAIT) { + if (const lock_t *wait_lock = trx->lock.wait_lock) { const lock_t* curr_lock; i_s_locks_row_t* blocking_lock_row; lock_queue_iterator_t iter; - ut_a(trx->lock.wait_lock != NULL); - uint16_t wait_lock_heap_no - = wait_lock_get_heap_no(trx->lock.wait_lock); + = wait_lock_get_heap_no(wait_lock); /* add the requested lock */ - *requested_lock_row - = add_lock_to_cache(cache, trx->lock.wait_lock, - wait_lock_heap_no); + *requested_lock_row = add_lock_to_cache(cache, wait_lock, + wait_lock_heap_no); /* memory could not be allocated */ if (*requested_lock_row == NULL) { @@ -1078,18 +1073,16 @@ add_trx_relevant_locks_to_cache( /* then iterate over the locks before the wait lock and add the ones that are blocking it */ - lock_queue_iterator_reset(&iter, trx->lock.wait_lock, - ULINT_UNDEFINED); + lock_queue_iterator_reset(&iter, wait_lock, ULINT_UNDEFINED); for (curr_lock = lock_queue_iterator_get_prev(&iter); curr_lock != NULL; curr_lock = lock_queue_iterator_get_prev(&iter)) { - if (lock_has_to_wait(trx->lock.wait_lock, - curr_lock)) { + if (lock_has_to_wait(wait_lock, curr_lock)) { /* add the lock that is - blocking trx->lock.wait_lock */ + blocking wait_lock */ blocking_lock_row = add_lock_to_cache( cache, curr_lock, @@ -1139,9 +1132,6 @@ static bool can_cache_be_updated(trx_i_s_cache_t* cache) we are currently holding an exclusive rw lock on the cache. So it is not possible for last_read to be updated while we are reading it. */ - - ut_ad(rw_lock_own(&cache->rw_lock, RW_LOCK_X)); - return my_interval_timer() - cache->last_read > CACHE_MIN_IDLE_TIME_NS; } @@ -1217,7 +1207,7 @@ static void fetch_data_into_cache_low(trx_i_s_cache_t *cache, const trx_t *trx) static void fetch_data_into_cache(trx_i_s_cache_t *cache) { - ut_ad(lock_mutex_own()); + LockMutexGuard g{SRW_LOCK_CALL}; trx_i_s_cache_clear(cache); /* Capture the state of transactions */ @@ -1225,10 +1215,10 @@ static void fetch_data_into_cache(trx_i_s_cache_t *cache) if (!cache->is_truncated && trx.state != TRX_STATE_NOT_STARTED && &trx != purge_sys.query->trx) { - mutex_enter(&trx.mutex); + trx.mutex_lock(); if (trx.state != TRX_STATE_NOT_STARTED) fetch_data_into_cache_low(cache, &trx); - mutex_exit(&trx.mutex); + trx.mutex_unlock(); } }); cache->is_truncated= false; @@ -1250,10 +1240,7 @@ trx_i_s_possibly_fetch_data_into_cache( } /* We need to read trx_sys and record/table lock queues */ - - lock_mutex_enter(); fetch_data_into_cache(cache); - lock_mutex_exit(); /* update cache last read time */ cache->last_read = my_interval_timer(); @@ -1281,15 +1268,14 @@ trx_i_s_cache_init( trx_i_s_cache_t* cache) /*!< out: cache to init */ { /* The latching is done in the following order: - acquire trx_i_s_cache_t::rw_lock, X - acquire lock mutex - release lock mutex + acquire trx_i_s_cache_t::rw_lock, rwlock + acquire exclusive lock_sys.latch + release exclusive lock_sys.latch release trx_i_s_cache_t::rw_lock - acquire trx_i_s_cache_t::rw_lock, S + acquire trx_i_s_cache_t::rw_lock, rdlock release trx_i_s_cache_t::rw_lock */ - rw_lock_create(trx_i_s_cache_lock_key, &cache->rw_lock, - SYNC_TRX_I_S_RWLOCK); + cache->rw_lock.SRW_LOCK_INIT(trx_i_s_cache_lock_key); cache->last_read = 0; @@ -1315,7 +1301,7 @@ trx_i_s_cache_free( /*===============*/ trx_i_s_cache_t* cache) /*!< in, own: cache to free */ { - rw_lock_free(&cache->rw_lock); + cache->rw_lock.destroy(); cache->locks_hash.free(); ha_storage_free(cache->storage); @@ -1331,7 +1317,7 @@ trx_i_s_cache_start_read( /*=====================*/ trx_i_s_cache_t* cache) /*!< in: cache */ { - rw_lock_s_lock(&cache->rw_lock); + cache->rw_lock.rd_lock(SRW_LOCK_CALL); } /*******************************************************************//** @@ -1342,7 +1328,7 @@ trx_i_s_cache_end_read( trx_i_s_cache_t* cache) /*!< in: cache */ { cache->last_read = my_interval_timer(); - rw_lock_s_unlock(&cache->rw_lock); + cache->rw_lock.rd_unlock(); } /*******************************************************************//** @@ -1352,7 +1338,7 @@ trx_i_s_cache_start_write( /*======================*/ trx_i_s_cache_t* cache) /*!< in: cache */ { - rw_lock_x_lock(&cache->rw_lock); + cache->rw_lock.wr_lock(SRW_LOCK_CALL); } /*******************************************************************//** @@ -1362,9 +1348,7 @@ trx_i_s_cache_end_write( /*====================*/ trx_i_s_cache_t* cache) /*!< in: cache */ { - ut_ad(rw_lock_own(&cache->rw_lock, RW_LOCK_X)); - - rw_lock_x_unlock(&cache->rw_lock); + cache->rw_lock.wr_unlock(); } /*******************************************************************//** @@ -1377,9 +1361,6 @@ cache_select_table( trx_i_s_cache_t* cache, /*!< in: whole cache */ enum i_s_table table) /*!< in: which table */ { - ut_ad(rw_lock_own_flagged(&cache->rw_lock, - RW_LOCK_FLAG_X | RW_LOCK_FLAG_S)); - switch (table) { case I_S_INNODB_TRX: return &cache->innodb_trx; diff --git a/storage/innobase/trx/trx0purge.cc b/storage/innobase/trx/trx0purge.cc index 4542e391dca..54919c260c8 100644 --- a/storage/innobase/trx/trx0purge.cc +++ b/storage/innobase/trx/trx0purge.cc @@ -36,7 +36,6 @@ Created 3/26/1996 Heikki Tuuri #include "srv0mon.h" #include "srv0srv.h" #include "srv0start.h" -#include "sync0sync.h" #include "trx0rec.h" #include "trx0roll.h" #include "trx0rseg.h" @@ -45,6 +44,10 @@ Created 3/26/1996 Heikki Tuuri #include <unordered_map> +#ifdef UNIV_PFS_RWLOCK +extern mysql_pfs_key_t trx_purge_latch_key; +#endif /* UNIV_PFS_RWLOCK */ + /** Maximum allowable purge history length. <=0 means 'infinite'. */ ulong srv_max_purge_lag = 0; @@ -76,7 +79,7 @@ Executed in the purge coordinator thread. @return whether anything is to be purged */ inline bool TrxUndoRsegsIterator::set_next() { - mutex_enter(&purge_sys.pq_mutex); + mysql_mutex_lock(&purge_sys.pq_mutex); /* Only purge consumes events from the priority queue, user threads only produce the events. */ @@ -99,15 +102,15 @@ inline bool TrxUndoRsegsIterator::set_next() } else { /* Queue is empty, reset iterator. */ purge_sys.rseg = NULL; - mutex_exit(&purge_sys.pq_mutex); + mysql_mutex_unlock(&purge_sys.pq_mutex); m_rsegs = NullElement; m_iter = m_rsegs.begin(); return false; } purge_sys.rseg = *m_iter++; - mutex_exit(&purge_sys.pq_mutex); - mutex_enter(&purge_sys.rseg->mutex); + mysql_mutex_unlock(&purge_sys.pq_mutex); + purge_sys.rseg->latch.rd_lock(); ut_a(purge_sys.rseg->last_page_no != FIL_NULL); ut_ad(purge_sys.rseg->last_trx_no() == m_rsegs.trx_no); @@ -123,8 +126,7 @@ inline bool TrxUndoRsegsIterator::set_next() purge_sys.hdr_offset = purge_sys.rseg->last_offset(); purge_sys.hdr_page_no = purge_sys.rseg->last_page_no; - mutex_exit(&purge_sys.rseg->mutex); - + purge_sys.rseg->latch.rd_unlock(); return(true); } @@ -145,8 +147,7 @@ purge_graph_build() trx->op_info = "purge trx"; mem_heap_t* heap = mem_heap_create(512); - que_fork_t* fork = que_fork_create( - NULL, NULL, QUE_FORK_PURGE, heap); + que_fork_t* fork = que_fork_create(heap); fork->trx = trx; for (auto i = innodb_purge_threads_MAX; i; i--) { @@ -165,6 +166,7 @@ void purge_sys_t::create() ut_ad(!heap); ut_ad(!enabled()); m_paused= 0; + m_SYS_paused= 0; query= purge_graph_build(); next_stored= false; rseg= NULL; @@ -172,8 +174,8 @@ void purge_sys_t::create() offset= 0; hdr_page_no= 0; hdr_offset= 0; - rw_lock_create(trx_purge_latch_key, &latch, SYNC_PURGE_LATCH); - mutex_create(LATCH_ID_PURGE_SYS_PQ, &pq_mutex); + latch.SRW_LOCK_INIT(trx_purge_latch_key); + mysql_mutex_init(purge_sys_pq_mutex_key, &pq_mutex, nullptr); truncate.current= NULL; truncate.last= NULL; heap= mem_heap_create(4096); @@ -193,8 +195,8 @@ void purge_sys_t::close() ut_ad(trx->state == TRX_STATE_ACTIVE); trx->state= TRX_STATE_NOT_STARTED; trx->free(); - rw_lock_free(&latch); - mutex_free(&pq_mutex); + latch.destroy(); + mysql_mutex_destroy(&pq_mutex); mem_heap_free(heap); heap= nullptr; } @@ -262,8 +264,8 @@ trx_purge_add_undo_to_history(const trx_t* trx, trx_undo_t*& undo, mtr_t* mtr) Before any transaction-generating background threads or the purge have been started, we can - start transactions in row_merge_drop_temp_indexes() and - fts_drop_orphaned_tables(), and roll back recovered transactions. + start transactions in row_merge_drop_temp_indexes(), + and roll back recovered transactions. Arbitrary user transactions may be executed when all the undo log related background processes (including purge) are disabled due to @@ -278,12 +280,11 @@ trx_purge_add_undo_to_history(const trx_t* trx, trx_undo_t*& undo, mtr_t* mtr) && (srv_is_being_started || trx_rollback_is_active || srv_force_recovery >= SRV_FORCE_NO_BACKGROUND)) - || ((trx->mysql_thd || trx->internal) - && srv_fast_shutdown)); + || srv_fast_shutdown); #ifdef WITH_WSREP - if (wsrep_is_wsrep_xid(trx->xid)) { - trx_rseg_update_wsrep_checkpoint(rseg_header, trx->xid, mtr); + if (wsrep_is_wsrep_xid(&trx->xid)) { + trx_rseg_update_wsrep_checkpoint(rseg_header, &trx->xid, mtr); } #endif @@ -309,10 +310,10 @@ trx_purge_add_undo_to_history(const trx_t* trx, trx_undo_t*& undo, mtr_t* mtr) rseg->last_page_no = undo->hdr_page_no; rseg->set_last_commit(undo->hdr_offset, trx->rw_trx_hash_element->no); - rseg->needs_purge = true; + rseg->set_needs_purge(); } - trx_sys.rseg_history_len++; + rseg->history_size++; if (undo->state == TRX_UNDO_CACHED) { UT_LIST_ADD_FIRST(rseg->undo_cached, undo); @@ -335,24 +336,25 @@ static void trx_purge_remove_log_hdr(buf_block_t *rseg, buf_block_t* log, { flst_remove(rseg, TRX_RSEG + TRX_RSEG_HISTORY, log, static_cast<uint16_t>(offset + TRX_UNDO_HISTORY_NODE), mtr); - trx_sys.rseg_history_len--; } /** Free an undo log segment, and remove the header from the history list. @param[in,out] rseg rollback segment @param[in] hdr_addr file address of log_hdr */ -static -void -trx_purge_free_segment(trx_rseg_t* rseg, fil_addr_t hdr_addr) +static void trx_purge_free_segment(trx_rseg_t *rseg, fil_addr_t hdr_addr) { mtr_t mtr; mtr.start(); - mutex_enter(&rseg->mutex); + const page_id_t hdr_page_id(rseg->space->id, hdr_addr.page); + + /* We only need the latch to maintain rseg->curr_size. To follow the + latching order, we must acquire it before acquiring any related + page latch. */ + rseg->latch.wr_lock(); buf_block_t* rseg_hdr = trx_rsegf_get(rseg->space, rseg->page_no, &mtr); - buf_block_t* block = trx_undo_page_get( - page_id_t(rseg->space->id, hdr_addr.page), &mtr); + buf_block_t* block = trx_undo_page_get(hdr_page_id, &mtr); /* Mark the last undo log totally purged, so that if the system crashes, the tail of the undo log will not get accessed @@ -365,17 +367,14 @@ trx_purge_free_segment(trx_rseg_t* rseg, fil_addr_t hdr_addr) while (!fseg_free_step_not_header( TRX_UNDO_SEG_HDR + TRX_UNDO_FSEG_HEADER + block->frame, &mtr)) { - mutex_exit(&rseg->mutex); - + rseg->latch.wr_unlock(); mtr.commit(); mtr.start(); - - mutex_enter(&rseg->mutex); + rseg->latch.wr_lock(); rseg_hdr = trx_rsegf_get(rseg->space, rseg->page_no, &mtr); - block = trx_undo_page_get( - page_id_t(rseg->space->id, hdr_addr.page), &mtr); + block = trx_undo_page_get(hdr_page_id, &mtr); } /* The page list may now be inconsistent, but the length field @@ -409,11 +408,12 @@ trx_purge_free_segment(trx_rseg_t* rseg, fil_addr_t hdr_addr) ut_ad(rseg->curr_size >= seg_size); + rseg->history_size--; rseg->curr_size -= seg_size; - mutex_exit(&(rseg->mutex)); + rseg->latch.wr_unlock(); - mtr_commit(&mtr); + mtr.commit(); } /** Remove unnecessary history data from a rollback segment. @@ -432,7 +432,7 @@ trx_purge_truncate_rseg_history( mtr.start(); ut_ad(rseg.is_persistent()); - mutex_enter(&rseg.mutex); + rseg.latch.wr_lock(); buf_block_t* rseg_hdr = trx_rsegf_get(rseg.space, rseg.page_no, &mtr); @@ -444,7 +444,7 @@ trx_purge_truncate_rseg_history( loop: if (hdr_addr.page == FIL_NULL) { func_exit: - mutex_exit(&rseg.mutex); + rseg.latch.wr_unlock(); mtr.commit(); return; } @@ -477,7 +477,7 @@ func_exit: /* We can free the whole log segment */ - mutex_exit(&rseg.mutex); + rseg.latch.wr_unlock(); mtr.commit(); /* calls the trx_purge_remove_log_hdr() @@ -487,13 +487,13 @@ func_exit: /* Remove the log hdr from the rseg history. */ trx_purge_remove_log_hdr(rseg_hdr, block, hdr_addr.boffset, &mtr); - - mutex_exit(&rseg.mutex); + rseg.history_size--; + rseg.latch.wr_unlock(); mtr.commit(); } mtr.start(); - mutex_enter(&rseg.mutex); + rseg.latch.wr_lock(); rseg_hdr = trx_rsegf_get(rseg.space, rseg.page_no, &mtr); @@ -510,7 +510,7 @@ static void trx_purge_cleanse_purge_queue(const fil_space_t& space) typedef std::vector<TrxUndoRsegs> purge_elem_list_t; purge_elem_list_t purge_elem_list; - mutex_enter(&purge_sys.pq_mutex); + mysql_mutex_lock(&purge_sys.pq_mutex); /* Remove rseg instances that are in the purge queue before we start truncate of corresponding UNDO truncate. */ @@ -537,7 +537,7 @@ static void trx_purge_cleanse_purge_queue(const fil_space_t& space) } } - mutex_exit(&purge_sys.pq_mutex); + mysql_mutex_unlock(&purge_sys.pq_mutex); } /** @@ -557,14 +557,9 @@ static void trx_purge_truncate_history() head.undo_no= 0; } - for (ulint i= 0; i < TRX_SYS_N_RSEGS; ++i) - { - if (trx_rseg_t *rseg= trx_sys.rseg_array[i]) - { - ut_ad(rseg->id == i); - trx_purge_truncate_rseg_history(*rseg, head); - } - } + for (auto &rseg : trx_sys.rseg_array) + if (rseg.space) + trx_purge_truncate_rseg_history(rseg, head); if (srv_undo_tablespaces_active < 2) return; @@ -605,34 +600,31 @@ static void trx_purge_truncate_history() DBUG_LOG("undo", "marking for truncate: " << file->name); - for (ulint i= 0; i < TRX_SYS_N_RSEGS; ++i) - if (trx_rseg_t *rseg= trx_sys.rseg_array[i]) - if (rseg->space == &space) - /* Once set, this rseg will not be allocated to subsequent - transactions, but we will wait for existing active - transactions to finish. */ - rseg->skip_allocation= true; + for (auto &rseg : trx_sys.rseg_array) + if (rseg.space == &space) + /* Once set, this rseg will not be allocated to subsequent + transactions, but we will wait for existing active + transactions to finish. */ + rseg.set_skip_allocation(); - for (ulint i= 0; i < TRX_SYS_N_RSEGS; ++i) + for (auto &rseg : trx_sys.rseg_array) { - trx_rseg_t *rseg= trx_sys.rseg_array[i]; - if (!rseg || rseg->space != &space) + if (rseg.space != &space) continue; - mutex_enter(&rseg->mutex); - ut_ad(rseg->skip_allocation); - ut_ad(rseg->is_persistent()); - if (rseg->trx_ref_count) + rseg.latch.rd_lock(); + ut_ad(rseg.skip_allocation()); + if (rseg.is_referenced()) { not_free: - mutex_exit(&rseg->mutex); + rseg.latch.rd_unlock(); return; } - if (rseg->curr_size != 1) + if (rseg.curr_size != 1) { /* Check if all segments are cached and safe to remove. */ ulint cached= 0; - for (trx_undo_t *undo= UT_LIST_GET_FIRST(rseg->undo_cached); undo; + for (trx_undo_t *undo= UT_LIST_GET_FIRST(rseg.undo_cached); undo; undo= UT_LIST_GET_NEXT(undo_list, undo)) { if (head.trx_no < undo->trx_id) @@ -641,13 +633,13 @@ not_free: cached+= undo->size; } - ut_ad(rseg->curr_size > cached); + ut_ad(rseg.curr_size > cached); - if (rseg->curr_size > cached + 1) + if (rseg.curr_size > cached + 1) goto not_free; } - mutex_exit(&rseg->mutex); + rseg.latch.rd_unlock(); } ib::info() << "Truncating " << file->name; @@ -657,7 +649,7 @@ not_free: mtr_t mtr; mtr.start(); - mtr_x_lock_space(&space, &mtr); + mtr.x_lock_space(&space); /* Lock all modified pages of the tablespace. @@ -682,14 +674,13 @@ not_free: ut_ad(bpage->state() == BUF_BLOCK_FILE_PAGE); auto block= reinterpret_cast<buf_block_t*>(bpage); block->fix(); - ut_ad(rw_lock_s_lock_nowait(block->debug_latch, __FILE__, __LINE__)); buf_pool.flush_hp.set(prev); mysql_mutex_unlock(&buf_pool.flush_list_mutex); #ifdef BTR_CUR_HASH_ADAPT ut_ad(!block->index); /* There is no AHI on undo tablespaces. */ #endif - rw_lock_x_lock(&block->lock); + block->lock.x_lock(); mysql_mutex_lock(&buf_pool.flush_list_mutex); ut_ad(bpage->io_fix() == BUF_IO_NONE); @@ -700,7 +691,7 @@ not_free: } else { - rw_lock_x_unlock(&block->lock); + block->lock.x_unlock(); block->unfix(); } @@ -717,72 +708,61 @@ not_free: mysql_mutex_unlock(&buf_pool.flush_list_mutex); + /* Re-initialize tablespace, in a single mini-transaction. */ + const ulint size= SRV_UNDO_TABLESPACE_SIZE_IN_PAGES; + /* Adjust the tablespace metadata. */ - if (!fil_truncate_prepare(space.id)) + mysql_mutex_lock(&fil_system.mutex); + space.set_stopping(); + space.is_being_truncated= true; + if (space.crypt_data) { - ib::error() << "Failed to find UNDO tablespace " << file->name; - mtr.commit(); - return; + space.reacquire(); + mysql_mutex_unlock(&fil_system.mutex); + fil_space_crypt_close_tablespace(&space); + space.release(); + } + else + mysql_mutex_unlock(&fil_system.mutex); + + for (auto i= 6000; space.referenced(); + std::this_thread::sleep_for(std::chrono::milliseconds(10))) + { + if (!--i) + { + mtr.commit(); + ib::error() << "Failed to freeze UNDO tablespace " << file->name; + return; + } } - /* Re-initialize tablespace, in a single mini-transaction. */ - const ulint size= SRV_UNDO_TABLESPACE_SIZE_IN_PAGES; /* Associate the undo tablespace with mtr. During mtr::commit_shrink(), InnoDB can use the undo tablespace object to clear all freed ranges */ mtr.set_named_space(&space); mtr.trim_pages(page_id_t(space.id, size)); fsp_header_init(&space, size, &mtr); - mutex_enter(&fil_system.mutex); + mysql_mutex_lock(&fil_system.mutex); space.size= file->size= size; - mutex_exit(&fil_system.mutex); + mysql_mutex_unlock(&fil_system.mutex); buf_block_t *sys_header= trx_sysf_get(&mtr); - for (ulint i= 0; i < TRX_SYS_N_RSEGS; ++i) + for (auto &rseg : trx_sys.rseg_array) { - trx_rseg_t *rseg= trx_sys.rseg_array[i]; - if (!rseg || rseg->space != &space) + if (rseg.space != &space) continue; - ut_ad(rseg->id == i); - ut_ad(rseg->is_persistent()); - ut_d(const auto old_page= rseg->page_no); - - buf_block_t *rblock= trx_rseg_header_create(&space, i, + buf_block_t *rblock= trx_rseg_header_create(&space, + &rseg - trx_sys.rseg_array, trx_sys.get_max_trx_id(), sys_header, &mtr); ut_ad(rblock); - rseg->page_no= rblock ? rblock->page.id().page_no() : FIL_NULL; - ut_ad(old_page == rseg->page_no); - - /* Before re-initialization ensure that we free the existing - structure. There can't be any active transactions. */ - ut_a(UT_LIST_GET_LEN(rseg->undo_list) == 0); - - for (trx_undo_t *undo= UT_LIST_GET_FIRST(rseg->undo_cached), *next_undo; - undo; undo= next_undo) - { - next_undo= UT_LIST_GET_NEXT(undo_list, undo); - UT_LIST_REMOVE(rseg->undo_cached, undo); - MONITOR_DEC(MONITOR_NUM_UNDO_SLOT_CACHED); - ut_free(undo); - } - - UT_LIST_INIT(rseg->undo_list, &trx_undo_t::undo_list); - UT_LIST_INIT(rseg->undo_cached, &trx_undo_t::undo_list); - /* These were written by trx_rseg_header_create(). */ ut_ad(!mach_read_from_4(TRX_RSEG + TRX_RSEG_FORMAT + rblock->frame)); ut_ad(!mach_read_from_4(TRX_RSEG + TRX_RSEG_HISTORY_SIZE + rblock->frame)); - /* Initialize the undo log lists according to - the rseg header */ - rseg->curr_size= 1; - rseg->trx_ref_count= 0; - rseg->last_page_no= FIL_NULL; - rseg->last_commit_and_offset= 0; - rseg->needs_purge= false; + rseg.reinit(rblock ? rblock->page.id().page_no() : FIL_NULL); } mtr.commit_shrink(space); @@ -806,16 +786,9 @@ not_free: log_buffer_flush_to_disk(); DBUG_SUICIDE();); - for (ulint i= 0; i < TRX_SYS_N_RSEGS; ++i) - { - if (trx_rseg_t *rseg= trx_sys.rseg_array[i]) - { - ut_ad(rseg->id == i); - ut_ad(rseg->is_persistent()); - if (rseg->space == &space) - rseg->skip_allocation= false; - } - } + for (auto &rseg : trx_sys.rseg_array) + if (rseg.space == &space) + rseg.clear_skip_allocation(); ib::info() << "Truncated " << file->name; purge_sys.truncate.last= purge_sys.truncate.current; @@ -835,7 +808,9 @@ static void trx_purge_rseg_get_next_history_log( trx_id_t trx_no; mtr_t mtr; - mutex_enter(&purge_sys.rseg->mutex); + mtr.start(); + + purge_sys.rseg->latch.wr_lock(); ut_a(purge_sys.rseg->last_page_no != FIL_NULL); @@ -843,8 +818,6 @@ static void trx_purge_rseg_get_next_history_log( purge_sys.tail.undo_no = 0; purge_sys.next_stored = false; - mtr.start(); - const buf_block_t* undo_page = trx_undo_page_get_s_latched( page_id_t(purge_sys.rseg->space->id, purge_sys.rseg->last_page_no), &mtr); @@ -868,7 +841,7 @@ static void trx_purge_rseg_get_next_history_log( purge_sys.rseg->last_page_no = FIL_NULL; } - mutex_exit(&purge_sys.rseg->mutex); + purge_sys.rseg->latch.wr_unlock(); mtr.commit(); if (empty) { @@ -889,24 +862,29 @@ static void trx_purge_rseg_get_next_history_log( mtr.commit(); - mutex_enter(&purge_sys.rseg->mutex); + purge_sys.rseg->latch.wr_lock(); purge_sys.rseg->last_page_no = prev_log_addr.page; purge_sys.rseg->set_last_commit(prev_log_addr.boffset, trx_no); - purge_sys.rseg->needs_purge = needs_purge != 0; + + if (needs_purge) { + purge_sys.rseg->set_needs_purge(); + } else { + purge_sys.rseg->clear_needs_purge(); + } /* Purge can also produce events, however these are already ordered in the rollback segment and any user generated event will be greater than the events that Purge produces. ie. Purge can never produce events from an empty rollback segment. */ - mutex_enter(&purge_sys.pq_mutex); + mysql_mutex_lock(&purge_sys.pq_mutex); purge_sys.purge_queue.push(*purge_sys.rseg); - mutex_exit(&purge_sys.pq_mutex); + mysql_mutex_unlock(&purge_sys.pq_mutex); - mutex_exit(&purge_sys.rseg->mutex); + purge_sys.rseg->latch.wr_unlock(); } /** Position the purge sys "iterator" on the undo record to use for purging. */ @@ -919,7 +897,7 @@ static void trx_purge_read_undo_rec() purge_sys.hdr_offset = purge_sys.rseg->last_offset(); page_no = purge_sys.hdr_page_no = purge_sys.rseg->last_page_no; - if (purge_sys.rseg->needs_purge) { + if (purge_sys.rseg->needs_purge()) { mtr_t mtr; mtr.start(); buf_block_t* undo_page; @@ -965,7 +943,7 @@ trx_purge_choose_next_log(void) trx_purge_read_undo_rec(); } else { /* There is nothing to do yet. */ - os_thread_yield(); + std::this_thread::yield(); } } @@ -1085,7 +1063,7 @@ trx_purge_fetch_next_rec( /* row_purge_record_func() will later set ROLL_PTR_INSERT_FLAG for TRX_UNDO_INSERT_REC */ false, - purge_sys.rseg->id, + trx_sys.rseg_id(purge_sys.rseg, true), purge_sys.page_no, purge_sys.offset); /* The following call will advance the stored values of the @@ -1150,8 +1128,6 @@ trx_purge_attach_undo_recs(ulint n_purge_threads) purge_node_t* node; trx_purge_rec_t purge_rec; - ut_a(!thr->is_active); - /* Get the purge node. */ node = (purge_node_t*) thr->child; ut_a(que_node_get_type(node) == QUE_NODE_PURGE); @@ -1221,7 +1197,7 @@ trx_purge_dml_delay(void) /* If purge lag is set then calculate the new DML delay. */ if (srv_max_purge_lag > 0) { - double ratio = static_cast<double>(trx_sys.rseg_history_len) / + double ratio = static_cast<double>(trx_sys.history_size()) / static_cast<double>(srv_max_purge_lag); if (ratio > 1.0) { diff --git a/storage/innobase/trx/trx0rec.cc b/storage/innobase/trx/trx0rec.cc index 438dfcf9352..08e05edb896 100644 --- a/storage/innobase/trx/trx0rec.cc +++ b/storage/innobase/trx/trx0rec.cc @@ -38,6 +38,7 @@ Created 3/26/1996 Heikki Tuuri #include "trx0rseg.h" #include "row0row.h" #include "row0mysql.h" +#include "row0ins.h" /** The search tuple corresponding to TRX_UNDO_INSERT_METADATA. */ const dtuple_t trx_undo_metadata = { @@ -371,19 +372,24 @@ trx_undo_report_insert_virtual( return(true); } -/**********************************************************************//** -Reports in the undo log of an insert of a clustered index record. +/** Reports in the undo log of an insert of a clustered index record. +@param undo_block undo log page +@param trx transaction +@param index clustered index +@param clust_entry index entry which will be inserted to the + clustered index +@param mtr mini-transaction +@param write_empty write empty table undo log record @return offset of the inserted entry on the page if succeed, 0 if fail */ static uint16_t trx_undo_page_report_insert( -/*========================*/ - buf_block_t* undo_block, /*!< in: undo log page */ - trx_t* trx, /*!< in: transaction */ - dict_index_t* index, /*!< in: clustered index */ - const dtuple_t* clust_entry, /*!< in: index entry which will be - inserted to the clustered index */ - mtr_t* mtr) /*!< in: mtr */ + buf_block_t* undo_block, + trx_t* trx, + dict_index_t* index, + const dtuple_t* clust_entry, + mtr_t* mtr, + bool write_empty) { ut_ad(index->is_primary()); /* MariaDB 10.3.1+ in trx_undo_page_init() always initializes @@ -411,6 +417,13 @@ trx_undo_page_report_insert( *ptr++ = TRX_UNDO_INSERT_REC; ptr += mach_u64_write_much_compressed(ptr, trx->undo_no); ptr += mach_u64_write_much_compressed(ptr, index->table->id); + + if (write_empty) { + /* Table is in bulk operation */ + undo_block->frame[first_free + 2] = TRX_UNDO_EMPTY; + goto done; + } + /*----------------------------------------*/ /* Store then the fields required to uniquely determine the record to be inserted in the clustered index */ @@ -488,7 +501,7 @@ trx_undo_rec_get_pars( type_cmpl &= ~TRX_UNDO_UPD_EXTERN; *type = type_cmpl & (TRX_UNDO_CMPL_INFO_MULT - 1); ut_ad(*type >= TRX_UNDO_RENAME_TABLE); - ut_ad(*type <= TRX_UNDO_DEL_MARK_REC); + ut_ad(*type <= TRX_UNDO_EMPTY); *cmpl_info = type_cmpl / TRX_UNDO_CMPL_INFO_MULT; *undo_no = mach_read_next_much_compressed(&ptr); @@ -1935,6 +1948,28 @@ dberr_t trx_undo_report_rename(trx_t* trx, const dict_table_t* table) return err; } +ATTRIBUTE_COLD ATTRIBUTE_NOINLINE +/** @return whether the transaction holds an exclusive lock on a table */ +static bool trx_has_lock_x(const trx_t &trx, dict_table_t& table) +{ + if (table.is_temporary()) + return true; + + table.lock_mutex_lock(); + const auto n= table.n_lock_x_or_s; + table.lock_mutex_unlock(); + + /* This thread is executing trx. No other thread can modify our table locks + (only record locks might be created, in an implicit-to-explicit conversion). + Hence, no mutex is needed here. */ + if (n == 1) + for (const lock_t *lock : trx.lock.table_locks) + if (lock && lock->type_mode == (LOCK_X | LOCK_TABLE)) + return true; + + return false; +} + /***********************************************************************//** Writes information to an undo log about an insert, update, or a delete marking of a clustered index record. This information is used in a rollback of the @@ -1965,7 +2000,6 @@ trx_undo_report_row_operation( undo log record */ { trx_t* trx; - mtr_t mtr; #ifdef UNIV_DEBUG int loop_count = 0; #endif /* UNIV_DEBUG */ @@ -1981,6 +2015,35 @@ trx_undo_report_row_operation( ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE)); ut_ad(!trx->in_rollback); + /* We must determine if this is the first time when this + transaction modifies this table. */ + auto m = trx->mod_tables.emplace(index->table, trx->undo_no); + ut_ad(m.first->second.valid(trx->undo_no)); + + bool bulk = !rec; + + if (!bulk) { + /* An UPDATE or DELETE must not be covered by an + earlier start_bulk_insert(). */ + ut_ad(!m.first->second.is_bulk_insert()); + } else if (m.first->second.is_bulk_insert()) { + /* Above, the emplace() tried to insert an object with + !is_bulk_insert(). Only an explicit start_bulk_insert() + (below) can set the flag. */ + ut_ad(!m.second); + /* We already wrote a TRX_UNDO_EMPTY record. */ + ut_ad(thr->run_node); + ut_ad(que_node_get_type(thr->run_node) == QUE_NODE_INSERT); + ut_ad(trx->bulk_insert); + return DB_SUCCESS; + } else if (m.second && trx->bulk_insert + && trx_has_lock_x(*trx, *index->table)) { + m.first->second.start_bulk_insert(); + } else { + bulk = false; + } + + mtr_t mtr; mtr.start(); trx_undo_t** pundo; trx_rseg_t* rseg; @@ -2002,10 +2065,11 @@ trx_undo_report_row_operation( buf_block_t* undo_block = trx_undo_assign_low(trx, rseg, pundo, &err, &mtr); trx_undo_t* undo = *pundo; - ut_ad((err == DB_SUCCESS) == (undo_block != NULL)); if (UNIV_UNLIKELY(undo_block == NULL)) { - goto err_exit; +err_exit: + mtr.commit(); + return err; } ut_ad(undo != NULL); @@ -2013,7 +2077,8 @@ trx_undo_report_row_operation( do { uint16_t offset = !rec ? trx_undo_page_report_insert( - undo_block, trx, index, clust_entry, &mtr) + undo_block, trx, index, clust_entry, &mtr, + bulk) : trx_undo_page_report_modify( undo_block, trx, index, rec, offsets, update, cmpl_info, clust_entry, &mtr); @@ -2039,7 +2104,7 @@ trx_undo_report_row_operation( tree latch, which is the rseg mutex. We must commit the mini-transaction first, because it may be holding lower-level - latches, such as SYNC_FSP and SYNC_FSP_PAGE. */ + latches, such as SYNC_FSP_PAGE. */ mtr.commit(); mtr.start(); @@ -2047,9 +2112,15 @@ trx_undo_report_row_operation( mtr.set_log_mode(MTR_LOG_NO_REDO); } - mutex_enter(&rseg->mutex); + rseg->latch.wr_lock(); trx_undo_free_last_page(undo, &mtr); - mutex_exit(&rseg->mutex); + rseg->latch.wr_unlock(); + + if (m.second) { + /* We are not going to modify + this table after all. */ + trx->mod_tables.erase(m.first); + } err = DB_UNDO_RECORD_TOO_BIG; goto err_exit; @@ -2081,28 +2152,24 @@ trx_undo_report_row_operation( ut_ad(!undo->empty()); if (!is_temp) { - const undo_no_t limit = undo->top_undo_no; - /* Determine if this is the first time - when this transaction modifies a - system-versioned column in this table. */ - trx_mod_table_time_t& time - = trx->mod_tables.insert( - trx_mod_tables_t::value_type( - index->table, limit)) - .first->second; - ut_ad(time.valid(limit)); + trx_mod_table_time_t& time = m.first->second; + ut_ad(time.valid(undo->top_undo_no)); if (!time.is_versioned() && index->table->versioned_by_id() && (!rec /* INSERT */ || (update && update->affects_versioned()))) { - time.set_versioned(limit); + time.set_versioned(undo->top_undo_no); } } - *roll_ptr = trx_undo_build_roll_ptr( - !rec, rseg->id, undo->top_page_no, offset); + if (!bulk) { + *roll_ptr = trx_undo_build_roll_ptr( + !rec, trx_sys.rseg_id(rseg, !is_temp), + undo->top_page_no, offset); + } + return(DB_SUCCESS); } @@ -2135,10 +2202,7 @@ trx_undo_report_row_operation( /* Did not succeed: out of space */ err = DB_OUT_OF_FILE_SPACE; - -err_exit: - mtr_commit(&mtr); - return(err); + goto err_exit; } /*============== BUILDING PREVIOUS VERSION OF A RECORD ===============*/ @@ -2156,7 +2220,6 @@ trx_undo_get_undo_rec_low( ulint rseg_id; uint32_t page_no; uint16_t offset; - trx_rseg_t* rseg; bool is_insert; mtr_t mtr; @@ -2164,7 +2227,7 @@ trx_undo_get_undo_rec_low( &offset); ut_ad(page_no > FSP_FIRST_INODE_PAGE_NO); ut_ad(offset >= TRX_UNDO_PAGE_HDR + TRX_UNDO_PAGE_HDR_SIZE); - rseg = trx_sys.rseg_array[rseg_id]; + trx_rseg_t* rseg = &trx_sys.rseg_array[rseg_id]; ut_ad(rseg->is_persistent()); mtr.start(); @@ -2200,14 +2263,14 @@ trx_undo_get_undo_rec( const table_name_t& name, trx_undo_rec_t** undo_rec) { - rw_lock_s_lock(&purge_sys.latch); + purge_sys.latch.rd_lock(SRW_LOCK_CALL); bool missing_history = purge_sys.changes_visible(trx_id, name); if (!missing_history) { *undo_rec = trx_undo_get_undo_rec_low(roll_ptr, heap); } - rw_lock_s_unlock(&purge_sys.latch); + purge_sys.latch.rd_unlock(); return(missing_history); } @@ -2270,7 +2333,6 @@ trx_undo_prev_version_build( byte* buf; ut_ad(!index->table->is_temporary()); - ut_ad(!rw_lock_own(&purge_sys.latch, RW_LOCK_S)); ut_ad(index_mtr->memo_contains_page_flagged(index_rec, MTR_MEMO_PAGE_S_FIX | MTR_MEMO_PAGE_X_FIX)); @@ -2364,14 +2426,12 @@ trx_undo_prev_version_build( if ((update->info_bits & REC_INFO_DELETED_FLAG) && row_upd_changes_disowned_external(update)) { - bool missing_extern; - - rw_lock_s_lock(&purge_sys.latch); + purge_sys.latch.rd_lock(SRW_LOCK_CALL); - missing_extern = purge_sys.changes_visible( + bool missing_extern = purge_sys.changes_visible( trx_id, index->table->name); - rw_lock_s_unlock(&purge_sys.latch); + purge_sys.latch.rd_unlock(); if (missing_extern) { /* treat as a fresh insert, not to diff --git a/storage/innobase/trx/trx0roll.cc b/storage/innobase/trx/trx0roll.cc index 23aa950a14a..a0582413d07 100644 --- a/storage/innobase/trx/trx0roll.cc +++ b/storage/innobase/trx/trx0roll.cc @@ -60,6 +60,7 @@ const trx_t* trx_roll_crash_recv_trx; inline bool trx_t::rollback_finish() { mod_tables.clear(); + if (UNIV_LIKELY(error_state == DB_SUCCESS)) { commit(); @@ -84,7 +85,7 @@ inline bool trx_t::rollback_finish() undo= nullptr; } commit_low(); - lock.que_state= TRX_QUE_RUNNING; + commit_cleanup(); return false; } @@ -144,13 +145,10 @@ inline void trx_t::rollback_low(trx_savept_t *savept) if (j->second.rollback(limit)) mod_tables.erase(j); } - lock.que_state= TRX_QUE_RUNNING; MONITOR_INC(MONITOR_TRX_ROLLBACK_SAVEPOINT); } mem_heap_free(heap); - - MONITOR_DEC(MONITOR_TRX_ACTIVE); } /** Initiate rollback. @@ -158,7 +156,7 @@ inline void trx_t::rollback_low(trx_savept_t *savept) @return error code or DB_SUCCESS */ dberr_t trx_t::rollback(trx_savept_t *savept) { - ut_ad(!trx_mutex_own(this)); + ut_ad(!mutex_is_owner()); if (state == TRX_STATE_NOT_STARTED) { error_state= DB_SUCCESS; @@ -212,7 +210,7 @@ dberr_t trx_rollback_for_mysql(trx_t* trx) ut_ad(trx->mysql_thd); #ifdef WITH_WSREP trx->wsrep= false; - trx->lock.was_chosen_as_wsrep_victim= false; + trx->lock.was_chosen_as_deadlock_victim= false; #endif return(DB_SUCCESS); @@ -245,12 +243,10 @@ dberr_t trx_rollback_for_mysql(trx_t* trx) == trx->rsegs.m_redo.rseg); mtr_t mtr; mtr.start(); - mutex_enter(&trx->rsegs.m_redo.rseg->mutex); if (trx_undo_t* undo = trx->rsegs.m_redo.undo) { trx_undo_set_state_at_prepare(trx, undo, true, &mtr); } - mutex_exit(&trx->rsegs.m_redo.rseg->mutex); /* Write the redo log for the XA ROLLBACK state change to the global buffer. It is not necessary to flush the redo log. If @@ -302,11 +298,11 @@ trx_rollback_last_sql_stat_for_mysql( if (trx->fts_trx != NULL) { fts_savepoint_rollback_last_stmt(trx); + fts_savepoint_laststmt_refresh(trx); } - /* The following call should not be needed, - but we play it safe: */ - trx_mark_sql_stat_end(trx); + trx->last_sql_stat_start.least_undo_no = trx->undo_no; + trx->end_bulk_insert(); trx->op_info = ""; @@ -362,24 +358,16 @@ trx_roll_savepoint_free( ut_free(savep); } -/*******************************************************************//** -Frees savepoint structs starting from savep. */ -void -trx_roll_savepoints_free( -/*=====================*/ - trx_t* trx, /*!< in: transaction handle */ - trx_named_savept_t* savep) /*!< in: free all savepoints starting - with this savepoint i*/ +/** Discard all savepoints starting from a particular savepoint. +@param savept first savepoint to discard */ +void trx_t::savepoints_discard(trx_named_savept_t *savept) { - while (savep != NULL) { - trx_named_savept_t* next_savep; - - next_savep = UT_LIST_GET_NEXT(trx_savepoints, savep); - - trx_roll_savepoint_free(trx, savep); - - savep = next_savep; - } + while (savept) + { + auto next= UT_LIST_GET_NEXT(trx_savepoints, savept); + trx_roll_savepoint_free(this, savept); + savept= next; + } } /*******************************************************************//** @@ -412,8 +400,7 @@ trx_rollback_to_savepoint_for_mysql_low( /* Free all savepoints strictly later than savep. */ - trx_roll_savepoints_free( - trx, UT_LIST_GET_NEXT(trx_savepoints, savep)); + trx->savepoints_discard(UT_LIST_GET_NEXT(trx_savepoints, savep)); *mysql_binlog_cache_pos = savep->mysql_binlog_cache_pos; @@ -429,7 +416,7 @@ trx_rollback_to_savepoint_for_mysql_low( trx->op_info = ""; #ifdef WITH_WSREP - trx->lock.was_chosen_as_wsrep_victim = false; + trx->lock.was_chosen_as_deadlock_victim = false; #endif return(err); } @@ -531,12 +518,15 @@ trx_savepoint_for_mysql( savep->name = mem_strdup(savepoint_name); - savep->savept = trx_savept_take(trx); + savep->savept.least_undo_no = trx->undo_no; + trx->last_sql_stat_start.least_undo_no = trx->undo_no; savep->mysql_binlog_cache_pos = binlog_cache_pos; UT_LIST_ADD_LAST(trx->trx_savepoints, savep); + trx->end_bulk_insert(); + return(DB_SUCCESS); } @@ -567,21 +557,6 @@ trx_release_savepoint_for_mysql( } /*******************************************************************//** -Returns a transaction savepoint taken at this point in time. -@return savepoint */ -trx_savept_t -trx_savept_take( -/*============*/ - trx_t* trx) /*!< in: transaction */ -{ - trx_savept_t savept; - - savept.least_undo_no = trx->undo_no; - - return(savept); -} - -/*******************************************************************//** Roll back an active transaction. */ static void @@ -599,7 +574,7 @@ trx_rollback_active( heap = mem_heap_create(512); - fork = que_fork_create(NULL, NULL, QUE_FORK_RECOVERY, heap); + fork = que_fork_create(heap); fork->trx = trx; thr = que_thr_create(fork, heap, NULL); @@ -615,8 +590,7 @@ trx_rollback_active( trx_roll_crash_recv_trx = trx; - const bool dictionary_locked = trx_get_dict_operation(trx) - != TRX_DICT_OP_NONE; + const bool dictionary_locked = trx->dict_operation; if (dictionary_locked) { row_mysql_lock_data_dictionary(trx); @@ -632,26 +606,10 @@ trx_rollback_active( if (UNIV_UNLIKELY(!trx->rollback_finish())) { ut_ad(!dictionary_locked); - goto func_exit; - } - - ut_a(trx->lock.que_state == TRX_QUE_RUNNING); - - if (!dictionary_locked || !trx->table_id) { - } else if (dict_table_t* table = dict_table_open_on_id( - trx->table_id, TRUE, DICT_TABLE_OP_NORMAL)) { - ib::info() << "Dropping table " << table->name - << ", with id " << trx->table_id - << " in recovery"; - - dict_table_close_and_drop(trx, table); - - trx_commit_for_mysql(trx); + } else { + ib::info() << "Rolled back recovered transaction " << trx_id; } - ib::info() << "Rolled back recovered transaction " << trx_id; - -func_exit: if (dictionary_locked) { row_mysql_unlock_data_dictionary(trx); } @@ -673,7 +631,7 @@ struct trx_roll_count_callback_arg static my_bool trx_roll_count_callback(rw_trx_hash_element_t *element, trx_roll_count_callback_arg *arg) { - mutex_enter(&element->mutex); + mysql_mutex_lock(&element->mutex); if (trx_t *trx= element->trx) { if (trx->is_recovered && trx_state_eq(trx, TRX_STATE_ACTIVE)) @@ -682,7 +640,7 @@ static my_bool trx_roll_count_callback(rw_trx_hash_element_t *element, arg->n_rows+= trx->undo_no; } } - mutex_exit(&element->mutex); + mysql_mutex_unlock(&element->mutex); return 0; } @@ -690,9 +648,9 @@ static my_bool trx_roll_count_callback(rw_trx_hash_element_t *element, void trx_roll_report_progress() { time_t now = time(NULL); - mutex_enter(&recv_sys.mutex); + mysql_mutex_lock(&recv_sys.mutex); bool report = recv_sys.report(now); - mutex_exit(&recv_sys.mutex); + mysql_mutex_unlock(&recv_sys.mutex); if (report) { trx_roll_count_callback_arg arg; @@ -720,19 +678,18 @@ void trx_roll_report_progress() static my_bool trx_rollback_recovered_callback(rw_trx_hash_element_t *element, std::vector<trx_t*> *trx_list) { - mutex_enter(&element->mutex); + mysql_mutex_lock(&element->mutex); if (trx_t *trx= element->trx) { - mutex_enter(&trx->mutex); + trx->mutex_lock(); if (trx_state_eq(trx, TRX_STATE_ACTIVE) && trx->is_recovered) trx_list->push_back(trx); - mutex_exit(&trx->mutex); + trx->mutex_unlock(); } - mutex_exit(&element->mutex); + mysql_mutex_unlock(&element->mutex); return 0; } - /** Rollback any incomplete transactions which were encountered in crash recovery. @@ -768,17 +725,16 @@ void trx_rollback_recovered(bool all) trx_list.pop_back(); ut_ad(trx); - ut_d(trx_mutex_enter(trx)); + ut_d(trx->mutex_lock()); ut_ad(trx->is_recovered); ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE)); - ut_d(trx_mutex_exit(trx)); + ut_d(trx->mutex_unlock()); if (srv_shutdown_state != SRV_SHUTDOWN_NONE && !srv_undo_sources && srv_fast_shutdown) goto discard; - if (all || trx_get_dict_operation(trx) != TRX_DICT_OP_NONE - || trx->has_stats_table_lock()) + if (all || trx->dict_operation || trx->has_stats_table_lock()) { trx_rollback_active(trx); if (trx->error_state != DB_SUCCESS) @@ -818,19 +774,11 @@ Rollback or clean up any incomplete transactions which were encountered in crash recovery. If the transaction already was committed, then we clean up a possible insert undo log. If the transaction was not yet committed, then we roll it back. -Note: this is done in a background thread. -@return a dummy parameter */ -extern "C" -os_thread_ret_t -DECLARE_THREAD(trx_rollback_all_recovered)(void*) +Note: this is done in a background thread. */ +void trx_rollback_all_recovered(void*) { - my_thread_init(); ut_ad(!srv_read_only_mode); -#ifdef UNIV_PFS_THREAD - pfs_register_thread(trx_rollback_clean_thread_key); -#endif /* UNIV_PFS_THREAD */ - if (trx_sys.rw_trx_hash.size()) { ib::info() << "Starting in background the rollback of" " recovered transactions"; @@ -840,14 +788,6 @@ DECLARE_THREAD(trx_rollback_all_recovered)(void*) } trx_rollback_is_active = false; - - my_thread_end(); - /* We count the number of threads in os_thread_exit(). A created - thread should always use that to exit and not use return() to exit. */ - - os_thread_exit(); - - OS_THREAD_DUMMY_RETURN; } /****************************************************************//** @@ -866,10 +806,9 @@ trx_roll_graph_build( que_fork_t* fork; que_thr_t* thr; - ut_ad(trx_mutex_own(trx)); - + ut_ad(trx->mutex_is_owner()); heap = mem_heap_create(512); - fork = que_fork_create(NULL, NULL, QUE_FORK_ROLLBACK, heap); + fork = que_fork_create(heap); fork->trx = trx; thr = que_thr_create(fork, heap, NULL); @@ -892,10 +831,9 @@ trx_rollback_start( partial undo), 0 if we are rolling back the entire transaction */ { - ut_ad(trx_mutex_own(trx)); - /* Initialize the rollback field in the transaction */ + ut_ad(trx->mutex_is_owner()); ut_ad(!trx->roll_limit); ut_ad(!trx->in_rollback); @@ -912,8 +850,6 @@ trx_rollback_start( trx->graph = roll_graph; - trx->lock.que_state = TRX_QUE_ROLLING_BACK; - return(que_fork_start_command(roll_graph)); } @@ -960,20 +896,19 @@ trx_rollback_step( trx = thr_get_trx(thr); - trx_mutex_enter(trx); - node->state = ROLL_NODE_WAIT; ut_a(node->undo_thr == NULL); roll_limit = node->savept ? node->savept->least_undo_no : 0; + trx->mutex_lock(); + trx_commit_or_rollback_prepare(trx); node->undo_thr = trx_rollback_start(trx, roll_limit); - trx_mutex_exit(trx); - + trx->mutex_unlock(); } else { ut_ad(node->state == ROLL_NODE_WAIT); diff --git a/storage/innobase/trx/trx0rseg.cc b/storage/innobase/trx/trx0rseg.cc index 34e1ccfc277..ace1d8531fc 100644 --- a/storage/innobase/trx/trx0rseg.cc +++ b/storage/innobase/trx/trx0rseg.cc @@ -134,7 +134,7 @@ trx_rseg_update_wsrep_checkpoint(const XID* xid, mtr_t* mtr) result. */ const bool must_clear_rsegs = memcmp(wsrep_uuid, xid_uuid, sizeof wsrep_uuid); - const trx_rseg_t* rseg = trx_sys.rseg_array[0]; + const trx_rseg_t* rseg = &trx_sys.rseg_array[0]; buf_block_t* rseg_header = trx_rsegf_get(rseg->space, rseg->page_no, mtr); @@ -151,11 +151,11 @@ trx_rseg_update_wsrep_checkpoint(const XID* xid, mtr_t* mtr) changed, and we must reset the XID in all rollback segment headers. */ for (ulint rseg_id = 1; rseg_id < TRX_SYS_N_RSEGS; ++rseg_id) { - if (const trx_rseg_t* rseg = - trx_sys.rseg_array[rseg_id]) { + const trx_rseg_t &rseg = trx_sys.rseg_array[rseg_id]; + if (rseg.space) { trx_rseg_clear_wsrep_checkpoint( - trx_rsegf_get(rseg->space, - rseg->page_no, mtr), + trx_rsegf_get(rseg.space, rseg.page_no, + mtr), mtr); } } @@ -325,8 +325,6 @@ trx_rseg_header_create( return block; } - buf_block_dbg_add_level(block, SYNC_RSEG_HEADER_NEW); - ut_ad(0 == mach_read_from_4(TRX_RSEG_FORMAT + TRX_RSEG + block->frame)); ut_ad(0 == mach_read_from_4(TRX_RSEG_HISTORY_SIZE + TRX_RSEG @@ -364,59 +362,59 @@ trx_rseg_header_create( return block; } -/** Free a rollback segment in memory. */ -void -trx_rseg_mem_free(trx_rseg_t* rseg) +void trx_rseg_t::destroy() { - trx_undo_t* undo; - trx_undo_t* next_undo; - - mutex_free(&rseg->mutex); - - /* There can't be any active transactions. */ - ut_a(UT_LIST_GET_LEN(rseg->undo_list) == 0); - - for (undo = UT_LIST_GET_FIRST(rseg->undo_cached); - undo != NULL; - undo = next_undo) { - - next_undo = UT_LIST_GET_NEXT(undo_list, undo); - - UT_LIST_REMOVE(rseg->undo_cached, undo); - - MONITOR_DEC(MONITOR_NUM_UNDO_SLOT_CACHED); + latch.destroy(); - ut_free(undo); - } + /* There can't be any active transactions. */ + ut_a(!UT_LIST_GET_LEN(undo_list)); - ut_free(rseg); + for (trx_undo_t *next, *undo= UT_LIST_GET_FIRST(undo_cached); undo; + undo= next) + { + next= UT_LIST_GET_NEXT(undo_list, undo); + UT_LIST_REMOVE(undo_cached, undo); + ut_free(undo); + } } -/** Create a rollback segment object. -@param[in] id rollback segment id -@param[in] space space where the segment is placed -@param[in] page_no page number of the segment header */ -static -trx_rseg_t* -trx_rseg_mem_create(ulint id, fil_space_t* space, uint32_t page_no) +void trx_rseg_t::init(fil_space_t *space, uint32_t page) { - trx_rseg_t* rseg = static_cast<trx_rseg_t*>( - ut_zalloc_nokey(sizeof *rseg)); + latch.init(); + ut_ad(!this->space); + this->space= space; + page_no= page; + last_page_no= FIL_NULL; + curr_size= 1; + + UT_LIST_INIT(undo_list, &trx_undo_t::undo_list); + UT_LIST_INIT(undo_cached, &trx_undo_t::undo_list); +} - rseg->id = id; - rseg->space = space; - rseg->page_no = page_no; - rseg->last_page_no = FIL_NULL; - rseg->curr_size = 1; +void trx_rseg_t::reinit(uint32_t page) +{ + ut_ad(is_persistent()); + ut_ad(page_no == page); + ut_a(!UT_LIST_GET_LEN(undo_list)); + ut_ad(!history_size || UT_LIST_GET_FIRST(undo_cached)); - mutex_create(rseg->is_persistent() - ? LATCH_ID_REDO_RSEG : LATCH_ID_NOREDO_RSEG, - &rseg->mutex); + history_size= 0; + page_no= page; - UT_LIST_INIT(rseg->undo_list, &trx_undo_t::undo_list); - UT_LIST_INIT(rseg->undo_cached, &trx_undo_t::undo_list); + for (trx_undo_t *next, *undo= UT_LIST_GET_FIRST(undo_cached); undo; + undo= next) + { + next= UT_LIST_GET_NEXT(undo_list, undo); + UT_LIST_REMOVE(undo_cached, undo); + MONITOR_DEC(MONITOR_NUM_UNDO_SLOT_CACHED); + ut_free(undo); + } - return(rseg); + ut_ad(!is_referenced()); + clear_needs_purge(); + last_commit_and_offset= 0; + last_page_no= FIL_NULL; + curr_size= 1; } /** Read the undo log lists. @@ -511,7 +509,7 @@ static dberr_t trx_rseg_mem_restore(trx_rseg_t *rseg, trx_id_t &max_trx_id, if (auto len = flst_get_len(TRX_RSEG + TRX_RSEG_HISTORY + rseg_hdr->frame)) { - trx_sys.rseg_history_len += len; + rseg->history_size += len; fil_addr_t node_addr = flst_get_last(TRX_RSEG + TRX_RSEG_HISTORY @@ -540,7 +538,9 @@ static dberr_t trx_rseg_mem_restore(trx_rseg_t *rseg, trx_id_t &max_trx_id, + node_addr.boffset + TRX_UNDO_NEEDS_PURGE); ut_ad(purge <= 1); - rseg->needs_purge = purge != 0; + if (purge != 0) { + rseg->set_needs_purge(); + } if (rseg->last_page_no != FIL_NULL) { @@ -609,17 +609,14 @@ dberr_t trx_rseg_array_init() const uint32_t page_no = trx_sysf_rseg_get_page_no( sys, rseg_id); if (page_no != FIL_NULL) { - trx_rseg_t* rseg = trx_rseg_mem_create( - rseg_id, - fil_space_get(trx_sysf_rseg_get_space( - sys, rseg_id)), - page_no); - ut_ad(rseg->is_persistent()); - ut_ad(rseg->id == rseg_id); - ut_ad(!trx_sys.rseg_array[rseg_id]); - trx_sys.rseg_array[rseg_id] = rseg; + trx_rseg_t& rseg = trx_sys.rseg_array[rseg_id]; + rseg.init(fil_space_get( + trx_sysf_rseg_get_space( + sys, rseg_id)), + page_no); + ut_ad(rseg.is_persistent()); if ((err = trx_rseg_mem_restore( - rseg, max_trx_id, &mtr)) + &rseg, max_trx_id, &mtr)) != DB_SUCCESS) { mtr.commit(); break; @@ -644,15 +641,10 @@ dberr_t trx_rseg_array_init() } if (err != DB_SUCCESS) { - for (ulint rseg_id = 0; rseg_id < TRX_SYS_N_RSEGS; rseg_id++) { - if (trx_rseg_t*& rseg = trx_sys.rseg_array[rseg_id]) { - while (trx_undo_t* u= UT_LIST_GET_FIRST( - rseg->undo_list)) { - UT_LIST_REMOVE(rseg->undo_list, u); - ut_free(u); - } - trx_rseg_mem_free(rseg); - rseg = NULL; + for (auto& rseg : trx_sys.rseg_array) { + while (auto u = UT_LIST_GET_FIRST(rseg.undo_list)) { + UT_LIST_REMOVE(rseg.undo_list, u); + ut_free(u); } } return err; @@ -683,62 +675,20 @@ dberr_t trx_rseg_array_init() return DB_SUCCESS; } -/** Create a persistent rollback segment. -@param[in] space_id system or undo tablespace id -@return pointer to new rollback segment -@retval NULL on failure */ -trx_rseg_t* -trx_rseg_create(ulint space_id) -{ - trx_rseg_t* rseg = NULL; - mtr_t mtr; - - mtr.start(); - - fil_space_t* space = mtr_x_lock_space(space_id, &mtr); - ut_ad(space->purpose == FIL_TYPE_TABLESPACE); - - if (buf_block_t* sys_header = trx_sysf_get(&mtr)) { - ulint rseg_id = trx_sys_rseg_find_free(sys_header); - if (buf_block_t* rblock = rseg_id == ULINT_UNDEFINED - ? NULL - : trx_rseg_header_create(space, rseg_id, 0, sys_header, - &mtr)) { - ut_ad(trx_sysf_rseg_get_space(sys_header, rseg_id) - == space_id); - rseg = trx_rseg_mem_create(rseg_id, space, - rblock->page.id(). - page_no()); - ut_ad(rseg->id == rseg_id); - ut_ad(rseg->is_persistent()); - ut_ad(!trx_sys.rseg_array[rseg->id]); - trx_sys.rseg_array[rseg->id] = rseg; - } - } - - mtr.commit(); - - return(rseg); -} - /** Create the temporary rollback segments. */ -void -trx_temp_rseg_create() +void trx_temp_rseg_create() { mtr_t mtr; - for (ulong i = 0; i < TRX_SYS_N_RSEGS; i++) { + for (ulong i = 0; i < array_elements(trx_sys.temp_rsegs); i++) { mtr.start(); mtr.set_log_mode(MTR_LOG_NO_REDO); - mtr_x_lock_space(fil_system.temp_space, &mtr); + mtr.x_lock_space(fil_system.temp_space); buf_block_t* rblock = trx_rseg_header_create( fil_system.temp_space, i, 0, NULL, &mtr); - trx_rseg_t* rseg = trx_rseg_mem_create( - i, fil_system.temp_space, rblock->page.id().page_no()); - ut_ad(!rseg->is_persistent()); - ut_ad(!trx_sys.temp_rsegs[i]); - trx_sys.temp_rsegs[i] = rseg; + trx_sys.temp_rsegs[i].init(fil_system.temp_space, + rblock->page.id().page_no()); mtr.commit(); } } diff --git a/storage/innobase/trx/trx0sys.cc b/storage/innobase/trx/trx0sys.cc index bcde969eb41..52e246c5c4f 100644 --- a/storage/innobase/trx/trx0sys.cc +++ b/storage/innobase/trx/trx0sys.cc @@ -154,14 +154,13 @@ trx_sysf_create( then enter the kernel: we must do it in this order to conform to the latching order rules. */ - mtr_x_lock_space(fil_system.sys_space, mtr); + mtr->x_lock_space(fil_system.sys_space); compile_time_assert(TRX_SYS_SPACE == 0); /* Create the trx sys file block in a new allocated file segment */ block = fseg_create(fil_system.sys_space, TRX_SYS + TRX_SYS_FSEG_HEADER, mtr); - buf_block_dbg_add_level(block, SYNC_TRX_SYS_HEADER); ut_a(block->page.id() == page_id_t(0, TRX_SYS_PAGE_NO)); @@ -198,17 +197,67 @@ trx_sysf_create( ut_a(rblock->page.id() == page_id_t(0, FSP_FIRST_RSEG_PAGE_NO)); } -/** Create the instance */ -void -trx_sys_t::create() +void trx_sys_t::create() +{ + ut_ad(this == &trx_sys); + ut_ad(!is_initialised()); + m_initialised= true; + trx_list.create(); + rw_trx_hash.init(); +} + +uint32_t trx_sys_t::history_size() +{ + ut_ad(is_initialised()); + uint32_t size= 0; + for (auto &rseg : rseg_array) + { + rseg.latch.rd_lock(); + size+= rseg.history_size; + } + for (auto &rseg : rseg_array) + rseg.latch.rd_unlock(); + return size; +} + +bool trx_sys_t::history_exceeds(uint32_t threshold) { - ut_ad(this == &trx_sys); - ut_ad(!is_initialised()); - m_initialised = true; - trx_list.create(); - rseg_history_len= 0; + ut_ad(is_initialised()); + uint32_t size= 0; + bool exceeds= false; + size_t i; + for (i= 0; i < array_elements(rseg_array); i++) + { + rseg_array[i].latch.rd_lock(); + size+= rseg_array[i].history_size; + if (size > threshold) + { + exceeds= true; + i++; + break; + } + } + while (i) + rseg_array[--i].latch.rd_unlock(); + return exceeds; +} - rw_trx_hash.init(); +TPOOL_SUPPRESS_TSAN bool trx_sys_t::history_exists() +{ + ut_ad(is_initialised()); + for (auto &rseg : rseg_array) + if (rseg.history_size) + return true; + return false; +} + +TPOOL_SUPPRESS_TSAN uint32_t trx_sys_t::history_size_approx() const +{ + ut_ad(is_initialised()); + uint32_t size= 0; + for (auto &rseg : rseg_array) + size+= rseg.history_size; + return size; } /*****************************************************************//** @@ -226,10 +275,42 @@ trx_sys_create_sys_pages(void) mtr_commit(&mtr); } +/** Create a persistent rollback segment. +@param space_id system or undo tablespace id +@return pointer to new rollback segment +@retval nullptr on failure */ +static trx_rseg_t *trx_rseg_create(ulint space_id) +{ + trx_rseg_t *rseg= nullptr; + mtr_t mtr; + + mtr.start(); + + if (fil_space_t *space= mtr.x_lock_space(space_id)) + { + ut_ad(space->purpose == FIL_TYPE_TABLESPACE); + if (buf_block_t *sys_header= trx_sysf_get(&mtr)) + { + ulint rseg_id= trx_sys_rseg_find_free(sys_header); + if (buf_block_t *rblock= rseg_id == ULINT_UNDEFINED + ? nullptr : trx_rseg_header_create(space, rseg_id, 0, sys_header, + &mtr)) + { + ut_ad(trx_sysf_rseg_get_space(sys_header, rseg_id) == space_id); + rseg= &trx_sys.rseg_array[rseg_id]; + rseg->init(space, rblock->page.id().page_no()); + ut_ad(rseg->is_persistent()); + } + } + } + + mtr.commit(); + return rseg; +} + /** Create the rollback segments. @return whether the creation succeeded */ -bool -trx_sys_create_rsegs() +bool trx_sys_create_rsegs() { /* srv_available_undo_logs reflects the number of persistent rollback segments that have been initialized in the @@ -309,14 +390,11 @@ trx_sys_t::close() /* There can't be any active transactions. */ - for (ulint i = 0; i < TRX_SYS_N_RSEGS; ++i) { - if (trx_rseg_t* rseg = rseg_array[i]) { - trx_rseg_mem_free(rseg); - } - - if (trx_rseg_t* rseg = temp_rsegs[i]) { - trx_rseg_mem_free(rseg); - } + for (ulint i = 0; i < array_elements(temp_rsegs); ++i) { + temp_rsegs[i].destroy(); + } + for (ulint i = 0; i < array_elements(rseg_array); ++i) { + rseg_array[i].destroy(); } ut_a(trx_list.empty()); diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc index cf8fa17cf1a..ad49d3e9c8e 100644 --- a/storage/innobase/trx/trx0trx.cc +++ b/storage/innobase/trx/trx0trx.cc @@ -64,12 +64,6 @@ const byte timestamp_max_bytes[7] = { static const ulint MAX_DETAILED_ERROR_LEN = 256; -/** Set of table_id */ -typedef std::set< - table_id_t, - std::less<table_id_t>, - ut_allocator<table_id_t> > table_id_set; - /*************************************************************//** Set detailed error message for the transaction. */ void @@ -119,9 +113,7 @@ trx_init( trx->lock.n_rec_locks = 0; - trx->dict_operation = TRX_DICT_OP_NONE; - - trx->table_id = 0; + trx->dict_operation = false; trx->error_state = DB_SUCCESS; @@ -139,9 +131,7 @@ trx_init( trx->will_lock = false; - trx->ddl = false; - - trx->internal = false; + trx->bulk_insert = false; ut_d(trx->start_file = 0); @@ -149,8 +139,6 @@ trx_init( trx->magic_n = TRX_MAGIC_N; - trx->lock.que_state = TRX_QUE_RUNNING; - trx->last_sql_stat_start.least_undo_no = 0; ut_ad(!trx->read_view.is_open()); @@ -160,7 +148,6 @@ trx_init( trx->lock.table_cached = 0; #ifdef WITH_WSREP ut_ad(!trx->wsrep); - ut_ad(!trx->wsrep_UK_scan); #endif /* WITH_WSREP */ } @@ -187,18 +174,16 @@ struct TrxFactory { trx->rw_trx_hash_pins = 0; trx_init(trx); - trx->dict_operation_lock_mode = 0; - - trx->xid = UT_NEW_NOKEY(xid_t()); + trx->dict_operation_lock_mode = false; trx->detailed_error = reinterpret_cast<char*>( ut_zalloc_nokey(MAX_DETAILED_ERROR_LEN)); trx->lock.lock_heap = mem_heap_create_typed( 1024, MEM_HEAP_FOR_LOCK_HEAP); + pthread_cond_init(&trx->lock.cond, nullptr); - lock_trx_lock_list_init(&trx->lock.trx_locks); - + UT_LIST_INIT(trx->lock.trx_locks, &lock_t::trx_locks); UT_LIST_INIT(trx->lock.evicted_tables, &dict_table_t::table_LRU); @@ -206,7 +191,7 @@ struct TrxFactory { trx->trx_savepoints, &trx_named_savept_t::trx_savepoints); - mutex_create(LATCH_ID_TRX, &trx->mutex); + trx->mutex_init(); } /** Release resources held by the transaction object. @@ -230,20 +215,21 @@ struct TrxFactory { ut_a(trx->lock.wait_lock == NULL); ut_a(trx->lock.wait_thr == NULL); - ut_a(trx->dict_operation_lock_mode == 0); + ut_a(!trx->dict_operation_lock_mode); if (trx->lock.lock_heap != NULL) { mem_heap_free(trx->lock.lock_heap); trx->lock.lock_heap = NULL; } + pthread_cond_destroy(&trx->lock.cond); + ut_a(UT_LIST_GET_LEN(trx->lock.trx_locks) == 0); ut_ad(UT_LIST_GET_LEN(trx->lock.evicted_tables) == 0); - UT_DELETE(trx->xid); ut_free(trx->detailed_error); - mutex_free(&trx->mutex); + trx->mutex_destroy(); trx->mod_tables.~trx_mod_tables_t(); @@ -256,49 +242,47 @@ struct TrxFactory { }; /** The lock strategy for TrxPool */ -struct TrxPoolLock { - TrxPoolLock() { } - - /** Create the mutex */ - void create() - { - mutex_create(LATCH_ID_TRX_POOL, &m_mutex); - } +class TrxPoolLock +{ + mysql_mutex_t mutex; - /** Acquire the mutex */ - void enter() { mutex_enter(&m_mutex); } +public: + /** Create the mutex */ + void create() + { + mysql_mutex_init(trx_pool_mutex_key, &mutex, nullptr); + } - /** Release the mutex */ - void exit() { mutex_exit(&m_mutex); } + /** Acquire the mutex */ + void enter() { mysql_mutex_lock(&mutex); } - /** Free the mutex */ - void destroy() { mutex_free(&m_mutex); } + /** Release the mutex */ + void exit() { mysql_mutex_unlock(&mutex); } - /** Mutex to use */ - ib_mutex_t m_mutex; + /** Free the mutex */ + void destroy() { mysql_mutex_destroy(&mutex); } }; /** The lock strategy for the TrxPoolManager */ -struct TrxPoolManagerLock { - TrxPoolManagerLock() { } - - /** Create the mutex */ - void create() - { - mutex_create(LATCH_ID_TRX_POOL_MANAGER, &m_mutex); - } +class TrxPoolManagerLock +{ + mysql_mutex_t mutex; - /** Acquire the mutex */ - void enter() { mutex_enter(&m_mutex); } +public: + /** Create the mutex */ + void create() + { + mysql_mutex_init(trx_pool_manager_mutex_key, &mutex, nullptr); + } - /** Release the mutex */ - void exit() { mutex_exit(&m_mutex); } + /** Acquire the mutex */ + void enter() { mysql_mutex_lock(&mutex); } - /** Free the mutex */ - void destroy() { mutex_free(&m_mutex); } + /** Release the mutex */ + void exit() { mysql_mutex_unlock(&mutex); } - /** Mutex to use */ - ib_mutex_t m_mutex; + /** Free the mutex */ + void destroy() { mysql_mutex_destroy(&mutex); } }; /** Use explicit mutexes for the trx_t pool and its manager. */ @@ -370,10 +354,6 @@ trx_t *trx_create() ut_ad(trx->lock.rec_cached == 0); ut_ad(UT_LIST_GET_LEN(trx->lock.evicted_tables) == 0); -#ifdef WITH_WSREP - ut_ad(!trx->wsrep_UK_scan); -#endif /* WITH_WSREP */ - trx_sys.register_trx(trx); return(trx); @@ -382,19 +362,25 @@ trx_t *trx_create() /** Free the memory to trx_pools */ void trx_t::free() { +#ifdef HAVE_MEM_CHECK + if (xid.is_null()) + MEM_MAKE_DEFINED(&xid, sizeof xid); + else + MEM_MAKE_DEFINED(&xid.data[xid.gtrid_length + xid.bqual_length], + sizeof xid.data - (xid.gtrid_length + xid.bqual_length)); +#endif MEM_CHECK_DEFINED(this, sizeof *this); ut_ad(!n_mysql_tables_in_use); ut_ad(!mysql_log_file_name); ut_ad(!mysql_n_tables_locked); - ut_ad(!internal); ut_ad(!will_lock); ut_ad(error_state == DB_SUCCESS); ut_ad(magic_n == TRX_MAGIC_N); ut_ad(!read_only); ut_ad(!lock.wait_lock); - dict_operation= TRX_DICT_OP_NONE; + dict_operation= false; trx_sys.deregister_trx(this); assert_freed(); trx_sys.rw_trx_hash.put_pins(this); @@ -410,8 +396,6 @@ void trx_t::free() autoinc_locks= NULL; } - mod_tables.clear(); - MEM_NOACCESS(&n_ref, sizeof n_ref); /* do not poison mutex */ MEM_NOACCESS(&id, sizeof id); @@ -436,7 +420,6 @@ void trx_t::free() MEM_NOACCESS(&start_time, sizeof start_time); MEM_NOACCESS(&start_time_micro, sizeof start_time_micro); MEM_NOACCESS(&commit_lsn, sizeof commit_lsn); - MEM_NOACCESS(&table_id, sizeof table_id); MEM_NOACCESS(&mysql_thd, sizeof mysql_thd); MEM_NOACCESS(&mysql_log_file_name, sizeof mysql_log_file_name); MEM_NOACCESS(&mysql_log_offset, sizeof mysql_log_offset); @@ -461,8 +444,6 @@ void trx_t::free() MEM_NOACCESS(&fts_trx, sizeof fts_trx); MEM_NOACCESS(&fts_next_doc_id, sizeof fts_next_doc_id); MEM_NOACCESS(&flush_tables, sizeof flush_tables); - MEM_NOACCESS(&ddl, sizeof ddl); - MEM_NOACCESS(&internal, sizeof internal); #ifdef UNIV_DEBUG MEM_NOACCESS(&start_line, sizeof start_line); MEM_NOACCESS(&start_file, sizeof start_file); @@ -470,10 +451,6 @@ void trx_t::free() MEM_NOACCESS(&xid, sizeof xid); MEM_NOACCESS(&mod_tables, sizeof mod_tables); MEM_NOACCESS(&detailed_error, sizeof detailed_error); -#ifdef WITH_WSREP - ut_ad(!wsrep_UK_scan); - MEM_NOACCESS(&wsrep_UK_scan, sizeof wsrep_UK_scan); -#endif /* WITH_WSREP */ MEM_NOACCESS(&magic_n, sizeof magic_n); trx_pools->mem_free(this); } @@ -496,9 +473,9 @@ inline void trx_t::commit_state() makes modifications to the database, will get an lsn larger than the committing transaction T. In the case where the log flush fails, and T never gets committed, also T2 will never get committed. */ - trx_mutex_enter(this); + mutex.wr_lock(); state= TRX_STATE_COMMITTED_IN_MEMORY; - trx_mutex_exit(this); + mutex.wr_unlock(); ut_ad(id || !is_referenced()); } @@ -511,7 +488,7 @@ inline void trx_t::release_locks() if (UT_LIST_GET_LEN(lock.trx_locks)) { lock_release(this); - lock.n_rec_locks = 0; + ut_ad(!lock.n_rec_locks); ut_ad(UT_LIST_GET_LEN(lock.trx_locks) == 0); ut_ad(ib_vector_is_empty(autoinc_locks)); mem_heap_empty(lock.lock_heap); @@ -539,6 +516,7 @@ trx_free_at_shutdown(trx_t *trx) trx->commit_state(); trx->release_locks(); + trx->mod_tables.clear(); trx_undo_free_at_shutdown(trx); ut_a(!trx->read_only); @@ -577,9 +555,6 @@ trx_resurrect_table_locks( trx_t* trx, /*!< in/out: transaction */ const trx_undo_t* undo) /*!< in: undo log */ { - mtr_t mtr; - table_id_set tables; - ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE) || trx_state_eq(trx, TRX_STATE_PREPARED)); ut_ad(undo->rseg == trx->rsegs.m_redo.rseg); @@ -588,7 +563,9 @@ trx_resurrect_table_locks( return; } - mtr_start(&mtr); + mtr_t mtr; + std::map<table_id_t, bool> tables; + mtr.start(); /* trx_rseg_mem_create() may have acquired an X-latch on this page, so we cannot acquire an S-latch. */ @@ -613,39 +590,39 @@ trx_resurrect_table_locks( trx_undo_rec_get_pars( undo_rec, &type, &cmpl_info, &updated_extern, &undo_no, &table_id); - tables.insert(table_id); + + tables.emplace(table_id, type == TRX_UNDO_EMPTY); undo_rec = trx_undo_get_prev_rec( block, page_offset(undo_rec), undo->hdr_page_no, undo->hdr_offset, false, &mtr); } while (undo_rec); - mtr_commit(&mtr); + mtr.commit(); - for (table_id_set::const_iterator i = tables.begin(); - i != tables.end(); i++) { + for (auto p : tables) { if (dict_table_t* table = dict_table_open_on_id( - *i, FALSE, DICT_TABLE_OP_LOAD_TABLESPACE)) { + p.first, FALSE, DICT_TABLE_OP_LOAD_TABLESPACE)) { if (!table->is_readable()) { - mutex_enter(&dict_sys.mutex); - dict_table_close(table, TRUE, FALSE); + dict_sys.lock(SRW_LOCK_CALL); + table->release(); dict_sys.remove(table); - mutex_exit(&dict_sys.mutex); + dict_sys.unlock(); continue; } if (trx->state == TRX_STATE_PREPARED) { - trx->mod_tables.insert( - trx_mod_tables_t::value_type(table, - 0)); + trx->mod_tables.emplace(table, 0); } - lock_table_ix_resurrect(table, trx); + + lock_table_resurrect(table, trx, + p.second ? LOCK_X : LOCK_IX); DBUG_LOG("ib_trx", "resurrect " << ib::hex(trx->id) - << " IX lock on " << table->name); + << " lock on " << table->name); - dict_table_close(table, FALSE, FALSE); + table->release(); } } } @@ -697,19 +674,13 @@ static void trx_resurrect(trx_undo_t *undo, trx_rseg_t *rseg, or will not qualify for purge limit criteria. So it is safe to increment this trx_ref_count w/o mutex protection. */ - ++trx->rsegs.m_redo.rseg->trx_ref_count; - *trx->xid= undo->xid; + trx->rsegs.m_redo.rseg->acquire(); + trx->xid= undo->xid; trx->id= undo->trx_id; trx->is_recovered= true; trx->start_time= start_time; trx->start_time_micro= start_time_micro; - - if (undo->dict_operation) - { - trx_set_dict_operation(trx, TRX_DICT_OP_TABLE); - if (!trx->table_id) - trx->table_id= undo->table_id; - } + trx->dict_operation= undo->dict_operation; trx_sys.rw_trx_hash.insert(trx); trx_sys.rw_trx_hash.put_pins(trx); @@ -748,31 +719,30 @@ dberr_t trx_lists_init_at_db_start() const ulonglong start_time_micro= microsecond_interval_timer(); uint64_t rows_to_undo = 0; - for (ulint i = 0; i < TRX_SYS_N_RSEGS; ++i) { + for (auto& rseg : trx_sys.rseg_array) { trx_undo_t* undo; - trx_rseg_t* rseg = trx_sys.rseg_array[i]; /* Some rollback segment may be unavailable, especially if the server was previously run with a non-default value of innodb_undo_logs. */ - if (rseg == NULL) { + if (!rseg.space) { continue; } /* Ressurrect other transactions. */ - for (undo = UT_LIST_GET_FIRST(rseg->undo_list); + for (undo = UT_LIST_GET_FIRST(rseg.undo_list); undo != NULL; undo = UT_LIST_GET_NEXT(undo_list, undo)) { trx_t *trx = trx_sys.find(0, undo->trx_id, false); if (!trx) { - trx_resurrect(undo, rseg, start_time, + trx_resurrect(undo, &rseg, start_time, start_time_micro, &rows_to_undo); } else { ut_ad(trx_state_eq(trx, TRX_STATE_ACTIVE) || trx_state_eq(trx, TRX_STATE_PREPARED)); ut_ad(trx->start_time == start_time); ut_ad(trx->is_recovered); - ut_ad(trx->rsegs.m_redo.rseg == rseg); - ut_ad(trx->rsegs.m_redo.rseg->trx_ref_count); + ut_ad(trx->rsegs.m_redo.rseg == &rseg); + ut_ad(rseg.is_referenced()); trx->rsegs.m_redo.undo = undo; if (undo->top_undo_no >= trx->undo_no) { @@ -816,7 +786,7 @@ static trx_rseg_t* trx_assign_rseg_low() ut_ad(srv_available_undo_logs == TRX_SYS_N_RSEGS); /* The first slot is always assigned to the system tablespace. */ - ut_ad(trx_sys.rseg_array[0]->space == fil_system.sys_space); + ut_ad(trx_sys.rseg_array[0].space == fil_system.sys_space); /* Choose a rollback segment evenly distributed between 0 and innodb_undo_logs-1 in a round-robin fashion, skipping those @@ -835,7 +805,7 @@ static trx_rseg_t* trx_assign_rseg_low() do { for (;;) { - rseg = trx_sys.rseg_array[slot]; + rseg = &trx_sys.rseg_array[slot]; #ifdef UNIV_DEBUG /* Ensure that we are not revisiting the same @@ -849,20 +819,20 @@ static trx_rseg_t* trx_assign_rseg_low() ut_d(if (!trx_rseg_n_slots_debug)) slot = (slot + 1) % TRX_SYS_N_RSEGS; - if (rseg == NULL) { + if (!rseg->space) { continue; } ut_ad(rseg->is_persistent()); if (rseg->space != fil_system.sys_space) { - if (rseg->skip_allocation + if (rseg->skip_allocation() || !srv_undo_tablespaces) { continue; } - } else if (trx_rseg_t* next - = trx_sys.rseg_array[slot]) { - if (next->space != fil_system.sys_space + } else if (const fil_space_t *space = + trx_sys.rseg_array[slot].space) { + if (space != fil_system.sys_space && srv_undo_tablespaces > 0) { /** If dedicated innodb_undo_tablespaces have @@ -878,15 +848,10 @@ static trx_rseg_t* trx_assign_rseg_low() /* By now we have only selected the rseg but not marked it allocated. By marking it allocated we are ensuring that it will never be selected for UNDO truncate purge. */ - mutex_enter(&rseg->mutex); - if (!rseg->skip_allocation) { - rseg->trx_ref_count++; - allocated = true; - } - mutex_exit(&rseg->mutex); + allocated = rseg->acquire_if_available(); } while (!allocated); - ut_ad(rseg->trx_ref_count > 0); + ut_ad(rseg->is_referenced()); ut_ad(rseg->is_persistent()); return(rseg); } @@ -902,7 +867,7 @@ trx_rseg_t *trx_t::assign_temp_rseg() /* Choose a temporary rollback segment between 0 and 127 in a round-robin fashion. */ static Atomic_counter<unsigned> rseg_slot; - trx_rseg_t* rseg = trx_sys.temp_rsegs[ + trx_rseg_t* rseg = &trx_sys.temp_rsegs[ rseg_slot++ & (TRX_SYS_N_RSEGS - 1)]; ut_ad(!rseg->is_persistent()); rsegs.m_noredo.rseg = rseg; @@ -911,7 +876,6 @@ trx_rseg_t *trx_t::assign_temp_rseg() trx_sys.register_rw(this); } - ut_ad(!rseg->is_persistent()); return(rseg); } @@ -939,7 +903,7 @@ trx_start_low( trx->auto_commit = thd_trx_is_auto_commit(trx->mysql_thd); trx->read_only = srv_read_only_mode - || (!trx->ddl && !trx->internal + || (!trx->dict_operation && thd_trx_is_read_only(trx->mysql_thd)); if (!trx->auto_commit) { @@ -949,7 +913,7 @@ trx_start_low( } #ifdef WITH_WSREP - trx->xid->null(); + trx->xid.null(); #endif /* WITH_WSREP */ ut_a(ib_vector_is_empty(trx->autoinc_locks)); @@ -971,7 +935,7 @@ trx_start_low( list too. */ if (!trx->read_only - && (trx->mysql_thd == 0 || read_write || trx->ddl)) { + && (!trx->mysql_thd || read_write || trx->dict_operation)) { /* Temporary rseg is assigned only if the transaction updates a temporary table */ @@ -999,12 +963,10 @@ trx_start_low( trx->start_time = time(NULL); trx->start_time_micro = trx->mysql_thd - ? thd_query_start_micro(trx->mysql_thd) + ? thd_start_utime(trx->mysql_thd) : microsecond_interval_timer(); ut_a(trx->error_state == DB_SUCCESS); - - MONITOR_INC(MONITOR_TRX_ACTIVE); } /** Set the serialisation number for a persistent committed transaction. @@ -1015,10 +977,9 @@ trx_serialise(trx_t* trx) { trx_rseg_t *rseg = trx->rsegs.m_redo.rseg; ut_ad(rseg); - ut_ad(mutex_own(&rseg->mutex)); if (rseg->last_page_no == FIL_NULL) { - mutex_enter(&purge_sys.pq_mutex); + mysql_mutex_lock(&purge_sys.pq_mutex); } trx_sys.assign_new_trx_no(trx); @@ -1030,7 +991,7 @@ trx_serialise(trx_t* trx) if (rseg->last_page_no == FIL_NULL) { purge_sys.purge_queue.push(TrxUndoRsegs(trx->rw_trx_hash_element->no, *rseg)); - mutex_exit(&purge_sys.pq_mutex); + mysql_mutex_unlock(&purge_sys.pq_mutex); } } @@ -1062,10 +1023,7 @@ trx_write_serialisation_history( mtr_t temp_mtr; temp_mtr.start(); temp_mtr.set_log_mode(MTR_LOG_NO_REDO); - - mutex_enter(&trx->rsegs.m_noredo.rseg->mutex); trx_undo_set_state_at_finish(undo, &temp_mtr); - mutex_exit(&trx->rsegs.m_noredo.rseg->mutex); temp_mtr.commit(); } @@ -1083,7 +1041,7 @@ trx_write_serialisation_history( ut_ad(!trx->read_only); ut_ad(!undo || undo->rseg == rseg); - mutex_enter(&rseg->mutex); + rseg->latch.wr_lock(); /* Assign the transaction serialisation number and add any undo log to the purge queue. */ @@ -1093,7 +1051,7 @@ trx_write_serialisation_history( trx_purge_add_undo_to_history(trx, undo, mtr); } - mutex_exit(&rseg->mutex); + rseg->latch.wr_unlock(); MONITOR_INC(MONITOR_TRX_COMMIT_UNDO); } @@ -1156,35 +1114,47 @@ trx_finalize_for_fts( trx->fts_trx = NULL; } -/**********************************************************************//** -If required, flushes the log to disk based on the value of -innodb_flush_log_at_trx_commit. */ -static -void -trx_flush_log_if_needed_low( -/*========================*/ - lsn_t lsn) /*!< in: lsn up to which logs are to be - flushed. */ + +extern "C" void thd_decrement_pending_ops(MYSQL_THD); + + +#include "../log/log0sync.h" + +/* + If required, initiates write and optionally flush of the log to + disk + @param lsn LSN up to which logs are to be flushed. + @param trx transaction; if trx->state is PREPARED, the function will + also wait for the flush to complete. +*/ +static void trx_flush_log_if_needed_low(lsn_t lsn, const trx_t *trx) { - bool flush = srv_file_flush_method != SRV_NOSYNC; + if (!srv_flush_log_at_trx_commit) + return; - switch (srv_flush_log_at_trx_commit) { - case 2: - /* Write the log but do not flush it to disk */ - flush = false; - /* fall through */ - case 1: - case 3: - /* Write the log and optionally flush it to disk */ - log_write_up_to(lsn, flush); - srv_inc_activity_count(); - return; - case 0: - /* Do nothing */ - return; - } + if (log_sys.get_flushed_lsn() > lsn) + return; - ut_error; + const bool flush= srv_file_flush_method != SRV_NOSYNC && + (srv_flush_log_at_trx_commit & 1); + + if (trx->state == TRX_STATE_PREPARED) + { + /* XA, which is used with binlog as well. + Be conservative, use synchronous wait.*/ +sync: + log_write_up_to(lsn, flush); + return; + } + + completion_callback cb; + if ((cb.m_param = innodb_thd_increment_pending_ops(trx->mysql_thd))) + { + cb.m_callback = (void (*)(void *)) thd_decrement_pending_ops; + log_write_up_to(lsn, flush, false, &cb); + } + else + goto sync; } /**********************************************************************//** @@ -1199,66 +1169,54 @@ trx_flush_log_if_needed( trx_t* trx) /*!< in/out: transaction */ { trx->op_info = "flushing log"; - trx_flush_log_if_needed_low(lsn); + trx_flush_log_if_needed_low(lsn, trx); trx->op_info = ""; } -/**********************************************************************//** -For each table that has been modified by the given transaction: update -its dict_table_t::update_time with the current timestamp. Clear the list -of the modified tables at the end. */ -static -void -trx_update_mod_tables_timestamp( -/*============================*/ - trx_t* trx) /*!< in: transaction */ +/** Process tables that were modified by the committing transaction. */ +inline void trx_t::commit_tables() { - /* consider using trx->start_time if calling time() is too - expensive here */ - const time_t now = time(NULL); - - trx_mod_tables_t::const_iterator end = trx->mod_tables.end(); - - for (trx_mod_tables_t::const_iterator it = trx->mod_tables.begin(); - it != end; - ++it) { - - /* This could be executed by multiple threads concurrently - on the same table object. This is fine because time_t is - word size or less. And _purely_ _theoretically_, even if - time_t write is not atomic, likely the value of 'now' is - the same in all threads and even if it is not, getting a - "garbage" in table->update_time is justified because - protecting it with a latch here would be too performance - intrusive. */ - dict_table_t* table = it->first; - table->update_time = now; - } + if (undo_no && !mod_tables.empty()) + { + const trx_id_t max_trx_id= trx_sys.get_max_trx_id(); + const auto now= start_time; - trx->mod_tables.clear(); + for (const auto &p : mod_tables) + { + dict_table_t *table= p.first; + table->update_time= now; + table->query_cache_inv_trx_id= max_trx_id; + } + } } /** Evict a table definition due to the rollback of ALTER TABLE. -@param[in] table_id table identifier */ -void trx_t::evict_table(table_id_t table_id) +@param table_id table identifier +@param reset_only whether to only reset dict_table_t::def_trx_id */ +void trx_t::evict_table(table_id_t table_id, bool reset_only) { ut_ad(in_rollback); - dict_table_t* table = dict_table_open_on_id( - table_id, true, DICT_TABLE_OP_OPEN_ONLY_IF_CACHED); + dict_table_t* table = dict_sys.find_table(table_id); if (!table) { return; } - if (!table->release()) { + table->def_trx_id = 0; + + if (auto ref_count = table->get_ref_count()) { /* This must be a DDL operation that is being rolled back in an active connection. */ - ut_a(table->get_ref_count() == 1); + ut_a(ref_count == 1); ut_ad(!is_recovered); ut_ad(mysql_thd); return; } + if (reset_only) { + return; + } + /* This table should only be locked by this transaction, if at all. */ ut_ad(UT_LIST_GET_LEN(table->locks) <= 1); const bool locked = UT_LIST_GET_LEN(table->locks); @@ -1285,10 +1243,8 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr) ut_ad(mysql_thd); ut_ad(state == TRX_STATE_ACTIVE); - /* Note: We are asserting without holding the lock mutex. But - that is OK because this transaction is not waiting and cannot - be rolled back and no new locks can (or should) be added - because it is flagged as a non-locking read-only transaction. */ + /* Note: We do not have to hold any lock_sys latch here, because + this is a non-locking transaction. */ ut_a(UT_LIST_GET_LEN(lock.trx_locks) == 0); /* This state change is not protected by any mutex, therefore @@ -1319,7 +1275,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr) /* Wait for any implicit-to-explicit lock conversions to cease, so that there will be no race condition in lock_release(). */ while (UNIV_UNLIKELY(is_referenced())) - ut_delay(srv_spin_wait_delay); + LF_BACKOFF(); } else ut_ad(read_only || !rsegs.m_redo.rseg); @@ -1330,7 +1286,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr) } else { - trx_update_mod_tables_timestamp(this); + commit_tables(); MONITOR_INC(MONITOR_TRX_RW_COMMIT); is_recovered= false; } @@ -1350,12 +1306,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr) ut_ad(UT_LIST_GET_LEN(lock.evicted_tables) == 0); if (trx_rseg_t *rseg= rsegs.m_redo.rseg) - { - mutex_enter(&rseg->mutex); - ut_ad(rseg->trx_ref_count > 0); - --rseg->trx_ref_count; - mutex_exit(&rseg->mutex); - } + rseg->release(); if (mtr) { @@ -1367,7 +1318,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr) } /* NOTE that we could possibly make a group commit more efficient - here: call os_thread_yield here to allow also other trxs to come + here: call std::this_thread::yield() here to allow also other trxs to come to commit! */ /*-------------------------------------*/ @@ -1393,7 +1344,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr) serialize all commits and prevent a group of transactions from gathering. */ - commit_lsn= mtr->commit_lsn(); + commit_lsn= undo_no || !xid.is_null() ? mtr->commit_lsn() : 0; if (!commit_lsn) /* Nothing to be done. */; else if (flush_log_later) @@ -1405,10 +1356,7 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr) ut_ad(!rsegs.m_noredo.undo); - /* Free all savepoints, starting from the first. */ - trx_named_savept_t *savep= UT_LIST_GET_FIRST(trx_savepoints); - - trx_roll_savepoints_free(this, savep); + savepoints_discard(); if (fts_trx) trx_finalize_for_fts(this, undo_no != 0); @@ -1422,21 +1370,25 @@ inline void trx_t::commit_in_memory(const mtr_t *mtr) wsrep= false; wsrep_commit_ordered(mysql_thd); } - lock.was_chosen_as_wsrep_victim= false; + ut_ad(!(lock.was_chosen_as_deadlock_victim & byte(~2U))); + lock.was_chosen_as_deadlock_victim= false; #endif /* WITH_WSREP */ - trx_mutex_enter(this); - dict_operation= TRX_DICT_OP_NONE; +} + +void trx_t::commit_cleanup() +{ + mutex.wr_lock(); + dict_operation= false; DBUG_LOG("trx", "Commit in memory: " << this); state= TRX_STATE_NOT_STARTED; + mod_tables.clear(); assert_freed(); trx_init(this); - trx_mutex_exit(this); + mutex.wr_unlock(); ut_a(error_state == DB_SUCCESS); - if (!srv_read_only_mode) - srv_wake_purge_thread_if_not_active(); } /** Commit the transaction in a mini-transaction. @@ -1448,16 +1400,19 @@ void trx_t::commit_low(mtr_t *mtr) ut_ad(!mtr == (aborted || !has_logged())); ut_ad(!mtr || !aborted); - /* undo_no is non-zero if we're doing the final commit. */ if (fts_trx && undo_no) { ut_a(!is_autocommit_non_locking()); - /* FTS-FIXME: Temporarily tolerate DB_DUPLICATE_KEY instead of - dying. This is a possible scenario if there is a crash between + /* MDEV-24088 FIXME: Invoke fts_commit() earlier (before possible + XA PREPARE), so that we will be able to return an error and rollback + the transaction, instead of violating consistency! + + The original claim about DB_DUPLICATE KEY was: + This is a possible scenario if there is a crash between insert to DELETED table committing and transaction committing. The fix would be able to return error from this function */ - if (dberr_t error= fts_commit(this)) - ut_a(error == DB_DUPLICATE_KEY); + if (ut_d(dberr_t error=) fts_commit(this)) + ut_ad(error == DB_DUPLICATE_KEY || error == DB_LOCK_WAIT_TIMEOUT); } #ifndef DBUG_OFF @@ -1494,7 +1449,7 @@ void trx_t::commit_low(mtr_t *mtr) } -void trx_t::commit() +void trx_t::commit_persist() { mtr_t *mtr= nullptr; mtr_t local_mtr; @@ -1507,6 +1462,15 @@ void trx_t::commit() commit_low(mtr); } + +void trx_t::commit() +{ + commit_persist(); + ut_d(for (const auto &p : mod_tables) ut_ad(!p.second.is_dropped())); + commit_cleanup(); +} + + /****************************************************************//** Prepares a transaction for commit/rollback. */ void @@ -1527,19 +1491,7 @@ trx_commit_or_rollback_prepare( case TRX_STATE_ACTIVE: case TRX_STATE_PREPARED: case TRX_STATE_PREPARED_RECOVERED: - /* If the trx is in a lock wait state, moves the waiting - query thread to the suspended state */ - - if (trx->lock.que_state == TRX_QUE_LOCK_WAIT) { - - ut_a(trx->lock.wait_thr != NULL); - trx->lock.wait_thr->state = QUE_THR_SUSPENDED; - trx->lock.wait_thr = NULL; - - trx->lock.que_state = TRX_QUE_RUNNING; - } - - ut_ad(trx->lock.n_active_thrs == 1); + trx->lock.wait_thr = NULL; return; case TRX_STATE_COMMITTED_IN_MEMORY: @@ -1592,14 +1544,11 @@ trx_commit_step( trx = thr_get_trx(thr); ut_a(trx->lock.wait_thr == NULL); - ut_a(trx->lock.que_state != TRX_QUE_LOCK_WAIT); trx_commit_or_rollback_prepare(trx); - trx->lock.que_state = TRX_QUE_COMMITTING; trx->commit(); ut_ad(trx->lock.wait_thr == NULL); - trx->lock.que_state = TRX_QUE_RUNNING; thr = NULL; } else { @@ -1627,17 +1576,12 @@ trx_commit_for_mysql( switch (trx->state) { case TRX_STATE_NOT_STARTED: - ut_d(trx->start_file = __FILE__); - ut_d(trx->start_line = __LINE__); - - trx_start_low(trx, true); - /* fall through */ + return DB_SUCCESS; case TRX_STATE_ACTIVE: case TRX_STATE_PREPARED: case TRX_STATE_PREPARED_RECOVERED: trx->op_info = "committing"; trx->commit(); - MONITOR_DEC(MONITOR_TRX_ACTIVE); trx->op_info = ""; return(DB_SUCCESS); case TRX_STATE_COMMITTED_IN_MEMORY: @@ -1685,12 +1629,18 @@ trx_mark_sql_stat_end( trx->undo_no = 0; /* fall through */ case TRX_STATE_ACTIVE: - trx->last_sql_stat_start.least_undo_no = trx->undo_no; - if (trx->fts_trx != NULL) { fts_savepoint_laststmt_refresh(trx); } + if (trx->is_bulk_insert()) { + /* Allow a subsequent INSERT into an empty table + if !unique_checks && !foreign_key_checks. */ + return; + } + + trx->last_sql_stat_start.least_undo_no = trx->undo_no; + trx->end_bulk_insert(); return; } @@ -1710,15 +1660,17 @@ trx_print_low( /*!< in: max query length to print, or 0 to use the default max length */ ulint n_rec_locks, - /*!< in: lock_number_of_rows_locked(&trx->lock) */ + /*!< in: trx->lock.n_rec_locks */ ulint n_trx_locks, /*!< in: length of trx->lock.trx_locks */ ulint heap_size) /*!< in: mem_heap_get_size(trx->lock.lock_heap) */ { - ibool newline; - - fprintf(f, "TRANSACTION " TRX_ID_FMT, trx_get_id_for_print(trx)); + if (const trx_id_t id = trx->id) { + fprintf(f, "TRANSACTION " TRX_ID_FMT, trx->id); + } else { + fprintf(f, "TRANSACTION (%p)", trx); + } switch (trx->state) { case TRX_STATE_NOT_STARTED: @@ -1759,27 +1711,18 @@ state_ok: (ulong) trx->mysql_n_tables_locked); } - newline = TRUE; - - /* trx->lock.que_state of an ACTIVE transaction may change - while we are not holding trx->mutex. We perform a dirty read - for performance reasons. */ - - switch (trx->lock.que_state) { - case TRX_QUE_RUNNING: - newline = FALSE; break; - case TRX_QUE_LOCK_WAIT: - fputs("LOCK WAIT ", f); break; - case TRX_QUE_ROLLING_BACK: - fputs("ROLLING BACK ", f); break; - case TRX_QUE_COMMITTING: - fputs("COMMITTING ", f); break; - default: - fprintf(f, "que state %lu ", (ulong) trx->lock.que_state); + bool newline = true; + + if (trx->in_rollback) { /* dirty read for performance reasons */ + fputs("ROLLING BACK ", f); + } else if (trx->lock.wait_lock) { + fputs("LOCK WAIT ", f); + } else { + newline = false; } if (n_trx_locks > 0 || heap_size > 400) { - newline = TRUE; + newline = true; fprintf(f, "%lu lock struct(s), heap size %lu," " %lu row lock(s)", @@ -1789,7 +1732,7 @@ state_ok: } if (trx->undo_no != 0) { - newline = TRUE; + newline = true; fprintf(f, ", undo log entries " TRX_ID_FMT, trx->undo_no); } @@ -1805,7 +1748,7 @@ state_ok: /**********************************************************************//** Prints info about a transaction. -The caller must hold lock_sys.mutex. +The caller must hold lock_sys.latch. When possible, use trx_print() instead. */ void trx_print_latched( @@ -1815,17 +1758,17 @@ trx_print_latched( ulint max_query_len) /*!< in: max query length to print, or 0 to use the default max length */ { - ut_ad(lock_mutex_own()); + lock_sys.assert_locked(); trx_print_low(f, trx, max_query_len, - lock_number_of_rows_locked(&trx->lock), + trx->lock.n_rec_locks, UT_LIST_GET_LEN(trx->lock.trx_locks), mem_heap_get_size(trx->lock.lock_heap)); } /**********************************************************************//** Prints info about a transaction. -Acquires and releases lock_sys.mutex. */ +Acquires and releases lock_sys.latch. */ void trx_print( /*======*/ @@ -1834,53 +1777,15 @@ trx_print( ulint max_query_len) /*!< in: max query length to print, or 0 to use the default max length */ { - ulint n_rec_locks; - ulint n_trx_locks; - ulint heap_size; - - lock_mutex_enter(); - n_rec_locks = lock_number_of_rows_locked(&trx->lock); - n_trx_locks = UT_LIST_GET_LEN(trx->lock.trx_locks); - heap_size = mem_heap_get_size(trx->lock.lock_heap); - lock_mutex_exit(); - - trx_print_low(f, trx, max_query_len, - n_rec_locks, n_trx_locks, heap_size); -} - -/*******************************************************************//** -Compares the "weight" (or size) of two transactions. Transactions that -have edited non-transactional tables are considered heavier than ones -that have not. -@return TRUE if weight(a) >= weight(b) */ -bool -trx_weight_ge( -/*==========*/ - const trx_t* a, /*!< in: transaction to be compared */ - const trx_t* b) /*!< in: transaction to be compared */ -{ - ibool a_notrans_edit; - ibool b_notrans_edit; - - /* If mysql_thd is NULL for a transaction we assume that it has - not edited non-transactional tables. */ - - a_notrans_edit = a->mysql_thd != NULL - && thd_has_edited_nontrans_tables(a->mysql_thd); - - b_notrans_edit = b->mysql_thd != NULL - && thd_has_edited_nontrans_tables(b->mysql_thd); - - if (a_notrans_edit != b_notrans_edit) { - - return(a_notrans_edit); - } - - /* Either both had edited non-transactional tables or both had - not, we fall back to comparing the number of altered/locked - rows. */ + ulint n_rec_locks, n_trx_locks, heap_size; + { + LockMutexGuard g{SRW_LOCK_CALL}; + n_rec_locks= trx->lock.n_rec_locks; + n_trx_locks= UT_LIST_GET_LEN(trx->lock.trx_locks); + heap_size= mem_heap_get_size(trx->lock.lock_heap); + } - return(TRX_WEIGHT(a) >= TRX_WEIGHT(b)); + trx_print_low(f, trx, max_query_len, n_rec_locks, n_trx_locks, heap_size); } /** Prepare a transaction. @@ -1897,11 +1802,7 @@ static lsn_t trx_prepare_low(trx_t *trx) mtr.start(); mtr.set_log_mode(MTR_LOG_NO_REDO); - - mutex_enter(&undo->rseg->mutex); trx_undo_set_state_at_prepare(trx, undo, false, &mtr); - mutex_exit(&undo->rseg->mutex); - mtr.commit(); } @@ -1912,8 +1813,7 @@ static lsn_t trx_prepare_low(trx_t *trx) return(0); } - trx_rseg_t* rseg = trx->rsegs.m_redo.rseg; - ut_ad(undo->rseg == rseg); + ut_ad(undo->rseg == trx->rsegs.m_redo.rseg); mtr.start(); @@ -1921,10 +1821,7 @@ static lsn_t trx_prepare_low(trx_t *trx) TRX_UNDO_PREPARED: these modifications to the file data structure define the transaction as prepared in the file-based world, at the serialization point of lsn. */ - - mutex_enter(&rseg->mutex); trx_undo_set_state_at_prepare(trx, undo, false, &mtr); - mutex_exit(&rseg->mutex); /* Make the XA PREPARE durable. */ mtr.commit(); @@ -1949,9 +1846,9 @@ trx_prepare( DBUG_EXECUTE_IF("ib_trx_crash_during_xa_prepare_step", DBUG_SUICIDE();); ut_a(trx->state == TRX_STATE_ACTIVE); - trx_mutex_enter(trx); + trx->mutex_lock(); trx->state = TRX_STATE_PREPARED; - trx_mutex_exit(trx); + trx->mutex_unlock(); if (lsn) { /* Depending on the my.cnf options, we may now write the log @@ -2000,7 +1897,7 @@ static my_bool trx_recover_for_mysql_callback(rw_trx_hash_element_t *element, trx_recover_for_mysql_callback_arg *arg) { DBUG_ASSERT(arg->len > 0); - mutex_enter(&element->mutex); + mysql_mutex_lock(&element->mutex); if (trx_t *trx= element->trx) { /* @@ -2022,11 +1919,11 @@ static my_bool trx_recover_for_mysql_callback(rw_trx_hash_element_t *element, << " in prepared state after recovery"; ib::info() << "Transaction contains changes to " << trx->undo_no << " rows"; - xid= *trx->xid; + xid= trx->xid; } } } - mutex_exit(&element->mutex); + mysql_mutex_unlock(&element->mutex); /* Do not terminate upon reaching arg->len; count all transactions */ return false; } @@ -2035,13 +1932,13 @@ static my_bool trx_recover_for_mysql_callback(rw_trx_hash_element_t *element, static my_bool trx_recover_reset_callback(rw_trx_hash_element_t *element, void*) { - mutex_enter(&element->mutex); + mysql_mutex_lock(&element->mutex); if (trx_t *trx= element->trx) { if (trx_state_eq(trx, TRX_STATE_PREPARED_RECOVERED)) trx->state= TRX_STATE_PREPARED; } - mutex_exit(&element->mutex); + mysql_mutex_unlock(&element->mutex); return false; } @@ -2090,29 +1987,29 @@ static my_bool trx_get_trx_by_xid_callback(rw_trx_hash_element_t *element, trx_get_trx_by_xid_callback_arg *arg) { my_bool found= 0; - mutex_enter(&element->mutex); + mysql_mutex_lock(&element->mutex); if (trx_t *trx= element->trx) { - trx_mutex_enter(trx); + trx->mutex_lock(); if (trx->is_recovered && (trx_state_eq(trx, TRX_STATE_PREPARED) || trx_state_eq(trx, TRX_STATE_PREPARED_RECOVERED)) && - arg->xid->eq(reinterpret_cast<XID*>(trx->xid))) + arg->xid->eq(&trx->xid)) { #ifdef WITH_WSREP /* The commit of a prepared recovered Galera transaction needs a valid trx->xid for invoking trx_sys_update_wsrep_checkpoint(). */ - if (!wsrep_is_wsrep_xid(trx->xid)) + if (!wsrep_is_wsrep_xid(&trx->xid)) #endif /* WITH_WSREP */ /* Invalidate the XID, so that subsequent calls will not find it. */ - trx->xid->null(); + trx->xid.null(); arg->trx= trx; found= 1; } - trx_mutex_exit(trx); + trx->mutex_unlock(); } - mutex_exit(&element->mutex); + mysql_mutex_unlock(&element->mutex); return found; } @@ -2193,65 +2090,24 @@ trx_start_if_not_started_low( ut_error; } -/*************************************************************//** -Starts a transaction for internal processing. */ -void -trx_start_internal_low( -/*===================*/ - trx_t* trx) /*!< in/out: transaction */ -{ - /* Ensure it is not flagged as an auto-commit-non-locking - transaction. */ - - trx->will_lock = true; - - trx->internal = true; - - trx_start_low(trx, true); -} - -/** Starts a read-only transaction for internal processing. -@param[in,out] trx transaction to be started */ -void -trx_start_internal_read_only_low( - trx_t* trx) +/** +Start a transaction for internal processing. +@param trx transaction +@param read_write whether writes may be performed */ +void trx_start_internal_low(trx_t *trx, bool read_write) { - /* Ensure it is not flagged as an auto-commit-non-locking - transaction. */ - - trx->will_lock = true; - - trx->internal = true; - - trx_start_low(trx, false); + trx->will_lock= true; + trx_start_low(trx, read_write); } -/*************************************************************//** -Starts the transaction for a DDL operation. */ -void -trx_start_for_ddl_low( -/*==================*/ - trx_t* trx, /*!< in/out: transaction */ - trx_dict_op_t op) /*!< in: dictionary operation type */ +/** Start a transaction for a DDL operation. +@param trx transaction */ +void trx_start_for_ddl_low(trx_t *trx) { - switch (trx->state) { - case TRX_STATE_NOT_STARTED: - /* Flag this transaction as a dictionary operation, so that - the data dictionary will be locked in crash recovery. */ - - trx_set_dict_operation(trx, op); - trx->ddl= true; - trx_start_internal_low(trx); - return; - - case TRX_STATE_ACTIVE: - case TRX_STATE_PREPARED: - case TRX_STATE_PREPARED_RECOVERED: - case TRX_STATE_COMMITTED_IN_MEMORY: - break; - } - - ut_error; + /* Flag this transaction as a dictionary operation, so that + the data dictionary will be locked in crash recovery. */ + trx->dict_operation= true; + trx_start_internal_low(trx, true); } /*************************************************************//** @@ -2285,16 +2141,3 @@ trx_set_rw_mode( trx->read_view.set_creator_trx_id(trx->id); } } - -bool trx_t::has_stats_table_lock() const -{ - for (lock_list::const_iterator it= lock.table_locks.begin(), - end= lock.table_locks.end(); it != end; ++it) - { - const lock_t *lock= *it; - if (lock && lock->un_member.tab_lock.table->is_stats_table()) - return true; - } - - return false; -} diff --git a/storage/innobase/trx/trx0undo.cc b/storage/innobase/trx/trx0undo.cc index 3d2d9752a42..da3f6b592fa 100644 --- a/storage/innobase/trx/trx0undo.cc +++ b/storage/innobase/trx/trx0undo.cc @@ -176,7 +176,6 @@ trx_undo_get_prev_rec_from_prev_page(buf_block_t *&block, uint16_t rec, block= buf_page_get(page_id_t(block->page.id().space(), prev_page_no), 0, shared ? RW_S_LATCH : RW_X_LATCH, mtr); - buf_block_dbg_add_level(block, SYNC_TRX_UNDO_PAGE); return trx_undo_page_get_last_rec(block, page_no, offset); } @@ -244,7 +243,6 @@ trx_undo_get_next_rec_from_next_page(buf_block_t *&block, uint32_t page_no, return NULL; block= buf_page_get(page_id_t(block->page.id().space(), next), 0, mode, mtr); - buf_block_dbg_add_level(block, SYNC_TRX_UNDO_PAGE); return trx_undo_page_get_first_rec(block, page_no, offset); } @@ -282,7 +280,6 @@ trx_undo_get_first_rec(const fil_space_t &space, uint32_t page_no, mtr_t *mtr) { block = buf_page_get(page_id_t(space.id, page_no), 0, mode, mtr); - buf_block_dbg_add_level(block, SYNC_TRX_UNDO_PAGE); if (trx_undo_rec_t *rec= trx_undo_page_get_first_rec(block, page_no, offset)) return rec; @@ -394,8 +391,6 @@ trx_undo_seg_create(fil_space_t *space, buf_block_t *rseg_hdr, ulint *id, return NULL; } - buf_block_dbg_add_level(block, SYNC_TRX_UNDO_PAGE); - mtr->undo_create(*block); trx_undo_page_init(*block); @@ -559,7 +554,7 @@ buf_block_t* trx_undo_add_page(trx_undo_t* undo, mtr_t* mtr) a pessimistic insert in a B-tree, and we must reserve the counterpart of the tree latch, which is the rseg mutex. */ - mutex_enter(&rseg->mutex); + rseg->latch.wr_lock(); buf_block_t* header_block = trx_undo_page_get( page_id_t(undo->rseg->space->id, undo->hdr_page_no), mtr); @@ -580,8 +575,6 @@ buf_block_t* trx_undo_add_page(trx_undo_t* undo, mtr_t* mtr) goto func_exit; } - ut_ad(rw_lock_get_x_lock_count(&new_block->lock) == 1); - buf_block_dbg_add_level(new_block, SYNC_TRX_UNDO_PAGE); undo->last_page_no = new_block->page.id().page_no(); mtr->undo_create(*new_block); @@ -593,7 +586,7 @@ buf_block_t* trx_undo_add_page(trx_undo_t* undo, mtr_t* mtr) rseg->curr_size++; func_exit: - mutex_exit(&rseg->mutex); + rseg->latch.wr_unlock(); return(new_block); } @@ -617,7 +610,6 @@ trx_undo_free_page( const ulint space = rseg->space->id; ut_a(hdr_page_no != page_no); - ut_ad(mutex_own(&(rseg->mutex))); buf_block_t* undo_block = trx_undo_page_get(page_id_t(space, page_no), mtr); @@ -631,7 +623,7 @@ trx_undo_free_page( fseg_free_page(TRX_UNDO_SEG_HDR + TRX_UNDO_FSEG_HEADER + header_block->frame, rseg->space, page_no, mtr); - buf_page_free(rseg->space, page_no, mtr, __FILE__, __LINE__); + buf_page_free(rseg->space, page_no, mtr); const fil_addr_t last_addr = flst_get_last( TRX_UNDO_SEG_HDR + TRX_UNDO_PAGE_LIST + header_block->frame); @@ -682,7 +674,7 @@ void trx_undo_truncate_end(trx_undo_t& undo, undo_no_t limit, bool is_temp) } trx_undo_rec_t* trunc_here = NULL; - mutex_enter(&undo.rseg->mutex); + undo.rseg->latch.wr_lock(); buf_block_t* undo_block = trx_undo_page_get( page_id_t(undo.rseg->space->id, undo.last_page_no), &mtr); @@ -702,13 +694,13 @@ void trx_undo_truncate_end(trx_undo_t& undo, undo_no_t limit, bool is_temp) if (undo.last_page_no != undo.hdr_page_no) { trx_undo_free_last_page(&undo, &mtr); - mutex_exit(&undo.rseg->mutex); + undo.rseg->latch.wr_unlock(); mtr.commit(); continue; } func_exit: - mutex_exit(&undo.rseg->mutex); + undo.rseg->latch.wr_unlock(); if (trunc_here) { mtr.write<2>(*undo_block, @@ -741,8 +733,6 @@ trx_undo_truncate_start( trx_undo_rec_t* last_rec; mtr_t mtr; - ut_ad(mutex_own(&(rseg->mutex))); - if (!limit) { return; } @@ -920,16 +910,13 @@ corrupted: max_trx_id = trx_id; } - mutex_enter(&rseg->mutex); trx_undo_t* undo = trx_undo_mem_create( rseg, id, trx_id, &xid, page_no, offset); - mutex_exit(&rseg->mutex); if (!undo) { return undo; } undo->dict_operation = undo_header[TRX_UNDO_DICT_TRANS]; - undo->table_id = mach_read_from_8(undo_header + TRX_UNDO_TABLE_ID); undo->size = flst_get_len(TRX_UNDO_SEG_HDR + TRX_UNDO_PAGE_LIST + block->frame); @@ -982,8 +969,6 @@ trx_undo_mem_create( { trx_undo_t* undo; - ut_ad(mutex_own(&(rseg->mutex))); - ut_a(id < TRX_RSEG_N_SLOTS); undo = static_cast<trx_undo_t*>(ut_malloc_nokey(sizeof(*undo))); @@ -1027,8 +1012,6 @@ trx_undo_mem_init_for_reuse( const XID* xid, /*!< in: X/Open XA transaction identification*/ uint16_t offset) /*!< in: undo log header byte offset on page */ { - ut_ad(mutex_own(&((undo->rseg)->mutex))); - ut_a(undo->id < TRX_RSEG_N_SLOTS); undo->state = TRX_UNDO_ACTIVE; @@ -1056,9 +1039,6 @@ trx_undo_create(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** undo, dberr_t* err, mtr_t* mtr) { ulint id; - - ut_ad(mutex_own(&(rseg->mutex))); - buf_block_t* block = trx_undo_seg_create( rseg->space, trx_rsegf_get(rseg->space, rseg->page_no, mtr), &id, err, mtr); @@ -1071,7 +1051,7 @@ trx_undo_create(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** undo, uint16_t offset = trx_undo_header_create(block, trx->id, mtr); - *undo = trx_undo_mem_create(rseg, id, trx->id, trx->xid, + *undo = trx_undo_mem_create(rseg, id, trx->id, &trx->xid, block->page.id().page_no(), offset); if (*undo == NULL) { *err = DB_OUT_OF_MEMORY; @@ -1081,21 +1061,12 @@ trx_undo_create(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** undo, return block; } - switch (trx_get_dict_operation(trx)) { - case TRX_DICT_OP_NONE: - break; - case TRX_DICT_OP_INDEX: - /* Do not discard the table on recovery. */ - trx->table_id = 0; - /* fall through */ - case TRX_DICT_OP_TABLE: - (*undo)->table_id = trx->table_id; - (*undo)->dict_operation = TRUE; + if (trx->dict_operation) { + (*undo)->dict_operation = true; mtr->write<1,mtr_t::MAYBE_NOP>(*block, block->frame + offset + TRX_UNDO_DICT_TRANS, 1U); mtr->write<8,mtr_t::MAYBE_NOP>(*block, block->frame + offset - + TRX_UNDO_TABLE_ID, - trx->table_id); + + TRX_UNDO_TABLE_ID, 0U); } *err = DB_SUCCESS; @@ -1116,8 +1087,6 @@ buf_block_t* trx_undo_reuse_cached(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** pundo, mtr_t* mtr) { - ut_ad(mutex_own(&rseg->mutex)); - trx_undo_t* undo = UT_LIST_GET_FIRST(rseg->undo_cached); if (!undo) { return NULL; @@ -1133,8 +1102,6 @@ trx_undo_reuse_cached(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** pundo, return NULL; } - buf_block_dbg_add_level(block, SYNC_TRX_UNDO_PAGE); - UT_LIST_REMOVE(rseg->undo_cached, undo); MONITOR_DEC(MONITOR_NUM_UNDO_SLOT_CACHED); @@ -1142,27 +1109,18 @@ trx_undo_reuse_cached(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** pundo, uint16_t offset = trx_undo_header_create(block, trx->id, mtr); - trx_undo_mem_init_for_reuse(undo, trx->id, trx->xid, offset); + trx_undo_mem_init_for_reuse(undo, trx->id, &trx->xid, offset); if (rseg != trx->rsegs.m_redo.rseg) { return block; } - switch (trx_get_dict_operation(trx)) { - case TRX_DICT_OP_NONE: - return block; - case TRX_DICT_OP_INDEX: - /* Do not discard the table on recovery. */ - trx->table_id = 0; - /* fall through */ - case TRX_DICT_OP_TABLE: - undo->table_id = trx->table_id; + if (trx->dict_operation) { undo->dict_operation = TRUE; mtr->write<1,mtr_t::MAYBE_NOP>(*block, block->frame + offset + TRX_UNDO_DICT_TRANS, 1U); mtr->write<8,mtr_t::MAYBE_NOP>(*block, block->frame + offset - + TRX_UNDO_TABLE_ID, - trx->table_id); + + TRX_UNDO_TABLE_ID, 0U); } return block; @@ -1186,12 +1144,12 @@ trx_undo_assign(trx_t* trx, dberr_t* err, mtr_t* mtr) return buf_page_get_gen( page_id_t(undo->rseg->space->id, undo->last_page_no), 0, RW_X_LATCH, undo->guess_block, - BUF_GET, __FILE__, __LINE__, mtr, err); + BUF_GET, mtr, err); } trx_rseg_t* rseg = trx->rsegs.m_redo.rseg; - mutex_enter(&rseg->mutex); + rseg->latch.wr_lock(); buf_block_t* block = trx_undo_reuse_cached( trx, rseg, &trx->rsegs.m_redo.undo, mtr); @@ -1209,7 +1167,7 @@ trx_undo_assign(trx_t* trx, dberr_t* err, mtr_t* mtr) UT_LIST_ADD_FIRST(rseg->undo_list, trx->rsegs.m_redo.undo); func_exit: - mutex_exit(&rseg->mutex); + rseg->latch.wr_unlock(); return block; } @@ -1226,8 +1184,7 @@ buf_block_t* trx_undo_assign_low(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** undo, dberr_t* err, mtr_t* mtr) { - const bool is_temp __attribute__((unused)) = rseg == trx->rsegs.m_noredo.rseg; - + ut_d(const bool is_temp = rseg == trx->rsegs.m_noredo.rseg); ut_ad(rseg == trx->rsegs.m_redo.rseg || rseg == trx->rsegs.m_noredo.rseg); ut_ad(undo == (is_temp @@ -1240,7 +1197,7 @@ trx_undo_assign_low(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** undo, return buf_page_get_gen( page_id_t(rseg->space->id, (*undo)->last_page_no), 0, RW_X_LATCH, (*undo)->guess_block, - BUF_GET, __FILE__, __LINE__, mtr, err); + BUF_GET, mtr, err); } DBUG_EXECUTE_IF( @@ -1248,7 +1205,7 @@ trx_undo_assign_low(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** undo, *err = DB_TOO_MANY_CONCURRENT_TRXS; return NULL; ); - mutex_enter(&rseg->mutex); + rseg->latch.wr_lock(); buf_block_t* block = trx_undo_reuse_cached(trx, rseg, undo, mtr); @@ -1265,7 +1222,7 @@ trx_undo_assign_low(trx_t* trx, trx_rseg_t* rseg, trx_undo_t** undo, UT_LIST_ADD_FIRST(rseg->undo_list, *undo); func_exit: - mutex_exit(&rseg->mutex); + rseg->latch.wr_unlock(); return block; } @@ -1320,7 +1277,7 @@ void trx_undo_set_state_at_prepare(trx_t *trx, trx_undo_t *undo, bool rollback, /*------------------------------*/ ut_ad(undo->state == TRX_UNDO_ACTIVE); undo->state = TRX_UNDO_PREPARED; - undo->xid = *trx->xid; + undo->xid = trx->xid; /*------------------------------*/ mtr->write<2>(*block, TRX_UNDO_SEG_HDR + TRX_UNDO_STATE + block->frame, @@ -1341,13 +1298,14 @@ void trx_undo_commit_cleanup(trx_undo_t *undo) trx_rseg_t* rseg = undo->rseg; ut_ad(rseg->space == fil_system.temp_space); - mutex_enter(&rseg->mutex); + rseg->latch.wr_lock(); UT_LIST_REMOVE(rseg->undo_list, undo); if (undo->state == TRX_UNDO_CACHED) { UT_LIST_ADD_FIRST(rseg->undo_cached, undo); MONITOR_INC(MONITOR_NUM_UNDO_SLOT_CACHED); + undo = nullptr; } else { ut_ad(undo->state == TRX_UNDO_TO_PURGE); @@ -1356,11 +1314,10 @@ void trx_undo_commit_cleanup(trx_undo_t *undo) ut_ad(rseg->curr_size > undo->size); rseg->curr_size -= undo->size; - - ut_free(undo); } - mutex_exit(&rseg->mutex); + rseg->latch.wr_unlock(); + ut_free(undo); } /** At shutdown, frees the undo logs of a transaction. */ |