From 542278479261aa9e5124d0095fb4f77ba5f3fa63 Mon Sep 17 00:00:00 2001 From: Vlad Lesin Date: Fri, 12 May 2023 17:20:03 +0300 Subject: [PATCH 1/7] MDEV-31256 fil_node_open_file() releases fil_system.mutex allowing other thread to open its file node There is room between mutex_exit(&fil_system.mutex) and mutex_enter(&fil_system.mutex) calls in fil_node_open_file(). During this room another thread can open the node, and ut_ad(!node->is_open()) assertion in fil_node_open_file_low() can fail. The fix is not to open node if it was already opened by another thread. --- storage/innobase/fil/fil0fil.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/storage/innobase/fil/fil0fil.cc b/storage/innobase/fil/fil0fil.cc index fd2404a009a..58261d27d8e 100644 --- a/storage/innobase/fil/fil0fil.cc +++ b/storage/innobase/fil/fil0fil.cc @@ -458,7 +458,9 @@ static bool fil_node_open_file(fil_node_t *node) } } - return fil_node_open_file_low(node); + /* The node can be opened beween releasing and acquiring fil_system.mutex + in the above code */ + return node->is_open() || fil_node_open_file_low(node); } /** Close the file handle. */ From f2c17cc9d9bcd634887846d3064bcb71243f9cc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20M=C3=A4kel=C3=A4?= Date: Fri, 19 May 2023 15:20:07 +0300 Subject: [PATCH 2/7] MDEV-29911 InnoDB recovery and mariadb-backup --prepare fail to report detailed progress This is a 10.6 port of commit 2f9e264781f702b8da1ed418ac9f4f5e8f8aa843 from MariaDB Server 10.9 that is missing some optimization due to a more complex redo log format and recovery logic (which was simplified in commit 685d958e38b825ad9829be311f26729cccf37c46). The progress reporting of InnoDB crash recovery was rather intermittent. Nothing was reported during the single-threaded log record parsing, which could consume minutes when parsing a large log. During log application, there only was progress reporting in background threads that would be invoked on data page read completion. The progress reporting here will be detailed like this: InnoDB: Starting crash recovery from checkpoint LSN=628599973,5653727799 InnoDB: Read redo log up to LSN=1963895808 InnoDB: Multi-batch recovery needed at LSN 2534560930 InnoDB: Read redo log up to LSN=3312233472 InnoDB: Read redo log up to LSN=1599646720 InnoDB: Read redo log up to LSN=2160831488 InnoDB: To recover: LSN 2806789376/2806819840; 195082 pages InnoDB: To recover: LSN 2806789376/2806819840; 63507 pages InnoDB: Read redo log up to LSN=3195776000 InnoDB: Read redo log up to LSN=3687099392 InnoDB: Read redo log up to LSN=4165315584 InnoDB: To recover: LSN 4374395699/4374440960; 241454 pages InnoDB: To recover: LSN 4374395699/4374440960; 123701 pages InnoDB: Read redo log up to LSN=4508724224 InnoDB: Read redo log up to LSN=5094550528 InnoDB: To recover: 205230 pages The previous messages "Starting a batch to recover" or "Starting a final batch to recover" will be replaced by "To recover: ... pages" messages. If a batch lasts longer than 15 seconds, then there will be progress reports every 15 seconds, showing the number of remaining pages. For the non-final batch, the "To recover:" message includes two end LSN: that of the batch, and of the recovered log. This is the primary measure of progress. The batch will end once the number of pages to recover reaches 0. If recovery is possible in a single batch, the output will look like this, with a shorter "To recover:" message that counts only the remaining pages: InnoDB: Starting crash recovery from checkpoint LSN=628599973,5653727799 InnoDB: Read redo log up to LSN=1984539648 InnoDB: Read redo log up to LSN=2710875136 InnoDB: Read redo log up to LSN=3358895104 InnoDB: Read redo log up to LSN=3965299712 InnoDB: Read redo log up to LSN=4557417472 InnoDB: Read redo log up to LSN=5219527680 InnoDB: To recover: 450915 pages We will also speed up recovery by improving the memory management and implementing multi-threaded recovery of data pages that will not need to be read into the buffer pool ("fake read"). Log application in the "fake read" threads will be protected by an atomic being_recovered field and exclusive buf_page_t::lock. Recovery will reserve for data pages two thirds of the buffer pool, or 256 pages, whichever is smaller. Previously, we could only use at most one third of the buffer pool for buffered log records. This would typically mean that with large buffer pools, recovery unnecessary consisted of multiple batches. If recovery runs out of memory, it will "roll back" or "rewind" the current mini-transaction. The recv_sys.recovered_lsn and recv_sys.pages will correspond to the "out of memory LSN", at the end of the previous complete mini-transaction. If recovery runs out of memory while executing the final recovery batch, we can simply invoke recv_sys.apply(false) to make room, and resume parsing. If recovery runs out of memory before the final batch, we will scan the redo log to the end and check for any missing or inconsistent files. In this version of the patch, we will throw away any previously buffered recv_sys.pages and rescan the log from the checkpoint onwards. recv_sys_t::pages_it: A cached iterator to recv_sys.pages. recv_sys_t::is_memory_exhausted(): Remove. We will have out-of-memory handling deep inside recv_sys_t::parse(). recv_sys_t::rewind(), page_recv_t::recs_t::rewind(): Remove all log starting with a specific LSN. IORequest::write_complete(), IORequest::read_complete(): Replaces fil_aio_callback(). read_io_callback(), write_io_callback(): Replaces io_callback(). IORequest::fake_read_complete(), fake_io_callback(), os_fake_read(): Process a "fake read" request for concurrent recovery. recv_sys_t::apply_batch(): Choose a number of successive pages for a recovery batch. recv_sys_t::erase(recv_sys_t::map::iterator): Remove log records for a page whose recovery is not in progress. Log application threads will not invoke this; they will only set being_recovered=-1 to indicate that the entry is no longer needed. recv_sys_t::garbage_collect(): Remove all being_recovered=-1 entries. recv_sys_t::wait_for_pool(): Wait for some space to become available in the buffer pool. mlog_init_t::mark_ibuf_exist(): Avoid calls to recv_sys::recover_low() via ibuf_page_exists() and buf_page_get_low(). Such calls would lead to double locking of recv_sys.mutex, which depending on implementation could cause a deadlock. We will use lower-level calls to look up index pages. buf_LRU_block_remove_hashed(): Disable consistency checks for freed ROW_FORMAT=COMPRESSED pages. Their contents could be uninitialized garbage. This fixes an occasional failure of the test innodb.innodb_bulk_create_index_debug. Tested by: Matthias Leich --- storage/innobase/buf/buf0flu.cc | 1 + storage/innobase/buf/buf0lru.cc | 9 +- storage/innobase/buf/buf0rea.cc | 86 +- storage/innobase/fil/fil0fil.cc | 70 +- storage/innobase/include/buf0buf.h | 3 +- storage/innobase/include/buf0rea.h | 13 +- storage/innobase/include/log0recv.h | 161 +-- storage/innobase/include/os0file.h | 9 + storage/innobase/log/log0recv.cc | 1507 ++++++++++++++++----------- storage/innobase/os/os0file.cc | 90 +- 10 files changed, 1135 insertions(+), 814 deletions(-) diff --git a/storage/innobase/buf/buf0flu.cc b/storage/innobase/buf/buf0flu.cc index 68dbaee5e7d..67c79702ec8 100644 --- a/storage/innobase/buf/buf0flu.cc +++ b/storage/innobase/buf/buf0flu.cc @@ -2483,6 +2483,7 @@ ATTRIBUTE_COLD void buf_flush_page_cleaner_init() /** Flush the buffer pool on shutdown. */ ATTRIBUTE_COLD void buf_flush_buffer_pool() { + ut_ad(!os_aio_pending_reads()); ut_ad(!buf_page_cleaner_is_active); ut_ad(!buf_flush_sync_lsn); diff --git a/storage/innobase/buf/buf0lru.cc b/storage/innobase/buf/buf0lru.cc index feb15fc226c..6cc4f1c3987 100644 --- a/storage/innobase/buf/buf0lru.cc +++ b/storage/innobase/buf/buf0lru.cc @@ -1095,7 +1095,11 @@ static bool buf_LRU_block_remove_hashed(buf_page_t *bpage, const page_id_t id, ut_a(!zip || !bpage->oldest_modification()); ut_ad(bpage->zip_size()); - + /* Skip consistency checks if the page was freed. + In recovery, we could get a sole FREE_PAGE record + and nothing else, for a ROW_FORMAT=COMPRESSED page. + Its contents would be garbage. */ + if (!bpage->is_freed()) switch (fil_page_get_type(page)) { case FIL_PAGE_TYPE_ALLOCATED: case FIL_PAGE_INODE: @@ -1226,6 +1230,7 @@ void buf_pool_t::corrupted_evict(buf_page_t *bpage, uint32_t state) buf_pool_t::hash_chain &chain= buf_pool.page_hash.cell_get(id.fold()); page_hash_latch &hash_lock= buf_pool.page_hash.lock_get(chain); + recv_sys.free_corrupted_page(id); mysql_mutex_lock(&mutex); hash_lock.lock(); @@ -1250,8 +1255,6 @@ void buf_pool_t::corrupted_evict(buf_page_t *bpage, uint32_t state) buf_LRU_block_free_hashed_page(reinterpret_cast(bpage)); mysql_mutex_unlock(&mutex); - - recv_sys.free_corrupted_page(id); } /** Update buf_pool.LRU_old_ratio. diff --git a/storage/innobase/buf/buf0rea.cc b/storage/innobase/buf/buf0rea.cc index c2ab50cc674..cf76a9bd93a 100644 --- a/storage/innobase/buf/buf0rea.cc +++ b/storage/innobase/buf/buf0rea.cc @@ -655,61 +655,35 @@ failed: return count; } -/** @return whether a page has been freed */ -inline bool fil_space_t::is_freed(uint32_t page) +/** Schedule a page for recovery. +@param space tablespace +@param page_id page identifier +@param recs log records +@param init page initialization, or nullptr if the page needs to be read */ +void buf_read_recover(fil_space_t *space, const page_id_t page_id, + page_recv_t &recs, recv_init *init) { - std::lock_guard freed_lock(freed_range_mutex); - return freed_ranges.contains(page); -} - -/** Issues read requests for pages which recovery wants to read in. -@param[in] space_id tablespace id -@param[in] page_nos array of page numbers to read, with the -highest page number the last in the array -@param[in] n number of page numbers in the array */ -void buf_read_recv_pages(ulint space_id, const uint32_t* page_nos, ulint n) -{ - fil_space_t* space = fil_space_t::get(space_id); - - if (!space) { - /* The tablespace is missing or unreadable: do nothing */ - return; - } - - const ulint zip_size = space->zip_size(); - - for (ulint i = 0; i < n; i++) { - - /* Ignore if the page already present in freed ranges. */ - if (space->is_freed(page_nos[i])) { - continue; - } - - const page_id_t cur_page_id(space_id, page_nos[i]); - - ulint limit = 0; - for (ulint j = 0; j < buf_pool.n_chunks; j++) { - limit += buf_pool.chunks[j].size / 2; - } - - if (os_aio_pending_reads() >= limit) { - os_aio_wait_until_no_pending_reads(false); - } - - space->reacquire(); - switch (buf_read_page_low(space, false, BUF_READ_ANY_PAGE, - cur_page_id, zip_size, true)) { - case DB_SUCCESS: case DB_SUCCESS_LOCKED_REC: - break; - default: - sql_print_error("InnoDB: Recovery failed to read page " - UINT32PF " from %s", - cur_page_id.page_no(), - space->chain.start->name); - } - } - - DBUG_PRINT("ib_buf", ("recovery read (%u pages) for %s", n, - space->chain.start->name)); - space->release(); + ut_ad(space->id == page_id.space()); + space->reacquire(); + const ulint zip_size= space->zip_size(); + + if (init) + { + if (buf_page_t *bpage= buf_page_init_for_read(BUF_READ_ANY_PAGE, page_id, + zip_size, true)) + { + ut_ad(bpage->in_file()); + os_fake_read(IORequest{bpage, (buf_tmp_buffer_t*) &recs, + UT_LIST_GET_FIRST(space->chain), + IORequest::READ_ASYNC}, ptrdiff_t(init)); + } + } + else if (dberr_t err= buf_read_page_low(space, false, BUF_READ_ANY_PAGE, + page_id, zip_size, true)) + { + if (err != DB_SUCCESS_LOCKED_REC) + sql_print_error("InnoDB: Recovery failed to read page " + UINT32PF " from %s", + page_id.page_no(), space->chain.start->name); + } } diff --git a/storage/innobase/fil/fil0fil.cc b/storage/innobase/fil/fil0fil.cc index 23f0cf75f39..71d96bf2a71 100644 --- a/storage/innobase/fil/fil0fil.cc +++ b/storage/innobase/fil/fil0fil.cc @@ -2823,53 +2823,55 @@ func_exit: #include -/** Callback for AIO completion */ -void fil_aio_callback(const IORequest &request) +void IORequest::write_complete() const { ut_ad(fil_validate_skip()); - ut_ad(request.node); + ut_ad(node); + ut_ad(is_write()); - if (!request.bpage) + if (!bpage) { ut_ad(!srv_read_only_mode); - if (request.type == IORequest::DBLWR_BATCH) - buf_dblwr.flush_buffered_writes_completed(request); + if (type == IORequest::DBLWR_BATCH) + buf_dblwr.flush_buffered_writes_completed(*this); else - ut_ad(request.type == IORequest::WRITE_ASYNC); -write_completed: - request.node->complete_write(); - } - else if (request.is_write()) - { - buf_page_write_complete(request); - goto write_completed; + ut_ad(type == IORequest::WRITE_ASYNC); } else + buf_page_write_complete(*this); + + node->complete_write(); + node->space->release(); +} + +void IORequest::read_complete() const +{ + ut_ad(fil_validate_skip()); + ut_ad(node); + ut_ad(is_read()); + ut_ad(bpage); + + /* IMPORTANT: since i/o handling for reads will read also the insert + buffer in fil_system.sys_space, we have to be very careful not to + introduce deadlocks. We never close fil_system.sys_space data files + and never issue asynchronous reads of change buffer pages. */ + const page_id_t id(bpage->id()); + + if (dberr_t err= bpage->read_complete(*node)) { - ut_ad(request.is_read()); - - /* IMPORTANT: since i/o handling for reads will read also the insert - buffer in fil_system.sys_space, we have to be very careful not to - introduce deadlocks. We never close fil_system.sys_space data - files and never issue asynchronous reads of change buffer pages. */ - const page_id_t id(request.bpage->id()); - - if (dberr_t err= request.bpage->read_complete(*request.node)) + if (recv_recovery_is_on() && !srv_force_recovery) { - if (recv_recovery_is_on() && !srv_force_recovery) - { - mysql_mutex_lock(&recv_sys.mutex); - recv_sys.set_corrupt_fs(); - mysql_mutex_unlock(&recv_sys.mutex); - } - - if (err != DB_FAIL) - ib::error() << "Failed to read page " << id.page_no() - << " from file '" << request.node->name << "': " << err; + mysql_mutex_lock(&recv_sys.mutex); + recv_sys.set_corrupt_fs(); + mysql_mutex_unlock(&recv_sys.mutex); } + + if (err != DB_FAIL) + ib::error() << "Failed to read page " << id.page_no() + << " from file '" << node->name << "': " << err; } - request.node->space->release(); + node->space->release(); } /** Flush to disk the writes in file spaces of the given type diff --git a/storage/innobase/include/buf0buf.h b/storage/innobase/include/buf0buf.h index 2b4732a64a0..d2b0112da7c 100644 --- a/storage/innobase/include/buf0buf.h +++ b/storage/innobase/include/buf0buf.h @@ -75,8 +75,7 @@ struct buf_pool_info_t ulint flush_list_len; /*!< Length of buf_pool.flush_list */ ulint n_pend_unzip; /*!< buf_pool.n_pend_unzip, pages pending decompress */ - ulint n_pend_reads; /*!< buf_pool.n_pend_reads, pages - pending read */ + ulint n_pend_reads; /*!< os_aio_pending_reads() */ ulint n_pending_flush_lru; /*!< Pages pending flush in LRU */ ulint n_pending_flush_list; /*!< Pages pending flush in FLUSH LIST */ diff --git a/storage/innobase/include/buf0rea.h b/storage/innobase/include/buf0rea.h index d898c5efc63..3dd085dda5c 100644 --- a/storage/innobase/include/buf0rea.h +++ b/storage/innobase/include/buf0rea.h @@ -102,12 +102,13 @@ which could result in a deadlock if the OS does not support asynchronous io. ulint buf_read_ahead_linear(const page_id_t page_id, ulint zip_size, bool ibuf); -/** Issues read requests for pages which recovery wants to read in. -@param[in] space_id tablespace id -@param[in] page_nos array of page numbers to read, with the -highest page number the last in the array -@param[in] n number of page numbers in the array */ -void buf_read_recv_pages(ulint space_id, const uint32_t* page_nos, ulint n); +/** Schedule a page for recovery. +@param space tablespace +@param page_id page identifier +@param recs log records +@param init page initialization, or nullptr if the page needs to be read */ +void buf_read_recover(fil_space_t *space, const page_id_t page_id, + page_recv_t &recs, recv_init *init); /** @name Modes used in read-ahead @{ */ /** read only pages belonging to the insert buffer tree */ diff --git a/storage/innobase/include/log0recv.h b/storage/innobase/include/log0recv.h index 5e8dc1c0160..2e4a4bfa794 100644 --- a/storage/innobase/include/log0recv.h +++ b/storage/innobase/include/log0recv.h @@ -45,9 +45,9 @@ recv_find_max_checkpoint(ulint* max_field) MY_ATTRIBUTE((nonnull, warn_unused_result)); ATTRIBUTE_COLD MY_ATTRIBUTE((nonnull, warn_unused_result)) -/** Apply any buffered redo log to a page that was just read from a data file. -@param[in,out] space tablespace -@param[in,out] bpage buffer pool page +/** Apply any buffered redo log to a page. +@param space tablespace +@param bpage buffer pool page @return whether the page was recovered correctly */ bool recv_recover_page(fil_space_t* space, buf_page_t* bpage); @@ -146,21 +146,15 @@ struct recv_dblwr_t list pages; }; -/** the recovery state and buffered records for a page */ +/** recv_sys.pages entry; protected by recv_sys.mutex */ struct page_recv_t { - /** Recovery state; protected by recv_sys.mutex */ - enum - { - /** not yet processed */ - RECV_NOT_PROCESSED, - /** not processed; the page will be reinitialized */ - RECV_WILL_NOT_READ, - /** page is being read */ - RECV_BEING_READ, - /** log records are being applied on the page */ - RECV_BEING_PROCESSED - } state= RECV_NOT_PROCESSED; + /** Recovery status: 0=not in progress, 1=log is being applied, + -1=log has been applied and the entry may be erased. + Transitions from 1 to -1 are NOT protected by recv_sys.mutex. */ + Atomic_relaxed being_processed{0}; + /** Whether reading the page will be skipped */ + bool skip_read= false; /** Latest written byte offset when applying the log records. @see mtr_t::m_last_offset */ uint16_t last_offset= 1; @@ -183,6 +177,9 @@ struct page_recv_t head= recs; tail= recs; } + /** Remove the last records for the page + @param start_lsn start of the removed log */ + ATTRIBUTE_COLD void rewind(lsn_t start_lsn); /** @return the last log snippet */ const log_rec_t* last() const { return tail; } @@ -201,8 +198,8 @@ struct page_recv_t iterator begin() { return head; } iterator end() { return NULL; } bool empty() const { ut_ad(!head == !tail); return !head; } - /** Clear and free the records; @see recv_sys_t::alloc() */ - inline void clear(); + /** Clear and free the records; @see recv_sys_t::add() */ + void clear(); } log; /** Trim old log records for a page. @@ -211,21 +208,27 @@ struct page_recv_t inline bool trim(lsn_t start_lsn); /** Ignore any earlier redo log records for this page. */ inline void will_not_read(); - /** @return whether the log records for the page are being processed */ - bool is_being_processed() const { return state == RECV_BEING_PROCESSED; } +}; + +/** A page initialization operation that was parsed from the redo log */ +struct recv_init +{ + /** log sequence number of the page initialization */ + lsn_t lsn; + /** Whether btr_page_create() avoided a read of the page. + At the end of the last recovery batch, mark_ibuf_exist() + will mark pages for which this flag is set. */ + bool created; }; /** Recovery system data structure */ struct recv_sys_t { - /** mutex protecting apply_log_recs and page_recv_t::state */ - mysql_mutex_t mutex; + using init= recv_init; + + /** mutex protecting this as well as some of page_recv_t */ + alignas(CPU_LEVEL1_DCACHE_LINESIZE) mysql_mutex_t mutex; private: - /** condition variable for - !apply_batch_on || pages.empty() || found_corrupt_log || found_corrupt_fs */ - pthread_cond_t cond; - /** whether recv_apply_hashed_log_recs() is running */ - bool apply_batch_on; /** set when finding a corrupt log block or record, or there is a log parsing buffer overflow */ bool found_corrupt_log; @@ -270,6 +273,9 @@ public: map pages; private: + /** iterator to pages, used by parse() */ + map::iterator pages_it; + /** Process a record that indicates that a tablespace size is being shrunk. @param page_id first page that is not in the file @param lsn log sequence number of the shrink operation */ @@ -296,23 +302,38 @@ public: inline size_t files_size(); void close_files() { files.clear(); files.shrink_to_fit(); } + /** Advance pages_it if it matches the iterator */ + void pages_it_invalidate(const map::iterator &p) + { + mysql_mutex_assert_owner(&mutex); + if (pages_it == p) + pages_it++; + } + /** Invalidate pages_it if it points to the given tablespace */ + void pages_it_invalidate(uint32_t space_id) + { + mysql_mutex_assert_owner(&mutex); + if (pages_it != pages.end() && pages_it->first.space() == space_id) + pages_it= pages.end(); + } + private: /** Attempt to initialize a page based on redo log records. - @param page_id page identifier - @param p iterator pointing to page_id + @param p iterator @param mtr mini-transaction @param b pre-allocated buffer pool block + @param init page initialization @return the recovered block @retval nullptr if the page cannot be initialized based on log records @retval -1 if the page cannot be recovered due to corruption */ - inline buf_block_t *recover_low(const page_id_t page_id, map::iterator &p, - mtr_t &mtr, buf_block_t *b); + inline buf_block_t *recover_low(const map::iterator &p, mtr_t &mtr, + buf_block_t *b, init &init); /** Attempt to initialize a page based on redo log records. @param page_id page identifier @return the recovered block @retval nullptr if the page cannot be initialized based on log records @retval -1 if the page cannot be recovered due to corruption */ - buf_block_t *recover_low(const page_id_t page_id); + ATTRIBUTE_COLD buf_block_t *recover_low(const page_id_t page_id); /** All found log files (multiple ones are possible if we are upgrading from before MariaDB Server 10.5.1) */ @@ -323,12 +344,27 @@ private: /** Base node of the redo block list. List elements are linked via buf_block_t::unzip_LRU. */ UT_LIST_BASE_NODE_T(buf_block_t) blocks; + + /** Allocate a block from the buffer pool for recv_sys.pages */ + ATTRIBUTE_COLD buf_block_t *add_block(); + + /** Wait for buffer pool to become available. + @param pages number of buffer pool pages needed */ + ATTRIBUTE_COLD void wait_for_pool(size_t pages); + + /** Free log for processed pages. */ + void garbage_collect(); + + /** Apply a recovery batch. + @param space_id current tablespace identifier + @param space current tablespace + @param free_block spare buffer block + @param last_batch whether it is possible to write more redo log + @return whether the caller must provide a new free_block */ + bool apply_batch(uint32_t space_id, fil_space_t *&space, + buf_block_t *&free_block, bool last_batch); + public: - /** Check whether the number of read redo log blocks exceeds the maximum. - Store last_stored_lsn if the recovery is not in the last phase. - @param[in,out] store whether to store page operations - @return whether the memory is exhausted */ - inline bool is_memory_exhausted(store_t *store); /** Apply buffered log to persistent data pages. @param last_batch whether it is possible to write more redo log */ void apply(bool last_batch); @@ -353,9 +389,10 @@ public: @param start_lsn start LSN of the mini-transaction @param lsn @see mtr_t::commit_lsn() @param l redo log snippet @see log_t::FORMAT_10_5 - @param len length of l, in bytes */ - inline void add(map::iterator it, lsn_t start_lsn, lsn_t lsn, - const byte *l, size_t len); + @param len length of l, in bytes + @return whether we ran out of memory */ + bool add(map::iterator it, lsn_t start_lsn, lsn_t lsn, + const byte *l, size_t len); /** Parse and register one mini-transaction in log_t::FORMAT_10_5. @param checkpoint_lsn the log sequence number of the latest checkpoint @@ -365,32 +402,31 @@ public: or corruption was noticed */ bool parse(lsn_t checkpoint_lsn, store_t *store, bool apply); - /** Clear a fully processed set of stored redo log records. */ - inline void clear(); + /** Erase log records for a page. */ + void erase(map::iterator p); + /** Clear a fully processed set of stored redo log records. */ + void clear(); + +private: + /** Rewind a mini-transaction when parse() runs out of memory. + @param end current position of the mini-transaction + @param begin start of the mini-transaction */ + ATTRIBUTE_COLD void rewind(const byte *end, const byte *begin) noexcept; + /** Report progress in terms of LSN or pages remaining */ + ATTRIBUTE_COLD void report_progress() const; +public: /** Determine whether redo log recovery progress should be reported. @param time the current time @return whether progress should be reported (the last report was at least 15 seconds ago) */ - bool report(time_t time) - { - if (time - progress_time < 15) - return false; - - progress_time= time; - return true; - } + bool report(time_t time); /** The alloc() memory alignment, in bytes */ static constexpr size_t ALIGNMENT= sizeof(size_t); - /** Allocate memory for log_rec_t - @param len allocation size, in bytes - @return pointer to len bytes of memory (never NULL) */ - inline void *alloc(size_t len); - /** Free a redo log snippet. - @param data buffer returned by alloc() */ + @param data buffer allocated in add() */ inline void free(const void *data); /** Remove records for a corrupted page. @@ -402,8 +438,6 @@ public: ATTRIBUTE_COLD void set_corrupt_fs(); /** Flag log file corruption during recovery. */ ATTRIBUTE_COLD void set_corrupt_log(); - /** Possibly finish a recovery batch. */ - inline void maybe_finish_batch(); /** @return whether data file corruption was found */ bool is_corrupt_fs() const { return UNIV_UNLIKELY(found_corrupt_fs); } @@ -421,13 +455,14 @@ public: } /** Try to recover a tablespace that was not readable earlier - @param p iterator, initially pointing to page_id_t{space_id,0}; - the records will be freed and the iterator advanced + @param p iterator @param name tablespace file name @param free_block spare buffer block - @return whether recovery failed */ - bool recover_deferred(map::iterator &p, const std::string &name, - buf_block_t *&free_block); + @return recovered tablespace + @retval nullptr if recovery failed */ + fil_space_t *recover_deferred(const map::iterator &p, + const std::string &name, + buf_block_t *&free_block); }; /** The recovery system */ diff --git a/storage/innobase/include/os0file.h b/storage/innobase/include/os0file.h index 7ac0579cc07..af3e7975e76 100644 --- a/storage/innobase/include/os0file.h +++ b/storage/innobase/include/os0file.h @@ -221,6 +221,10 @@ public: bool is_LRU() const { return (type & (WRITE_LRU ^ WRITE_ASYNC)) != 0; } bool is_async() const { return (type & (READ_SYNC ^ READ_ASYNC)) != 0; } + void write_complete() const; + void read_complete() const; + void fake_read_complete(os_offset_t offset) const; + /** If requested, free storage space associated with a section of the file. @param off byte offset from the start (SEEK_SET) @param len size of the hole in bytes @@ -1050,6 +1054,11 @@ int os_aio_init(); Frees the asynchronous io system. */ void os_aio_free(); +/** Submit a fake read request during crash recovery. +@param type fake read request +@param offset additional context */ +void os_fake_read(const IORequest &type, os_offset_t offset); + /** Request a read or write. @param type I/O request @param buf buffer diff --git a/storage/innobase/log/log0recv.cc b/storage/innobase/log/log0recv.cc index 78ba8b70a49..1d80345a5e0 100644 --- a/storage/innobase/log/log0recv.cc +++ b/storage/innobase/log/log0recv.cc @@ -748,7 +748,7 @@ static struct { retry: mysql_mutex_unlock(&log_sys.mutex); - bool fail= false; + fil_space_t *space= fil_system.sys_space; buf_block_t *free_block= buf_LRU_get_free_block(false); mysql_mutex_lock(&log_sys.mutex); mysql_mutex_lock(&recv_sys.mutex); @@ -765,11 +765,12 @@ retry: there were no buffered records. Either way, we must create a dummy tablespace with the latest known name, for dict_drop_index_tree(). */ + recv_sys.pages_it_invalidate(space_id); while (p != recv_sys.pages.end() && p->first.space() == space_id) { + ut_ad(!p->second.being_processed); recv_sys_t::map::iterator r= p++; - r->second.log.clear(); - recv_sys.pages.erase(r); + recv_sys.erase(r); } recv_spaces_t::iterator it{recv_spaces.find(space_id)}; if (it != recv_spaces.end()) @@ -792,11 +793,14 @@ retry: } } else - fail= recv_sys.recover_deferred(p, d->second.file_name, free_block); + space= recv_sys.recover_deferred(p, d->second.file_name, free_block); processed: - defers.erase(d++); - if (fail) + auto e= d++; + defers.erase(e); + if (!space) break; + if (space != fil_system.sys_space) + space->release(); if (free_block) continue; mysql_mutex_unlock(&recv_sys.mutex); @@ -807,7 +811,7 @@ processed: mysql_mutex_unlock(&recv_sys.mutex); if (free_block) buf_pool.free_block(free_block); - return fail; + return !space; } /** Create tablespace metadata for a data file that was initially @@ -927,28 +931,191 @@ free_space: } deferred_spaces; +/** Report an operation to create, delete, or rename a file during backup. +@param[in] space_id tablespace identifier +@param[in] type redo log type +@param[in] name file name (not NUL-terminated) +@param[in] len length of name, in bytes +@param[in] new_name new file name (NULL if not rename) +@param[in] new_len length of new_name, in bytes (0 if NULL) */ +void (*log_file_op)(ulint space_id, int type, + const byte* name, ulint len, + const byte* new_name, ulint new_len); + +void (*undo_space_trunc)(uint32_t space_id); + +void (*first_page_init)(ulint space_id); + +/** Information about initializing page contents during redo log processing. +FIXME: Rely on recv_sys.pages! */ +class mlog_init_t +{ + using map= std::map, + ut_allocator>>; + /** Map of page initialization operations. + FIXME: Merge this to recv_sys.pages! */ + map inits; + + /** Iterator to the last add() or will_avoid_read(), for speeding up + will_avoid_read(). */ + map::iterator i; +public: + /** Constructor */ + mlog_init_t() : i(inits.end()) {} + + /** Record that a page will be initialized by the redo log. + @param page_id page identifier + @param lsn log sequence number + @return whether the state was changed */ + bool add(const page_id_t page_id, lsn_t lsn) + { + mysql_mutex_assert_owner(&recv_sys.mutex); + const recv_init init = { lsn, false }; + std::pair p= + inits.insert(map::value_type(page_id, init)); + ut_ad(!p.first->second.created); + if (p.second) return true; + if (p.first->second.lsn >= lsn) return false; + p.first->second = init; + i = p.first; + return true; + } + + /** Get the last stored lsn of the page id and its respective + init/load operation. + @param page_id page identifier + @return the latest page initialization; + not valid after releasing recv_sys.mutex. */ + recv_init &last(page_id_t page_id) + { + mysql_mutex_assert_owner(&recv_sys.mutex); + return inits.find(page_id)->second; + } + + /** Determine if a page will be initialized or freed after a time. + @param page_id page identifier + @param lsn log sequence number + @return whether page_id will be freed or initialized after lsn */ + bool will_avoid_read(page_id_t page_id, lsn_t lsn) + { + mysql_mutex_assert_owner(&recv_sys.mutex); + if (i != inits.end() && i->first == page_id) + return i->second.lsn > lsn; + i = inits.lower_bound(page_id); + return i != inits.end() && i->first == page_id && i->second.lsn > lsn; + } + + /** At the end of each recovery batch, reset the 'created' flags. */ + void reset() + { + mysql_mutex_assert_owner(&recv_sys.mutex); + ut_ad(recv_no_ibuf_operations); + for (map::value_type &i : inits) + i.second.created= false; + } + + /** During the last recovery batch, mark whether there exist + buffered changes for the pages that were initialized + by buf_page_create() and still reside in the buffer pool. */ + void mark_ibuf_exist() + { + mysql_mutex_assert_owner(&recv_sys.mutex); + + for (const map::value_type &i : inits) + if (i.second.created) + { + auto &chain= buf_pool.page_hash.cell_get(i.first.fold()); + page_hash_latch &hash_lock= buf_pool.page_hash.lock_get(chain); + + hash_lock.lock_shared(); + buf_block_t *block= reinterpret_cast + (buf_pool.page_hash.get(i.first, chain)); + bool got_latch= block && block->page.lock.x_lock_try(); + hash_lock.unlock_shared(); + + if (!block) + continue; + + uint32_t state; + + if (!got_latch) + { + mysql_mutex_lock(&buf_pool.mutex); + block= reinterpret_cast + (buf_pool.page_hash.get(i.first, chain)); + if (!block) + { + mysql_mutex_unlock(&buf_pool.mutex); + continue; + } + + state= block->page.fix(); + mysql_mutex_unlock(&buf_pool.mutex); + if (state < buf_page_t::UNFIXED) + { + block->page.unfix(); + continue; + } + block->page.lock.x_lock(); + state= block->page.unfix(); + ut_ad(state < buf_page_t::READ_FIX); + if (state >= buf_page_t::UNFIXED && block->page.id() == i.first) + goto check_ibuf; + } + else + { + state= block->page.state(); + ut_ad(state >= buf_page_t::FREED); + ut_ad(state < buf_page_t::READ_FIX); + + if (state >= buf_page_t::UNFIXED) + { + check_ibuf: + mysql_mutex_unlock(&recv_sys.mutex); + if (ibuf_page_exists(block->page.id(), block->zip_size())) + block->page.set_ibuf_exist(); + mysql_mutex_lock(&recv_sys.mutex); + } + } + + block->page.lock.x_unlock(); + } + } + + /** Clear the data structure */ + void clear() { inits.clear(); i = inits.end(); } +}; + +static mlog_init_t mlog_init; + /** Try to recover a tablespace that was not readable earlier -@param p iterator, initially pointing to page_id_t{space_id,0}; - the records will be freed and the iterator advanced +@param p iterator to the page @param name tablespace file name @param free_block spare buffer block -@return whether recovery failed */ -bool recv_sys_t::recover_deferred(recv_sys_t::map::iterator &p, - const std::string &name, - buf_block_t *&free_block) +@return recovered tablespace +@retval nullptr if recovery failed */ +fil_space_t *recv_sys_t::recover_deferred(const recv_sys_t::map::iterator &p, + const std::string &name, + buf_block_t *&free_block) { mysql_mutex_assert_owner(&mutex); - const page_id_t first{p->first}; - ut_ad(first.space()); + ut_ad(p->first.space()); - recv_spaces_t::iterator it{recv_spaces.find(first.space())}; + recv_spaces_t::iterator it{recv_spaces.find(p->first.space())}; ut_ad(it != recv_spaces.end()); - if (!first.page_no() && p->second.state == page_recv_t::RECV_WILL_NOT_READ) + if (!p->first.page_no() && p->second.skip_read) { mtr_t mtr; - buf_block_t *block= recover_low(first, p, mtr, free_block); + ut_ad(!p->second.being_processed); + p->second.being_processed= 1; + init &init= mlog_init.last(p->first); + mysql_mutex_unlock(&mutex); + buf_block_t *block= recover_low(p, mtr, free_block, init); + mysql_mutex_lock(&mutex); + p->second.being_processed= -1; ut_ad(block == free_block || block == reinterpret_cast(-1)); free_block= nullptr; if (UNIV_UNLIKELY(!block || block == reinterpret_cast(-1))) @@ -961,10 +1128,7 @@ bool recv_sys_t::recover_deferred(recv_sys_t::map::iterator &p, const uint32_t page_no= mach_read_from_4(page + FIL_PAGE_OFFSET); const uint32_t size= fsp_header_get_field(page, FSP_SIZE); - ut_ad(it != recv_spaces.end()); - - if (page_id_t{space_id, page_no} == first && size >= 4 && - it != recv_spaces.end() && + if (page_id_t{space_id, page_no} == p->first && size >= 4 && fil_space_t::is_valid_flags(flags, space_id) && fil_space_t::logical_size(flags) == srv_page_size) { @@ -1018,10 +1182,10 @@ bool recv_sys_t::recover_deferred(recv_sys_t::map::iterator &p, } size_set: node->deferred= false; - space->release(); it->second.space= space; block->page.lock.x_unlock(); - return false; + p->second.being_processed= -1; + return space; } release_and_fail: @@ -1029,179 +1193,34 @@ bool recv_sys_t::recover_deferred(recv_sys_t::map::iterator &p, } fail: - ib::error() << "Cannot apply log to " << first + ib::error() << "Cannot apply log to " << p->first << " of corrupted file '" << name << "'"; - return true; + return nullptr; } -/** Report an operation to create, delete, or rename a file during backup. -@param[in] space_id tablespace identifier -@param[in] type redo log type -@param[in] name file name (not NUL-terminated) -@param[in] len length of name, in bytes -@param[in] new_name new file name (NULL if not rename) -@param[in] new_len length of new_name, in bytes (0 if NULL) */ -void (*log_file_op)(ulint space_id, int type, - const byte* name, ulint len, - const byte* new_name, ulint new_len); - -void (*undo_space_trunc)(uint32_t space_id); - -void (*first_page_init)(ulint space_id); - -/** Information about initializing page contents during redo log processing. -FIXME: Rely on recv_sys.pages! */ -class mlog_init_t -{ -public: - /** A page initialization operation that was parsed from - the redo log */ - struct init { - /** log sequence number of the page initialization */ - lsn_t lsn; - /** Whether btr_page_create() avoided a read of the page. - - At the end of the last recovery batch, mark_ibuf_exist() - will mark pages for which this flag is set. */ - bool created; - }; - -private: - typedef std::map, - ut_allocator > > - map; - /** Map of page initialization operations. - FIXME: Merge this to recv_sys.pages! */ - map inits; -public: - /** Record that a page will be initialized by the redo log. - @param[in] page_id page identifier - @param[in] lsn log sequence number - @return whether the state was changed */ - bool add(const page_id_t page_id, lsn_t lsn) - { - mysql_mutex_assert_owner(&recv_sys.mutex); - const init init = { lsn, false }; - std::pair p = inits.insert( - map::value_type(page_id, init)); - ut_ad(!p.first->second.created); - if (p.second) return true; - if (p.first->second.lsn >= init.lsn) return false; - p.first->second = init; - return true; - } - - /** Get the last stored lsn of the page id and its respective - init/load operation. - @param[in] page_id page id - @param[in,out] init initialize log or load log - @return the latest page initialization; - not valid after releasing recv_sys.mutex. */ - init& last(page_id_t page_id) - { - mysql_mutex_assert_owner(&recv_sys.mutex); - return inits.find(page_id)->second; - } - - /** Determine if a page will be initialized or freed after a time. - @param page_id page identifier - @param lsn log sequence number - @return whether page_id will be freed or initialized after lsn */ - bool will_avoid_read(page_id_t page_id, lsn_t lsn) const - { - mysql_mutex_assert_owner(&recv_sys.mutex); - auto i= inits.find(page_id); - return i != inits.end() && i->second.lsn > lsn; - } - - /** At the end of each recovery batch, reset the 'created' flags. */ - void reset() - { - mysql_mutex_assert_owner(&recv_sys.mutex); - ut_ad(recv_no_ibuf_operations); - for (map::value_type& i : inits) { - i.second.created = false; - } - } - - /** On the last recovery batch, mark whether there exist - buffered changes for the pages that were initialized - by buf_page_create() and still reside in the buffer pool. - @param[in,out] mtr dummy mini-transaction */ - void mark_ibuf_exist(mtr_t& mtr) - { - mysql_mutex_assert_owner(&recv_sys.mutex); - mtr.start(); - - for (const map::value_type& i : inits) { - if (!i.second.created) { - continue; - } - if (buf_block_t* block = buf_page_get_low( - i.first, 0, RW_X_LATCH, nullptr, - BUF_GET_IF_IN_POOL, - &mtr, nullptr, false)) { - if (UNIV_LIKELY_NULL(block->page.zip.data)) { - switch (fil_page_get_type( - block->page.zip.data)) { - case FIL_PAGE_INDEX: - case FIL_PAGE_RTREE: - if (page_zip_decompress( - &block->page.zip, - block->page.frame, - true)) { - break; - } - ib::error() << "corrupted " - << block->page.id(); - } - } - if (recv_no_ibuf_operations) { - mtr.commit(); - mtr.start(); - continue; - } - mysql_mutex_unlock(&recv_sys.mutex); - if (ibuf_page_exists(block->page.id(), - block->zip_size())) { - block->page.set_ibuf_exist(); - } - mtr.commit(); - mtr.start(); - mysql_mutex_lock(&recv_sys.mutex); - } - } - - mtr.commit(); - clear(); - } - - /** Clear the data structure */ - void clear() { inits.clear(); } -}; - -static mlog_init_t mlog_init; - /** Process a record that indicates that a tablespace is being shrunk in size. @param page_id first page identifier that is not in the file @param lsn log sequence number of the shrink operation */ inline void recv_sys_t::trim(const page_id_t page_id, lsn_t lsn) { - DBUG_ENTER("recv_sys_t::trim"); - DBUG_LOG("ib_log", - "discarding log beyond end of tablespace " - << page_id << " before LSN " << lsn); - mysql_mutex_assert_owner(&mutex); - for (recv_sys_t::map::iterator p = pages.lower_bound(page_id); - p != pages.end() && p->first.space() == page_id.space();) { - recv_sys_t::map::iterator r = p++; - if (r->second.trim(lsn)) { - pages.erase(r); - } - } - DBUG_VOID_RETURN; + DBUG_ENTER("recv_sys_t::trim"); + DBUG_LOG("ib_log", "discarding log beyond end of tablespace " + << page_id << " before LSN " << lsn); + mysql_mutex_assert_owner(&mutex); + if (pages_it != pages.end() && pages_it->first.space() == page_id.space()) + pages_it= pages.end(); + for (recv_sys_t::map::iterator p = pages.lower_bound(page_id); + p != pages.end() && p->first.space() == page_id.space();) + { + recv_sys_t::map::iterator r = p++; + if (r->second.trim(lsn)) + { + ut_ad(!r->second.being_processed); + pages.erase(r); + } + } + DBUG_VOID_RETURN; } void recv_sys_t::open_log_files_if_needed() @@ -1400,7 +1419,6 @@ void recv_sys_t::close() last_stored_lsn= 0; mysql_mutex_destroy(&mutex); - pthread_cond_destroy(&cond); } recv_spaces.clear(); @@ -1415,10 +1433,8 @@ void recv_sys_t::create() ut_ad(this == &recv_sys); ut_ad(!is_initialised()); mysql_mutex_init(recv_sys_mutex_key, &mutex, nullptr); - pthread_cond_init(&cond, nullptr); apply_log_recs = false; - apply_batch_on = false; buf = static_cast(ut_malloc_dontdump(RECV_PARSING_BUF_SIZE, PSI_INSTRUMENT_ME)); @@ -1433,6 +1449,8 @@ void recv_sys_t::create() mlog_checkpoint_lsn = 0; progress_time = time(NULL); + ut_ad(pages.empty()); + pages_it = pages.end(); recv_max_page_lsn = 0; memset(truncated_undo_spaces, 0, sizeof truncated_undo_spaces); @@ -1441,13 +1459,13 @@ void recv_sys_t::create() } /** Clear a fully processed set of stored redo log records. */ -inline void recv_sys_t::clear() +void recv_sys_t::clear() { mysql_mutex_assert_owner(&mutex); apply_log_recs= false; - apply_batch_on= false; ut_ad(!after_apply || found_corrupt_fs || !UT_LIST_GET_LAST(blocks)); pages.clear(); + pages_it= pages.end(); for (buf_block_t *block= UT_LIST_GET_LAST(blocks); block; ) { @@ -1458,8 +1476,6 @@ inline void recv_sys_t::clear() buf_block_free(block); block= prev_block; } - - pthread_cond_broadcast(&cond); } /** Free most recovery data structures. */ @@ -1471,6 +1487,7 @@ void recv_sys_t::debug_free() recovery_on= false; pages.clear(); + pages_it= pages.end(); ut_free_dodump(buf, RECV_PARSING_BUF_SIZE); buf= nullptr; @@ -1478,48 +1495,9 @@ void recv_sys_t::debug_free() mysql_mutex_unlock(&mutex); } -inline void *recv_sys_t::alloc(size_t len) -{ - mysql_mutex_assert_owner(&mutex); - ut_ad(len); - ut_ad(len <= srv_page_size); - - buf_block_t *block= UT_LIST_GET_FIRST(blocks); - if (UNIV_UNLIKELY(!block)) - { -create_block: - block= buf_block_alloc(); - block->page.access_time= 1U << 16 | - ut_calc_align(static_cast(len), ALIGNMENT); - static_assert(ut_is_2pow(ALIGNMENT), "ALIGNMENT must be a power of 2"); - UT_LIST_ADD_FIRST(blocks, block); - MEM_MAKE_ADDRESSABLE(block->page.frame, len); - MEM_NOACCESS(block->page.frame + len, srv_page_size - len); - return my_assume_aligned(block->page.frame); - } - - size_t free_offset= static_cast(block->page.access_time); - ut_ad(!ut_2pow_remainder(free_offset, ALIGNMENT)); - if (UNIV_UNLIKELY(!free_offset)) - { - ut_ad(srv_page_size == 65536); - goto create_block; - } - ut_ad(free_offset <= srv_page_size); - free_offset+= len; - - if (free_offset > srv_page_size) - goto create_block; - - block->page.access_time= ((block->page.access_time >> 16) + 1) << 16 | - ut_calc_align(static_cast(free_offset), ALIGNMENT); - MEM_MAKE_ADDRESSABLE(block->page.frame + free_offset - len, len); - return my_assume_aligned(block->page.frame + free_offset - len); -} - /** Free a redo log snippet. -@param data buffer returned by alloc() */ +@param data buffer allocated in add() */ inline void recv_sys_t::free(const void *data) { ut_ad(!ut_align_offset(data, ALIGNMENT)); @@ -1544,8 +1522,11 @@ inline void recv_sys_t::free(const void *data) ut_ad(block->page.state() == buf_page_t::MEMORY); ut_ad(static_cast(block->page.access_time - 1) < srv_page_size); - ut_ad(block->page.access_time >= 1U << 16); - if (!((block->page.access_time -= 1U << 16) >> 16)) + unsigned a= block->page.access_time; + ut_ad(a >= 1U << 16); + a-= 1U << 16; + block->page.access_time= a; + if (!(a >> 16)) { UT_LIST_REMOVE(blocks, block); MEM_MAKE_ADDRESSABLE(block->page.frame, srv_page_size); @@ -2109,7 +2090,31 @@ inline bool page_recv_t::trim(lsn_t start_lsn) } -inline void page_recv_t::recs_t::clear() +void page_recv_t::recs_t::rewind(lsn_t start_lsn) +{ + mysql_mutex_assert_owner(&recv_sys.mutex); + log_phys_t *trim= static_cast(head); + ut_ad(trim); + while (log_phys_t *next= static_cast(trim->next)) + { + ut_ad(trim->start_lsn < start_lsn); + if (next->start_lsn == start_lsn) + break; + trim= next; + } + tail= trim; + log_rec_t *l= tail->next; + tail->next= nullptr; + while (l) + { + log_rec_t *next= l->next; + recv_sys.free(l); + l= next; + } +} + + +void page_recv_t::recs_t::clear() { mysql_mutex_assert_owner(&recv_sys.mutex); for (const log_rec_t *l= head; l; ) @@ -2121,33 +2126,99 @@ inline void page_recv_t::recs_t::clear() head= tail= nullptr; } - /** Ignore any earlier redo log records for this page. */ inline void page_recv_t::will_not_read() { - ut_ad(state == RECV_NOT_PROCESSED || state == RECV_WILL_NOT_READ); - state= RECV_WILL_NOT_READ; + ut_ad(!being_processed); + skip_read= true; log.clear(); } +void recv_sys_t::erase(map::iterator p) +{ + ut_ad(p->second.being_processed <= 0); + p->second.log.clear(); + pages.erase(p); +} + +/** Free log for processed pages. */ +void recv_sys_t::garbage_collect() +{ + mysql_mutex_assert_owner(&mutex); + + if (pages_it != pages.end() && pages_it->second.being_processed < 0) + pages_it= pages.end(); + + for (map::iterator p= pages.begin(); p != pages.end(); ) + { + if (p->second.being_processed < 0) + { + map::iterator r= p++; + erase(r); + } + else + p++; + } +} + +/** Allocate a block from the buffer pool for recv_sys.pages */ +ATTRIBUTE_COLD buf_block_t *recv_sys_t::add_block() +{ + for (bool freed= false;;) + { + const auto rs= UT_LIST_GET_LEN(blocks) * 2; + mysql_mutex_lock(&buf_pool.mutex); + const auto bs= + UT_LIST_GET_LEN(buf_pool.free) + UT_LIST_GET_LEN(buf_pool.LRU); + if (UNIV_LIKELY(bs > BUF_LRU_MIN_LEN || rs < bs)) + { + buf_block_t *block= buf_LRU_get_free_block(true); + mysql_mutex_unlock(&buf_pool.mutex); + return block; + } + /* out of memory: redo log occupies more than 1/3 of buf_pool + and there are fewer than BUF_LRU_MIN_LEN pages left */ + mysql_mutex_unlock(&buf_pool.mutex); + if (freed) + return nullptr; + freed= true; + garbage_collect(); + } +} + +/** Wait for buffer pool to become available. */ +ATTRIBUTE_COLD void recv_sys_t::wait_for_pool(size_t pages) +{ + mysql_mutex_unlock(&mutex); + os_aio_wait_until_no_pending_reads(false); + mysql_mutex_lock(&mutex); + garbage_collect(); + mysql_mutex_lock(&buf_pool.mutex); + bool need_more= UT_LIST_GET_LEN(buf_pool.free) < pages; + mysql_mutex_unlock(&buf_pool.mutex); + if (need_more) + buf_flush_sync_batch(recovered_lsn); +} /** Register a redo log snippet for a page. @param it page iterator @param start_lsn start LSN of the mini-transaction @param lsn @see mtr_t::commit_lsn() -@param recs redo log snippet @see log_t::FORMAT_10_5 -@param len length of l, in bytes */ -inline void recv_sys_t::add(map::iterator it, lsn_t start_lsn, lsn_t lsn, - const byte *l, size_t len) +@param l redo log snippet +@param len length of l, in bytes +@return whether we ran out of memory */ +ATTRIBUTE_NOINLINE +bool recv_sys_t::add(map::iterator it, lsn_t start_lsn, lsn_t lsn, + const byte *l, size_t len) { mysql_mutex_assert_owner(&mutex); - page_id_t page_id = it->first; page_recv_t &recs= it->second; + buf_block_t *block; switch (*l & 0x70) { case FREE_PAGE: case INIT_PAGE: recs.will_not_read(); - mlog_init.add(page_id, start_lsn); /* FIXME: remove this! */ + mlog_init.add(it->first, start_lsn); /* FIXME: remove this! */ /* fall through */ default: log_phys_t *tail= static_cast(recs.log.last()); @@ -2156,7 +2227,7 @@ inline void recv_sys_t::add(map::iterator it, lsn_t start_lsn, lsn_t lsn, if (tail->start_lsn != start_lsn) break; ut_ad(tail->lsn == lsn); - buf_block_t *block= UT_LIST_GET_LAST(blocks); + block= UT_LIST_GET_LAST(blocks); ut_ad(block); const size_t used= static_cast(block->page.access_time - 1) + 1; ut_ad(used >= ALIGNMENT); @@ -2169,7 +2240,7 @@ append: MEM_MAKE_ADDRESSABLE(end + 1, len); /* Append to the preceding record for the page */ tail->append(l, len); - return; + return false; } if (end <= &block->page.frame[used - ALIGNMENT] || &block->page.frame[used] >= end) @@ -2183,8 +2254,49 @@ append: ut_calc_align(static_cast(new_used), ALIGNMENT); goto append; } - recs.log.append(new (alloc(log_phys_t::alloc_size(len))) - log_phys_t(start_lsn, lsn, l, len)); + + const size_t size{log_phys_t::alloc_size(len)}; + ut_ad(size <= srv_page_size); + void *buf; + block= UT_LIST_GET_FIRST(blocks); + if (UNIV_UNLIKELY(!block)) + { + create_block: + block= add_block(); + if (UNIV_UNLIKELY(!block)) + return true; + block->page.access_time= 1U << 16 | + ut_calc_align(static_cast(size), ALIGNMENT); + static_assert(ut_is_2pow(ALIGNMENT), "ALIGNMENT must be a power of 2"); + UT_LIST_ADD_FIRST(blocks, block); + MEM_MAKE_ADDRESSABLE(block->page.frame, size); + MEM_NOACCESS(block->page.frame + size, srv_page_size - size); + buf= block->page.frame; + } + else + { + size_t free_offset= static_cast(block->page.access_time); + ut_ad(!ut_2pow_remainder(free_offset, ALIGNMENT)); + if (UNIV_UNLIKELY(!free_offset)) + { + ut_ad(srv_page_size == 65536); + goto create_block; + } + ut_ad(free_offset <= srv_page_size); + free_offset+= size; + + if (free_offset > srv_page_size) + goto create_block; + + block->page.access_time= ((block->page.access_time >> 16) + 1) << 16 | + ut_calc_align(static_cast(free_offset), ALIGNMENT); + MEM_MAKE_ADDRESSABLE(block->page.frame + free_offset - size, size); + buf= block->page.frame + free_offset - size; + } + + recs.log.append(new (my_assume_aligned(buf)) + log_phys_t{start_lsn, lsn, l, len}); + return false; } /** Store/remove the freed pages in fil_name_t of recv_spaces. @@ -2220,6 +2332,70 @@ static void store_freed_or_init_rec(page_id_t page_id, bool freed) } } +ATTRIBUTE_COLD +void recv_sys_t::rewind(const byte *end, const byte *begin) noexcept +{ + ut_ad(srv_operation != SRV_OPERATION_BACKUP); + mysql_mutex_assert_owner(&mutex); + + uint32_t rlen; + for (const byte *l= begin; !(l == end); l+= rlen) + { + const byte b= *l++; + ut_ad(UNIV_LIKELY((b & 0x70) != RESERVED) || srv_force_recovery); + + rlen= b & 0xf; + if (!rlen) + { + if (!b) + continue; + const uint32_t lenlen= mlog_decode_varint_length(*l); + const uint32_t addlen= mlog_decode_varint(l); + ut_ad(addlen != MLOG_DECODE_ERROR); + rlen= addlen + 15 - lenlen; + l+= lenlen; + } + ut_ad(l + rlen <= end); + if (b & 0x80) + continue; + + uint32_t idlen= mlog_decode_varint_length(*l); + if (UNIV_UNLIKELY(idlen > 5 || idlen >= rlen)) + continue; + const uint32_t space_id= mlog_decode_varint(l); + if (UNIV_UNLIKELY(space_id == MLOG_DECODE_ERROR)) + continue; + l+= idlen; + rlen-= idlen; + idlen= mlog_decode_varint_length(*l); + if (UNIV_UNLIKELY(idlen > 5 || idlen > rlen)) + continue; + const uint32_t page_no= mlog_decode_varint(l); + if (UNIV_UNLIKELY(page_no == MLOG_DECODE_ERROR)) + continue; + const page_id_t id{space_id, page_no}; + if (pages_it == pages.end() || pages_it->first != id) + { + pages_it= pages.find(id); + if (pages_it == pages.end()) + continue; + } + + ut_ad(!pages_it->second.being_processed); + const log_phys_t *head= + static_cast(*pages_it->second.log.begin()); + if (!head || head->start_lsn == recovered_lsn) + { + erase(pages_it); + pages_it= pages.end(); + } + else + pages_it->second.log.rewind(recovered_lsn); + } + + pages_it= pages.end(); +} + /** Parse and register one mini-transaction in log_t::FORMAT_10_5. @param checkpoint_lsn the log sequence number of the latest checkpoint @param store whether to store the records @@ -2228,17 +2404,16 @@ static void store_freed_or_init_rec(page_id_t page_id, bool freed) or corruption was noticed */ bool recv_sys_t::parse(lsn_t checkpoint_lsn, store_t *store, bool apply) { + restart: mysql_mutex_assert_owner(&log_sys.mutex); mysql_mutex_assert_owner(&mutex); ut_ad(parse_start_lsn); ut_ad(log_sys.is_physical()); - bool last_phase= (*store == STORE_IF_EXISTS); const byte *const end= buf + len; loop: const byte *const log= buf + recovered_offset; const lsn_t start_lsn= recovered_lsn; - map::iterator cached_pages_it = pages.end(); /* Check that the entire mini-transaction is included within the buffer */ const byte *l; @@ -2554,7 +2729,6 @@ same_page: ut_ad(modified.emplace(id).second || (b & 0x70) != INIT_PAGE); } #endif - const bool is_init= (b & 0x70) <= INIT_PAGE; switch (*store) { case STORE_IF_EXISTS: if (fil_space_t *space= fil_space_t::get(space_id)) @@ -2568,23 +2742,48 @@ same_page: continue; /* fall through */ case STORE_YES: - if (!mlog_init.will_avoid_read(id, start_lsn)) + if (mlog_init.will_avoid_read(id, start_lsn)) + continue; + if (pages_it == pages.end() || pages_it->first != id) + pages_it= pages.emplace(id, page_recv_t{}).first; + if (UNIV_UNLIKELY(add(pages_it, start_lsn, end_lsn, recs, + l - recs + rlen))) { - if (cached_pages_it == pages.end() || cached_pages_it->first != id) - cached_pages_it= pages.emplace(id, page_recv_t()).first; - add(cached_pages_it, start_lsn, end_lsn, recs, - static_cast(l + rlen - recs)); + recovered_lsn= start_lsn; + recovered_offset= log - buf; + rewind(l + rlen, log); + if (*store == STORE_IF_EXISTS) + { + log_sys.set_lsn(recovered_lsn); + log_sys.set_flushed_lsn(recovered_lsn); + mysql_mutex_unlock(&mutex); + this->apply(false); + mysql_mutex_lock(&mutex); + if (is_corrupt_fs()) + return true; + } + else + { + last_stored_lsn= recovered_lsn; + sql_print_information("InnoDB: Multi-batch recovery needed at LSN " + LSN_PF, recovered_lsn); + *store= STORE_NO; + } + goto restart; } continue; case STORE_NO: - if (!is_init) + if ((b & 0x70) > INIT_PAGE) continue; mlog_init.add(id, start_lsn); - map::iterator i= pages.find(id); - if (i == pages.end()) - continue; - i->second.log.clear(); - pages.erase(i); + if (pages_it == pages.end() || pages_it->first != id) + { + pages_it= pages.find(id); + if (pages_it == pages.end()) + continue; + } + map::iterator r= pages_it++; + erase(r); } } else if (rlen) @@ -2706,8 +2905,6 @@ same_page: ut_ad(l == el); recovered_offset= l - buf; recovered_lsn= end_lsn; - if (is_memory_exhausted(store) && last_phase) - return false; goto loop; } @@ -2715,23 +2912,22 @@ same_page: lsn of a log record. @param[in,out] block buffer pool page @param[in,out] mtr mini-transaction -@param[in,out] p recovery address +@param[in,out] recs log records to apply @param[in,out] space tablespace, or NULL if not looked up yet @param[in,out] init page initialization operation, or NULL @return the recovered page @retval nullptr on failure */ static buf_block_t *recv_recover_page(buf_block_t *block, mtr_t &mtr, - const recv_sys_t::map::iterator &p, - fil_space_t *space= nullptr, - mlog_init_t::init *init= nullptr) + page_recv_t &recs, + fil_space_t *space, + recv_init *init) { - mysql_mutex_assert_owner(&recv_sys.mutex); + mysql_mutex_assert_not_owner(&recv_sys.mutex); ut_ad(recv_sys.apply_log_recs); ut_ad(recv_needed_recovery); ut_ad(!init || init->created); ut_ad(!init || init->lsn); - ut_ad(block->page.id() == p->first); - ut_ad(!p->second.is_being_processed()); + ut_ad(recs.being_processed == 1); ut_ad(!space || space->id == block->page.id().space()); ut_ad(log_sys.is_physical()); @@ -2743,10 +2939,6 @@ static buf_block_t *recv_recover_page(buf_block_t *block, mtr_t &mtr, block->page.id().space(), block->page.id().page_no())); - p->second.state = page_recv_t::RECV_BEING_PROCESSED; - - mysql_mutex_unlock(&recv_sys.mutex); - byte *frame = UNIV_LIKELY_NULL(block->page.zip.data) ? block->page.zip.data : block->page.frame; @@ -2760,7 +2952,7 @@ static buf_block_t *recv_recover_page(buf_block_t *block, mtr_t &mtr, bool skipped_after_init = false; - for (const log_rec_t* recv : p->second.log) { + for (const log_rec_t* recv : recs.log) { const log_phys_t* l = static_cast(recv); ut_ad(l->lsn); ut_ad(end_lsn <= l->lsn); @@ -2817,8 +3009,7 @@ static buf_block_t *recv_recover_page(buf_block_t *block, mtr_t &mtr, block->page.id().space(), block->page.id().page_no())); - log_phys_t::apply_status a= l->apply(*block, - p->second.last_offset); + log_phys_t::apply_status a= l->apply(*block, recs.last_offset); switch (a) { case log_phys_t::APPLIED_NO: @@ -2937,24 +3128,11 @@ set_start_lsn: mtr.commit(); done: - time_t now = time(NULL); - - mysql_mutex_lock(&recv_sys.mutex); - + /* FIXME: do this in page read, protected with recv_sys.mutex! */ if (recv_max_page_lsn < page_lsn) { recv_max_page_lsn = page_lsn; } - ut_ad(!block || p->second.is_being_processed()); - ut_ad(!block || !recv_sys.pages.empty()); - - if (recv_sys.report(now)) { - const ulint n = recv_sys.pages.size(); - ib::info() << "To recover: " << n << " pages from log"; - service_manager_extend_timeout( - INNODB_EXTEND_TIMEOUT_INTERVAL, "To recover: " ULINTPF " pages from log", n); - } - return block; } @@ -2968,146 +3146,350 @@ ATTRIBUTE_COLD void recv_sys_t::free_corrupted_page(page_id_t page_id) mysql_mutex_lock(&mutex); map::iterator p= pages.find(page_id); - if (p != pages.end()) + if (p == pages.end()) { - p->second.log.clear(); - pages.erase(p); - if (!srv_force_recovery) - { - set_corrupt_fs(); - ib::error() << "Unable to apply log to corrupted page " << page_id - << "; set innodb_force_recovery to ignore"; - } - else - ib::warn() << "Discarding log for corrupted page " << page_id; + mysql_mutex_unlock(&mutex); + return; } - if (pages.empty()) - pthread_cond_broadcast(&cond); + p->second.being_processed= -1; + if (!srv_force_recovery) + set_corrupt_fs(); mysql_mutex_unlock(&mutex); -} -/** Possibly finish a recovery batch. */ -inline void recv_sys_t::maybe_finish_batch() -{ - mysql_mutex_assert_owner(&mutex); - ut_ad(recovery_on); - if (!apply_batch_on || pages.empty() || is_corrupt_log() || is_corrupt_fs()) - pthread_cond_broadcast(&cond); + ib::error_or_warn(!srv_force_recovery) + << "Unable to apply log to corrupted page " << page_id; } ATTRIBUTE_COLD void recv_sys_t::set_corrupt_log() { mysql_mutex_lock(&mutex); found_corrupt_log= true; - pthread_cond_broadcast(&cond); mysql_mutex_unlock(&mutex); } ATTRIBUTE_COLD void recv_sys_t::set_corrupt_fs() { mysql_mutex_assert_owner(&mutex); + if (!srv_force_recovery) + sql_print_information("InnoDB: Set innodb_force_recovery=1" + " to ignore corrupted pages."); found_corrupt_fs= true; - pthread_cond_broadcast(&cond); } -/** Apply any buffered redo log to a page that was just read from a data file. -@param[in,out] space tablespace -@param[in,out] bpage buffer pool page +/** Apply any buffered redo log to a page. +@param space tablespace +@param bpage buffer pool page @return whether the page was recovered correctly */ bool recv_recover_page(fil_space_t* space, buf_page_t* bpage) { - mtr_t mtr; - mtr.start(); - mtr.set_log_mode(MTR_LOG_NO_REDO); + mtr_t mtr; + mtr.start(); + mtr.set_log_mode(MTR_LOG_NO_REDO); - ut_ad(bpage->frame); - /* Move the ownership of the x-latch on the page to - this OS thread, so that we can acquire a second - x-latch on it. This is needed for the operations to - the page to pass the debug checks. */ - bpage->lock.claim_ownership(); - bpage->lock.x_lock_recursive(); - bpage->fix_on_recovery(); - mtr.memo_push(reinterpret_cast(bpage), - MTR_MEMO_PAGE_X_FIX); + ut_ad(bpage->frame); + /* Move the ownership of the x-latch on the page to this OS thread, + so that we can acquire a second x-latch on it. This is needed for + the operations to the page to pass the debug checks. */ + bpage->lock.claim_ownership(); + bpage->lock.x_lock_recursive(); + bpage->fix_on_recovery(); + mtr.memo_push(reinterpret_cast(bpage), MTR_MEMO_PAGE_X_FIX); - buf_block_t* success = reinterpret_cast(bpage); + buf_block_t *success= reinterpret_cast(bpage); - mysql_mutex_lock(&recv_sys.mutex); - if (recv_sys.apply_log_recs) { - recv_sys_t::map::iterator p = recv_sys.pages.find(bpage->id()); - if (p != recv_sys.pages.end() - && !p->second.is_being_processed()) { - success = recv_recover_page(success, mtr, p, space); - if (UNIV_LIKELY(!!success)) { - p->second.log.clear(); - recv_sys.pages.erase(p); - } - recv_sys.maybe_finish_batch(); - goto func_exit; - } - } - - mtr.commit(); -func_exit: - mysql_mutex_unlock(&recv_sys.mutex); - ut_ad(mtr.has_committed()); - return success; -} - -/** Read pages for which log needs to be applied. -@param page_id first page identifier to read -@param i iterator to recv_sys.pages */ -TRANSACTIONAL_TARGET -static void recv_read_in_area(page_id_t page_id, recv_sys_t::map::iterator i) -{ - uint32_t page_nos[32]; - ut_ad(page_id == i->first); - page_id.set_page_no(ut_2pow_round(page_id.page_no(), 32U)); - const page_id_t up_limit{page_id + 31}; - uint32_t* p= page_nos; - - for (; i != recv_sys.pages.end() && i->first <= up_limit; i++) + mysql_mutex_lock(&recv_sys.mutex); + if (recv_sys.apply_log_recs) { - if (i->second.state == page_recv_t::RECV_NOT_PROCESSED) + const page_id_t id{bpage->id()}; + recv_sys_t::map::iterator p= recv_sys.pages.find(id); + if (p == recv_sys.pages.end()); + else if (p->second.being_processed < 0) { - i->second.state= page_recv_t::RECV_BEING_READ; - *p++= i->first.page_no(); + recv_sys.pages_it_invalidate(p); + recv_sys.erase(p); + } + else + { + p->second.being_processed= 1; + recv_sys_t::init *init= nullptr; + if (p->second.skip_read) + (init= &mlog_init.last(id))->created= true; + mysql_mutex_unlock(&recv_sys.mutex); + success= recv_recover_page(success, mtr, p->second, space, init); + p->second.being_processed= -1; + goto func_exit; } } - if (p != page_nos) + mysql_mutex_unlock(&recv_sys.mutex); + mtr.commit(); +func_exit: + ut_ad(mtr.has_committed()); + return success; +} + +void IORequest::fake_read_complete(os_offset_t offset) const +{ + ut_ad(node); + ut_ad(is_read()); + ut_ad(bpage); + ut_ad(bpage->frame); + ut_ad(recv_recovery_is_on()); + ut_ad(offset); + + mtr_t mtr; + mtr.start(); + mtr.set_log_mode(MTR_LOG_NO_REDO); + + ut_ad(bpage->frame); + /* Move the ownership of the x-latch on the page to this OS thread, + so that we can acquire a second x-latch on it. This is needed for + the operations to the page to pass the debug checks. */ + bpage->lock.claim_ownership(); + bpage->lock.x_lock_recursive(); + bpage->fix_on_recovery(); + mtr.memo_push(reinterpret_cast(bpage), MTR_MEMO_PAGE_X_FIX); + + page_recv_t &recs= *reinterpret_cast(slot); + ut_ad(recs.being_processed == 1); + recv_init &init= *reinterpret_cast(offset); + ut_ad(init.lsn > 1); + init.created= true; + + if (recv_recover_page(reinterpret_cast(bpage), + mtr, recs, node->space, &init)) { - mysql_mutex_unlock(&recv_sys.mutex); - buf_read_recv_pages(page_id.space(), page_nos, ulint(p - page_nos)); - mysql_mutex_lock(&recv_sys.mutex); + ut_ad(bpage->oldest_modification() || bpage->is_freed()); + bpage->lock.x_unlock(true); + } + recs.being_processed= -1; + ut_ad(mtr.has_committed()); + + node->space->release(); +} + +/** @return whether a page has been freed */ +inline bool fil_space_t::is_freed(uint32_t page) +{ + std::lock_guard freed_lock(freed_range_mutex); + return freed_ranges.contains(page); +} + +bool recv_sys_t::report(time_t time) +{ + if (time - progress_time < 15) + return false; + progress_time= time; + return true; +} + +ATTRIBUTE_COLD +void recv_sys_t::report_progress() const +{ + mysql_mutex_assert_owner(&mutex); + const size_t n{pages.size()}; + if (recv_sys.scanned_lsn == recv_sys.recovered_lsn) + { + sql_print_information("InnoDB: To recover: %zu pages", n); + service_manager_extend_timeout(INNODB_EXTEND_TIMEOUT_INTERVAL, + "To recover: %zu pages", n); + } + else + { + sql_print_information("InnoDB: To recover: LSN " LSN_PF + "/" LSN_PF "; %zu pages", + recv_sys.recovered_lsn, recv_sys.scanned_lsn, n); + service_manager_extend_timeout(INNODB_EXTEND_TIMEOUT_INTERVAL, + "To recover: LSN " LSN_PF + "/" LSN_PF "; %zu pages", + recv_sys.recovered_lsn, + recv_sys.scanned_lsn, n); } } +/** Apply a recovery batch. +@param space_id current tablespace identifier +@param space current tablespace +@param free_block spare buffer block +@param last_batch whether it is possible to write more redo log +@return whether the caller must provide a new free_block */ +bool recv_sys_t::apply_batch(uint32_t space_id, fil_space_t *&space, + buf_block_t *&free_block, bool last_batch) +{ + mysql_mutex_assert_owner(&mutex); + ut_ad(pages_it != pages.end()); + ut_ad(!pages_it->second.log.empty()); + + mysql_mutex_lock(&buf_pool.mutex); + size_t n= 0, max_n= std::min(BUF_LRU_MIN_LEN, + UT_LIST_GET_LEN(buf_pool.LRU) + + UT_LIST_GET_LEN(buf_pool.free)); + mysql_mutex_unlock(&buf_pool.mutex); + + map::iterator begin= pages.end(); + page_id_t begin_id{~0ULL}; + + while (pages_it != pages.end() && n < max_n) + { + ut_ad(!buf_dblwr.is_inside(pages_it->first)); + if (!pages_it->second.being_processed) + { + if (space_id != pages_it->first.space()) + { + space_id= pages_it->first.space(); + if (space) + space->release(); + space= fil_space_t::get(space_id); + if (!space) + { + auto d= deferred_spaces.defers.find(space_id); + if (d == deferred_spaces.defers.end() || d->second.deleted) + /* For deleted files we preserve the deferred_spaces entry */; + else if (!free_block) + return true; + else + { + space= recover_deferred(pages_it, d->second.file_name, free_block); + deferred_spaces.defers.erase(d); + if (!space && !srv_force_recovery) + { + set_corrupt_fs(); + return false; + } + } + } + } + if (!space || space->is_freed(pages_it->first.page_no())) + pages_it->second.being_processed= -1; + else if (!n++) + { + begin= pages_it; + begin_id= pages_it->first; + } + } + pages_it++; + } + + if (!last_batch) + mysql_mutex_unlock(&log_sys.mutex); + + mysql_mutex_assert_not_owner(&log_sys.mutex); + + pages_it= begin; + + if (report(time(nullptr))) + report_progress(); + + if (!n) + goto wait; + + mysql_mutex_lock(&buf_pool.mutex); + + if (UNIV_UNLIKELY(UT_LIST_GET_LEN(buf_pool.free) < n)) + { + mysql_mutex_unlock(&buf_pool.mutex); + wait: + wait_for_pool(n); + if (n); + else if (!last_batch) + goto unlock_relock; + else + goto get_last; + pages_it= pages.lower_bound(begin_id); + ut_ad(pages_it != pages.end()); + } + else + mysql_mutex_unlock(&buf_pool.mutex); + + while (pages_it != pages.end()) + { + ut_ad(!buf_dblwr.is_inside(pages_it->first)); + if (!pages_it->second.being_processed) + { + const page_id_t id{pages_it->first}; + + if (space_id != id.space()) + { + space_id= id.space(); + if (space) + space->release(); + space= fil_space_t::get(space_id); + } + if (!space) + { + const auto it= deferred_spaces.defers.find(space_id); + if (it != deferred_spaces.defers.end() && !it->second.deleted) + /* The records must be processed after recover_deferred(). */ + goto next; + goto space_not_found; + } + else if (space->is_freed(id.page_no())) + { + space_not_found: + pages_it->second.being_processed= -1; + goto next; + } + else + { + page_recv_t &recs= pages_it->second; + ut_ad(!recs.log.empty()); + recs.being_processed= 1; + init *init= recs.skip_read ? &mlog_init.last(id) : nullptr; + mysql_mutex_unlock(&mutex); + buf_read_recover(space, id, recs, init); + } + + if (!--n) + { + if (last_batch) + goto relock_last; + goto relock; + } + mysql_mutex_lock(&mutex); + pages_it= pages.lower_bound(id); + } + else + next: + pages_it++; + } + + if (!last_batch) + { + unlock_relock: + mysql_mutex_unlock(&mutex); + relock: + mysql_mutex_lock(&log_sys.mutex); + relock_last: + mysql_mutex_lock(&mutex); + get_last: + pages_it= pages.lower_bound(begin_id); + } + + return false; +} + /** Attempt to initialize a page based on redo log records. -@param page_id page identifier -@param p iterator pointing to page_id +@param p iterator @param mtr mini-transaction @param b pre-allocated buffer pool block +@param init page initialization @return the recovered block @retval nullptr if the page cannot be initialized based on log records @retval -1 if the page cannot be recovered due to corruption */ -inline buf_block_t *recv_sys_t::recover_low(const page_id_t page_id, - map::iterator &p, mtr_t &mtr, - buf_block_t *b) +inline buf_block_t *recv_sys_t::recover_low(const map::iterator &p, mtr_t &mtr, + buf_block_t *b, init &init) { - mysql_mutex_assert_owner(&mutex); - ut_ad(p->first == page_id); + mysql_mutex_assert_not_owner(&mutex); page_recv_t &recs= p->second; - ut_ad(recs.state == page_recv_t::RECV_WILL_NOT_READ); + ut_ad(recs.skip_read); + ut_ad(recs.being_processed == 1); buf_block_t* block= nullptr; - mlog_init_t::init &i= mlog_init.last(page_id); - const lsn_t end_lsn = recs.log.last()->lsn; - if (end_lsn < i.lsn) - DBUG_LOG("ib_log", "skip log for page " << page_id - << " LSN " << end_lsn << " < " << i.lsn); - fil_space_t *space= fil_space_t::get(page_id.space()); + const lsn_t end_lsn= recs.log.last()->lsn; + if (end_lsn < init.lsn) + DBUG_LOG("ib_log", "skip log for page " << p->first + << " LSN " << end_lsn << " < " << init.lsn); + fil_space_t *space= fil_space_t::get(p->first.space()); mtr.start(); mtr.set_log_mode(MTR_LOG_NO_REDO); @@ -3116,82 +3498,77 @@ inline buf_block_t *recv_sys_t::recover_low(const page_id_t page_id, if (!space) { - if (page_id.page_no() != 0) + if (p->first.page_no() != 0) { nothing_recoverable: mtr.commit(); return nullptr; } - auto it= recv_spaces.find(page_id.space()); + auto it= recv_spaces.find(p->first.space()); ut_ad(it != recv_spaces.end()); uint32_t flags= it->second.flags; zip_size= fil_space_t::zip_size(flags); - block= buf_page_create_deferred(page_id.space(), zip_size, &mtr, b); + block= buf_page_create_deferred(p->first.space(), zip_size, &mtr, b); ut_ad(block == b); block->page.lock.x_lock_recursive(); } else { - block= buf_page_create(space, page_id.page_no(), zip_size, &mtr, b); + block= buf_page_create(space, p->first.page_no(), zip_size, &mtr, b); if (UNIV_UNLIKELY(block != b)) { /* The page happened to exist in the buffer pool, or it was just being read in. Before the exclusive page latch was acquired by buf_page_create(), all changes to the page must have been applied. */ - ut_ad(pages.find(page_id) == pages.end()); + ut_d(mysql_mutex_lock(&mutex)); + ut_ad(pages.find(p->first) == pages.end()); + ut_d(mysql_mutex_unlock(&mutex)); space->release(); goto nothing_recoverable; } } - ut_ad(&recs == &pages.find(page_id)->second); - i.created= true; - map::iterator r= p++; - block= recv_recover_page(block, mtr, r, space, &i); + ut_d(mysql_mutex_lock(&mutex)); + ut_ad(&recs == &pages.find(p->first)->second); + ut_d(mysql_mutex_unlock(&mutex)); + init.created= true; + block= recv_recover_page(block, mtr, recs, space, &init); ut_ad(mtr.has_committed()); - if (block) - { - recs.log.clear(); - pages.erase(r); - } - else - block= reinterpret_cast(-1); - - if (pages.empty()) - pthread_cond_signal(&cond); - if (space) space->release(); - return block; + return block ? block : reinterpret_cast(-1); } /** Attempt to initialize a page based on redo log records. @param page_id page identifier @return recovered block @retval nullptr if the page cannot be initialized based on log records */ -buf_block_t *recv_sys_t::recover_low(const page_id_t page_id) +ATTRIBUTE_COLD buf_block_t *recv_sys_t::recover_low(const page_id_t page_id) { - buf_block_t *free_block= buf_LRU_get_free_block(false); - buf_block_t *block= nullptr; - mysql_mutex_lock(&mutex); map::iterator p= pages.find(page_id); - if (p != pages.end() && p->second.state == page_recv_t::RECV_WILL_NOT_READ) + if (p != pages.end() && !p->second.being_processed && p->second.skip_read) { + p->second.being_processed= 1; + init &init= mlog_init.last(page_id); + mysql_mutex_unlock(&mutex); + buf_block_t *free_block= buf_LRU_get_free_block(false); mtr_t mtr; - block= recover_low(page_id, p, mtr, free_block); + buf_block_t *block= recover_low(p, mtr, free_block, init); + p->second.being_processed= -1; ut_ad(!block || block == reinterpret_cast(-1) || block == free_block); + if (UNIV_UNLIKELY(!block)) + buf_pool.free_block(free_block); + return block; } mysql_mutex_unlock(&mutex); - if (UNIV_UNLIKELY(!block)) - buf_pool.free_block(free_block); - return block; + return nullptr; } inline fil_space_t *fil_system_t::find(const char *path) const @@ -3242,46 +3619,18 @@ void recv_sys_t::apply(bool last_batch) #endif /* SAFE_MUTEX */ mysql_mutex_lock(&mutex); - timespec abstime; - - while (apply_batch_on) - { - if (is_corrupt_log()) - { - mysql_mutex_unlock(&mutex); - return; - } - if (last_batch) - { - mysql_mutex_assert_not_owner(&log_sys.mutex); - my_cond_wait(&cond, &mutex.m_mutex); - } - else - { - mysql_mutex_unlock(&mutex); - set_timespec_nsec(abstime, 500000000ULL); /* 0.5s */ - my_cond_timedwait(&cond, &log_sys.mutex.m_mutex, &abstime); - mysql_mutex_lock(&mutex); - } - } - - recv_no_ibuf_operations = !last_batch || - srv_operation == SRV_OPERATION_RESTORE || - srv_operation == SRV_OPERATION_RESTORE_EXPORT; - - mtr_t mtr; + garbage_collect(); if (!pages.empty()) { - const char *msg= last_batch - ? "Starting final batch to recover " - : "Starting a batch to recover "; - const ulint n= pages.size(); - ib::info() << msg << n << " pages from redo log."; - sd_notifyf(0, "STATUS=%s" ULINTPF " pages from redo log", msg, n); + recv_no_ibuf_operations = !last_batch || + srv_operation == SRV_OPERATION_RESTORE || + srv_operation == SRV_OPERATION_RESTORE_EXPORT; + ut_ad(!last_batch || recovered_lsn == scanned_lsn); + progress_time= time(nullptr); + report_progress(); apply_log_recs= true; - apply_batch_on= true; for (auto id= srv_undo_tablespaces_open; id--;) { @@ -3307,130 +3656,70 @@ void recv_sys_t::apply(bool last_batch) fil_system.extend_to_recv_size(); - /* We must release log_sys.mutex and recv_sys.mutex before - invoking buf_LRU_get_free_block(). Allocating a block may initiate - a redo log write and therefore acquire log_sys.mutex. To avoid - deadlocks, log_sys.mutex must not be acquired while holding - recv_sys.mutex. */ - mysql_mutex_unlock(&mutex); - if (!last_batch) - mysql_mutex_unlock(&log_sys.mutex); + fil_space_t *space= nullptr; + uint32_t space_id= ~0; + buf_block_t *free_block= nullptr; - mysql_mutex_assert_not_owner(&log_sys.mutex); - buf_block_t *free_block= buf_LRU_get_free_block(false); - - if (!last_batch) - mysql_mutex_lock(&log_sys.mutex); - mysql_mutex_lock(&mutex); - - for (map::iterator p= pages.begin(); p != pages.end(); ) + for (pages_it= pages.begin(); pages_it != pages.end(); + pages_it= pages.begin()) { - const page_id_t page_id= p->first; - ut_ad(!p->second.log.empty()); - - const uint32_t space_id= page_id.space(); - auto d= deferred_spaces.defers.find(space_id); - if (d != deferred_spaces.defers.end()) + if (!free_block) { - if (d->second.deleted) - { - /* For deleted files we must preserve the entry in deferred_spaces */ -erase_for_space: - while (p != pages.end() && p->first.space() == space_id) - { - map::iterator r= p++; - r->second.log.clear(); - pages.erase(r); - } - } - else if (recover_deferred(p, d->second.file_name, free_block)) - { - if (!srv_force_recovery) - set_corrupt_fs(); - deferred_spaces.defers.erase(d); - goto erase_for_space; - } - else - deferred_spaces.defers.erase(d); - if (!free_block) - goto next_free_block; - p= pages.lower_bound(page_id); - continue; + if (!last_batch) + mysql_mutex_unlock(&log_sys.mutex); + wait_for_pool(1); + pages_it= pages.begin(); + mysql_mutex_unlock(&mutex); + /* We must release log_sys.mutex and recv_sys.mutex before + invoking buf_LRU_get_free_block(). Allocating a block may initiate + a redo log write and therefore acquire log_sys.mutex. To avoid + deadlocks, log_sys.mutex must not be acquired while holding + recv_sys.mutex. */ + free_block= buf_LRU_get_free_block(false); + if (!last_batch) + mysql_mutex_lock(&log_sys.mutex); + mysql_mutex_lock(&mutex); + pages_it= pages.begin(); } - switch (p->second.state) { - case page_recv_t::RECV_BEING_READ: - case page_recv_t::RECV_BEING_PROCESSED: - p++; - continue; - case page_recv_t::RECV_WILL_NOT_READ: - if (UNIV_LIKELY(!!recover_low(page_id, p, mtr, free_block))) + while (pages_it != pages.end()) + { + if (is_corrupt_fs() || is_corrupt_log()) { -next_free_block: + if (space) + space->release(); mysql_mutex_unlock(&mutex); - if (!last_batch) - mysql_mutex_unlock(&log_sys.mutex); - mysql_mutex_assert_not_owner(&log_sys.mutex); - free_block= buf_LRU_get_free_block(false); - if (!last_batch) - mysql_mutex_lock(&log_sys.mutex); - mysql_mutex_lock(&mutex); - break; + if (free_block) + { + mysql_mutex_lock(&buf_pool.mutex); + buf_LRU_block_free_non_file_page(free_block); + mysql_mutex_unlock(&buf_pool.mutex); + } + return; } - ut_ad(p == pages.end() || p->first > page_id); - continue; - case page_recv_t::RECV_NOT_PROCESSED: - recv_read_in_area(page_id, p); + if (apply_batch(space_id, space, free_block, last_batch)) + break; } - p= pages.lower_bound(page_id); - /* Ensure that progress will be made. */ - ut_ad(p == pages.end() || p->first > page_id || - p->second.state >= page_recv_t::RECV_BEING_READ); } - buf_pool.free_block(free_block); + if (space) + space->release(); - /* Wait until all the pages have been processed */ - for (;;) + if (free_block) { - const bool empty= pages.empty(); - if (empty && !os_aio_pending_reads()) - break; - - if (!is_corrupt_fs() && !is_corrupt_log()) - { - if (last_batch) - { - mysql_mutex_assert_not_owner(&log_sys.mutex); - if (!empty) - my_cond_wait(&cond, &mutex.m_mutex); - else - { - mysql_mutex_unlock(&mutex); - os_aio_wait_until_no_pending_reads(false); - mysql_mutex_lock(&mutex); - ut_ad(pages.empty()); - } - } - else - { - mysql_mutex_unlock(&mutex); - set_timespec_nsec(abstime, 500000000ULL); /* 0.5s */ - my_cond_timedwait(&cond, &log_sys.mutex.m_mutex, &abstime); - mysql_mutex_lock(&mutex); - } - continue; - } - if (is_corrupt_fs() && !srv_force_recovery) - ib::info() << "Set innodb_force_recovery=1 to ignore corrupted pages."; - mysql_mutex_unlock(&mutex); - return; + mysql_mutex_lock(&buf_pool.mutex); + buf_LRU_block_free_non_file_page(free_block); + mysql_mutex_unlock(&buf_pool.mutex); } } if (last_batch) - /* We skipped this in buf_page_create(). */ - mlog_init.mark_ibuf_exist(mtr); + { + if (!recv_no_ibuf_operations) + /* We skipped this in buf_page_create(). */ + mlog_init.mark_ibuf_exist(); + mlog_init.clear(); + } else { mlog_init.reset(); @@ -3440,21 +3729,19 @@ next_free_block: mysql_mutex_assert_not_owner(&log_sys.mutex); mysql_mutex_unlock(&mutex); - if (last_batch && srv_operation != SRV_OPERATION_RESTORE && - srv_operation != SRV_OPERATION_RESTORE_EXPORT) - log_sort_flush_list(); - else - { - /* Instead of flushing, last_batch could sort the buf_pool.flush_list - in ascending order of buf_page_t::oldest_modification. */ - buf_flush_sync_batch(recovered_lsn); - } - if (!last_batch) { + buf_flush_sync_batch(recovered_lsn); buf_pool_invalidate(); mysql_mutex_lock(&log_sys.mutex); } + else if (srv_operation == SRV_OPERATION_RESTORE || + srv_operation == SRV_OPERATION_RESTORE_EXPORT) + buf_flush_sync_batch(recovered_lsn); + else + /* Instead of flushing, last_batch could sort the buf_pool.flush_list + in ascending order of buf_page_t::oldest_modification() */ + log_sort_flush_list(); mysql_mutex_lock(&mutex); @@ -3463,24 +3750,6 @@ next_free_block: mysql_mutex_unlock(&mutex); } -/** Check whether the number of read redo log blocks exceeds the maximum. -Store last_stored_lsn if the recovery is not in the last phase. -@param[in,out] store whether to store page operations -@return whether the memory is exhausted */ -inline bool recv_sys_t::is_memory_exhausted(store_t *store) -{ - if (*store == STORE_NO || - UT_LIST_GET_LEN(blocks) * 3 < buf_pool.get_n_pages()) - return false; - if (*store == STORE_YES) - last_stored_lsn= recovered_lsn; - *store= STORE_NO; - DBUG_PRINT("ib_log",("Ran out of memory and last stored lsn " LSN_PF - " last stored offset " ULINTPF "\n", - recovered_lsn, recovered_offset)); - return true; -} - /** Adds data from a new log block to the parsing buffer of recv_sys if recv_sys.parse_start_lsn is non-zero. @param[in] log_block log block to add @@ -3588,7 +3857,7 @@ static bool recv_scan_log_recs( bool more_data = false; bool apply = recv_sys.mlog_checkpoint_lsn != 0; ulint recv_parsing_buf_size = RECV_PARSING_BUF_SIZE; - const bool last_phase = (*store == STORE_IF_EXISTS); + const store_t old_store = *store; ut_ad(start_lsn % OS_FILE_LOG_BLOCK_SIZE == 0); ut_ad(end_lsn % OS_FILE_LOG_BLOCK_SIZE == 0); ut_ad(end_lsn >= start_lsn + OS_FILE_LOG_BLOCK_SIZE); @@ -3710,8 +3979,8 @@ static bool recv_scan_log_recs( } /* During last phase of scanning, there can be redo logs - left in recv_sys.buf to parse & store it in recv_sys.heap */ - if (last_phase + left in recv_sys.buf to parse & store it in recv_sys.pages */ + if (old_store == STORE_IF_EXISTS && recv_sys.recovered_lsn < recv_sys.scanned_lsn) { more_data = true; } @@ -3732,33 +4001,21 @@ static bool recv_scan_log_recs( if (more_data && !recv_sys.is_corrupt_log()) { /* Try to parse more log records */ if (recv_sys.parse(checkpoint_lsn, store, apply)) { + finished = true; ut_ad(recv_sys.is_corrupt_log() || recv_sys.is_corrupt_fs() || recv_sys.mlog_checkpoint_lsn == recv_sys.recovered_lsn); - finished = true; - goto func_exit; - } - - recv_sys.is_memory_exhausted(store); - - if (recv_sys.recovered_offset > recv_parsing_buf_size / 4 - || (recv_sys.recovered_offset - && recv_sys.len - >= recv_parsing_buf_size - RECV_SCAN_SIZE)) { + } else if (recv_sys.recovered_offset + > recv_parsing_buf_size / 4 + || (recv_sys.recovered_offset + && recv_sys.len + >= recv_parsing_buf_size - RECV_SCAN_SIZE)) { /* Move parsing buffer data to the buffer start */ recv_sys_justify_left_parsing_buf(); } - - /* Need to re-parse the redo log which're stored - in recv_sys.buf */ - if (last_phase && *store == STORE_NO) { - finished = false; - } } -func_exit: - recv_sys.maybe_finish_batch(); mysql_mutex_unlock(&recv_sys.mutex); return(finished); } @@ -3802,13 +4059,6 @@ recv_group_scan_log_recs( ut_d(recv_sys.after_apply = last_phase); do { - if (last_phase && store == STORE_NO) { - store = STORE_IF_EXISTS; - recv_sys.apply(false); - /* Rescan the redo logs from last stored lsn */ - end_lsn = recv_sys.recovered_lsn; - } - start_lsn = ut_uint64_align_down(end_lsn, OS_FILE_LOG_BLOCK_SIZE); end_lsn = start_lsn; @@ -3913,8 +4163,8 @@ next: /* fall through */ case file_name_t::DELETED: recv_sys_t::map::iterator r = p++; - r->second.log.clear(); - recv_sys.pages.erase(r); + recv_sys.pages_it_invalidate(r); + recv_sys.erase(r); continue; } ut_ad(0); @@ -3938,8 +4188,6 @@ func_exit: continue; } - missing_tablespace = true; - if (srv_force_recovery > 0) { ib::warn() << "Tablespace " << rs.first <<" was not found at " << rs.second.name @@ -3954,14 +4202,11 @@ func_exit: << " was not found at '" << rs.second.name << "', but there" <<" were no modifications either."; + } else { + missing_tablespace = true; } } - if (!rescan || srv_force_recovery > 0) { - missing_tablespace = false; - } - - err = DB_SUCCESS; goto func_exit; } @@ -4344,6 +4589,12 @@ completed: return(DB_ERROR); } + ut_ad(contiguous_lsn <= recv_sys.recovered_lsn); + ut_ad(recv_sys.scanned_lsn == recv_sys.scanned_lsn); + + log_sys.set_lsn(recv_sys.recovered_lsn); + log_sys.set_flushed_lsn(recv_sys.recovered_lsn); + /* In case of multi-batch recovery, redo log for the last batch is not applied yet. */ diff --git a/storage/innobase/os/os0file.cc b/storage/innobase/os/os0file.cc index 3b81dc7ee07..f8fb9270230 100644 --- a/storage/innobase/os/os0file.cc +++ b/storage/innobase/os/os0file.cc @@ -3431,15 +3431,12 @@ os_file_get_status( return(ret); } - -extern void fil_aio_callback(const IORequest &request); - -static void io_callback(tpool::aiocb *cb) +static void io_callback_errorcheck(const tpool::aiocb *cb) { - const IORequest &request= *static_cast - (static_cast(cb->m_userdata)); if (cb->m_err != DB_SUCCESS) { + const IORequest &request= *static_cast + (static_cast(cb->m_userdata)); ib::fatal() << "IO Error: " << cb->m_err << " during " << (request.is_async() ? "async " : "sync ") << (request.is_LRU() ? "lru " : "") << @@ -3447,19 +3444,36 @@ static void io_callback(tpool::aiocb *cb) " of " << cb->m_len << " bytes, for file " << cb->m_fh << ", returned " << cb->m_ret_len; } - /* Return cb back to cache*/ - if (cb->m_opcode == tpool::aio_opcode::AIO_PREAD) - { - ut_ad(read_slots->contains(cb)); - fil_aio_callback(request); - read_slots->release(cb); - } - else - { - ut_ad(write_slots->contains(cb)); - fil_aio_callback(request); - write_slots->release(cb); - } +} + +static void fake_io_callback(void *c) +{ + tpool::aiocb *cb= static_cast(c); + ut_ad(read_slots->contains(cb)); + static_cast(static_cast(cb->m_userdata))-> + fake_read_complete(cb->m_offset); + read_slots->release(cb); +} + +static void read_io_callback(void *c) +{ + tpool::aiocb *cb= static_cast(c); + ut_ad(cb->m_opcode == tpool::aio_opcode::AIO_PREAD); + io_callback_errorcheck(cb); + ut_ad(read_slots->contains(cb)); + static_cast + (static_cast(cb->m_userdata))->read_complete(); + read_slots->release(cb); +} + +static void write_io_callback(void *c) +{ + tpool::aiocb *cb= static_cast(c); + ut_ad(cb->m_opcode == tpool::aio_opcode::AIO_PWRITE); + ut_ad(write_slots->contains(cb)); + static_cast + (static_cast(cb->m_userdata))->write_complete(); + write_slots->release(cb); } #ifdef LINUX_NATIVE_AIO @@ -3704,6 +3718,28 @@ void os_aio_wait_until_no_pending_reads(bool declare) tpool::tpool_wait_end(); } +/** Submit a fake read request during crash recovery. +@param type fake read request +@param offset additional context */ +void os_fake_read(const IORequest &type, os_offset_t offset) +{ + tpool::aiocb *cb= read_slots->acquire(); + + cb->m_group= read_slots->get_task_group(); + cb->m_fh= type.node->handle.m_file; + cb->m_buffer= nullptr; + cb->m_len= 0; + cb->m_offset= offset; + cb->m_opcode= tpool::aio_opcode::AIO_PREAD; + new (cb->m_userdata) IORequest{type}; + cb->m_internal_task.m_func= fake_io_callback; + cb->m_internal_task.m_arg= cb; + cb->m_internal_task.m_group= cb->m_group; + + srv_thread_pool->submit_task(&cb->m_internal_task); +} + + /** Request a read or write. @param type I/O request @param buf buffer @@ -3748,23 +3784,32 @@ func_exit: return err; } + io_slots* slots; + tpool::callback_func callback; + tpool::aio_opcode opcode; + if (type.is_read()) { ++os_n_file_reads; + slots = read_slots; + callback = read_io_callback; + opcode = tpool::aio_opcode::AIO_PREAD; } else { ++os_n_file_writes; + slots = write_slots; + callback = write_io_callback; + opcode = tpool::aio_opcode::AIO_PWRITE; } compile_time_assert(sizeof(IORequest) <= tpool::MAX_AIO_USERDATA_LEN); - io_slots* slots= type.is_read() ? read_slots : write_slots; tpool::aiocb* cb = slots->acquire(); cb->m_buffer = buf; - cb->m_callback = (tpool::callback_func)io_callback; + cb->m_callback = callback; cb->m_group = slots->get_task_group(); cb->m_fh = type.node->handle.m_file; cb->m_len = (int)n; cb->m_offset = offset; - cb->m_opcode = type.is_read() ? tpool::aio_opcode::AIO_PREAD : tpool::aio_opcode::AIO_PWRITE; + cb->m_opcode = opcode; new (cb->m_userdata) IORequest{type}; ut_a(reinterpret_cast(cb->m_buffer) % OS_FILE_LOG_BLOCK_SIZE @@ -3777,6 +3822,7 @@ func_exit: os_file_handle_error(type.node->name, type.is_read() ? "aio read" : "aio write"); err = DB_IO_ERROR; + type.node->space->release(); } goto func_exit; From df524dc06f7a92ebeb737755e8bd56c790fcf002 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20M=C3=A4kel=C3=A4?= Date: Fri, 19 May 2023 15:29:26 +0300 Subject: [PATCH 3/7] MDEV-31308 InnoDB monitor trx_rseg_history_len was accidentally disabled by default innodb_counter_info[]: Revert a change that was accidentally made in commit 204e7225dce32130ac2c96f469611d2cb421241e --- mysql-test/suite/innodb/r/monitor.result | 138 ++++++++++++----------- mysql-test/suite/innodb/t/monitor.test | 4 +- storage/innobase/srv/srv0mon.cc | 2 +- 3 files changed, 74 insertions(+), 70 deletions(-) diff --git a/mysql-test/suite/innodb/r/monitor.result b/mysql-test/suite/innodb/r/monitor.result index c874a84d26b..22e6fbea1e3 100644 --- a/mysql-test/suite/innodb/r/monitor.result +++ b/mysql-test/suite/innodb/r/monitor.result @@ -1,10 +1,9 @@ -set global innodb_monitor_disable = All; select name, if(enabled,'enabled','disabled') status from information_schema.innodb_metrics; name status metadata_table_handles_opened disabled -lock_deadlocks disabled -lock_timeouts disabled +lock_deadlocks enabled +lock_timeouts enabled lock_rec_lock_waits disabled lock_table_lock_waits disabled lock_rec_lock_requests disabled @@ -14,32 +13,32 @@ lock_rec_locks disabled lock_table_lock_created disabled lock_table_lock_removed disabled lock_table_locks disabled -lock_row_lock_current_waits disabled -lock_row_lock_time disabled -lock_row_lock_time_max disabled -lock_row_lock_waits disabled -lock_row_lock_time_avg disabled -buffer_pool_size disabled -buffer_pool_reads disabled -buffer_pool_read_requests disabled -buffer_pool_write_requests disabled -buffer_pool_wait_free disabled -buffer_pool_read_ahead disabled -buffer_pool_read_ahead_evicted disabled -buffer_pool_pages_total disabled -buffer_pool_pages_misc disabled -buffer_pool_pages_data disabled -buffer_pool_bytes_data disabled -buffer_pool_pages_dirty disabled -buffer_pool_bytes_dirty disabled -buffer_pool_pages_free disabled -buffer_pages_created disabled -buffer_pages_written disabled -buffer_pages_read disabled -buffer_index_sec_rec_cluster_reads disabled -buffer_index_sec_rec_cluster_reads_avoided disabled -buffer_data_reads disabled -buffer_data_written disabled +lock_row_lock_current_waits enabled +lock_row_lock_time enabled +lock_row_lock_time_max enabled +lock_row_lock_waits enabled +lock_row_lock_time_avg enabled +buffer_pool_size enabled +buffer_pool_reads enabled +buffer_pool_read_requests enabled +buffer_pool_write_requests enabled +buffer_pool_wait_free enabled +buffer_pool_read_ahead enabled +buffer_pool_read_ahead_evicted enabled +buffer_pool_pages_total enabled +buffer_pool_pages_misc enabled +buffer_pool_pages_data enabled +buffer_pool_bytes_data enabled +buffer_pool_pages_dirty enabled +buffer_pool_bytes_dirty enabled +buffer_pool_pages_free enabled +buffer_pages_created enabled +buffer_pages_written enabled +buffer_pages_read enabled +buffer_index_sec_rec_cluster_reads enabled +buffer_index_sec_rec_cluster_reads_avoided enabled +buffer_data_reads enabled +buffer_data_written enabled buffer_flush_batch_scanned disabled buffer_flush_batch_num_scan disabled buffer_flush_batch_scanned_per_call disabled @@ -72,8 +71,8 @@ buffer_flush_background_pages disabled buffer_LRU_batch_scanned disabled buffer_LRU_batch_num_scan disabled buffer_LRU_batch_scanned_per_call disabled -buffer_LRU_batch_flush_total_pages disabled -buffer_LRU_batch_evict_total_pages disabled +buffer_LRU_batch_flush_total_pages enabled +buffer_LRU_batch_evict_total_pages enabled buffer_LRU_single_flush_failure_count disabled buffer_LRU_get_free_search disabled buffer_LRU_search_scanned disabled @@ -114,24 +113,24 @@ buffer_page_written_blob disabled buffer_page_written_zblob disabled buffer_page_written_zblob2 disabled buffer_page_written_other disabled -os_data_reads disabled -os_data_writes disabled -os_data_fsyncs disabled -os_pending_reads disabled -os_pending_writes disabled -os_log_bytes_written disabled -os_log_fsyncs disabled -os_log_pending_fsyncs disabled -os_log_pending_writes disabled +os_data_reads enabled +os_data_writes enabled +os_data_fsyncs enabled +os_pending_reads enabled +os_pending_writes enabled +os_log_bytes_written enabled +os_log_fsyncs enabled +os_log_pending_fsyncs enabled +os_log_pending_writes enabled trx_rw_commits disabled trx_ro_commits disabled trx_nl_ro_commits disabled trx_commits_insert_update disabled trx_rollbacks disabled trx_rollbacks_savepoint disabled -trx_rseg_history_len disabled +trx_rseg_history_len enabled trx_undo_slots_used disabled -trx_undo_slots_cached disabled +trx_undo_slots_cached enabled trx_rseg_current_size disabled purge_del_mark_records disabled purge_upd_exist_or_extern_records disabled @@ -150,10 +149,10 @@ log_max_modified_age_async disabled log_pending_log_flushes disabled log_pending_checkpoint_writes disabled log_num_log_io disabled -log_waits disabled -log_write_requests disabled -log_writes disabled -log_padded disabled +log_waits enabled +log_write_requests enabled +log_writes enabled +log_padded enabled compress_pages_compressed disabled compress_pages_decompressed disabled compression_pad_increments disabled @@ -171,42 +170,42 @@ index_page_merge_successful disabled index_page_reorg_attempts disabled index_page_reorg_successful disabled index_page_discards disabled -adaptive_hash_searches disabled -adaptive_hash_searches_btree disabled +adaptive_hash_searches enabled +adaptive_hash_searches_btree enabled adaptive_hash_pages_added disabled adaptive_hash_pages_removed disabled adaptive_hash_rows_added disabled adaptive_hash_rows_removed disabled adaptive_hash_rows_deleted_no_hash_entry disabled adaptive_hash_rows_updated disabled -file_num_open_files disabled -ibuf_merges_insert disabled -ibuf_merges_delete_mark disabled -ibuf_merges_delete disabled -ibuf_merges_discard_insert disabled -ibuf_merges_discard_delete_mark disabled -ibuf_merges_discard_delete disabled -ibuf_merges disabled -ibuf_size disabled +file_num_open_files enabled +ibuf_merges_insert enabled +ibuf_merges_delete_mark enabled +ibuf_merges_delete enabled +ibuf_merges_discard_insert enabled +ibuf_merges_discard_delete_mark enabled +ibuf_merges_discard_delete enabled +ibuf_merges enabled +ibuf_size enabled innodb_master_thread_sleeps disabled -innodb_activity_count disabled +innodb_activity_count enabled innodb_master_active_loops disabled innodb_master_idle_loops disabled innodb_log_flush_usec disabled innodb_dict_lru_usec disabled innodb_dict_lru_count_active disabled innodb_dict_lru_count_idle disabled -innodb_dblwr_writes disabled -innodb_dblwr_pages_written disabled -innodb_page_size disabled +innodb_dblwr_writes enabled +innodb_dblwr_pages_written enabled +innodb_page_size enabled dml_reads disabled -dml_inserts disabled -dml_deletes disabled -dml_updates disabled -dml_system_reads disabled -dml_system_inserts disabled -dml_system_deletes disabled -dml_system_updates disabled +dml_inserts enabled +dml_deletes enabled +dml_updates enabled +dml_system_reads enabled +dml_system_inserts enabled +dml_system_deletes enabled +dml_system_updates enabled ddl_background_drop_indexes disabled ddl_online_create_index disabled ddl_pending_alter_table disabled @@ -216,6 +215,9 @@ icp_attempts disabled icp_no_match disabled icp_out_of_range disabled icp_match disabled +set global innodb_monitor_disable = All; +select name from information_schema.innodb_metrics where enabled; +name set global innodb_monitor_enable = all; select name from information_schema.innodb_metrics where not enabled; name diff --git a/mysql-test/suite/innodb/t/monitor.test b/mysql-test/suite/innodb/t/monitor.test index d6fa3f2fbc9..65a93e5a97a 100644 --- a/mysql-test/suite/innodb/t/monitor.test +++ b/mysql-test/suite/innodb/t/monitor.test @@ -5,12 +5,14 @@ # sys_vars.innodb_monitor_enable_basic --source include/have_innodb.inc -set global innodb_monitor_disable = All; # Test turn on/off the monitor counter with "all" option # By default, they will be off. select name, if(enabled,'enabled','disabled') status from information_schema.innodb_metrics; +set global innodb_monitor_disable = All; +select name from information_schema.innodb_metrics where enabled; + # Turn on all monitor counters set global innodb_monitor_enable = all; diff --git a/storage/innobase/srv/srv0mon.cc b/storage/innobase/srv/srv0mon.cc index 971f4f330c8..3065ab19462 100644 --- a/storage/innobase/srv/srv0mon.cc +++ b/storage/innobase/srv/srv0mon.cc @@ -704,7 +704,7 @@ static monitor_info_t innodb_counter_info[] = {"trx_rseg_history_len", "transaction", "Length of the TRX_RSEG_HISTORY list", static_cast( - MONITOR_EXISTING | MONITOR_DISPLAY_CURRENT), + MONITOR_EXISTING | MONITOR_DISPLAY_CURRENT | MONITOR_DEFAULT_ON), MONITOR_DEFAULT_START, MONITOR_RSEG_HISTORY_LEN}, {"trx_undo_slots_used", "transaction", "Number of undo slots used", From d2420669bd07a29276e14f52a71bab0e7d5d8587 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20M=C3=A4kel=C3=A4?= Date: Fri, 19 May 2023 15:38:48 +0300 Subject: [PATCH 4/7] MDEV-31309 Innodb_buffer_pool_read_requests is not updated correctly srv_export_innodb_status(): Update export_vars.innodb_buffer_pool_read_requests as it was done before commit a55b951e6082a4ce9a1f2ed5ee176ea7dbbaf1f2 (MDEV-26827). If innodb_status_variables[] pointed to a sharded variable, it would only access the first shard. --- storage/innobase/handler/ha_innodb.cc | 3 ++- storage/innobase/include/srv0srv.h | 2 ++ storage/innobase/srv/srv0srv.cc | 3 +++ 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc index 0b117c02e29..2937ca40752 100644 --- a/storage/innobase/handler/ha_innodb.cc +++ b/storage/innobase/handler/ha_innodb.cc @@ -946,7 +946,8 @@ static SHOW_VAR innodb_status_variables[]= { {"buffer_pool_read_ahead", &buf_pool.stat.n_ra_pages_read, SHOW_SIZE_T}, {"buffer_pool_read_ahead_evicted", &buf_pool.stat.n_ra_pages_evicted, SHOW_SIZE_T}, - {"buffer_pool_read_requests", &buf_pool.stat.n_page_gets, SHOW_SIZE_T}, + {"buffer_pool_read_requests", + &export_vars.innodb_buffer_pool_read_requests, SHOW_SIZE_T}, {"buffer_pool_reads", &buf_pool.stat.n_pages_read, SHOW_SIZE_T}, {"buffer_pool_wait_free", &buf_pool.stat.LRU_waits, SHOW_SIZE_T}, {"buffer_pool_write_requests", diff --git a/storage/innobase/include/srv0srv.h b/storage/innobase/include/srv0srv.h index 96cfe886c02..c4c854f6b9c 100644 --- a/storage/innobase/include/srv0srv.h +++ b/storage/innobase/include/srv0srv.h @@ -674,6 +674,8 @@ struct export_var_t{ #ifdef UNIV_DEBUG ulint innodb_buffer_pool_pages_latched; /*!< Latched pages */ #endif /* UNIV_DEBUG */ + /** buf_pool.stat.n_page_gets (a sharded counter) */ + ulint innodb_buffer_pool_read_requests; ulint innodb_buffer_pool_write_requests;/*!< srv_stats.buf_pool_write_requests */ ulint innodb_checkpoint_age; ulint innodb_checkpoint_max_age; diff --git a/storage/innobase/srv/srv0srv.cc b/storage/innobase/srv/srv0srv.cc index af2845085ad..41ef2bccdc4 100644 --- a/storage/innobase/srv/srv0srv.cc +++ b/storage/innobase/srv/srv0srv.cc @@ -1011,6 +1011,9 @@ srv_export_innodb_status(void) export_vars.innodb_data_written = srv_stats.data_written + (dblwr << srv_page_size_shift); + export_vars.innodb_buffer_pool_read_requests + = buf_pool.stat.n_page_gets; + export_vars.innodb_buffer_pool_write_requests = srv_stats.buf_pool_write_requests; From 03d4fd3214bae64856255f4de1eb533b05f88e8d Mon Sep 17 00:00:00 2001 From: Robin Newhouse Date: Fri, 27 Jan 2023 23:10:47 +0000 Subject: [PATCH 5/7] Backport GitLab CI to 10.5 Add .gitlab-ci.yml file to earliest supported branch to enable automated building and testing for all MariaDB major branches. Note to mergers: GitLab CI is available for branches >= 10.6. This commit includes a GitLab CI file identical to that in branches >= 10.6, except for the MARIADB_MAJOR_VERSION variable which should reflect the branch version. A modified CI will be included in branches 10.4 with PR !2418. Also changed is the `allow_failure: true` for the MSAN build, which should be merged up to later branches. All new code of the whole pull request, including one or several files that are either new files or modified ones, are contributed under the BSD-new license. I am contributing on behalf of my employer Amazon Web Services, Inc. --- .gitlab-ci.yml | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/.gitlab-ci.yml b/.gitlab-ci.yml index 39dae0facb8..0a9113a36c7 100644 --- a/.gitlab-ci.yml +++ b/.gitlab-ci.yml @@ -39,10 +39,10 @@ default: # submodules (a commit in this repo does not affect their builds anyway) and # many components that are otherwise slow to build. variables: - CMAKE_FLAGS: "-DPLUGIN_COLUMNSTORE=NO -DPLUGIN_ROCKSDB=NO -DPLUGIN_S3=NO -DPLUGIN_MROONGA=NO -DPLUGIN_CONNECT=NO -DPLUGIN_MROONGA=NO -DPLUGIN_TOKUDB=NO -DPLUGIN_PERFSCHEMA=NO -DWITH_WSREP=OFF" + CMAKE_FLAGS: "-DWITH_SSL=system -DPLUGIN_COLUMNSTORE=NO -DPLUGIN_ROCKSDB=NO -DPLUGIN_S3=NO -DPLUGIN_MROONGA=NO -DPLUGIN_CONNECT=NO -DPLUGIN_MROONGA=NO -DPLUGIN_TOKUDB=NO -DPLUGIN_PERFSCHEMA=NO -DWITH_WSREP=OFF" # Major version dictates which branches share the same ccache. E.g. 10.6-abc # and 10.6-xyz will have the same cache. - MARIADB_MAJOR_VERSION: "10.6" + MARIADB_MAJOR_VERSION: "10.5" # NOTE! Currently ccache is only used on the Centos8 build. As each job has # sufficiently different environments they are unable to benefit from each # other's ccaches. As each build generates about 1 GB of ccache, having @@ -82,13 +82,13 @@ fedora: GIT_STRATEGY: fetch GIT_SUBMODULE_STRATEGY: normal script: - - yum install -y yum-utils rpm-build openssl-devel graphviz clang gnutls-devel + - yum install -y yum-utils rpm-build openssl-devel graphviz # Accelerate builds with unsafe disk access, as we can afford to loose the entire build anyway - yum install -y https://github.com/stewartsmith/libeatmydata/releases/download/v129/libeatmydata-129-1.fc33.x86_64.rpm # This repository does not have any .spec files, so install dependencies based on Fedora spec file - yum-builddep -y mariadb-server - mkdir builddir; cd builddir - - cmake -DRPM=$CI_JOB_NAME $CMAKE_FLAGS -DWITH_SSL=bundled .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log + - cmake -DRPM=$CI_JOB_NAME $CMAKE_FLAGS .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log - cmake --graphviz=../dependencies.dot .. && dot -Tpng -o ../dependencies.png ../dependencies.dot - eatmydata make package -j 2 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log # @TODO: Don't use -j without the limit of 2 on Gitlab.com as builds just @@ -113,13 +113,13 @@ fedora-ninja: GIT_STRATEGY: fetch GIT_SUBMODULE_STRATEGY: normal script: - - yum install -y yum-utils rpm-build openssl-devel graphviz ninja-build gnutls-devel + - yum install -y yum-utils rpm-build openssl-devel graphviz ninja-build # Accelerate builds with unsafe disk access, as we can afford to loose the entire build anyway - yum install -y https://github.com/stewartsmith/libeatmydata/releases/download/v129/libeatmydata-129-1.fc33.x86_64.rpm # This repository does not have any .spec files, so install dependencies based on Fedora spec file - yum-builddep -y mariadb-server - mkdir builddir; cd builddir - - cmake -DRPM=generic $CMAKE_FLAGS -DWITH_SSL=bundled -DCMAKE_BUILD_WITH_INSTALL_RPATH=ON -G Ninja .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log + - cmake -DRPM=generic $CMAKE_FLAGS -DCMAKE_BUILD_WITH_INSTALL_RPATH=ON -G Ninja .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log - ninja -t graph > ../dependencies.dot && dot -Tpng -o ../dependencies.png ../dependencies.dot - eatmydata ninja package -j 2 --verbose 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log # @TODO: Unlike other builds, the Ninja builds using Gitlab.com runners don't get stuck, but they do get @@ -144,7 +144,7 @@ fedora-clang: GIT_STRATEGY: fetch GIT_SUBMODULE_STRATEGY: normal script: - - yum install -y yum-utils rpm-build openssl-devel graphviz clang gnutls-devel + - yum install -y yum-utils rpm-build openssl-devel graphviz clang # Accelerate builds with unsafe disk access, as we can afford to loose the entire build anyway - yum install -y https://github.com/stewartsmith/libeatmydata/releases/download/v129/libeatmydata-129-1.fc33.x86_64.rpm # This repository does not have any .spec files, so install dependencies based on Fedora spec file @@ -156,7 +156,7 @@ fedora-clang: - export CC_FOR_BUILD=${CC_FOR_BUILD:-clang} - export CFLAGS='-Wno-unused-command-line-argument' - export CXXFLAGS='-Wno-unused-command-line-argument' - - cmake -DRPM=generic $CMAKE_FLAGS -DWITH_SSL=bundled .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log + - cmake -DRPM=generic $CMAKE_FLAGS .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log - cmake --graphviz=../dependencies.dot .. && dot -Tpng -o ../dependencies.png ../dependencies.dot - eatmydata make package -j 2 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log # @TODO: Don't use -j without the limit of 2 on Gitlab.com as builds just @@ -181,7 +181,7 @@ fedora-sanitizer: GIT_STRATEGY: fetch GIT_SUBMODULE_STRATEGY: normal script: - - yum install -y yum-utils rpm-build openssl-devel clang gnutls-devel + - yum install -y yum-utils rpm-build openssl-devel clang - yum install -y libasan libtsan libubsan # This repository does not have any .spec files, so install dependencies based on Fedora spec file - yum-builddep -y mariadb-server @@ -192,7 +192,7 @@ fedora-sanitizer: - export CC_FOR_BUILD=${CC_FOR_BUILD:-clang} - export CFLAGS='-Wno-unused-command-line-argument' - export CXXFLAGS='-Wno-unused-command-line-argument' - - cmake -DRPM=$CI_JOB_NAME $CMAKE_FLAGS -DWITH_SSL=bundled $SANITIZER .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log + - cmake -DRPM=$CI_JOB_NAME $CMAKE_FLAGS $SANITIZER .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log # @TODO: the build will fail consistently at 24% when trying to make using eatmydata - make package -j 2 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log - *rpm_listfiles @@ -233,7 +233,7 @@ centos8: # This repository does not have any .spec files, so install dependencies based on CentOS spec file - yum-builddep -y mariadb-server - mkdir builddir; cd builddir - - cmake -DRPM=$CI_JOB_NAME $CMAKE_FLAGS -DWITH_SSL=system .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log + - cmake -DRPM=$CI_JOB_NAME $CMAKE_FLAGS .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log - eatmydata make package -j 2 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log # @TODO: Don't use -j without the limit of 2 on Gitlab.com as builds just # get stuck when running multi-proc and out of memory, see https://jira.mariadb.org/browse/MDEV-25968 @@ -266,7 +266,7 @@ centos7: # ..with a few extra ones, as CentOS 7 is very old and these are added in newer MariaDB releases - yum install -y yum-utils rpm-build gcc gcc-c++ bison libxml2-devel libevent-devel openssl-devel pcre2-devel - mkdir builddir; cd builddir - - cmake -DRPM=$CI_JOB_NAME $CMAKE_FLAGS -DWITH_SSL=system .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log + - cmake -DRPM=$CI_JOB_NAME $CMAKE_FLAGS .. 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log - make package -j 2 2>&1 | tee -a ../build-$CI_JOB_NAME-$CI_COMMIT_REF_SLUG.log # @TODO: Don't use -j without the limit of 2 on Gitlab.com as builds just # get stuck when running multi-proc and out of memory, see https://jira.mariadb.org/browse/MDEV-25968 @@ -331,6 +331,7 @@ mysql-test-run-asan: needs: - "fedora-sanitizer: [-DWITH_ASAN=YES]" <<: *mysql-test-run-def + allow_failure: true artifacts: when: always # Also show results when tests fail reports: From 3f59bbeeaec751e9aabdc544324546f3c8326f0f Mon Sep 17 00:00:00 2001 From: Teemu Ollakka Date: Mon, 17 Apr 2023 16:04:01 +0300 Subject: [PATCH 6/7] MDEV-29293 MariaDB stuck on starting commit state MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The problem seems to be a deadlock between KILL command execution and BF abort issued by an applier, where: * KILL has locked victim's LOCK_thd_kill and LOCK_thd_data. * Applier has innodb side global lock mutex and victim trx mutex. * KILL is calling innobase_kill_query, and is blocked by innodb global lock mutex. * Applier is in wsrep_innobase_kill_one_trx and is blocked by victim's LOCK_thd_kill. The fix in this commit removes the TOI replication of KILL command and makes KILL execution less intrusive operation. Aborting the victim happens now by using awake_no_mutex() and ha_abort_transaction(). If the KILL happens when the transaction is committing, the KILL operation is postponed to happen after the statement has completed in order to avoid KILL to interrupt commit processing. Notable changes in this commit: * wsrep client connections's error state may remain sticky after client connection is closed. This error message will then pop up for the next client session issuing first SQL statement. This problem raised with test galera.galera_bf_kill. The fix is to reset wsrep client error state, before a THD is reused for next connetion. * Release THD locks in wsrep_abort_transaction when locking innodb mutexes. This guarantees same locking order as with applier BF aborting. * BF abort from MDL was changed to do BF abort on server/wsrep-lib side first, and only then do the BF abort on InnoDB side. This removes the need to call back from InnoDB for BF aborts which originate from MDL and simplifies the locking. * Removed wsrep_thd_set_wsrep_aborter() from service_wsrep.h. The manipulation of the wsrep_aborter can be done solely on server side. Moreover, it is now debug only variable and could be excluded from optimized builds. * Remove LOCK_thd_kill from wsrep_thd_LOCK/UNLOCK to allow more fine grained locking for SR BF abort which may require locking of victim LOCK_thd_kill. Added explicit call for wsrep_thd_kill_LOCK/UNLOCK where appropriate. * Wsrep-lib was updated to version which allows external locking for BF abort calls. Changes to MTR tests: * Disable galera_bf_abort_group_commit. This test is going to be removed (MDEV-30855). * Record galera_gcache_recover_manytrx as result file was incomplete. Trivial change. * Make galera_create_table_as_select more deterministic: Wait until CTAS execution has reached MDL wait for multi-master conflict case. Expected error from multi-master conflict is ER_QUERY_INTERRUPTED. This is because CTAS does not yet have open wsrep transaction when it is waiting for MDL, query gets interrupted instead of BF aborted. This should be addressed in separate task. * A new test galera_kill_group_commit to verify correct behavior when KILL is executed while the transaction is committing. Co-authored-by: Seppo Jaakola Co-authored-by: Jan Lindström Signed-off-by: Julius Goryavsky --- include/mysql/service_wsrep.h | 9 +- mysql-test/suite/galera/disabled.def | 2 + mysql-test/suite/galera/r/MDEV-29293.result | 21 ++ .../r/galera_create_table_as_select.result | 1 + .../r/galera_gcache_recover_manytrx.result | 3 - .../galera/r/galera_kill_group_commit.result | 27 +++ mysql-test/suite/galera/t/MDEV-29293.test | 41 ++++ .../t/galera_create_table_as_select.test | 6 +- .../galera/t/galera_kill_group_commit.cnf | 5 + .../galera/t/galera_kill_group_commit.test | 69 ++++++ sql/handler.cc | 7 + sql/service_wsrep.cc | 39 +--- sql/sql_class.cc | 27 ++- sql/sql_class.h | 9 +- sql/sql_parse.cc | 58 +---- sql/sql_plugin_services.inl | 2 +- sql/wsrep_dummy.cc | 7 +- sql/wsrep_high_priority_service.cc | 1 + sql/wsrep_mysqld.cc | 45 ++-- sql/wsrep_server_service.cc | 8 +- sql/wsrep_server_service.h | 3 +- sql/wsrep_thd.cc | 214 +++++++++++++----- sql/wsrep_thd.h | 36 ++- sql/wsrep_trans_observer.h | 23 +- storage/innobase/handler/ha_innodb.cc | 190 +++++++++------- wsrep-lib | 2 +- 26 files changed, 592 insertions(+), 263 deletions(-) create mode 100644 mysql-test/suite/galera/r/MDEV-29293.result create mode 100644 mysql-test/suite/galera/r/galera_kill_group_commit.result create mode 100644 mysql-test/suite/galera/t/MDEV-29293.test create mode 100644 mysql-test/suite/galera/t/galera_kill_group_commit.cnf create mode 100644 mysql-test/suite/galera/t/galera_kill_group_commit.test diff --git a/include/mysql/service_wsrep.h b/include/mysql/service_wsrep.h index 42b758c03f3..f3588da4b46 100644 --- a/include/mysql/service_wsrep.h +++ b/include/mysql/service_wsrep.h @@ -57,6 +57,7 @@ extern struct wsrep_service_st { my_bool (*wsrep_on_func)(const MYSQL_THD thd); bool (*wsrep_prepare_key_for_innodb_func)(MYSQL_THD thd, const unsigned char*, size_t, const unsigned char*, size_t, struct wsrep_buf*, size_t*); void (*wsrep_thd_LOCK_func)(const MYSQL_THD thd); + int (*wsrep_thd_TRYLOCK_func)(const MYSQL_THD thd); void (*wsrep_thd_UNLOCK_func)(const MYSQL_THD thd); const char * (*wsrep_thd_query_func)(const MYSQL_THD thd); int (*wsrep_thd_retry_counter_func)(const MYSQL_THD thd); @@ -86,7 +87,6 @@ extern struct wsrep_service_st { ulong (*wsrep_OSU_method_get_func)(const MYSQL_THD thd); my_bool (*wsrep_thd_has_ignored_error_func)(const MYSQL_THD thd); void (*wsrep_thd_set_ignored_error_func)(MYSQL_THD thd, my_bool val); - bool (*wsrep_thd_set_wsrep_aborter_func)(MYSQL_THD bf_thd, MYSQL_THD thd); void (*wsrep_report_bf_lock_wait_func)(const MYSQL_THD thd, unsigned long long trx_id); void (*wsrep_thd_kill_LOCK_func)(const MYSQL_THD thd); @@ -108,6 +108,7 @@ extern struct wsrep_service_st { #define wsrep_on(thd) (thd) && WSREP_ON && wsrep_service->wsrep_on_func(thd) #define wsrep_prepare_key_for_innodb(A,B,C,D,E,F,G) wsrep_service->wsrep_prepare_key_for_innodb_func(A,B,C,D,E,F,G) #define wsrep_thd_LOCK(T) wsrep_service->wsrep_thd_LOCK_func(T) +#define wsrep_thd_TRYLOCK(T) wsrep_service->wsrep_thd_TRYLOCK_func(T) #define wsrep_thd_UNLOCK(T) wsrep_service->wsrep_thd_UNLOCK_func(T) #define wsrep_thd_kill_LOCK(T) wsrep_service->wsrep_thd_kill_LOCK_func(T) #define wsrep_thd_kill_UNLOCK(T) wsrep_service->wsrep_thd_kill_UNLOCK_func(T) @@ -136,7 +137,6 @@ extern struct wsrep_service_st { #define wsrep_OSU_method_get(T) wsrep_service->wsrep_OSU_method_get_func(T) #define wsrep_thd_has_ignored_error(T) wsrep_service->wsrep_thd_has_ignored_error_func(T) #define wsrep_thd_set_ignored_error(T,V) wsrep_service->wsrep_thd_set_ignored_error_func(T,V) -#define wsrep_thd_set_wsrep_aborter(T) wsrep_service->wsrep_thd_set_wsrep_aborter_func(T1, T2) #define wsrep_report_bf_lock_wait(T,I) wsrep_service->wsrep_report_bf_lock_wait(T,I) #define wsrep_thd_set_PA_unsafe(T) wsrep_service->wsrep_thd_set_PA_unsafe_func(T) #else @@ -170,6 +170,8 @@ void wsrep_set_data_home_dir(const char *data_dir); extern "C" my_bool wsrep_on(const MYSQL_THD thd); /* Lock thd wsrep lock */ extern "C" void wsrep_thd_LOCK(const MYSQL_THD thd); +/* Try thd wsrep lock. Return non-zero if lock could not be taken. */ +extern "C" int wsrep_thd_TRYLOCK(const MYSQL_THD thd); /* Unlock thd wsrep lock */ extern "C" void wsrep_thd_UNLOCK(const MYSQL_THD thd); @@ -192,8 +194,6 @@ extern "C" my_bool wsrep_thd_is_local(const MYSQL_THD thd); /* Return true if thd is in high priority mode */ /* todo: rename to is_high_priority() */ extern "C" my_bool wsrep_thd_is_applying(const MYSQL_THD thd); -/* set wsrep_aborter for the target THD */ -extern "C" bool wsrep_thd_set_wsrep_aborter(MYSQL_THD bf_thd, MYSQL_THD victim_thd); /* Return true if thd is in TOI mode */ extern "C" my_bool wsrep_thd_is_toi(const MYSQL_THD thd); /* Return true if thd is in replicating TOI mode */ @@ -237,7 +237,6 @@ extern "C" my_bool wsrep_thd_is_applying(const MYSQL_THD thd); extern "C" ulong wsrep_OSU_method_get(const MYSQL_THD thd); extern "C" my_bool wsrep_thd_has_ignored_error(const MYSQL_THD thd); extern "C" void wsrep_thd_set_ignored_error(MYSQL_THD thd, my_bool val); -extern "C" bool wsrep_thd_set_wsrep_aborter(MYSQL_THD bf_thd, MYSQL_THD victim_thd); extern "C" void wsrep_report_bf_lock_wait(const THD *thd, unsigned long long trx_id); /* declare parallel applying unsafety for the THD */ diff --git a/mysql-test/suite/galera/disabled.def b/mysql-test/suite/galera/disabled.def index d15f2d271ca..0c98133ab5f 100644 --- a/mysql-test/suite/galera/disabled.def +++ b/mysql-test/suite/galera/disabled.def @@ -22,3 +22,5 @@ MDEV-26575 : MDEV-29878 Galera test failure on MDEV-26575 galera_bf_abort_shutdown : MDEV-29918 Assertion failure on galera_bf_abort_shutdown galera_wan : [ERROR] WSREP: /home/buildbot/buildbot/build/gcs/src/gcs_state_msg.cpp:gcs_state_msg_get_quorum():947: Failed to establish quorum. galera_var_ignore_apply_errors : 28: "Server did not transition to READY state" +MDEV-27713 : test is using get_lock(), which is now rejected in cluster +galera_bf_abort_group_commit : MDEV-30855 PR to remove the test exists diff --git a/mysql-test/suite/galera/r/MDEV-29293.result b/mysql-test/suite/galera/r/MDEV-29293.result new file mode 100644 index 00000000000..70c0cc84a31 --- /dev/null +++ b/mysql-test/suite/galera/r/MDEV-29293.result @@ -0,0 +1,21 @@ +connection node_2; +connection node_1; +connect node_1a, 127.0.0.1, root, , test, $NODE_MYPORT_1; +connect node_1b, 127.0.0.1, root, , test, $NODE_MYPORT_1; +set wsrep_sync_wait = 0; +CREATE TABLE t1(a int not null primary key auto_increment, b int) engine=InnoDB; +INSERT INTO t1 VALUES (1,2); +connection node_1a; +BEGIN; +UPDATE t1 SET b=3 WHERE a=1; +connection node_1; +set debug_sync='wsrep_kill_before_awake_no_mutex SIGNAL before_kill WAIT_FOR continue'; +connection node_1b; +set debug_sync= 'now WAIT_FOR before_kill'; +connection node_2; +UPDATE t1 SET b=7 WHERE a=1; +connection node_1b; +set debug_sync= 'now SIGNAL continue'; +connection node_1; +DROP TABLE t1; +SET DEBUG_SYNC= 'RESET'; diff --git a/mysql-test/suite/galera/r/galera_create_table_as_select.result b/mysql-test/suite/galera/r/galera_create_table_as_select.result index 6f65ee99f0a..beda5f30fe2 100644 --- a/mysql-test/suite/galera/r/galera_create_table_as_select.result +++ b/mysql-test/suite/galera/r/galera_create_table_as_select.result @@ -82,6 +82,7 @@ connect node_1a, 127.0.0.1, root, , test, $NODE_MYPORT_1; LOCK TABLE t2 WRITE; connection node_1; CREATE TABLE t1 AS SELECT * FROM t2;; +connection node_1a; connection node_2; SELECT COUNT(*) = 5 FROM t2; COUNT(*) = 5 diff --git a/mysql-test/suite/galera/r/galera_gcache_recover_manytrx.result b/mysql-test/suite/galera/r/galera_gcache_recover_manytrx.result index 8495bfde2f9..9b1e8105c1c 100644 --- a/mysql-test/suite/galera/r/galera_gcache_recover_manytrx.result +++ b/mysql-test/suite/galera/r/galera_gcache_recover_manytrx.result @@ -134,6 +134,3 @@ connection node_1; call mtr.add_suppression("Error in Log_event::read_log_event():.*"); CALL mtr.add_suppression("conflict state 7 after post commit"); CALL mtr.add_suppression("Skipped GCache ring buffer recovery"); -connection node_2; -call mtr.add_suppression("Error in Log_event::read_log_event():.*"); -CALL mtr.add_suppression("Skipped GCache ring buffer recovery"); diff --git a/mysql-test/suite/galera/r/galera_kill_group_commit.result b/mysql-test/suite/galera/r/galera_kill_group_commit.result new file mode 100644 index 00000000000..bb59ce1486f --- /dev/null +++ b/mysql-test/suite/galera/r/galera_kill_group_commit.result @@ -0,0 +1,27 @@ +connection node_2; +connection node_1; +connect node_1_kill, 127.0.0.1, root, , test, $NODE_MYPORT_1; +connect node_1_ctrl, 127.0.0.1, root, , test, $NODE_MYPORT_1; +SET SESSION wsrep_sync_wait = 0; +connect node_1_follower, 127.0.0.1, root, , test, $NODE_MYPORT_1; +SET SESSION wsrep_sync_wait = 0; +connection node_1; +CREATE TABLE t1 (f1 INT PRIMARY KEY) ENGINE=InnoDB; +SET SESSION DEBUG_SYNC = "commit_before_enqueue SIGNAL leader_before_enqueue_reached WAIT_FOR leader_before_enqueue_continue"; +INSERT INTO t1 VALUES (1); +connection node_1_ctrl; +SET DEBUG_SYNC = "now WAIT_FOR leader_before_enqueue_reached"; +connection node_1_follower; +INSERT INTO t1 VALUES (2);; +connection node_1_ctrl; +connection node_1_kill; +# Execute KILL QUERY for group commit follower +SET DEBUG_SYNC = "now SIGNAL leader_before_enqueue_continue"; +connection node_1_follower; +connection node_1; +SELECT * FROM t1; +f1 +1 +2 +SET DEBUG_SYNC = "RESET"; +DROP TABLE t1; diff --git a/mysql-test/suite/galera/t/MDEV-29293.test b/mysql-test/suite/galera/t/MDEV-29293.test new file mode 100644 index 00000000000..dacbf714c06 --- /dev/null +++ b/mysql-test/suite/galera/t/MDEV-29293.test @@ -0,0 +1,41 @@ +--source include/galera_cluster.inc +--source include/have_innodb.inc +--source include/have_debug_sync.inc +--source include/galera_have_debug_sync.inc + +--connect node_1a, 127.0.0.1, root, , test, $NODE_MYPORT_1 +--connect node_1b, 127.0.0.1, root, , test, $NODE_MYPORT_1 +set wsrep_sync_wait = 0; + +CREATE TABLE t1(a int not null primary key auto_increment, b int) engine=InnoDB; +INSERT INTO t1 VALUES (1,2); + +--connection node_1a +--let $victim_id = `SELECT CONNECTION_ID()` +BEGIN; +UPDATE t1 SET b=3 WHERE a=1; + +--connection node_1 +set debug_sync='wsrep_kill_before_awake_no_mutex SIGNAL before_kill WAIT_FOR continue'; +--disable_query_log +--disable_result_log +--send_eval KILL CONNECTION $victim_id +--enable_result_log +--enable_query_log + +--connection node_1b +set debug_sync= 'now WAIT_FOR before_kill'; + +--connection node_2 +UPDATE t1 SET b=7 WHERE a=1; + +--connection node_1b +--let $wait_condition = SELECT COUNT(*) = 1 FROM INFORMATION_SCHEMA.PROCESSLIST WHERE User = 'system user' AND State LIKE 'Update_rows_log_event%'; +--source include/wait_condition.inc +set debug_sync= 'now SIGNAL continue'; + +--connection node_1 +--reap +DROP TABLE t1; +SET DEBUG_SYNC= 'RESET'; + diff --git a/mysql-test/suite/galera/t/galera_create_table_as_select.test b/mysql-test/suite/galera/t/galera_create_table_as_select.test index a6c1f657280..cfee63e5e27 100644 --- a/mysql-test/suite/galera/t/galera_create_table_as_select.test +++ b/mysql-test/suite/galera/t/galera_create_table_as_select.test @@ -113,6 +113,10 @@ LOCK TABLE t2 WRITE; --connection node_1 --send CREATE TABLE t1 AS SELECT * FROM t2; +--connection node_1a +--let $wait_condition = SELECT COUNT(*) = 1 FROM information_schema.processlist WHERE STATE LIKE 'Waiting for table metadata lock%' +--source include/wait_condition.inc + --connection node_2 SELECT COUNT(*) = 5 FROM t2; CREATE TABLE t1 AS SELECT * FROM t2; @@ -121,7 +125,7 @@ CREATE TABLE t1 AS SELECT * FROM t2; UNLOCK TABLES; --connection node_1 ---error ER_TABLE_EXISTS_ERROR,ER_LOCK_DEADLOCK +--error ER_TABLE_EXISTS_ERROR,ER_QUERY_INTERRUPTED --reap DROP TABLE t1, t2; diff --git a/mysql-test/suite/galera/t/galera_kill_group_commit.cnf b/mysql-test/suite/galera/t/galera_kill_group_commit.cnf new file mode 100644 index 00000000000..60f4f776409 --- /dev/null +++ b/mysql-test/suite/galera/t/galera_kill_group_commit.cnf @@ -0,0 +1,5 @@ +!include ../galera_2nodes.cnf + +[mysqld] +log-bin +log-slave-updates diff --git a/mysql-test/suite/galera/t/galera_kill_group_commit.test b/mysql-test/suite/galera/t/galera_kill_group_commit.test new file mode 100644 index 00000000000..4b84f2d90ef --- /dev/null +++ b/mysql-test/suite/galera/t/galera_kill_group_commit.test @@ -0,0 +1,69 @@ +# +# Verify that transaction which has reached group commit queue +# cannot be killed. If the kill succeeds, assertion for +# wsrep transaction state will fail. +# +# If the bug is present, i.e. wsrep transaction gets killed during +# group commit wait, this test is enough to reproduce the crash +# most of the time. +# + +--source include/have_innodb.inc +--source include/have_debug_sync.inc +--source include/galera_cluster.inc + +# Connection for KILL commands +--connect node_1_kill, 127.0.0.1, root, , test, $NODE_MYPORT_1 +# Connection for sync point control +--connect node_1_ctrl, 127.0.0.1, root, , test, $NODE_MYPORT_1 +SET SESSION wsrep_sync_wait = 0; +# Connection for group commit follower +--connect node_1_follower, 127.0.0.1, root, , test, $NODE_MYPORT_1 +# Need to disable sync wait to reach commit queue when leader +# is blocked. +SET SESSION wsrep_sync_wait = 0; +--let $follower_id = `SELECT CONNECTION_ID()` + +--connection node_1 +CREATE TABLE t1 (f1 INT PRIMARY KEY) ENGINE=InnoDB; + +SET SESSION DEBUG_SYNC = "commit_before_enqueue SIGNAL leader_before_enqueue_reached WAIT_FOR leader_before_enqueue_continue"; +--send INSERT INTO t1 VALUES (1) + +--connection node_1_ctrl +SET DEBUG_SYNC = "now WAIT_FOR leader_before_enqueue_reached"; + +--connection node_1_follower +# SET SESSION DEBUG_SYNC = "group_commit_waiting_for_prior SIGNAL follower_waiting_for_prior_reached WAIT_FOR follower_waiting_for_prior_continue"; +--send INSERT INTO t1 VALUES (2); + +--connection node_1_ctrl +# TODO: Is it possible to use sync points to enforce group commit to happen? +# The leader will hold commit monitor in commit_before_enqueue sync point, +# which prevents the follower to reach the group commit wait state. +# We now sleep and expect the follower to reach group commit, but this +# may cause false negatives. +--sleep 1 + +--connection node_1_kill +--echo # Execute KILL QUERY for group commit follower +--disable_query_log +--disable_result_log +# Because it is currently impossible to verify that the +# follower has reached group commit queue, the KILL may +# sometimes return success. +--error 0,ER_KILL_DENIED_ERROR +--eval KILL QUERY $follower_id +--enable_result_log +--enable_query_log + +SET DEBUG_SYNC = "now SIGNAL leader_before_enqueue_continue"; +--connection node_1_follower +--reap + +--connection node_1 +--reap +SELECT * FROM t1; + +SET DEBUG_SYNC = "RESET"; +DROP TABLE t1; diff --git a/sql/handler.cc b/sql/handler.cc index eaaf4664c07..7f591b8456c 100644 --- a/sql/handler.cc +++ b/sql/handler.cc @@ -7599,6 +7599,9 @@ Compare_keys handler::compare_key_parts(const Field &old_field, concurrent accesses. And it's an overkill to take LOCK_plugin and iterate the whole installed_htons[] array every time. + @note Object victim_thd is not guaranteed to exist after this + function returns. + @param bf_thd brute force THD asking for the abort @param victim_thd victim THD to be aborted @@ -7612,6 +7615,8 @@ int ha_abort_transaction(THD *bf_thd, THD *victim_thd, my_bool signal) if (!WSREP(bf_thd) && !(bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU && wsrep_thd_is_toi(bf_thd))) { + mysql_mutex_unlock(&victim_thd->LOCK_thd_data); + mysql_mutex_unlock(&victim_thd->LOCK_thd_kill); DBUG_RETURN(0); } @@ -7623,6 +7628,8 @@ int ha_abort_transaction(THD *bf_thd, THD *victim_thd, my_bool signal) else { WSREP_WARN("Cannot abort InnoDB transaction"); + mysql_mutex_unlock(&victim_thd->LOCK_thd_data); + mysql_mutex_unlock(&victim_thd->LOCK_thd_kill); } DBUG_RETURN(0); diff --git a/sql/service_wsrep.cc b/sql/service_wsrep.cc index 2d8eff2bd4b..ccce076d8a9 100644 --- a/sql/service_wsrep.cc +++ b/sql/service_wsrep.cc @@ -29,14 +29,17 @@ extern "C" my_bool wsrep_on(const THD *thd) extern "C" void wsrep_thd_LOCK(const THD *thd) { - mysql_mutex_lock(&thd->LOCK_thd_kill); mysql_mutex_lock(&thd->LOCK_thd_data); } +extern "C" int wsrep_thd_TRYLOCK(const THD *thd) +{ + return mysql_mutex_trylock(&thd->LOCK_thd_data); +} + extern "C" void wsrep_thd_UNLOCK(const THD *thd) { mysql_mutex_unlock(&thd->LOCK_thd_data); - mysql_mutex_unlock(&thd->LOCK_thd_kill); } extern "C" void wsrep_thd_kill_LOCK(const THD *thd) @@ -248,21 +251,12 @@ extern "C" my_bool wsrep_thd_bf_abort(THD *bf_thd, THD *victim_thd, if ((ret || !wsrep_on(victim_thd)) && signal) { - if (victim_thd->wsrep_aborter && victim_thd->wsrep_aborter != bf_thd->thread_id) - { - WSREP_DEBUG("victim is killed already by %llu, skipping awake", - victim_thd->wsrep_aborter); - wsrep_thd_UNLOCK(victim_thd); - return false; - } - - victim_thd->wsrep_aborter= bf_thd->thread_id; victim_thd->awake_no_mutex(KILL_QUERY_HARD); } else - WSREP_DEBUG("wsrep_thd_bf_abort skipped awake for %llu", thd_get_thread_id(victim_thd)); + WSREP_DEBUG("wsrep_thd_bf_abort skipped awake for %llu", + thd_get_thread_id(victim_thd)); - wsrep_thd_UNLOCK(victim_thd); return ret; } @@ -385,25 +379,6 @@ extern "C" ulong wsrep_OSU_method_get(const MYSQL_THD thd) return(global_system_variables.wsrep_OSU_method); } -extern "C" bool wsrep_thd_set_wsrep_aborter(THD *bf_thd, THD *victim_thd) -{ - mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); - if (!bf_thd) - { - victim_thd->wsrep_aborter= 0; - WSREP_DEBUG("wsrep_thd_set_wsrep_aborter resetting wsrep_aborter"); - return false; - } - if (victim_thd->wsrep_aborter && victim_thd->wsrep_aborter != bf_thd->thread_id) - { - return true; - } - victim_thd->wsrep_aborter= bf_thd->thread_id; - WSREP_DEBUG("wsrep_thd_set_wsrep_aborter setting wsrep_aborter %u", - victim_thd->wsrep_aborter); - return false; -} - extern "C" void wsrep_report_bf_lock_wait(const THD *thd, unsigned long long trx_id) { diff --git a/sql/sql_class.cc b/sql/sql_class.cc index 95a777c75cf..b22b766c409 100644 --- a/sql/sql_class.cc +++ b/sql/sql_class.cc @@ -1305,6 +1305,11 @@ void THD::init() wsrep_affected_rows = 0; m_wsrep_next_trx_id = WSREP_UNDEFINED_TRX_ID; wsrep_aborter = 0; + wsrep_abort_by_kill = NOT_KILLED; + wsrep_abort_by_kill_err = 0; +#ifndef DBUG_OFF + wsrep_killed_state = 0; +#endif /* DBUG_OFF */ wsrep_desynced_backup_stage= false; #endif /* WITH_WSREP */ @@ -1656,6 +1661,13 @@ void THD::reset_for_reuse() #endif #ifdef WITH_WSREP wsrep_free_status(this); + wsrep_cs().reset_error(); + wsrep_aborter= 0; + wsrep_abort_by_kill= NOT_KILLED; + wsrep_abort_by_kill_err= 0; +#ifndef DBUG_OFF + wsrep_killed_state= 0; +#endif /* DBUG_OFF */ #endif /* WITH_WSREP */ } @@ -1911,7 +1923,9 @@ void THD::awake_no_mutex(killed_state state_to_set) } /* Interrupt target waiting inside a storage engine. */ - if (state_to_set != NOT_KILLED && !wsrep_is_bf_aborted(this)) + if (state_to_set != NOT_KILLED && + IF_WSREP(!wsrep_is_bf_aborted(this) && wsrep_abort_by_kill == NOT_KILLED, + true)) ha_kill_query(this, thd_kill_level(this)); abort_current_cond_wait(false); @@ -2153,6 +2167,17 @@ void THD::reset_killed() mysql_mutex_unlock(&LOCK_thd_kill); } #ifdef WITH_WSREP + if (WSREP_NNULL(this)) + { + if (wsrep_abort_by_kill != NOT_KILLED) + { + mysql_mutex_assert_not_owner(&LOCK_thd_kill); + mysql_mutex_lock(&LOCK_thd_kill); + wsrep_abort_by_kill= NOT_KILLED; + wsrep_abort_by_kill_err= 0; + mysql_mutex_unlock(&LOCK_thd_kill); + } + } mysql_mutex_assert_not_owner(&LOCK_thd_data); mysql_mutex_lock(&LOCK_thd_data); wsrep_aborter= 0; diff --git a/sql/sql_class.h b/sql/sql_class.h index 68a69762354..c373c0f6a43 100644 --- a/sql/sql_class.h +++ b/sql/sql_class.h @@ -5147,7 +5147,14 @@ public: bool wsrep_ignore_table; /* thread who has started kill for this THD protected by LOCK_thd_data*/ my_thread_id wsrep_aborter; - + /* Kill signal used, if thread was killed by manual KILL. Protected by + LOCK_thd_kill. */ + std::atomic wsrep_abort_by_kill; + /* */ + struct err_info* wsrep_abort_by_kill_err; +#ifndef DBUG_OFF + int wsrep_killed_state; +#endif /* DBUG_OFF */ /* true if BF abort is observed in do_command() right after reading client's packet, and if the client has sent PS execute command. */ bool wsrep_delayed_BF_abort; diff --git a/sql/sql_parse.cc b/sql/sql_parse.cc index 4c7313265e8..59f21247445 100644 --- a/sql/sql_parse.cc +++ b/sql/sql_parse.cc @@ -7945,7 +7945,7 @@ static bool wsrep_mysql_parse(THD *thd, char *rawbuf, uint length, thd->wsrep_retry_counter < thd->variables.wsrep_retry_autocommit) { #ifdef ENABLED_DEBUG_SYNC - DBUG_EXECUTE_IF("sync.wsrep_retry_autocommit", + DBUG_EXECUTE_IF("sync.wsrep_retry_autocommit", { const char act[]= "now " @@ -9309,21 +9309,15 @@ kill_one_thread(THD *thd, my_thread_id id, killed_state kill_signal, killed_type thd->security_ctx->user_matches(tmp->security_ctx)) #endif /* WITH_WSREP */ { + { #ifdef WITH_WSREP - DEBUG_SYNC(thd, "before_awake_no_mutex"); - if (tmp->wsrep_aborter && tmp->wsrep_aborter != thd->thread_id) - { - /* victim is in hit list already, bail out */ - WSREP_DEBUG("victim %lld has wsrep aborter: %lu, skipping awake()", - id, tmp->wsrep_aborter); - error= 0; - } - else + if (WSREP(tmp)) + { + /* Object tmp is not guaranteed to exist after wsrep_kill_thd() + returns, so do early return from this function. */ + DBUG_RETURN(wsrep_kill_thd(thd, tmp, kill_signal)); + } #endif /* WITH_WSREP */ - { - WSREP_DEBUG("kill_one_thread victim: %lld wsrep_aborter %lu" - " by signal %d", - id, tmp->wsrep_aborter, kill_signal); tmp->awake_no_mutex(kill_signal); error= 0; } @@ -9448,18 +9442,6 @@ static void sql_kill(THD *thd, my_thread_id id, killed_state state, killed_type type) { uint error; -#ifdef WITH_WSREP - if (WSREP(thd)) - { - WSREP_DEBUG("sql_kill called"); - if (thd->wsrep_applier) - { - WSREP_DEBUG("KILL in applying, bailing out here"); - return; - } - WSREP_TO_ISOLATION_BEGIN(WSREP_MYSQL_DB, NULL, NULL) - } -#endif /* WITH_WSREP */ if (likely(!(error= kill_one_thread(thd, id, state, type)))) { if (!thd->killed) @@ -9469,13 +9451,6 @@ void sql_kill(THD *thd, my_thread_id id, killed_state state, killed_type type) } else my_error(error, MYF(0), id); -#ifdef WITH_WSREP - return; - wsrep_error_label: - error= (type == KILL_TYPE_QUERY ? ER_KILL_QUERY_DENIED_ERROR : - ER_KILL_DENIED_ERROR); - my_error(error, MYF(0), (long long) id); -#endif /* WITH_WSREP */ } @@ -9484,18 +9459,6 @@ sql_kill_user(THD *thd, LEX_USER *user, killed_state state) { uint error; ha_rows rows; -#ifdef WITH_WSREP - if (WSREP(thd)) - { - WSREP_DEBUG("sql_kill_user called"); - if (thd->wsrep_applier) - { - WSREP_DEBUG("KILL in applying, bailing out here"); - return; - } - WSREP_TO_ISOLATION_BEGIN(WSREP_MYSQL_DB, NULL, NULL) - } -#endif /* WITH_WSREP */ switch (error= kill_threads_for_user(thd, user, state, &rows)) { case 0: @@ -9511,11 +9474,6 @@ sql_kill_user(THD *thd, LEX_USER *user, killed_state state) default: my_error(error, MYF(0)); } -#ifdef WITH_WSREP - return; - wsrep_error_label: - my_error(ER_CANNOT_USER, MYF(0), user ? user->user.str : "NULL"); -#endif /* WITH_WSREP */ } diff --git a/sql/sql_plugin_services.inl b/sql/sql_plugin_services.inl index c6f07158003..8863f581afb 100644 --- a/sql/sql_plugin_services.inl +++ b/sql/sql_plugin_services.inl @@ -151,6 +151,7 @@ static struct wsrep_service_st wsrep_handler = { wsrep_on, wsrep_prepare_key_for_innodb, wsrep_thd_LOCK, + wsrep_thd_TRYLOCK, wsrep_thd_UNLOCK, wsrep_thd_query, wsrep_thd_retry_counter, @@ -177,7 +178,6 @@ static struct wsrep_service_st wsrep_handler = { wsrep_OSU_method_get, wsrep_thd_has_ignored_error, wsrep_thd_set_ignored_error, - wsrep_thd_set_wsrep_aborter, wsrep_report_bf_lock_wait, wsrep_thd_kill_LOCK, wsrep_thd_kill_UNLOCK, diff --git a/sql/wsrep_dummy.cc b/sql/wsrep_dummy.cc index ac14fc4597a..a67da77c472 100644 --- a/sql/wsrep_dummy.cc +++ b/sql/wsrep_dummy.cc @@ -56,6 +56,11 @@ my_bool wsrep_on(const THD *) void wsrep_thd_LOCK(const THD *) { } +int wsrep_thd_TRYLOCK(const THD *) +{ + return 0; +} + void wsrep_thd_UNLOCK(const THD *) { } @@ -148,8 +153,6 @@ void wsrep_thd_set_ignored_error(THD*, my_bool) { } ulong wsrep_OSU_method_get(const THD*) { return 0;} -bool wsrep_thd_set_wsrep_aborter(THD*, THD*) -{ return 0;} void wsrep_report_bf_lock_wait(const THD*, unsigned long long) diff --git a/sql/wsrep_high_priority_service.cc b/sql/wsrep_high_priority_service.cc index 53ef20f3e78..0a2fa273723 100644 --- a/sql/wsrep_high_priority_service.cc +++ b/sql/wsrep_high_priority_service.cc @@ -510,6 +510,7 @@ int Wsrep_high_priority_service::log_dummy_write_set(const wsrep::ws_handle& ws_ m_thd->wait_for_prior_commit(); } + WSREP_DEBUG("checkpointing dummy write set %lld", ws_meta.seqno().get()); wsrep_set_SE_checkpoint(ws_meta.gtid(), wsrep_gtid_server.gtid()); if (!WSREP_EMULATE_BINLOG(m_thd)) diff --git a/sql/wsrep_mysqld.cc b/sql/wsrep_mysqld.cc index db4a2a2e7b9..bf28f7fd39a 100644 --- a/sql/wsrep_mysqld.cc +++ b/sql/wsrep_mysqld.cc @@ -2249,11 +2249,6 @@ static int wsrep_TOI_event_buf(THD* thd, uchar** buf, size_t* buf_len) case SQLCOM_DROP_TABLE: err= wsrep_drop_table_query(thd, buf, buf_len); break; - case SQLCOM_KILL: - WSREP_DEBUG("KILL as TOI: %s", thd->query()); - err= wsrep_to_buf_helper(thd, thd->query(), thd->query_length(), - buf, buf_len); - break; case SQLCOM_CREATE_ROLE: if (sp_process_definer(thd)) { @@ -2675,8 +2670,15 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, /* Here we will call wsrep_abort_transaction so we should hold THD::LOCK_thd_data to protect victim from concurrent usage - and THD::LOCK_thd_kill to protect from disconnect or delete. */ - wsrep_thd_LOCK(granted_thd); + and THD::LOCK_thd_kill to protect from disconnect or delete. + + Note that all calls to wsrep_abort_thd() and ha_abort_transaction() + unlock LOCK_thd_kill for granted_thd, so granted_thd must not be + accessed after any of those calls. Moreover all other if branches + must release those locks. + */ + mysql_mutex_lock(&granted_thd->LOCK_thd_kill); + mysql_mutex_lock(&granted_thd->LOCK_thd_data); if (wsrep_thd_is_toi(granted_thd) || wsrep_thd_is_applying(granted_thd)) @@ -2685,22 +2687,22 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, { WSREP_DEBUG("BF thread waiting for SR in aborting state"); ticket->wsrep_report(wsrep_debug); - wsrep_thd_UNLOCK(granted_thd); + mysql_mutex_unlock(&granted_thd->LOCK_thd_data); + mysql_mutex_unlock(&granted_thd->LOCK_thd_kill); } else if (wsrep_thd_is_SR(granted_thd) && !wsrep_thd_is_SR(request_thd)) { WSREP_MDL_LOG(INFO, "MDL conflict, DDL vs SR", schema, schema_len, request_thd, granted_thd); wsrep_abort_thd(request_thd, granted_thd, 1); - mysql_mutex_assert_not_owner(&granted_thd->LOCK_thd_data); - mysql_mutex_assert_not_owner(&granted_thd->LOCK_thd_kill); } else { WSREP_MDL_LOG(INFO, "MDL BF-BF conflict", schema, schema_len, request_thd, granted_thd); ticket->wsrep_report(true); - wsrep_thd_UNLOCK(granted_thd); + mysql_mutex_unlock(&granted_thd->LOCK_thd_data); + mysql_mutex_unlock(&granted_thd->LOCK_thd_kill); unireg_abort(1); } } @@ -2709,7 +2711,8 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, { WSREP_DEBUG("BF thread waiting for FLUSH"); ticket->wsrep_report(wsrep_debug); - wsrep_thd_UNLOCK(granted_thd); + mysql_mutex_unlock(&granted_thd->LOCK_thd_data); + mysql_mutex_unlock(&granted_thd->LOCK_thd_kill); } else if (request_thd->lex->sql_command == SQLCOM_DROP_TABLE) { @@ -2717,8 +2720,6 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, wsrep_thd_transaction_state_str(granted_thd)); ticket->wsrep_report(wsrep_debug); wsrep_abort_thd(request_thd, granted_thd, 1); - mysql_mutex_assert_not_owner(&granted_thd->LOCK_thd_data); - mysql_mutex_assert_not_owner(&granted_thd->LOCK_thd_kill); } else { @@ -2728,8 +2729,6 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, if (granted_thd->wsrep_trx().active()) { wsrep_abort_thd(request_thd, granted_thd, true); - mysql_mutex_assert_not_owner(&granted_thd->LOCK_thd_data); - mysql_mutex_assert_not_owner(&granted_thd->LOCK_thd_kill); } else { @@ -2739,15 +2738,16 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, */ if (wsrep_thd_is_BF(request_thd, FALSE)) { + granted_thd->awake_no_mutex(KILL_QUERY_HARD); ha_abort_transaction(request_thd, granted_thd, TRUE); - mysql_mutex_assert_not_owner(&granted_thd->LOCK_thd_data); - mysql_mutex_assert_not_owner(&granted_thd->LOCK_thd_kill); } else { WSREP_MDL_LOG(INFO, "MDL unknown BF-BF conflict", schema, schema_len, request_thd, granted_thd); ticket->wsrep_report(true); + mysql_mutex_unlock(&granted_thd->LOCK_thd_data); + mysql_mutex_unlock(&granted_thd->LOCK_thd_kill); unireg_abort(1); } } @@ -2763,17 +2763,22 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, static bool abort_replicated(THD *thd) { bool ret_code= false; + wsrep_thd_kill_LOCK(thd); wsrep_thd_LOCK(thd); if (thd->wsrep_trx().state() == wsrep::transaction::s_committing) { WSREP_DEBUG("aborting replicated trx: %llu", (ulonglong)(thd->real_id)); - (void)wsrep_abort_thd(thd, thd, TRUE); + wsrep_abort_thd(thd, thd, TRUE); ret_code= true; } else + { + /* wsrep_abort_thd() above releases LOCK_thd_data and LOCK_thd_kill, so + must do it here too. */ wsrep_thd_UNLOCK(thd); - + wsrep_thd_kill_UNLOCK(thd); + } return ret_code; } diff --git a/sql/wsrep_server_service.cc b/sql/wsrep_server_service.cc index ac7226b9948..a1b96a60672 100644 --- a/sql/wsrep_server_service.cc +++ b/sql/wsrep_server_service.cc @@ -143,9 +143,13 @@ void Wsrep_server_service::release_high_priority_service(wsrep::high_priority_se wsrep_delete_threadvars(); } -void Wsrep_server_service::background_rollback(wsrep::client_state& client_state) +void Wsrep_server_service::background_rollback( + wsrep::unique_lock &lock WSREP_UNUSED, + wsrep::client_state &client_state) { - Wsrep_client_state& cs= static_cast(client_state); + DBUG_ASSERT(lock.owns_lock()); + Wsrep_client_state &cs= static_cast(client_state); + mysql_mutex_assert_owner(&cs.thd()->LOCK_thd_data); wsrep_fire_rollbacker(cs.thd()); } diff --git a/sql/wsrep_server_service.h b/sql/wsrep_server_service.h index 168e98206e3..0fc48402024 100644 --- a/sql/wsrep_server_service.h +++ b/sql/wsrep_server_service.h @@ -46,7 +46,8 @@ public: void release_high_priority_service(wsrep::high_priority_service*); - void background_rollback(wsrep::client_state&); + void background_rollback(wsrep::unique_lock &, + wsrep::client_state &); void bootstrap(); void log_message(enum wsrep::log::level, const char*); diff --git a/sql/wsrep_thd.cc b/sql/wsrep_thd.cc index ccb32fb13af..d05ddcbae16 100644 --- a/sql/wsrep_thd.cc +++ b/sql/wsrep_thd.cc @@ -307,50 +307,9 @@ void wsrep_fire_rollbacker(THD *thd) } } - -int wsrep_abort_thd(THD *bf_thd, - THD *victim_thd, - my_bool signal) +static bool wsrep_bf_abort_low(THD *bf_thd, THD *victim_thd) { - DBUG_ENTER("wsrep_abort_thd"); - mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); - mysql_mutex_assert_owner(&victim_thd->LOCK_thd_kill); - - /* Note that when you use RSU node is desynced from cluster, thus WSREP(thd) - might not be true. - */ - if ((WSREP(bf_thd) || - ((WSREP_ON || bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU) && - wsrep_thd_is_toi(bf_thd))) && - !wsrep_thd_is_aborting(victim_thd)) - { - WSREP_DEBUG("wsrep_abort_thd, by: %llu, victim: %llu", - (long long)bf_thd->real_id, (long long)victim_thd->real_id); - ha_abort_transaction(bf_thd, victim_thd, signal); - } - else - { - WSREP_DEBUG("wsrep_abort_thd not effective: bf %llu victim %llu " - "wsrep %d wsrep_on %d RSU %d TOI %d aborting %d", - (long long)bf_thd->real_id, (long long)victim_thd->real_id, - WSREP_NNULL(bf_thd), WSREP_ON, - bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU, - wsrep_thd_is_toi(bf_thd), - wsrep_thd_is_aborting(victim_thd)); - wsrep_thd_UNLOCK(victim_thd); - } - - DBUG_RETURN(1); -} - -bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd) -{ - WSREP_LOG_THD(bf_thd, "BF aborter before"); - WSREP_LOG_THD(victim_thd, "victim before"); - - mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); - mysql_mutex_assert_owner(&victim_thd->LOCK_thd_kill); #ifdef ENABLED_DEBUG_SYNC DBUG_EXECUTE_IF("sync.wsrep_bf_abort", @@ -364,6 +323,88 @@ bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd) };); #endif + wsrep::seqno bf_seqno(bf_thd->wsrep_trx().ws_meta().seqno()); + bool ret; + + { + /* Adopt the lock, it is being held by the caller. */ + Wsrep_mutex wsm{&victim_thd->LOCK_thd_data}; + wsrep::unique_lock lock{wsm, std::adopt_lock}; + + if (wsrep_thd_is_toi(bf_thd)) + { + ret= victim_thd->wsrep_cs().total_order_bf_abort(lock, bf_seqno); + } + else + { + DBUG_ASSERT(WSREP(victim_thd) ? victim_thd->wsrep_trx().active() : 1); + ret= victim_thd->wsrep_cs().bf_abort(lock, bf_seqno); + } + if (ret) + { + /* BF abort should be allowed only once by wsrep-lib.*/ + DBUG_ASSERT(victim_thd->wsrep_aborter == 0); + victim_thd->wsrep_aborter= bf_thd->thread_id; + wsrep_bf_aborts_counter++; + } + lock.release(); /* No unlock at the end of the scope. */ + } + + /* Sanity check for wsrep-lib calls to return with LOCK_thd_data held. */ + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); + + return ret; +} + + +void wsrep_abort_thd(THD *bf_thd, + THD *victim_thd, + my_bool signal) +{ + DBUG_ENTER("wsrep_abort_thd"); + + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_kill); + + /* Note that when you use RSU node is desynced from cluster, thus WSREP(thd) + might not be true. + */ + if ((WSREP(bf_thd) + || ((WSREP_ON || bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU) + && wsrep_thd_is_toi(bf_thd)) + || bf_thd->lex->sql_command == SQLCOM_KILL) + && !wsrep_thd_is_aborting(victim_thd) && + wsrep_bf_abort_low(bf_thd, victim_thd) && + !victim_thd->wsrep_cs().is_rollbacker_active()) + { + WSREP_DEBUG("wsrep_abort_thd, by: %llu, victim: %llu", + (long long)bf_thd->real_id, (long long)victim_thd->real_id); + victim_thd->awake_no_mutex(KILL_QUERY_HARD); + ha_abort_transaction(bf_thd, victim_thd, signal); + } + else + { + WSREP_DEBUG("wsrep_abort_thd not effective: bf %llu victim %llu " + "wsrep %d wsrep_on %d RSU %d TOI %d aborting %d", + (long long)bf_thd->real_id, (long long)victim_thd->real_id, + WSREP_NNULL(bf_thd), WSREP_ON, + bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU, + wsrep_thd_is_toi(bf_thd), + wsrep_thd_is_aborting(victim_thd)); + mysql_mutex_unlock(&victim_thd->LOCK_thd_data); + mysql_mutex_unlock(&victim_thd->LOCK_thd_kill); + } + + DBUG_VOID_RETURN; +} + +bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd) +{ + WSREP_LOG_THD(bf_thd, "BF aborter before"); + WSREP_LOG_THD(victim_thd, "victim before"); + + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); + if (WSREP(victim_thd) && !victim_thd->wsrep_trx().active()) { WSREP_DEBUG("wsrep_bf_abort, BF abort for non active transaction"); @@ -385,32 +426,81 @@ bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd) mysql_mutex_lock(&victim_thd->LOCK_thd_data); } - bool ret; - wsrep::seqno bf_seqno(bf_thd->wsrep_trx().ws_meta().seqno()); + return wsrep_bf_abort_low(bf_thd, victim_thd); +} - if (wsrep_thd_is_toi(bf_thd)) +uint wsrep_kill_thd(THD *thd, THD *victim_thd, killed_state kill_signal) +{ + DBUG_ENTER("wsrep_kill_thd"); + DBUG_ASSERT(WSREP(victim_thd)); + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_kill); + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); + using trans= wsrep::transaction; + auto trx_state= victim_thd->wsrep_trx().state(); +#ifndef DBUG_OFF + victim_thd->wsrep_killed_state= trx_state; +#endif /* DBUG_OFF */ + /* + Already killed or in commit codepath. Mark the victim as killed, + the killed status will be restored in wsrep_after_commit() and + will be processed after the commit is over. In case of multiple + KILLs happened on commit codepath, the last one will be effective. + */ + if (victim_thd->wsrep_abort_by_kill || + trx_state == trans::s_preparing || + trx_state == trans::s_committing || + trx_state == trans::s_ordered_commit) { - /* Here we enter wsrep-lib were LOCK_thd_data will be acquired, - thus we need to release it. However, we can still hold - LOCK_thd_kill to protect from disconnect or delete. */ + victim_thd->wsrep_abort_by_kill= kill_signal; mysql_mutex_unlock(&victim_thd->LOCK_thd_data); - ret= victim_thd->wsrep_cs().total_order_bf_abort(bf_seqno); - mysql_mutex_lock(&victim_thd->LOCK_thd_data); + mysql_mutex_unlock(&victim_thd->LOCK_thd_kill); + DBUG_RETURN(0); } - else + /* + Mark killed victim_thd with kill_signal so that awake_no_mutex does + not dive into storage engine. We use ha_abort_transaction() + to do the storage engine part for wsrep THDs. + */ + DEBUG_SYNC(thd, "wsrep_kill_before_awake_no_mutex"); + victim_thd->wsrep_abort_by_kill= kill_signal; + victim_thd->awake_no_mutex(kill_signal); + /* ha_abort_transaction() releases tmp->LOCK_thd_kill, so tmp + is not safe to access anymore. */ + ha_abort_transaction(thd, victim_thd, 1); + DBUG_RETURN(0); +} + +void wsrep_backup_kill_for_commit(THD *thd) +{ + DBUG_ASSERT(WSREP(thd)); + mysql_mutex_assert_owner(&thd->LOCK_thd_kill); + DBUG_ASSERT(thd->killed != NOT_KILLED); + mysql_mutex_lock(&thd->LOCK_thd_data); + /* If the transaction will roll back, keep the killed state. + For must replay, the replay will happen in different THD context + which is high priority and cannot be killed. The owning thread will + pick the killed state in after statement processing. */ + if (thd->wsrep_trx().state() != wsrep::transaction::s_cert_failed && + thd->wsrep_trx().state() != wsrep::transaction::s_must_abort && + thd->wsrep_trx().state() != wsrep::transaction::s_aborting && + thd->wsrep_trx().state() != wsrep::transaction::s_must_replay) { - /* Test: mysql-wsrep-features#165. Here we enter wsrep-lib - were LOCK_thd_data will be acquired and later LOCK_thd_kill - thus we need to release them. */ - wsrep_thd_UNLOCK(victim_thd); - ret= victim_thd->wsrep_cs().bf_abort(bf_seqno); - wsrep_thd_LOCK(victim_thd); + thd->wsrep_abort_by_kill= thd->killed; + thd->wsrep_abort_by_kill_err= thd->killed_err; + thd->killed= NOT_KILLED; + thd->killed_err= 0; } - if (ret) - { - wsrep_bf_aborts_counter++; - } - return ret; + mysql_mutex_unlock(&thd->LOCK_thd_data); +} + +void wsrep_restore_kill_after_commit(THD *thd) +{ + DBUG_ASSERT(WSREP(thd)); + mysql_mutex_assert_owner(&thd->LOCK_thd_kill); + thd->killed= thd->wsrep_abort_by_kill; + thd->killed_err= thd->wsrep_abort_by_kill_err; + thd->wsrep_abort_by_kill= NOT_KILLED; + thd->wsrep_abort_by_kill_err= 0; } int wsrep_create_threadvars() diff --git a/sql/wsrep_thd.h b/sql/wsrep_thd.h index 0ce612d6097..6f5a70a30a4 100644 --- a/sql/wsrep_thd.h +++ b/sql/wsrep_thd.h @@ -88,10 +88,44 @@ bool wsrep_create_appliers(long threads, bool mutex_protected=false); void wsrep_create_rollbacker(); bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd); -int wsrep_abort_thd(THD *bf_thd, +/* + Abort transaction for victim_thd. This function is called from + MDL BF abort codepath. + + @note This thread unlocks victim_thd->LOCK_thd_kill, so accessing + victim_thd after the function returns is not safe anymore. +*/ +void wsrep_abort_thd(THD *bf_thd, THD *victim_thd, my_bool signal) __attribute__((nonnull(1,2))); +/** + Kill wsrep connection with kill_signal. Object thd is not + guaranteed to exist anymore when this function returns. + + Asserts that the caller holds victim_thd->LOCK_thd_kill, + victim_thd->LOCK_thd_data. + + Releases victim_thd->LOCK_thd_kill, victim_thd->LOCK_thd_data. + + @param thd THD object for connection that executes the KILL. + @param victim_thd THD object for connection to be killed. + @param kill_signal Kill signal. + + @return Zero if the kill was successful, otherwise non-zero error code. + */ +uint wsrep_kill_thd(THD *thd, THD *victim_thd, killed_state kill_signal); + +/* + Backup kill status for commit. + */ +void wsrep_backup_kill_for_commit(THD *); + +/* + Restore KILL status after commit. + */ +void wsrep_restore_kill_after_commit(THD *); + /* Helper methods to deal with thread local storage. The purpose of these methods is to hide the details of thread diff --git a/sql/wsrep_trans_observer.h b/sql/wsrep_trans_observer.h index 8f998244ee6..40fbf80cfb3 100644 --- a/sql/wsrep_trans_observer.h +++ b/sql/wsrep_trans_observer.h @@ -256,6 +256,11 @@ static inline int wsrep_before_prepare(THD* thd, bool all) thd->wsrep_trx().ws_meta().gtid(), wsrep_gtid_server.gtid()); } + + mysql_mutex_lock(&thd->LOCK_thd_kill); + if (thd->killed) wsrep_backup_kill_for_commit(thd); + mysql_mutex_unlock(&thd->LOCK_thd_kill); + DBUG_RETURN(ret); } @@ -323,6 +328,11 @@ static inline int wsrep_before_commit(THD* thd, bool all) wsrep_gtid_server.gtid()); wsrep_register_for_group_commit(thd); } + + mysql_mutex_lock(&thd->LOCK_thd_kill); + if (thd->killed) wsrep_backup_kill_for_commit(thd); + mysql_mutex_unlock(&thd->LOCK_thd_kill); + DBUG_RETURN(ret); } @@ -341,7 +351,8 @@ static inline int wsrep_before_commit(THD* thd, bool all) static inline int wsrep_ordered_commit(THD* thd, bool all) { DBUG_ENTER("wsrep_ordered_commit"); - WSREP_DEBUG("wsrep_ordered_commit: %d", wsrep_is_real(thd, all)); + WSREP_DEBUG("wsrep_ordered_commit: %d %lld", wsrep_is_real(thd, all), + (long long) wsrep_thd_trx_seqno(thd)); DBUG_ASSERT(wsrep_run_commit_hook(thd, all)); DBUG_RETURN(thd->wsrep_cs().ordered_commit()); } @@ -449,10 +460,18 @@ int wsrep_after_statement(THD* thd) wsrep::to_c_string(thd->wsrep_cs().state()), wsrep::to_c_string(thd->wsrep_cs().mode()), wsrep::to_c_string(thd->wsrep_cs().transaction().state())); - DBUG_RETURN((thd->wsrep_cs().state() != wsrep::client_state::s_none && + int ret= ((thd->wsrep_cs().state() != wsrep::client_state::s_none && thd->wsrep_cs().mode() == Wsrep_client_state::m_local) && !thd->internal_transaction() ? thd->wsrep_cs().after_statement() : 0); + + if (wsrep_is_active(thd)) + { + mysql_mutex_lock(&thd->LOCK_thd_kill); + wsrep_restore_kill_after_commit(thd); + mysql_mutex_unlock(&thd->LOCK_thd_kill); + } + DBUG_RETURN(ret); } static inline void wsrep_after_apply(THD* thd) diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc index f7dd18e0e36..98385e3b5dd 100644 --- a/storage/innobase/handler/ha_innodb.cc +++ b/storage/innobase/handler/ha_innodb.cc @@ -18709,50 +18709,6 @@ static struct st_mysql_storage_engine innobase_storage_engine= #ifdef WITH_WSREP -static -void -wsrep_kill_victim( - MYSQL_THD const bf_thd, - MYSQL_THD thd, - trx_t* victim_trx, - my_bool signal) -{ - DBUG_ENTER("wsrep_kill_victim"); - - /* Mark transaction as a victim for Galera abort */ - victim_trx->lock.was_chosen_as_wsrep_victim= true; - if (wsrep_thd_set_wsrep_aborter(bf_thd, thd)) - { - WSREP_DEBUG("innodb kill transaction skipped due to wsrep_aborter set"); - wsrep_thd_UNLOCK(thd); - DBUG_VOID_RETURN; - } - - if (wsrep_thd_bf_abort(bf_thd, thd, signal)) - { - lock_t* wait_lock= victim_trx->lock.wait_lock; - if (wait_lock) - { - DBUG_ASSERT(victim_trx->is_wsrep()); - WSREP_DEBUG("victim has wait flag: %lu", thd_get_thread_id(thd)); - victim_trx->lock.was_chosen_as_deadlock_victim= TRUE; - lock_cancel_waiting_and_release(wait_lock); - } - } - else - { - wsrep_thd_LOCK(thd); - victim_trx->lock.was_chosen_as_wsrep_victim= false; - wsrep_thd_set_wsrep_aborter(NULL, thd); - wsrep_thd_UNLOCK(thd); - - WSREP_DEBUG("wsrep_thd_bf_abort has failed, victim %lu will survive", - thd_get_thread_id(thd)); - } - - DBUG_VOID_RETURN; -} - /** This function is used to kill one transaction. This transaction was open on this node (not-yet-committed), and a @@ -18799,10 +18755,45 @@ wsrep_innobase_kill_one_trx( DBUG_VOID_RETURN; } - /* Here we need to lock THD::LOCK_thd_data to protect from - concurrent usage or disconnect or delete. */ + /* Grab reference to victim_trx before releasing the mutex, this will + prevent victim to release locks or commit while the mutex is + unlocked. The state may change to TRX_STATE_COMMITTED_IN_MEMORY. + See skip_lock_inheritance_n_ref in trx0trx.h. */ + const trx_id_t victim_trx_id= victim_trx->id; +retry_lock: + victim_trx->reference(); + trx_mutex_exit(victim_trx); + DEBUG_SYNC(bf_thd, "wsrep_before_BF_victim_lock"); - wsrep_thd_LOCK(thd); + wsrep_thd_kill_LOCK(thd); + /* + There is now a cycle + + trx reference + -> LOCK_commit_order + -> LOCK_thd_data + -> trx reference + + which may prevent the transaction committing because reference was grabbed + above. Try to lock LOCK_thd_data, and if not successul, enter the + trx mutex again to release the reference and try again. + */ + if (wsrep_thd_TRYLOCK(thd)) + { + wsrep_thd_kill_UNLOCK(thd); + trx_mutex_enter(victim_trx); + victim_trx->release_reference(); + if (victim_trx_id != victim_trx->id || + victim_trx->state == TRX_STATE_COMMITTED_IN_MEMORY || + victim_trx->state == TRX_STATE_NOT_STARTED) + { + WSREP_DEBUG("wsrep_innobase_kill_one_trx: Victim committed in memory"); + DBUG_VOID_RETURN; + } + goto retry_lock; + } + + DEBUG_SYNC(bf_thd, "wsrep_after_BF_victim_lock"); WSREP_LOG_CONFLICT(bf_thd, thd, TRUE); @@ -18833,7 +18824,31 @@ wsrep_innobase_kill_one_trx( wsrep_thd_transaction_state_str(thd), wsrep_thd_query(thd)); - wsrep_kill_victim(bf_thd, thd, victim_trx, signal); + const bool success= wsrep_thd_bf_abort(bf_thd, thd, signal); + + wsrep_thd_UNLOCK(thd); + wsrep_thd_kill_UNLOCK(thd); + trx_mutex_enter(victim_trx); + + if (success && victim_trx->state == TRX_STATE_ACTIVE) + { + lock_t* wait_lock= victim_trx->lock.wait_lock; + if (wait_lock) + { + victim_trx->lock.was_chosen_as_deadlock_victim= TRUE; + DBUG_ASSERT(victim_trx->is_wsrep()); + WSREP_DEBUG("victim has wait flag: %lu", thd_get_thread_id(thd)); + lock_cancel_waiting_and_release(wait_lock); + } + } + else + { + victim_trx->lock.was_chosen_as_wsrep_victim= false; + WSREP_DEBUG("wsrep_thd_bf_abort has failed, victim %lu will survive", + thd_get_thread_id(thd)); + } + victim_trx->release_reference(); + DBUG_VOID_RETURN; } @@ -18854,42 +18869,61 @@ wsrep_abort_transaction( THD *victim_thd, my_bool signal) { - /* Note that victim thd is protected with - THD::LOCK_thd_data and THD::LOCK_thd_kill here. */ + /* Unlock LOCK_thd_kill and LOCK_thd_data temporarily to grab mutexes + in the right order: + lock_sys.mutex + LOCK_thd_kill + LOCK_thd_data + trx.mutex + */ trx_t* victim_trx= thd_to_trx(victim_thd); - trx_t* bf_trx= thd_to_trx(bf_thd); - WSREP_DEBUG("wsrep_abort_transaction: BF:" - " thread %ld client_state %s client_mode %s" - " trans_state %s query %s trx " TRX_ID_FMT, - thd_get_thread_id(bf_thd), - wsrep_thd_client_state_str(bf_thd), - wsrep_thd_client_mode_str(bf_thd), - wsrep_thd_transaction_state_str(bf_thd), - wsrep_thd_query(bf_thd), - bf_trx ? bf_trx->id : 0); + trx_id_t victim_trx_id= victim_trx ? victim_trx->id : 0; + wsrep_thd_UNLOCK(victim_thd); + wsrep_thd_kill_UNLOCK(victim_thd); + /* After this point must use find_thread_by_id() if victim_thd + is needed again. */ - WSREP_DEBUG("wsrep_abort_transaction: victim:" - " thread %ld client_state %s client_mode %s" - " trans_state %s query %s trx " TRX_ID_FMT, - thd_get_thread_id(victim_thd), - wsrep_thd_client_state_str(victim_thd), - wsrep_thd_client_mode_str(victim_thd), - wsrep_thd_transaction_state_str(victim_thd), - wsrep_thd_query(victim_thd), - victim_trx ? victim_trx->id : 0); - - if (victim_trx) + /* Victim didn't have active RW transaction. Note that tere is a possible + race when the victim transaction is just starting write operation + as is still read only. This however will be resolved eventually since + all the possible blocking transactions are also BF aborted, + and the victim will find that it was BF aborted on server level after + the write operation in InnoDB completes. */ + if (!victim_trx_id) { - lock_mutex_enter(); - trx_mutex_enter(victim_trx); - wsrep_kill_victim(bf_thd, victim_thd, victim_trx, signal); +#ifdef ENABLED_DEBUG_SYNC + DBUG_EXECUTE_IF( + "sync.wsrep_abort_transaction_read_only", + {const char act[]= + "now " + "SIGNAL sync.wsrep_abort_transaction_read_only_reached " + "WAIT_FOR signal.wsrep_abort_transaction_read_only"; + DBUG_ASSERT(!debug_sync_set_action(bf_thd, STRING_WITH_LEN(act))); + };); +#endif /* ENABLED_DEBUG_SYNC*/ + return; + } + lock_mutex_enter(); + + /* Check if victim trx still exists. */ + /* Note based on comment on trx0sys.h only ACTIVE or PREPARED trx + objects may participate in hash. However, transaction may get committed + before this method returns. */ + if(!(victim_trx= trx_sys.find(nullptr, victim_trx_id, true))) { + WSREP_DEBUG("wsrep_abort_transaction: Victim trx does not exist anymore"); lock_mutex_exit(); - trx_mutex_exit(victim_trx); + return; } - else - { - wsrep_thd_bf_abort(bf_thd, victim_thd, signal); + trx_mutex_enter(victim_trx); + + if (victim_trx->state == TRX_STATE_ACTIVE && victim_trx->lock.wait_lock) { + victim_trx->lock.was_chosen_as_deadlock_victim= TRUE; + lock_cancel_waiting_and_release(victim_trx->lock.wait_lock); } + + trx_mutex_exit(victim_trx); + victim_trx->release_reference(); + lock_mutex_exit(); } static diff --git a/wsrep-lib b/wsrep-lib index 4951c383577..e238c0d240c 160000 --- a/wsrep-lib +++ b/wsrep-lib @@ -1 +1 @@ -Subproject commit 4951c38357737d568b554402bc5b6abe88a38fe1 +Subproject commit e238c0d240c2557229b0523a4a032f3cf8b41639 From f307160218f8f9ed2528ffc685f49c4e2ae050b3 Mon Sep 17 00:00:00 2001 From: Teemu Ollakka Date: Wed, 19 Apr 2023 16:51:55 +0300 Subject: [PATCH 7/7] MDEV-29293 MariaDB stuck on starting commit state MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit contains a merge from 10.5-MDEV-29293-squash into 10.6. Although the bug MDEV-29293 was not reproducible with 10.6, the fix contains several improvements for wsrep KILL query and BF abort handling, and addresses the following issues: * MDEV-30307 KILL command issued inside a transaction is problematic for galera replication: This commit will remove KILL TOI replication, so Galera side transaction context is not lost during KILL. * MDEV-21075 KILL QUERY maintains nodes data consistency but breaks GTID sequence: This is fixed as well as KILL does not use TOI, and thus does not change GTID state. * MDEV-30372 Assertion in wsrep-lib state: This was caused by BF abort or KILL when local transaction was in the middle of group commit. This commit disables THD::killed handling during commit, so the problem is avoided. * MDEV-30963 Assertion failure !lock.was_chosen_as_deadlock_victim in trx0trx.h:1065: The assertion happened when the victim was BF aborted via MDL while it was committing. This commit changes MDL BF aborts so that transactions which are committing cannot be BF aborted via MDL. The RQG grammar attached in the issue could not reproduce the crash anymore. Original commit message from 10.5 fix: MDEV-29293 MariaDB stuck on starting commit state The problem seems to be a deadlock between KILL command execution and BF abort issued by an applier, where: * KILL has locked victim's LOCK_thd_kill and LOCK_thd_data. * Applier has innodb side global lock mutex and victim trx mutex. * KILL is calling innobase_kill_query, and is blocked by innodb global lock mutex. * Applier is in wsrep_innobase_kill_one_trx and is blocked by victim's LOCK_thd_kill. The fix in this commit removes the TOI replication of KILL command and makes KILL execution less intrusive operation. Aborting the victim happens now by using awake_no_mutex() and ha_abort_transaction(). If the KILL happens when the transaction is committing, the KILL operation is postponed to happen after the statement has completed in order to avoid KILL to interrupt commit processing. Notable changes in this commit: * wsrep client connections's error state may remain sticky after client connection is closed. This error message will then pop up for the next client session issuing first SQL statement. This problem raised with test galera.galera_bf_kill. The fix is to reset wsrep client error state, before a THD is reused for next connetion. * Release THD locks in wsrep_abort_transaction when locking innodb mutexes. This guarantees same locking order as with applier BF aborting. * BF abort from MDL was changed to do BF abort on server/wsrep-lib side first, and only then do the BF abort on InnoDB side. This removes the need to call back from InnoDB for BF aborts which originate from MDL and simplifies the locking. * Removed wsrep_thd_set_wsrep_aborter() from service_wsrep.h. The manipulation of the wsrep_aborter can be done solely on server side. Moreover, it is now debug only variable and could be excluded from optimized builds. * Remove LOCK_thd_kill from wsrep_thd_LOCK/UNLOCK to allow more fine grained locking for SR BF abort which may require locking of victim LOCK_thd_kill. Added explicit call for wsrep_thd_kill_LOCK/UNLOCK where appropriate. * Wsrep-lib was updated to version which allows external locking for BF abort calls. Changes to MTR tests: * Disable galera_bf_abort_group_commit. This test is going to be removed (MDEV-30855). * Make galera_var_retry_autocommit result more readable by echoing cases and expectations into result. Only one expected result for reap to verify that server returns expected status for query. * Record galera_gcache_recover_manytrx as result file was incomplete. Trivial change. * Make galera_create_table_as_select more deterministic: Wait until CTAS execution has reached MDL wait for multi-master conflict case. Expected error from multi-master conflict is ER_QUERY_INTERRUPTED. This is because CTAS does not yet have open wsrep transaction when it is waiting for MDL, query gets interrupted instead of BF aborted. This should be addressed in separate task. * A new test galera_bf_abort_registering to check that registering trx gets BF aborted through MDL. * A new test galera_kill_group_commit to verify correct behavior when KILL is executed while the transaction is committing. Co-authored-by: Seppo Jaakola Co-authored-by: Jan Lindström Signed-off-by: Julius Goryavsky --- include/mysql/service_wsrep.h | 9 +- mysql-test/suite/galera/disabled.def | 2 + mysql-test/suite/galera/r/MDEV-29293.result | 21 ++ .../r/galera_create_table_as_select.result | 1 + .../r/galera_gcache_recover_manytrx.result | 3 - .../galera/r/galera_kill_group_commit.result | 27 +++ .../r/galera_var_retry_autocommit.result | 5 +- mysql-test/suite/galera/t/MDEV-29293.test | 41 ++++ .../t/galera_create_table_as_select.test | 6 +- .../galera/t/galera_kill_group_commit.cnf | 5 + .../galera/t/galera_kill_group_commit.test | 69 ++++++ .../galera/t/galera_var_retry_autocommit.test | 1 + sql/handler.h | 7 +- sql/service_wsrep.cc | 39 +--- sql/sql_class.cc | 27 ++- sql/sql_class.h | 9 +- sql/sql_parse.cc | 59 +---- sql/sql_plugin_services.inl | 2 +- sql/wsrep_dummy.cc | 7 +- sql/wsrep_high_priority_service.cc | 1 + sql/wsrep_mysqld.cc | 34 ++- sql/wsrep_server_service.cc | 8 +- sql/wsrep_server_service.h | 3 +- sql/wsrep_thd.cc | 204 +++++++++++++----- sql/wsrep_thd.h | 31 ++- sql/wsrep_trans_observer.h | 23 +- storage/innobase/handler/ha_innodb.cc | 138 ++++++------ storage/innobase/include/lock0lock.h | 4 + storage/innobase/lock/lock0lock.cc | 21 +- wsrep-lib | 2 +- 30 files changed, 567 insertions(+), 242 deletions(-) create mode 100644 mysql-test/suite/galera/r/MDEV-29293.result create mode 100644 mysql-test/suite/galera/r/galera_kill_group_commit.result create mode 100644 mysql-test/suite/galera/t/MDEV-29293.test create mode 100644 mysql-test/suite/galera/t/galera_kill_group_commit.cnf create mode 100644 mysql-test/suite/galera/t/galera_kill_group_commit.test diff --git a/include/mysql/service_wsrep.h b/include/mysql/service_wsrep.h index 8541b348ae4..8add709362e 100644 --- a/include/mysql/service_wsrep.h +++ b/include/mysql/service_wsrep.h @@ -57,6 +57,7 @@ extern struct wsrep_service_st { my_bool (*wsrep_on_func)(const MYSQL_THD thd); bool (*wsrep_prepare_key_for_innodb_func)(MYSQL_THD thd, const unsigned char*, size_t, const unsigned char*, size_t, struct wsrep_buf*, size_t*); void (*wsrep_thd_LOCK_func)(const MYSQL_THD thd); + int (*wsrep_thd_TRYLOCK_func)(const MYSQL_THD thd); void (*wsrep_thd_UNLOCK_func)(const MYSQL_THD thd); const char * (*wsrep_thd_query_func)(const MYSQL_THD thd); int (*wsrep_thd_retry_counter_func)(const MYSQL_THD thd); @@ -89,7 +90,6 @@ extern struct wsrep_service_st { ulong (*wsrep_OSU_method_get_func)(const MYSQL_THD thd); my_bool (*wsrep_thd_has_ignored_error_func)(const MYSQL_THD thd); void (*wsrep_thd_set_ignored_error_func)(MYSQL_THD thd, my_bool val); - bool (*wsrep_thd_set_wsrep_aborter_func)(MYSQL_THD bf_thd, MYSQL_THD thd); void (*wsrep_report_bf_lock_wait_func)(const MYSQL_THD thd, unsigned long long trx_id); void (*wsrep_thd_kill_LOCK_func)(const MYSQL_THD thd); @@ -111,6 +111,7 @@ extern struct wsrep_service_st { #define wsrep_on(thd) (thd) && WSREP_ON && wsrep_service->wsrep_on_func(thd) #define wsrep_prepare_key_for_innodb(A,B,C,D,E,F,G) wsrep_service->wsrep_prepare_key_for_innodb_func(A,B,C,D,E,F,G) #define wsrep_thd_LOCK(T) wsrep_service->wsrep_thd_LOCK_func(T) +#define wsrep_thd_TRYLOCK(T) wsrep_service->wsrep_thd_TRYLOCK_func(T) #define wsrep_thd_UNLOCK(T) wsrep_service->wsrep_thd_UNLOCK_func(T) #define wsrep_thd_kill_LOCK(T) wsrep_service->wsrep_thd_kill_LOCK_func(T) #define wsrep_thd_kill_UNLOCK(T) wsrep_service->wsrep_thd_kill_UNLOCK_func(T) @@ -141,7 +142,6 @@ extern struct wsrep_service_st { #define wsrep_OSU_method_get(T) wsrep_service->wsrep_OSU_method_get_func(T) #define wsrep_thd_has_ignored_error(T) wsrep_service->wsrep_thd_has_ignored_error_func(T) #define wsrep_thd_set_ignored_error(T,V) wsrep_service->wsrep_thd_set_ignored_error_func(T,V) -#define wsrep_thd_set_wsrep_aborter(T) wsrep_service->wsrep_thd_set_wsrep_aborter_func(T1, T2) #define wsrep_report_bf_lock_wait(T,I) wsrep_service->wsrep_report_bf_lock_wait(T,I) #define wsrep_thd_set_PA_unsafe(T) wsrep_service->wsrep_thd_set_PA_unsafe_func(T) #else @@ -175,6 +175,8 @@ void wsrep_set_data_home_dir(const char *data_dir); extern "C" my_bool wsrep_on(const MYSQL_THD thd); /* Lock thd wsrep lock */ extern "C" void wsrep_thd_LOCK(const MYSQL_THD thd); +/* Try thd wsrep lock. Return non-zero if lock could not be taken. */ +extern "C" int wsrep_thd_TRYLOCK(const MYSQL_THD thd); /* Unlock thd wsrep lock */ extern "C" void wsrep_thd_UNLOCK(const MYSQL_THD thd); @@ -197,8 +199,6 @@ extern "C" my_bool wsrep_thd_is_local(const MYSQL_THD thd); /* Return true if thd is in high priority mode */ /* todo: rename to is_high_priority() */ extern "C" my_bool wsrep_thd_is_applying(const MYSQL_THD thd); -/* set wsrep_aborter for the target THD */ -extern "C" bool wsrep_thd_set_wsrep_aborter(MYSQL_THD bf_thd, MYSQL_THD victim_thd); /* Return true if thd is in TOI mode */ extern "C" my_bool wsrep_thd_is_toi(const MYSQL_THD thd); /* Return true if thd is in replicating TOI mode */ @@ -249,7 +249,6 @@ extern "C" my_bool wsrep_thd_is_applying(const MYSQL_THD thd); extern "C" ulong wsrep_OSU_method_get(const MYSQL_THD thd); extern "C" my_bool wsrep_thd_has_ignored_error(const MYSQL_THD thd); extern "C" void wsrep_thd_set_ignored_error(MYSQL_THD thd, my_bool val); -extern "C" bool wsrep_thd_set_wsrep_aborter(MYSQL_THD bf_thd, MYSQL_THD victim_thd); extern "C" void wsrep_report_bf_lock_wait(const THD *thd, unsigned long long trx_id); /* declare parallel applying unsafety for the THD */ diff --git a/mysql-test/suite/galera/disabled.def b/mysql-test/suite/galera/disabled.def index 2d68598c03b..af1f6cd8861 100644 --- a/mysql-test/suite/galera/disabled.def +++ b/mysql-test/suite/galera/disabled.def @@ -26,3 +26,5 @@ galera_var_ignore_apply_errors : 28: "Server did not transition to READY state" galera_bf_kill_debug : timeout after 900 seconds galera_ssl_upgrade : [Warning] Failed to load slave replication state from table mysql.gtid_slave_pos: 130: Incorrect file format 'gtid_slave_pos' galera_insert_bulk : MDEV-30536 no expected deadlock in galera_insert_bulk test +MDEV-27713 : test is using get_lock(), which is now rejected in cluster +galera_bf_abort_group_commit : MDEV-30855 PR to remove the test exists diff --git a/mysql-test/suite/galera/r/MDEV-29293.result b/mysql-test/suite/galera/r/MDEV-29293.result new file mode 100644 index 00000000000..70c0cc84a31 --- /dev/null +++ b/mysql-test/suite/galera/r/MDEV-29293.result @@ -0,0 +1,21 @@ +connection node_2; +connection node_1; +connect node_1a, 127.0.0.1, root, , test, $NODE_MYPORT_1; +connect node_1b, 127.0.0.1, root, , test, $NODE_MYPORT_1; +set wsrep_sync_wait = 0; +CREATE TABLE t1(a int not null primary key auto_increment, b int) engine=InnoDB; +INSERT INTO t1 VALUES (1,2); +connection node_1a; +BEGIN; +UPDATE t1 SET b=3 WHERE a=1; +connection node_1; +set debug_sync='wsrep_kill_before_awake_no_mutex SIGNAL before_kill WAIT_FOR continue'; +connection node_1b; +set debug_sync= 'now WAIT_FOR before_kill'; +connection node_2; +UPDATE t1 SET b=7 WHERE a=1; +connection node_1b; +set debug_sync= 'now SIGNAL continue'; +connection node_1; +DROP TABLE t1; +SET DEBUG_SYNC= 'RESET'; diff --git a/mysql-test/suite/galera/r/galera_create_table_as_select.result b/mysql-test/suite/galera/r/galera_create_table_as_select.result index 6f65ee99f0a..beda5f30fe2 100644 --- a/mysql-test/suite/galera/r/galera_create_table_as_select.result +++ b/mysql-test/suite/galera/r/galera_create_table_as_select.result @@ -82,6 +82,7 @@ connect node_1a, 127.0.0.1, root, , test, $NODE_MYPORT_1; LOCK TABLE t2 WRITE; connection node_1; CREATE TABLE t1 AS SELECT * FROM t2;; +connection node_1a; connection node_2; SELECT COUNT(*) = 5 FROM t2; COUNT(*) = 5 diff --git a/mysql-test/suite/galera/r/galera_gcache_recover_manytrx.result b/mysql-test/suite/galera/r/galera_gcache_recover_manytrx.result index 5caf22b39ca..5718807b5c4 100644 --- a/mysql-test/suite/galera/r/galera_gcache_recover_manytrx.result +++ b/mysql-test/suite/galera/r/galera_gcache_recover_manytrx.result @@ -134,6 +134,3 @@ connection node_1; call mtr.add_suppression("Error in Log_event::read_log_event():.*"); CALL mtr.add_suppression("conflict state 7 after post commit"); CALL mtr.add_suppression("Skipped GCache ring buffer recovery"); -connection node_2; -call mtr.add_suppression("Error in Log_event::read_log_event():.*"); -CALL mtr.add_suppression("Skipped GCache ring buffer recovery"); diff --git a/mysql-test/suite/galera/r/galera_kill_group_commit.result b/mysql-test/suite/galera/r/galera_kill_group_commit.result new file mode 100644 index 00000000000..bb59ce1486f --- /dev/null +++ b/mysql-test/suite/galera/r/galera_kill_group_commit.result @@ -0,0 +1,27 @@ +connection node_2; +connection node_1; +connect node_1_kill, 127.0.0.1, root, , test, $NODE_MYPORT_1; +connect node_1_ctrl, 127.0.0.1, root, , test, $NODE_MYPORT_1; +SET SESSION wsrep_sync_wait = 0; +connect node_1_follower, 127.0.0.1, root, , test, $NODE_MYPORT_1; +SET SESSION wsrep_sync_wait = 0; +connection node_1; +CREATE TABLE t1 (f1 INT PRIMARY KEY) ENGINE=InnoDB; +SET SESSION DEBUG_SYNC = "commit_before_enqueue SIGNAL leader_before_enqueue_reached WAIT_FOR leader_before_enqueue_continue"; +INSERT INTO t1 VALUES (1); +connection node_1_ctrl; +SET DEBUG_SYNC = "now WAIT_FOR leader_before_enqueue_reached"; +connection node_1_follower; +INSERT INTO t1 VALUES (2);; +connection node_1_ctrl; +connection node_1_kill; +# Execute KILL QUERY for group commit follower +SET DEBUG_SYNC = "now SIGNAL leader_before_enqueue_continue"; +connection node_1_follower; +connection node_1; +SELECT * FROM t1; +f1 +1 +2 +SET DEBUG_SYNC = "RESET"; +DROP TABLE t1; diff --git a/mysql-test/suite/galera/r/galera_var_retry_autocommit.result b/mysql-test/suite/galera/r/galera_var_retry_autocommit.result index 50667b0a4fa..eee740b6036 100644 --- a/mysql-test/suite/galera/r/galera_var_retry_autocommit.result +++ b/mysql-test/suite/galera/r/galera_var_retry_autocommit.result @@ -36,7 +36,10 @@ SET DEBUG_SYNC = 'now SIGNAL wsrep_retry_autocommit_continue'; connection node_1; SELECT COUNT(*) FROM t1; COUNT(*) -1 +connection node_1; +SELECT COUNT(*) FROM t1; +COUNT(*) +0 SET DEBUG_SYNC = 'RESET'; SET GLOBAL debug_dbug = NULL; DROP TABLE t1; diff --git a/mysql-test/suite/galera/t/MDEV-29293.test b/mysql-test/suite/galera/t/MDEV-29293.test new file mode 100644 index 00000000000..dacbf714c06 --- /dev/null +++ b/mysql-test/suite/galera/t/MDEV-29293.test @@ -0,0 +1,41 @@ +--source include/galera_cluster.inc +--source include/have_innodb.inc +--source include/have_debug_sync.inc +--source include/galera_have_debug_sync.inc + +--connect node_1a, 127.0.0.1, root, , test, $NODE_MYPORT_1 +--connect node_1b, 127.0.0.1, root, , test, $NODE_MYPORT_1 +set wsrep_sync_wait = 0; + +CREATE TABLE t1(a int not null primary key auto_increment, b int) engine=InnoDB; +INSERT INTO t1 VALUES (1,2); + +--connection node_1a +--let $victim_id = `SELECT CONNECTION_ID()` +BEGIN; +UPDATE t1 SET b=3 WHERE a=1; + +--connection node_1 +set debug_sync='wsrep_kill_before_awake_no_mutex SIGNAL before_kill WAIT_FOR continue'; +--disable_query_log +--disable_result_log +--send_eval KILL CONNECTION $victim_id +--enable_result_log +--enable_query_log + +--connection node_1b +set debug_sync= 'now WAIT_FOR before_kill'; + +--connection node_2 +UPDATE t1 SET b=7 WHERE a=1; + +--connection node_1b +--let $wait_condition = SELECT COUNT(*) = 1 FROM INFORMATION_SCHEMA.PROCESSLIST WHERE User = 'system user' AND State LIKE 'Update_rows_log_event%'; +--source include/wait_condition.inc +set debug_sync= 'now SIGNAL continue'; + +--connection node_1 +--reap +DROP TABLE t1; +SET DEBUG_SYNC= 'RESET'; + diff --git a/mysql-test/suite/galera/t/galera_create_table_as_select.test b/mysql-test/suite/galera/t/galera_create_table_as_select.test index a6c1f657280..cfee63e5e27 100644 --- a/mysql-test/suite/galera/t/galera_create_table_as_select.test +++ b/mysql-test/suite/galera/t/galera_create_table_as_select.test @@ -113,6 +113,10 @@ LOCK TABLE t2 WRITE; --connection node_1 --send CREATE TABLE t1 AS SELECT * FROM t2; +--connection node_1a +--let $wait_condition = SELECT COUNT(*) = 1 FROM information_schema.processlist WHERE STATE LIKE 'Waiting for table metadata lock%' +--source include/wait_condition.inc + --connection node_2 SELECT COUNT(*) = 5 FROM t2; CREATE TABLE t1 AS SELECT * FROM t2; @@ -121,7 +125,7 @@ CREATE TABLE t1 AS SELECT * FROM t2; UNLOCK TABLES; --connection node_1 ---error ER_TABLE_EXISTS_ERROR,ER_LOCK_DEADLOCK +--error ER_TABLE_EXISTS_ERROR,ER_QUERY_INTERRUPTED --reap DROP TABLE t1, t2; diff --git a/mysql-test/suite/galera/t/galera_kill_group_commit.cnf b/mysql-test/suite/galera/t/galera_kill_group_commit.cnf new file mode 100644 index 00000000000..60f4f776409 --- /dev/null +++ b/mysql-test/suite/galera/t/galera_kill_group_commit.cnf @@ -0,0 +1,5 @@ +!include ../galera_2nodes.cnf + +[mysqld] +log-bin +log-slave-updates diff --git a/mysql-test/suite/galera/t/galera_kill_group_commit.test b/mysql-test/suite/galera/t/galera_kill_group_commit.test new file mode 100644 index 00000000000..4b84f2d90ef --- /dev/null +++ b/mysql-test/suite/galera/t/galera_kill_group_commit.test @@ -0,0 +1,69 @@ +# +# Verify that transaction which has reached group commit queue +# cannot be killed. If the kill succeeds, assertion for +# wsrep transaction state will fail. +# +# If the bug is present, i.e. wsrep transaction gets killed during +# group commit wait, this test is enough to reproduce the crash +# most of the time. +# + +--source include/have_innodb.inc +--source include/have_debug_sync.inc +--source include/galera_cluster.inc + +# Connection for KILL commands +--connect node_1_kill, 127.0.0.1, root, , test, $NODE_MYPORT_1 +# Connection for sync point control +--connect node_1_ctrl, 127.0.0.1, root, , test, $NODE_MYPORT_1 +SET SESSION wsrep_sync_wait = 0; +# Connection for group commit follower +--connect node_1_follower, 127.0.0.1, root, , test, $NODE_MYPORT_1 +# Need to disable sync wait to reach commit queue when leader +# is blocked. +SET SESSION wsrep_sync_wait = 0; +--let $follower_id = `SELECT CONNECTION_ID()` + +--connection node_1 +CREATE TABLE t1 (f1 INT PRIMARY KEY) ENGINE=InnoDB; + +SET SESSION DEBUG_SYNC = "commit_before_enqueue SIGNAL leader_before_enqueue_reached WAIT_FOR leader_before_enqueue_continue"; +--send INSERT INTO t1 VALUES (1) + +--connection node_1_ctrl +SET DEBUG_SYNC = "now WAIT_FOR leader_before_enqueue_reached"; + +--connection node_1_follower +# SET SESSION DEBUG_SYNC = "group_commit_waiting_for_prior SIGNAL follower_waiting_for_prior_reached WAIT_FOR follower_waiting_for_prior_continue"; +--send INSERT INTO t1 VALUES (2); + +--connection node_1_ctrl +# TODO: Is it possible to use sync points to enforce group commit to happen? +# The leader will hold commit monitor in commit_before_enqueue sync point, +# which prevents the follower to reach the group commit wait state. +# We now sleep and expect the follower to reach group commit, but this +# may cause false negatives. +--sleep 1 + +--connection node_1_kill +--echo # Execute KILL QUERY for group commit follower +--disable_query_log +--disable_result_log +# Because it is currently impossible to verify that the +# follower has reached group commit queue, the KILL may +# sometimes return success. +--error 0,ER_KILL_DENIED_ERROR +--eval KILL QUERY $follower_id +--enable_result_log +--enable_query_log + +SET DEBUG_SYNC = "now SIGNAL leader_before_enqueue_continue"; +--connection node_1_follower +--reap + +--connection node_1 +--reap +SELECT * FROM t1; + +SET DEBUG_SYNC = "RESET"; +DROP TABLE t1; diff --git a/mysql-test/suite/galera/t/galera_var_retry_autocommit.test b/mysql-test/suite/galera/t/galera_var_retry_autocommit.test index c58eba1410e..8009fe88c65 100644 --- a/mysql-test/suite/galera/t/galera_var_retry_autocommit.test +++ b/mysql-test/suite/galera/t/galera_var_retry_autocommit.test @@ -64,6 +64,7 @@ SELECT COUNT(*) FROM t1; SET DEBUG_SYNC = 'now SIGNAL wsrep_retry_autocommit_continue'; --connection node_1 +--error 0,ER_LOCK_DEADLOCK --reap SELECT COUNT(*) FROM t1; diff --git a/sql/handler.h b/sql/handler.h index e3d968808ee..ca118813656 100644 --- a/sql/handler.h +++ b/sql/handler.h @@ -45,6 +45,7 @@ #include "sql_sequence.h" #include "mem_root_array.h" #include // pair +#include /* __attribute__ */ class Alter_info; class Virtual_column_info; @@ -1530,9 +1531,9 @@ struct handlerton const char *query, uint query_length, const char *db, const char *table_name); - void (*abort_transaction)(handlerton *hton, THD *bf_thd, - THD *victim_thd, my_bool signal); - int (*set_checkpoint)(handlerton *hton, const XID* xid); + void (*abort_transaction)(handlerton *hton, THD *bf_thd, THD *victim_thd, + my_bool signal) __attribute__((nonnull)); + int (*set_checkpoint)(handlerton *hton, const XID *xid); int (*get_checkpoint)(handlerton *hton, XID* xid); /** Check if the version of the table matches the version in the .frm diff --git a/sql/service_wsrep.cc b/sql/service_wsrep.cc index dd12149ff48..e1a4a25b27a 100644 --- a/sql/service_wsrep.cc +++ b/sql/service_wsrep.cc @@ -32,6 +32,11 @@ extern "C" void wsrep_thd_LOCK(const THD *thd) mysql_mutex_lock(&thd->LOCK_thd_data); } +extern "C" int wsrep_thd_TRYLOCK(const THD *thd) +{ + return mysql_mutex_trylock(&thd->LOCK_thd_data); +} + extern "C" void wsrep_thd_UNLOCK(const THD *thd) { mysql_mutex_unlock(&thd->LOCK_thd_data); @@ -196,6 +201,7 @@ extern "C" void wsrep_handle_SR_rollback(THD *bf_thd, /* Note: do not store/reset globals before wsrep_bf_abort() call to avoid losing BF thd context. */ + mysql_mutex_lock(&victim_thd->LOCK_thd_data); if (!(bf_thd && bf_thd != victim_thd)) { DEBUG_SYNC(victim_thd, "wsrep_before_SR_rollback"); @@ -208,6 +214,7 @@ extern "C" void wsrep_handle_SR_rollback(THD *bf_thd, { wsrep_thd_self_abort(victim_thd); } + mysql_mutex_unlock(&victim_thd->LOCK_thd_data); if (bf_thd) { wsrep_store_threadvars(bf_thd); @@ -218,7 +225,7 @@ extern "C" my_bool wsrep_thd_bf_abort(THD *bf_thd, THD *victim_thd, my_bool signal) { mysql_mutex_assert_owner(&victim_thd->LOCK_thd_kill); - mysql_mutex_assert_not_owner(&victim_thd->LOCK_thd_data); + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); my_bool ret= wsrep_bf_abort(bf_thd, victim_thd); /* Send awake signal if victim was BF aborted or does not @@ -227,19 +234,8 @@ extern "C" my_bool wsrep_thd_bf_abort(THD *bf_thd, THD *victim_thd, */ if ((ret || !wsrep_on(victim_thd)) && signal) { - mysql_mutex_lock(&victim_thd->LOCK_thd_data); - - if (victim_thd->wsrep_aborter && victim_thd->wsrep_aborter != bf_thd->thread_id) - { - WSREP_DEBUG("victim is killed already by %llu, skipping awake", - victim_thd->wsrep_aborter); - mysql_mutex_unlock(&victim_thd->LOCK_thd_data); - return false; - } - victim_thd->wsrep_aborter= bf_thd->thread_id; victim_thd->awake_no_mutex(KILL_QUERY_HARD); - mysql_mutex_unlock(&victim_thd->LOCK_thd_data); } else { WSREP_DEBUG("wsrep_thd_bf_abort skipped awake, signal %d", signal); } @@ -368,25 +364,6 @@ extern "C" ulong wsrep_OSU_method_get(const MYSQL_THD thd) return(global_system_variables.wsrep_OSU_method); } -extern "C" bool wsrep_thd_set_wsrep_aborter(THD *bf_thd, THD *victim_thd) -{ - mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); - if (!bf_thd) - { - victim_thd->wsrep_aborter= 0; - WSREP_DEBUG("wsrep_thd_set_wsrep_aborter resetting wsrep_aborter"); - return false; - } - if (victim_thd->wsrep_aborter && victim_thd->wsrep_aborter != bf_thd->thread_id) - { - return true; - } - victim_thd->wsrep_aborter= bf_thd->thread_id; - WSREP_DEBUG("wsrep_thd_set_wsrep_aborter setting wsrep_aborter %u", - victim_thd->wsrep_aborter); - return false; -} - extern "C" void wsrep_report_bf_lock_wait(const THD *thd, unsigned long long trx_id) { diff --git a/sql/sql_class.cc b/sql/sql_class.cc index 56d0d1682cb..a49271125cc 100644 --- a/sql/sql_class.cc +++ b/sql/sql_class.cc @@ -1310,6 +1310,11 @@ void THD::init() wsrep_affected_rows = 0; m_wsrep_next_trx_id = WSREP_UNDEFINED_TRX_ID; wsrep_aborter = 0; + wsrep_abort_by_kill = NOT_KILLED; + wsrep_abort_by_kill_err = 0; +#ifndef DBUG_OFF + wsrep_killed_state = 0; +#endif /* DBUG_OFF */ wsrep_desynced_backup_stage= false; #endif /* WITH_WSREP */ @@ -1661,6 +1666,13 @@ void THD::reset_for_reuse() #endif #ifdef WITH_WSREP wsrep_free_status(this); + wsrep_cs().reset_error(); + wsrep_aborter= 0; + wsrep_abort_by_kill= NOT_KILLED; + wsrep_abort_by_kill_err= 0; +#ifndef DBUG_OFF + wsrep_killed_state= 0; +#endif /* DBUG_OFF */ #endif /* WITH_WSREP */ } @@ -1917,7 +1929,9 @@ void THD::awake_no_mutex(killed_state state_to_set) } /* Interrupt target waiting inside a storage engine. */ - if (state_to_set != NOT_KILLED && !wsrep_is_bf_aborted(this)) + if (state_to_set != NOT_KILLED && + IF_WSREP(!wsrep_is_bf_aborted(this) && wsrep_abort_by_kill == NOT_KILLED, + true)) ha_kill_query(this, thd_kill_level(this)); abort_current_cond_wait(false); @@ -2144,6 +2158,17 @@ void THD::reset_killed() mysql_mutex_unlock(&LOCK_thd_kill); } #ifdef WITH_WSREP + if (WSREP_NNULL(this)) + { + if (wsrep_abort_by_kill != NOT_KILLED) + { + mysql_mutex_assert_not_owner(&LOCK_thd_kill); + mysql_mutex_lock(&LOCK_thd_kill); + wsrep_abort_by_kill= NOT_KILLED; + wsrep_abort_by_kill_err= 0; + mysql_mutex_unlock(&LOCK_thd_kill); + } + } mysql_mutex_assert_not_owner(&LOCK_thd_data); mysql_mutex_lock(&LOCK_thd_data); wsrep_aborter= 0; diff --git a/sql/sql_class.h b/sql/sql_class.h index 6265d8060ce..2bca2c993fc 100644 --- a/sql/sql_class.h +++ b/sql/sql_class.h @@ -5402,7 +5402,14 @@ public: bool wsrep_ignore_table; /* thread who has started kill for this THD protected by LOCK_thd_data*/ my_thread_id wsrep_aborter; - + /* Kill signal used, if thread was killed by manual KILL. Protected by + LOCK_thd_kill. */ + std::atomic wsrep_abort_by_kill; + /* */ + struct err_info* wsrep_abort_by_kill_err; +#ifndef DBUG_OFF + int wsrep_killed_state; +#endif /* DBUG_OFF */ /* true if BF abort is observed in do_command() right after reading client's packet, and if the client has sent PS execute command. */ bool wsrep_delayed_BF_abort; diff --git a/sql/sql_parse.cc b/sql/sql_parse.cc index f2f622e78c8..1b9504abe07 100644 --- a/sql/sql_parse.cc +++ b/sql/sql_parse.cc @@ -7886,7 +7886,7 @@ static bool wsrep_mysql_parse(THD *thd, char *rawbuf, uint length, thd->wsrep_retry_counter < thd->variables.wsrep_retry_autocommit) { #ifdef ENABLED_DEBUG_SYNC - DBUG_EXECUTE_IF("sync.wsrep_retry_autocommit", + DBUG_EXECUTE_IF("sync.wsrep_retry_autocommit", { const char act[]= "now " @@ -9248,23 +9248,20 @@ kill_one_thread(THD *thd, my_thread_id id, killed_state kill_signal, killed_type thd->security_ctx->user_matches(tmp->security_ctx)) #endif /* WITH_WSREP */ { + { #ifdef WITH_WSREP - DEBUG_SYNC(thd, "before_awake_no_mutex"); - if (tmp->wsrep_aborter && tmp->wsrep_aborter != thd->thread_id) - { - /* victim is in hit list already, bail out */ - WSREP_DEBUG("victim %lld has wsrep aborter: %lu, skipping awake()", - id, tmp->wsrep_aborter); - error= 0; - } - else + if (WSREP(tmp)) + { + error = wsrep_kill_thd(thd, tmp, kill_signal); + } + else + { #endif /* WITH_WSREP */ - { - WSREP_DEBUG("kill_one_thread victim: %lld wsrep_aborter %lu" - " by signal %d", - id, tmp->wsrep_aborter, kill_signal); tmp->awake_no_mutex(kill_signal); error= 0; +#ifdef WITH_WSREP + } +#endif /* WITH_WSREP */ } } else @@ -9387,18 +9384,6 @@ static void sql_kill(THD *thd, my_thread_id id, killed_state state, killed_type type) { uint error; -#ifdef WITH_WSREP - if (WSREP(thd)) - { - WSREP_DEBUG("sql_kill called"); - if (thd->wsrep_applier) - { - WSREP_DEBUG("KILL in applying, bailing out here"); - return; - } - WSREP_TO_ISOLATION_BEGIN(WSREP_MYSQL_DB, NULL, NULL) - } -#endif /* WITH_WSREP */ if (likely(!(error= kill_one_thread(thd, id, state, type)))) { if (!thd->killed) @@ -9408,11 +9393,6 @@ void sql_kill(THD *thd, my_thread_id id, killed_state state, killed_type type) } else my_error(error, MYF(0), id); -#ifdef WITH_WSREP - return; - wsrep_error_label: - my_error(ER_KILL_DENIED_ERROR, MYF(0), (long long) thd->thread_id); -#endif /* WITH_WSREP */ } @@ -9421,18 +9401,6 @@ sql_kill_user(THD *thd, LEX_USER *user, killed_state state) { uint error; ha_rows rows; -#ifdef WITH_WSREP - if (WSREP(thd)) - { - WSREP_DEBUG("sql_kill_user called"); - if (thd->wsrep_applier) - { - WSREP_DEBUG("KILL in applying, bailing out here"); - return; - } - WSREP_TO_ISOLATION_BEGIN(WSREP_MYSQL_DB, NULL, NULL) - } -#endif /* WITH_WSREP */ switch (error= kill_threads_for_user(thd, user, state, &rows)) { case 0: @@ -9448,11 +9416,6 @@ sql_kill_user(THD *thd, LEX_USER *user, killed_state state) default: my_error(error, MYF(0)); } -#ifdef WITH_WSREP - return; - wsrep_error_label: - my_error(ER_KILL_DENIED_ERROR, MYF(0), (long long) thd->thread_id); -#endif /* WITH_WSREP */ } diff --git a/sql/sql_plugin_services.inl b/sql/sql_plugin_services.inl index 86b2fb69b22..7bd38368f3d 100644 --- a/sql/sql_plugin_services.inl +++ b/sql/sql_plugin_services.inl @@ -151,6 +151,7 @@ static struct wsrep_service_st wsrep_handler = { wsrep_on, wsrep_prepare_key_for_innodb, wsrep_thd_LOCK, + wsrep_thd_TRYLOCK, wsrep_thd_UNLOCK, wsrep_thd_query, wsrep_thd_retry_counter, @@ -179,7 +180,6 @@ static struct wsrep_service_st wsrep_handler = { wsrep_OSU_method_get, wsrep_thd_has_ignored_error, wsrep_thd_set_ignored_error, - wsrep_thd_set_wsrep_aborter, wsrep_report_bf_lock_wait, wsrep_thd_kill_LOCK, wsrep_thd_kill_UNLOCK, diff --git a/sql/wsrep_dummy.cc b/sql/wsrep_dummy.cc index 9bfaf9285f3..e1508884075 100644 --- a/sql/wsrep_dummy.cc +++ b/sql/wsrep_dummy.cc @@ -56,6 +56,11 @@ my_bool wsrep_on(const THD *) void wsrep_thd_LOCK(const THD *) { } +int wsrep_thd_TRYLOCK(const THD *) +{ + return 0; +} + void wsrep_thd_UNLOCK(const THD *) { } @@ -154,8 +159,6 @@ void wsrep_thd_set_ignored_error(THD*, my_bool) { } ulong wsrep_OSU_method_get(const THD*) { return 0;} -bool wsrep_thd_set_wsrep_aborter(THD*, THD*) -{ return 0;} void wsrep_report_bf_lock_wait(const THD*, unsigned long long) diff --git a/sql/wsrep_high_priority_service.cc b/sql/wsrep_high_priority_service.cc index 53ef20f3e78..0a2fa273723 100644 --- a/sql/wsrep_high_priority_service.cc +++ b/sql/wsrep_high_priority_service.cc @@ -510,6 +510,7 @@ int Wsrep_high_priority_service::log_dummy_write_set(const wsrep::ws_handle& ws_ m_thd->wait_for_prior_commit(); } + WSREP_DEBUG("checkpointing dummy write set %lld", ws_meta.seqno().get()); wsrep_set_SE_checkpoint(ws_meta.gtid(), wsrep_gtid_server.gtid()); if (!WSREP_EMULATE_BINLOG(m_thd)) diff --git a/sql/wsrep_mysqld.cc b/sql/wsrep_mysqld.cc index e0db00a3de3..50c85aa7173 100644 --- a/sql/wsrep_mysqld.cc +++ b/sql/wsrep_mysqld.cc @@ -51,6 +51,7 @@ #include "log_event.h" #include "sql_connect.h" #include "thread_cache.h" +#include "debug_sync.h" #include @@ -3037,7 +3038,22 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, request_thd, granted_thd); ticket->wsrep_report(wsrep_debug); + DEBUG_SYNC(request_thd, "before_wsrep_thd_abort"); + DBUG_EXECUTE_IF("sync.before_wsrep_thd_abort", { + const char act[]= "now " + "SIGNAL sync.before_wsrep_thd_abort_reached " + "WAIT_FOR signal.before_wsrep_thd_abort"; + DBUG_ASSERT(!debug_sync_set_action(request_thd, STRING_WITH_LEN(act))); + };); + + /* Here we will call wsrep_abort_transaction so we should hold + THD::LOCK_thd_data to protect victim from concurrent usage + and THD::LOCK_thd_kill to protect from disconnect or delete. + + */ + mysql_mutex_lock(&granted_thd->LOCK_thd_kill); mysql_mutex_lock(&granted_thd->LOCK_thd_data); + if (wsrep_thd_is_toi(granted_thd) || wsrep_thd_is_applying(granted_thd)) { @@ -3045,13 +3061,11 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, { WSREP_DEBUG("BF thread waiting for SR in aborting state"); ticket->wsrep_report(wsrep_debug); - mysql_mutex_unlock(&granted_thd->LOCK_thd_data); } else if (wsrep_thd_is_SR(granted_thd) && !wsrep_thd_is_SR(request_thd)) { WSREP_MDL_LOG(INFO, "MDL conflict, DDL vs SR", schema, schema_len, request_thd, granted_thd); - mysql_mutex_unlock(&granted_thd->LOCK_thd_data); wsrep_abort_thd(request_thd, granted_thd, 1); } else @@ -3060,6 +3074,7 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, request_thd, granted_thd); ticket->wsrep_report(true); mysql_mutex_unlock(&granted_thd->LOCK_thd_data); + mysql_mutex_unlock(&granted_thd->LOCK_thd_kill); unireg_abort(1); } } @@ -3068,7 +3083,6 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, { WSREP_DEBUG("BF thread waiting for FLUSH"); ticket->wsrep_report(wsrep_debug); - mysql_mutex_unlock(&granted_thd->LOCK_thd_data); if (granted_thd->current_backup_stage != BACKUP_FINISHED && wsrep_check_mode(WSREP_MODE_BF_MARIABACKUP)) { @@ -3080,7 +3094,6 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, WSREP_DEBUG("DROP caused BF abort, conf %s", wsrep_thd_transaction_state_str(granted_thd)); ticket->wsrep_report(wsrep_debug); - mysql_mutex_unlock(&granted_thd->LOCK_thd_data); wsrep_abort_thd(request_thd, granted_thd, 1); } else @@ -3090,7 +3103,6 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, ticket->wsrep_report(wsrep_debug); if (granted_thd->wsrep_trx().active()) { - mysql_mutex_unlock(&granted_thd->LOCK_thd_data); wsrep_abort_thd(request_thd, granted_thd, 1); } else @@ -3099,9 +3111,9 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, Granted_thd is likely executing with wsrep_on=0. If the requesting thd is BF, BF abort and wait. */ - mysql_mutex_unlock(&granted_thd->LOCK_thd_data); if (wsrep_thd_is_BF(request_thd, FALSE)) { + granted_thd->awake_no_mutex(KILL_QUERY_HARD); ha_abort_transaction(request_thd, granted_thd, TRUE); } else @@ -3109,10 +3121,14 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, WSREP_MDL_LOG(INFO, "MDL unknown BF-BF conflict", schema, schema_len, request_thd, granted_thd); ticket->wsrep_report(true); + mysql_mutex_unlock(&granted_thd->LOCK_thd_data); + mysql_mutex_unlock(&granted_thd->LOCK_thd_kill); unireg_abort(1); } } } + mysql_mutex_unlock(&granted_thd->LOCK_thd_data); + mysql_mutex_unlock(&granted_thd->LOCK_thd_kill); } else { @@ -3124,13 +3140,17 @@ void wsrep_handle_mdl_conflict(MDL_context *requestor_ctx, static bool abort_replicated(THD *thd) { bool ret_code= false; + mysql_mutex_lock(&thd->LOCK_thd_kill); + mysql_mutex_lock(&thd->LOCK_thd_data); if (thd->wsrep_trx().state() == wsrep::transaction::s_committing) { WSREP_DEBUG("aborting replicated trx: %llu", (ulonglong)(thd->real_id)); - (void)wsrep_abort_thd(thd, thd, TRUE); + wsrep_abort_thd(thd, thd, TRUE); ret_code= true; } + mysql_mutex_unlock(&thd->LOCK_thd_data); + mysql_mutex_unlock(&thd->LOCK_thd_kill); return ret_code; } diff --git a/sql/wsrep_server_service.cc b/sql/wsrep_server_service.cc index 9be6af71c56..52a0a9753c1 100644 --- a/sql/wsrep_server_service.cc +++ b/sql/wsrep_server_service.cc @@ -148,9 +148,13 @@ void Wsrep_server_service::release_high_priority_service(wsrep::high_priority_se wsrep_delete_threadvars(); } -void Wsrep_server_service::background_rollback(wsrep::client_state& client_state) +void Wsrep_server_service::background_rollback( + wsrep::unique_lock &lock WSREP_UNUSED, + wsrep::client_state &client_state) { - Wsrep_client_state& cs= static_cast(client_state); + DBUG_ASSERT(lock.owns_lock()); + Wsrep_client_state &cs= static_cast(client_state); + mysql_mutex_assert_owner(&cs.thd()->LOCK_thd_data); wsrep_fire_rollbacker(cs.thd()); } diff --git a/sql/wsrep_server_service.h b/sql/wsrep_server_service.h index 168e98206e3..0fc48402024 100644 --- a/sql/wsrep_server_service.h +++ b/sql/wsrep_server_service.h @@ -46,7 +46,8 @@ public: void release_high_priority_service(wsrep::high_priority_service*); - void background_rollback(wsrep::client_state&); + void background_rollback(wsrep::unique_lock &, + wsrep::client_state &); void bootstrap(); void log_message(enum wsrep::log::level, const char*); diff --git a/sql/wsrep_thd.cc b/sql/wsrep_thd.cc index 420a25dd2ae..682e64859b4 100644 --- a/sql/wsrep_thd.cc +++ b/sql/wsrep_thd.cc @@ -307,48 +307,9 @@ void wsrep_fire_rollbacker(THD *thd) } } - -int wsrep_abort_thd(THD *bf_thd, - THD *victim_thd, - my_bool signal) +static bool wsrep_bf_abort_low(THD *bf_thd, THD *victim_thd) { - DBUG_ENTER("wsrep_abort_thd"); - - mysql_mutex_lock(&victim_thd->LOCK_thd_data); - - /* Note that when you use RSU node is desynced from cluster, thus WSREP(thd) - might not be true. - */ - if ((WSREP_NNULL(bf_thd) || - ((WSREP_ON || bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU) && - wsrep_thd_is_toi(bf_thd))) && - !wsrep_thd_is_aborting(victim_thd)) - { - WSREP_DEBUG("wsrep_abort_thd, by: %llu, victim: %llu", - (long long)bf_thd->real_id, (long long)victim_thd->real_id); - mysql_mutex_unlock(&victim_thd->LOCK_thd_data); - ha_abort_transaction(bf_thd, victim_thd, signal); - DBUG_RETURN(1); - } - else - { - WSREP_DEBUG("wsrep_abort_thd not effective: bf %llu victim %llu " - "wsrep %d wsrep_on %d RSU %d TOI %d aborting %d", - (long long)bf_thd->real_id, (long long)victim_thd->real_id, - WSREP_NNULL(bf_thd), WSREP_ON, - bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU, - wsrep_thd_is_toi(bf_thd), - wsrep_thd_is_aborting(victim_thd)); - } - - mysql_mutex_unlock(&victim_thd->LOCK_thd_data); - DBUG_RETURN(1); -} - -bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd) -{ - WSREP_LOG_THD(bf_thd, "BF aborter before"); - WSREP_LOG_THD(victim_thd, "victim before"); + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); #ifdef ENABLED_DEBUG_SYNC DBUG_EXECUTE_IF("sync.wsrep_bf_abort", @@ -362,6 +323,85 @@ bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd) };); #endif + wsrep::seqno bf_seqno(bf_thd->wsrep_trx().ws_meta().seqno()); + bool ret; + + { + /* Adopt the lock, it is being held by the caller. */ + Wsrep_mutex wsm{&victim_thd->LOCK_thd_data}; + wsrep::unique_lock lock{wsm, std::adopt_lock}; + + if (wsrep_thd_is_toi(bf_thd)) + { + ret= victim_thd->wsrep_cs().total_order_bf_abort(lock, bf_seqno); + } + else + { + DBUG_ASSERT(WSREP(victim_thd) ? victim_thd->wsrep_trx().active() : 1); + ret= victim_thd->wsrep_cs().bf_abort(lock, bf_seqno); + } + if (ret) + { + /* BF abort should be allowed only once by wsrep-lib.*/ + DBUG_ASSERT(victim_thd->wsrep_aborter == 0); + victim_thd->wsrep_aborter= bf_thd->thread_id; + wsrep_bf_aborts_counter++; + } + lock.release(); /* No unlock at the end of the scope. */ + } + + /* Sanity check for wsrep-lib calls to return with LOCK_thd_data held. */ + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); + + return ret; +} + +void wsrep_abort_thd(THD *bf_thd, + THD *victim_thd, + my_bool signal) +{ + DBUG_ENTER("wsrep_abort_thd"); + + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_kill); + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); + + /* Note that when you use RSU node is desynced from cluster, thus WSREP(thd) + might not be true. + */ + if ((WSREP(bf_thd) + || ((WSREP_ON || bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU) + && wsrep_thd_is_toi(bf_thd)) + || bf_thd->lex->sql_command == SQLCOM_KILL) + && !wsrep_thd_is_aborting(victim_thd) && + wsrep_bf_abort_low(bf_thd, victim_thd) && + !victim_thd->wsrep_cs().is_rollbacker_active()) + { + WSREP_DEBUG("wsrep_abort_thd, by: %llu, victim: %llu", + (long long)bf_thd->real_id, (long long)victim_thd->real_id); + victim_thd->awake_no_mutex(KILL_QUERY_HARD); + ha_abort_transaction(bf_thd, victim_thd, signal); + } + else + { + WSREP_DEBUG("wsrep_abort_thd not effective: bf %llu victim %llu " + "wsrep %d wsrep_on %d RSU %d TOI %d aborting %d", + (long long)bf_thd->real_id, (long long)victim_thd->real_id, + WSREP_NNULL(bf_thd), WSREP_ON, + bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU, + wsrep_thd_is_toi(bf_thd), + wsrep_thd_is_aborting(victim_thd)); + } + + DBUG_VOID_RETURN; +} + +bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd) +{ + WSREP_LOG_THD(bf_thd, "BF aborter before"); + WSREP_LOG_THD(victim_thd, "victim before"); + + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); + if (WSREP(victim_thd) && !victim_thd->wsrep_trx().active()) { WSREP_DEBUG("wsrep_bf_abort, BF abort for non active transaction." @@ -384,30 +424,84 @@ bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd) wsrep_check_mode(WSREP_MODE_BF_MARIABACKUP)) { WSREP_DEBUG("killing connection for non wsrep session"); - mysql_mutex_lock(&victim_thd->LOCK_thd_data); victim_thd->awake_no_mutex(KILL_CONNECTION); - mysql_mutex_unlock(&victim_thd->LOCK_thd_data); } return false; } - bool ret; - wsrep::seqno bf_seqno(bf_thd->wsrep_trx().ws_meta().seqno()); + return wsrep_bf_abort_low(bf_thd, victim_thd); +} - if (wsrep_thd_is_toi(bf_thd)) +uint wsrep_kill_thd(THD *thd, THD *victim_thd, killed_state kill_signal) +{ + DBUG_ENTER("wsrep_kill_thd"); + DBUG_ASSERT(WSREP(victim_thd)); + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_kill); + mysql_mutex_assert_owner(&victim_thd->LOCK_thd_data); + using trans= wsrep::transaction; + auto trx_state= victim_thd->wsrep_trx().state(); +#ifndef DBUG_OFF + victim_thd->wsrep_killed_state= trx_state; +#endif /* DBUG_OFF */ + /* + Already killed or in commit codepath. Mark the victim as killed, + the killed status will be restored in wsrep_after_commit() and + will be processed after the commit is over. In case of multiple + KILLs happened on commit codepath, the last one will be effective. + */ + if (victim_thd->wsrep_abort_by_kill || + trx_state == trans::s_preparing || + trx_state == trans::s_committing || + trx_state == trans::s_ordered_commit) { - ret= victim_thd->wsrep_cs().total_order_bf_abort(bf_seqno); + victim_thd->wsrep_abort_by_kill= kill_signal; + DBUG_RETURN(0); } - else + /* + Mark killed victim_thd with kill_signal so that awake_no_mutex does + not dive into storage engine. We use ha_abort_transaction() + to do the storage engine part for wsrep THDs. + */ + DEBUG_SYNC(thd, "wsrep_kill_before_awake_no_mutex"); + victim_thd->wsrep_abort_by_kill= kill_signal; + victim_thd->awake_no_mutex(kill_signal); + /* ha_abort_transaction() releases tmp->LOCK_thd_kill, so tmp + is not safe to access anymore. */ + ha_abort_transaction(thd, victim_thd, 1); + DBUG_RETURN(0); +} + +void wsrep_backup_kill_for_commit(THD *thd) +{ + DBUG_ASSERT(WSREP(thd)); + mysql_mutex_assert_owner(&thd->LOCK_thd_kill); + DBUG_ASSERT(thd->killed != NOT_KILLED); + mysql_mutex_lock(&thd->LOCK_thd_data); + /* If the transaction will roll back, keep the killed state. + For must replay, the replay will happen in different THD context + which is high priority and cannot be killed. The owning thread will + pick the killed state in after statement processing. */ + if (thd->wsrep_trx().state() != wsrep::transaction::s_cert_failed && + thd->wsrep_trx().state() != wsrep::transaction::s_must_abort && + thd->wsrep_trx().state() != wsrep::transaction::s_aborting && + thd->wsrep_trx().state() != wsrep::transaction::s_must_replay) { - DBUG_ASSERT(WSREP(victim_thd) ? victim_thd->wsrep_trx().active() : 1); - ret= victim_thd->wsrep_cs().bf_abort(bf_seqno); + thd->wsrep_abort_by_kill= thd->killed; + thd->wsrep_abort_by_kill_err= thd->killed_err; + thd->killed= NOT_KILLED; + thd->killed_err= 0; } - if (ret) - { - wsrep_bf_aborts_counter++; - } - return ret; + mysql_mutex_unlock(&thd->LOCK_thd_data); +} + +void wsrep_restore_kill_after_commit(THD *thd) +{ + DBUG_ASSERT(WSREP(thd)); + mysql_mutex_assert_owner(&thd->LOCK_thd_kill); + thd->killed= thd->wsrep_abort_by_kill; + thd->killed_err= thd->wsrep_abort_by_kill_err; + thd->wsrep_abort_by_kill= NOT_KILLED; + thd->wsrep_abort_by_kill_err= 0; } int wsrep_create_threadvars() diff --git a/sql/wsrep_thd.h b/sql/wsrep_thd.h index 0ce612d6097..f3790887bf5 100644 --- a/sql/wsrep_thd.h +++ b/sql/wsrep_thd.h @@ -88,10 +88,39 @@ bool wsrep_create_appliers(long threads, bool mutex_protected=false); void wsrep_create_rollbacker(); bool wsrep_bf_abort(THD* bf_thd, THD* victim_thd); -int wsrep_abort_thd(THD *bf_thd, +/* + Abort transaction for victim_thd. This function is called from + MDL BF abort codepath. +*/ +void wsrep_abort_thd(THD *bf_thd, THD *victim_thd, my_bool signal) __attribute__((nonnull(1,2))); +/** + Kill wsrep connection with kill_signal. Object thd is not + guaranteed to exist anymore when this function returns. + + Asserts that the caller holds victim_thd->LOCK_thd_kill, + victim_thd->LOCK_thd_data. + + @param thd THD object for connection that executes the KILL. + @param victim_thd THD object for connection to be killed. + @param kill_signal Kill signal. + + @return Zero if the kill was successful, otherwise non-zero error code. + */ +uint wsrep_kill_thd(THD *thd, THD *victim_thd, killed_state kill_signal); + +/* + Backup kill status for commit. + */ +void wsrep_backup_kill_for_commit(THD *); + +/* + Restore KILL status after commit. + */ +void wsrep_restore_kill_after_commit(THD *); + /* Helper methods to deal with thread local storage. The purpose of these methods is to hide the details of thread diff --git a/sql/wsrep_trans_observer.h b/sql/wsrep_trans_observer.h index 8f998244ee6..40fbf80cfb3 100644 --- a/sql/wsrep_trans_observer.h +++ b/sql/wsrep_trans_observer.h @@ -256,6 +256,11 @@ static inline int wsrep_before_prepare(THD* thd, bool all) thd->wsrep_trx().ws_meta().gtid(), wsrep_gtid_server.gtid()); } + + mysql_mutex_lock(&thd->LOCK_thd_kill); + if (thd->killed) wsrep_backup_kill_for_commit(thd); + mysql_mutex_unlock(&thd->LOCK_thd_kill); + DBUG_RETURN(ret); } @@ -323,6 +328,11 @@ static inline int wsrep_before_commit(THD* thd, bool all) wsrep_gtid_server.gtid()); wsrep_register_for_group_commit(thd); } + + mysql_mutex_lock(&thd->LOCK_thd_kill); + if (thd->killed) wsrep_backup_kill_for_commit(thd); + mysql_mutex_unlock(&thd->LOCK_thd_kill); + DBUG_RETURN(ret); } @@ -341,7 +351,8 @@ static inline int wsrep_before_commit(THD* thd, bool all) static inline int wsrep_ordered_commit(THD* thd, bool all) { DBUG_ENTER("wsrep_ordered_commit"); - WSREP_DEBUG("wsrep_ordered_commit: %d", wsrep_is_real(thd, all)); + WSREP_DEBUG("wsrep_ordered_commit: %d %lld", wsrep_is_real(thd, all), + (long long) wsrep_thd_trx_seqno(thd)); DBUG_ASSERT(wsrep_run_commit_hook(thd, all)); DBUG_RETURN(thd->wsrep_cs().ordered_commit()); } @@ -449,10 +460,18 @@ int wsrep_after_statement(THD* thd) wsrep::to_c_string(thd->wsrep_cs().state()), wsrep::to_c_string(thd->wsrep_cs().mode()), wsrep::to_c_string(thd->wsrep_cs().transaction().state())); - DBUG_RETURN((thd->wsrep_cs().state() != wsrep::client_state::s_none && + int ret= ((thd->wsrep_cs().state() != wsrep::client_state::s_none && thd->wsrep_cs().mode() == Wsrep_client_state::m_local) && !thd->internal_transaction() ? thd->wsrep_cs().after_statement() : 0); + + if (wsrep_is_active(thd)) + { + mysql_mutex_lock(&thd->LOCK_thd_kill); + wsrep_restore_kill_after_commit(thd); + mysql_mutex_unlock(&thd->LOCK_thd_kill); + } + DBUG_RETURN(ret); } static inline void wsrep_after_apply(THD* thd) diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc index 2937ca40752..0a56c2b691c 100644 --- a/storage/innobase/handler/ha_innodb.cc +++ b/storage/innobase/handler/ha_innodb.cc @@ -1987,8 +1987,9 @@ static void innodb_disable_internal_writes(bool disable) sst_enable_innodb_writes(); } -static void wsrep_abort_transaction(handlerton*, THD *, THD *, my_bool); -static int innobase_wsrep_set_checkpoint(handlerton* hton, const XID* xid); +static void wsrep_abort_transaction(handlerton *, THD *, THD *, my_bool) + __attribute__((nonnull)); +static int innobase_wsrep_set_checkpoint(handlerton *hton, const XID *xid); static int innobase_wsrep_get_checkpoint(handlerton* hton, XID* xid); #endif /* WITH_WSREP */ @@ -18672,36 +18673,45 @@ void lock_wait_wsrep_kill(trx_t *bf_trx, ulong thd_id, trx_id_t trx_id) wsrep_thd_client_mode_str(vthd), wsrep_thd_transaction_state_str(vthd), wsrep_thd_query(vthd)); - /* Mark transaction as a victim for Galera abort */ - vtrx->lock.set_wsrep_victim(); - if (!wsrep_thd_set_wsrep_aborter(bf_thd, vthd)) - aborting= true; - else - WSREP_DEBUG("kill transaction skipped due to wsrep_aborter set"); + aborting= true; } } mysql_mutex_unlock(&lock_sys.wait_mutex); vtrx->mutex_unlock(); } - wsrep_thd_UNLOCK(vthd); - if (aborting) + + DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort"); + if (aborting && wsrep_thd_bf_abort(bf_thd, vthd, true)) { + /* Need to grab mutexes again to ensure that the trx is still in + right state. */ + lock_sys.wr_lock(SRW_LOCK_CALL); + mysql_mutex_lock(&lock_sys.wait_mutex); + vtrx->mutex_lock(); + /* if victim is waiting for some other lock, we have to cancel that waiting */ - lock_sys.cancel_lock_wait_for_trx(vtrx); - - DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort"); - if (!wsrep_thd_bf_abort(bf_thd, vthd, true)) + if (vtrx->id == trx_id) { - wsrep_thd_LOCK(vthd); - wsrep_thd_set_wsrep_aborter(NULL, vthd); - wsrep_thd_UNLOCK(vthd); - - WSREP_DEBUG("wsrep_thd_bf_abort has failed, victim %lu will survive", - thd_get_thread_id(vthd)); + switch (vtrx->state) { + default: + break; + case TRX_STATE_ACTIVE: + case TRX_STATE_PREPARED: + lock_sys.cancel_lock_wait_for_wsrep_bf_abort(vtrx); + } } + lock_sys.wr_unlock(); + mysql_mutex_unlock(&lock_sys.wait_mutex); + vtrx->mutex_unlock(); } + else + { + WSREP_DEBUG("wsrep_thd_bf_abort has failed, victim %lu will survive", + thd_get_thread_id(vthd)); + } + wsrep_thd_UNLOCK(vthd); wsrep_thd_kill_UNLOCK(vthd); } } @@ -18709,68 +18719,50 @@ void lock_wait_wsrep_kill(trx_t *bf_trx, ulong thd_id, trx_id_t trx_id) /** This function forces the victim transaction to abort. Aborting the transaction does NOT end it, it still has to be rolled back. + The caller must lock LOCK_thd_kill and LOCK_thd_data. + @param bf_thd brute force THD asking for the abort @param victim_thd victim THD to be aborted - - @return 0 victim was aborted - @return -1 victim thread was aborted (no transaction) */ -static -void -wsrep_abort_transaction( - handlerton*, - THD *bf_thd, - THD *victim_thd, - my_bool signal) +static void wsrep_abort_transaction(handlerton *, THD *bf_thd, THD *victim_thd, + my_bool signal) { - DBUG_ENTER("wsrep_abort_transaction"); - ut_ad(bf_thd); - ut_ad(victim_thd); + DBUG_ENTER("wsrep_abort_transaction"); + ut_ad(bf_thd); + ut_ad(victim_thd); - wsrep_thd_kill_LOCK(victim_thd); - wsrep_thd_LOCK(victim_thd); - trx_t* victim_trx= thd_to_trx(victim_thd); - wsrep_thd_UNLOCK(victim_thd); + trx_t *victim_trx= thd_to_trx(victim_thd); - WSREP_DEBUG("abort transaction: BF: %s victim: %s victim conf: %s", - wsrep_thd_query(bf_thd), - wsrep_thd_query(victim_thd), - wsrep_thd_transaction_state_str(victim_thd)); + WSREP_DEBUG("abort transaction: BF: %s victim: %s victim conf: %s", + wsrep_thd_query(bf_thd), wsrep_thd_query(victim_thd), + wsrep_thd_transaction_state_str(victim_thd)); - if (victim_trx) { - victim_trx->lock.set_wsrep_victim(); + if (!victim_trx) + { + WSREP_DEBUG("abort transaction: victim did not exist"); + DBUG_VOID_RETURN; + } - wsrep_thd_LOCK(victim_thd); - bool aborting= !wsrep_thd_set_wsrep_aborter(bf_thd, victim_thd); - wsrep_thd_UNLOCK(victim_thd); - if (aborting) { - DEBUG_SYNC(bf_thd, "before_wsrep_thd_abort"); - DBUG_EXECUTE_IF("sync.before_wsrep_thd_abort", - { - const char act[]= - "now " - "SIGNAL sync.before_wsrep_thd_abort_reached " - "WAIT_FOR signal.before_wsrep_thd_abort"; - DBUG_ASSERT(!debug_sync_set_action(bf_thd, - STRING_WITH_LEN(act))); - };); - wsrep_thd_bf_abort(bf_thd, victim_thd, signal); - } - } else { - DBUG_EXECUTE_IF("sync.before_wsrep_thd_abort", - { - const char act[]= - "now " - "SIGNAL sync.before_wsrep_thd_abort_reached " - "WAIT_FOR signal.before_wsrep_thd_abort"; - DBUG_ASSERT(!debug_sync_set_action(bf_thd, - STRING_WITH_LEN(act))); - };); - wsrep_thd_bf_abort(bf_thd, victim_thd, signal); - } + lock_sys.wr_lock(SRW_LOCK_CALL); + mysql_mutex_lock(&lock_sys.wait_mutex); + victim_trx->mutex_lock(); - wsrep_thd_kill_UNLOCK(victim_thd); - DBUG_VOID_RETURN; + switch (victim_trx->state) { + default: + break; + case TRX_STATE_ACTIVE: + case TRX_STATE_PREPARED: + /* Cancel lock wait if the victim is waiting for a lock in InnoDB. + The transaction which is blocked somewhere else (e.g. waiting + for next command or MDL) has been interrupted by THD::awake_no_mutex() + on server level before calling this function. */ + lock_sys.cancel_lock_wait_for_wsrep_bf_abort(victim_trx); + } + lock_sys.wr_unlock(); + mysql_mutex_unlock(&lock_sys.wait_mutex); + victim_trx->mutex_unlock(); + + DBUG_VOID_RETURN; } static diff --git a/storage/innobase/include/lock0lock.h b/storage/innobase/include/lock0lock.h index 16acd031177..e8299bb1189 100644 --- a/storage/innobase/include/lock0lock.h +++ b/storage/innobase/include/lock0lock.h @@ -955,6 +955,10 @@ public: /** Cancel possible lock waiting for a transaction */ static void cancel_lock_wait_for_trx(trx_t *trx); +#ifdef WITH_WSREP + /** Cancel lock waiting for a wsrep BF abort. */ + static void cancel_lock_wait_for_wsrep_bf_abort(trx_t *trx); +#endif /* WITH_WSREP */ }; /** The lock system */ diff --git a/storage/innobase/lock/lock0lock.cc b/storage/innobase/lock/lock0lock.cc index 3c7c3d348af..08547f169f3 100644 --- a/storage/innobase/lock/lock0lock.cc +++ b/storage/innobase/lock/lock0lock.cc @@ -5732,13 +5732,14 @@ static void lock_release_autoinc_locks(trx_t *trx) } /** Cancel a waiting lock request and release possibly waiting transactions */ -template +template void lock_cancel_waiting_and_release(lock_t *lock) { lock_sys.assert_locked(*lock); mysql_mutex_assert_owner(&lock_sys.wait_mutex); trx_t *trx= lock->trx; - trx->mutex_lock(); + if (inner_trx_lock) + trx->mutex_lock(); ut_d(const auto trx_state= trx->state); ut_ad(trx_state == TRX_STATE_COMMITTED_IN_MEMORY || trx_state == TRX_STATE_ACTIVE); @@ -5762,7 +5763,8 @@ void lock_cancel_waiting_and_release(lock_t *lock) lock_wait_end(trx); - trx->mutex_unlock(); + if (inner_trx_lock) + trx->mutex_unlock(); } void lock_sys_t::cancel_lock_wait_for_trx(trx_t *trx) @@ -5779,6 +5781,19 @@ void lock_sys_t::cancel_lock_wait_for_trx(trx_t *trx) mysql_mutex_unlock(&lock_sys.wait_mutex); } +#ifdef WITH_WSREP +void lock_sys_t::cancel_lock_wait_for_wsrep_bf_abort(trx_t *trx) +{ + lock_sys.assert_locked(); + mysql_mutex_assert_owner(&lock_sys.wait_mutex); + ut_ad(trx->mutex_is_owner()); + ut_ad(trx->state == TRX_STATE_ACTIVE || trx->state == TRX_STATE_PREPARED); + trx->lock.set_wsrep_victim(); + if (lock_t *lock= trx->lock.wait_lock) + lock_cancel_waiting_and_release(lock); +} +#endif /* WITH_WSREP */ + /** Cancel a waiting lock request. @tparam check_victim whether to check for DB_DEADLOCK @param trx active transaction diff --git a/wsrep-lib b/wsrep-lib index 4951c383577..e238c0d240c 160000 --- a/wsrep-lib +++ b/wsrep-lib @@ -1 +1 @@ -Subproject commit 4951c38357737d568b554402bc5b6abe88a38fe1 +Subproject commit e238c0d240c2557229b0523a4a032f3cf8b41639