2014-02-26 19:11:54 +01:00
|
|
|
/*****************************************************************************
|
|
|
|
|
2017-05-15 17:17:16 +03:00
|
|
|
Copyright (c) 1996, 2017, Oracle and/or its affiliates. All rights reserved.
|
2014-02-26 19:11:54 +01:00
|
|
|
Copyright (c) 2008, Google Inc.
|
|
|
|
Copyright (c) 2009, Percona Inc.
|
MDEV-24671: Replace lock_wait_timeout_task with mysql_cond_timedwait()
lock_wait(): Replaces lock_wait_suspend_thread(). Wait for the lock to
be granted or the transaction to be killed using mysql_cond_timedwait()
or mysql_cond_wait().
lock_wait_end(): Replaces que_thr_end_lock_wait() and
lock_wait_release_thread_if_suspended().
lock_wait_timeout_task: Remove. The operating system kernel will
resume the mysql_cond_timedwait() in lock_wait(). An added benefit
is that innodb_lock_wait_timeout no longer has a 'jitter' of 1 second,
which was caused by this wake-up task waking up only once per second,
and then waking up any threads for which the timeout (which was only
measured in seconds) was exceeded.
innobase_kill_query(): Set trx->error_state=DB_INTERRUPTED,
so that a call trx_is_interrupted(trx) in lock_wait() can be avoided.
We will protect things more consistently with lock_sys.wait_mutex,
which will be moved below lock_sys.mutex in the latching order.
trx_lock_t::cond: Condition variable for !wait_lock, used with
lock_sys.wait_mutex.
srv_slot_t: Remove. Replaced by trx_lock_t::cond,
lock_grant_after_reset(): Merged to to lock_grant().
lock_rec_get_index_name(): Remove.
lock_sys_t: Introduce wait_pending, wait_count, wait_time, wait_time_max
that are protected by wait_mutex.
trx_lock_t::que_state: Remove.
que_thr_state_t: Remove QUE_THR_COMMAND_WAIT, QUE_THR_LOCK_WAIT.
que_thr_t: Remove is_active, start_running(), stop_no_error().
que_fork_t::n_active_thrs, trx_lock_t::n_active_thrs: Remove.
2021-01-26 16:39:56 +02:00
|
|
|
Copyright (c) 2013, 2021, MariaDB Corporation.
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
Portions of this file contain modifications contributed and copyrighted by
|
|
|
|
Google, Inc. Those modifications are gratefully acknowledged and are described
|
|
|
|
briefly in the InnoDB documentation. The contributions by Google are
|
|
|
|
incorporated with their permission, and subject to the conditions contained in
|
|
|
|
the file COPYING.Google.
|
|
|
|
|
|
|
|
Portions of this file contain modifications contributed and copyrighted
|
|
|
|
by Percona Inc.. Those modifications are
|
|
|
|
gratefully acknowledged and are described briefly in the InnoDB
|
|
|
|
documentation. The contributions by Percona Inc. are incorporated with
|
|
|
|
their permission, and subject to the conditions contained in the file
|
|
|
|
COPYING.Percona.
|
|
|
|
|
|
|
|
This program is free software; you can redistribute it and/or modify it under
|
|
|
|
the terms of the GNU General Public License as published by the Free Software
|
|
|
|
Foundation; version 2 of the License.
|
|
|
|
|
|
|
|
This program is distributed in the hope that it will be useful, but WITHOUT
|
|
|
|
ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
|
|
|
|
FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
|
|
|
|
|
|
|
|
You should have received a copy of the GNU General Public License along with
|
|
|
|
this program; if not, write to the Free Software Foundation, Inc.,
|
2019-05-11 19:25:02 +03:00
|
|
|
51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
*****************************************************************************/
|
|
|
|
|
|
|
|
/********************************************************************//**
|
|
|
|
@file srv/srv0start.cc
|
|
|
|
Starts the InnoDB database server
|
|
|
|
|
|
|
|
Created 2/16/1996 Heikki Tuuri
|
|
|
|
*************************************************************************/
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
#include "my_global.h"
|
|
|
|
|
2014-02-26 19:23:04 +01:00
|
|
|
#include "mysqld.h"
|
2016-08-12 11:17:45 +03:00
|
|
|
#include "mysql/psi/mysql_stage.h"
|
|
|
|
#include "mysql/psi/psi.h"
|
|
|
|
|
2014-02-26 19:23:04 +01:00
|
|
|
#include "row0ftsort.h"
|
2014-02-26 19:11:54 +01:00
|
|
|
#include "ut0mem.h"
|
|
|
|
#include "mem0mem.h"
|
|
|
|
#include "data0data.h"
|
|
|
|
#include "data0type.h"
|
|
|
|
#include "dict0dict.h"
|
|
|
|
#include "buf0buf.h"
|
2020-01-21 15:46:20 +02:00
|
|
|
#include "buf0dblwr.h"
|
2014-02-26 19:11:54 +01:00
|
|
|
#include "buf0dump.h"
|
|
|
|
#include "os0file.h"
|
|
|
|
#include "os0thread.h"
|
|
|
|
#include "fil0fil.h"
|
2015-04-01 11:50:21 +03:00
|
|
|
#include "fil0crypt.h"
|
2014-02-26 19:11:54 +01:00
|
|
|
#include "fsp0fsp.h"
|
|
|
|
#include "rem0rec.h"
|
|
|
|
#include "mtr0mtr.h"
|
2017-02-10 12:11:42 +02:00
|
|
|
#include "log0crypt.h"
|
2014-02-26 19:11:54 +01:00
|
|
|
#include "log0recv.h"
|
|
|
|
#include "page0page.h"
|
|
|
|
#include "page0cur.h"
|
|
|
|
#include "trx0trx.h"
|
|
|
|
#include "trx0sys.h"
|
|
|
|
#include "btr0btr.h"
|
|
|
|
#include "btr0cur.h"
|
|
|
|
#include "rem0rec.h"
|
|
|
|
#include "ibuf0ibuf.h"
|
|
|
|
#include "srv0start.h"
|
|
|
|
#include "srv0srv.h"
|
2014-08-06 15:28:58 +03:00
|
|
|
#include "btr0defragment.h"
|
2017-08-31 08:27:59 +03:00
|
|
|
#include "mysql/service_wsrep.h" /* wsrep_recovery */
|
2016-12-30 15:04:10 +02:00
|
|
|
#include "trx0rseg.h"
|
|
|
|
#include "buf0flu.h"
|
|
|
|
#include "buf0rea.h"
|
|
|
|
#include "dict0boot.h"
|
|
|
|
#include "dict0load.h"
|
|
|
|
#include "dict0stats_bg.h"
|
|
|
|
#include "que0que.h"
|
|
|
|
#include "lock0lock.h"
|
|
|
|
#include "trx0roll.h"
|
|
|
|
#include "trx0purge.h"
|
|
|
|
#include "lock0lock.h"
|
|
|
|
#include "pars0pars.h"
|
|
|
|
#include "btr0sea.h"
|
|
|
|
#include "rem0cmp.h"
|
|
|
|
#include "dict0crea.h"
|
|
|
|
#include "row0ins.h"
|
|
|
|
#include "row0sel.h"
|
|
|
|
#include "row0upd.h"
|
|
|
|
#include "row0row.h"
|
|
|
|
#include "row0mysql.h"
|
|
|
|
#include "btr0pcur.h"
|
|
|
|
#include "zlib.h"
|
|
|
|
#include "ut0crc32.h"
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2020-08-04 06:59:29 +03:00
|
|
|
/** We are prepared for a situation that we have this many threads waiting for
|
2020-12-04 18:07:25 +02:00
|
|
|
a transactional lock inside InnoDB. srv_start() sets the value. */
|
2020-08-04 06:59:29 +03:00
|
|
|
ulint srv_max_n_threads;
|
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
/** Log sequence number at shutdown */
|
2016-08-12 11:17:45 +03:00
|
|
|
lsn_t srv_shutdown_lsn;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
/** TRUE if a raw partition is in use */
|
2017-04-25 09:26:01 +03:00
|
|
|
ibool srv_start_raw_disk_in_use;
|
2016-08-12 11:17:45 +03:00
|
|
|
|
|
|
|
/** Number of IO threads to use */
|
2020-12-14 15:27:03 +02:00
|
|
|
uint srv_n_file_io_threads;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2017-05-15 17:17:16 +03:00
|
|
|
/** UNDO tablespaces starts with space id. */
|
|
|
|
ulint srv_undo_space_id_start;
|
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
/** TRUE if the server is being started, before rolling back any
|
|
|
|
incomplete transactions */
|
2017-04-25 09:26:01 +03:00
|
|
|
bool srv_startup_is_before_trx_rollback_phase;
|
2014-02-26 19:11:54 +01:00
|
|
|
/** TRUE if the server is being started */
|
2017-04-25 09:26:01 +03:00
|
|
|
bool srv_is_being_started;
|
2014-02-26 19:11:54 +01:00
|
|
|
/** TRUE if the server was successfully started */
|
2017-04-25 09:26:01 +03:00
|
|
|
bool srv_was_started;
|
Reduce the granularity of innodb_log_file_size
In Mariabackup, we would want the backed-up redo log file size to be
a multiple of 512 bytes, or OS_FILE_LOG_BLOCK_SIZE. However, at startup,
InnoDB would be picky, requiring the file size to be a multiple of
innodb_page_size.
Furthermore, InnoDB would require the parameter to be a multiple of
one megabyte, while the minimum granularity is 512 bytes. Because
the data-file-oriented fil_io() API is being used for writing the
InnoDB redo log, writes will for now require innodb_log_file_size to
be a multiple of the maximum innodb_page_size (65536 bytes).
To complicate matters, InnoDB startup divided srv_log_file_size by
UNIV_PAGE_SIZE, so that initially, the unit was bytes, and later it
was innodb_page_size. We will simplify this and keep srv_log_file_size
in bytes at all times.
innobase_log_file_size: Remove. Remove some obsolete checks against
overflow on 32-bit systems. srv_log_file_size is always 64 bits, and
the maximum size 512GiB in multiples of innodb_page_size always fits
in ulint (which is 32 or 64 bits). 512GiB would be 8,388,608*64KiB or
134,217,728*4KiB.
log_init(): Remove the parameter file_size that was always passed as
srv_log_file_size.
log_set_capacity(): Add a parameter for passing the requested file size.
srv_log_file_size_requested: Declare static in srv0start.cc.
create_log_file(), create_log_files(),
innobase_start_or_create_for_mysql(): Invoke fil_node_create()
with srv_log_file_size expressed in multiples of innodb_page_size.
innobase_start_or_create_for_mysql(): Require the redo log file sizes
to be multiples of 512 bytes.
2017-06-05 22:47:20 +03:00
|
|
|
/** The original value of srv_log_file_size (innodb_log_file_size) */
|
|
|
|
static ulonglong srv_log_file_size_requested;
|
2018-04-29 09:41:42 +03:00
|
|
|
/** whether srv_start() has been called */
|
2017-06-12 17:43:07 +03:00
|
|
|
static bool srv_start_has_been_called;
|
MDEV-13039 innodb_fast_shutdown=0 may fail to purge all undo log
When a slow shutdown is performed soon after spawning some work for
background threads that can create or commit transactions, it is possible
that new transactions are started or committed after the purge has finished.
This is violating the specification of innodb_fast_shutdown=0, namely that
the purge must be completed. (None of the history of the recent transactions
would be purged.)
Also, it is possible that the purge threads would exit in slow shutdown
while there exist active transactions, such as recovered incomplete
transactions that are being rolled back. Thus, the slow shutdown could
fail to purge some undo log that becomes purgeable after the transaction
commit or rollback.
srv_undo_sources: A flag that indicates if undo log can be generated
or the persistent, whether by background threads or by user SQL.
Even when this flag is clear, active transactions that already exist
in the system may be committed or rolled back.
innodb_shutdown(): Renamed from innobase_shutdown_for_mysql().
Do not return an error code; the operation never fails.
Clear the srv_undo_sources flag, and also ensure that the background
DROP TABLE queue is empty.
srv_purge_should_exit(): Do not allow the purge to exit if
srv_undo_sources are active or the background DROP TABLE queue is not
empty, or in slow shutdown, if any active transactions exist
(and are being rolled back).
srv_purge_coordinator_thread(): Remove some previous workarounds
for this bug.
innobase_start_or_create_for_mysql(): Set buf_page_cleaner_is_active
and srv_dict_stats_thread_active directly. Set srv_undo_sources before
starting the purge subsystem, to prevent immediate shutdown of the purge.
Create dict_stats_thread and fts_optimize_thread immediately
after setting srv_undo_sources, so that shutdown can use this flag to
determine if these subsystems were started.
dict_stats_shutdown(): Shut down dict_stats_thread. Backported from 10.2.
srv_shutdown_table_bg_threads(): Remove (unused).
2017-06-08 15:43:06 +03:00
|
|
|
|
|
|
|
/** Whether any undo log records can be generated */
|
2017-06-12 14:26:32 +03:00
|
|
|
UNIV_INTERN bool srv_undo_sources;
|
|
|
|
|
MDEV-11556 InnoDB redo log apply fails to adjust data file sizes
fil_space_t::recv_size: New member: recovered tablespace size in pages;
0 if no size change was read from the redo log,
or if the size change was implemented.
fil_space_set_recv_size(): New function for setting space->recv_size.
innodb_data_file_size_debug: A debug parameter for setting the system
tablespace size in recovery even when the redo log does not contain
any size changes. It is hard to write a small test case that would
cause the system tablespace to be extended at the critical moment.
recv_parse_log_rec(): Note those tablespaces whose size is being changed
by the redo log, by invoking fil_space_set_recv_size().
innobase_init(): Correct an error message, and do not require a larger
innodb_buffer_pool_size when starting up with a smaller innodb_page_size.
innobase_start_or_create_for_mysql(): Allow startup with any initial
size of the ibdata1 file if the autoextend attribute is set. Require
the minimum size of fixed-size system tablespaces to be 640 pages,
not 10 megabytes. Implement innodb_data_file_size_debug.
open_or_create_data_files(): Round the system tablespace size down
to pages, not to full megabytes, (Our test truncates the system
tablespace to more than 800 pages with innodb_page_size=4k.
InnoDB should not imagine that it was truncated to 768 pages
and then overwrite good pages in the tablespace.)
fil_flush_low(): Refactored from fil_flush().
fil_space_extend_must_retry(): Refactored from
fil_extend_space_to_desired_size().
fil_mutex_enter_and_prepare_for_io(): Extend the tablespace if
fil_space_set_recv_size() was called.
The test case has been successfully run with all the
innodb_page_size values 4k, 8k, 16k, 32k, 64k.
2016-12-28 12:05:43 +02:00
|
|
|
#ifdef UNIV_DEBUG
|
|
|
|
/** InnoDB system tablespace to set during recovery */
|
2017-01-07 15:27:59 +02:00
|
|
|
UNIV_INTERN uint srv_sys_space_size_debug;
|
2020-01-12 02:05:28 +07:00
|
|
|
/** whether redo log file have been created at startup */
|
|
|
|
UNIV_INTERN bool srv_log_file_created;
|
MDEV-11556 InnoDB redo log apply fails to adjust data file sizes
fil_space_t::recv_size: New member: recovered tablespace size in pages;
0 if no size change was read from the redo log,
or if the size change was implemented.
fil_space_set_recv_size(): New function for setting space->recv_size.
innodb_data_file_size_debug: A debug parameter for setting the system
tablespace size in recovery even when the redo log does not contain
any size changes. It is hard to write a small test case that would
cause the system tablespace to be extended at the critical moment.
recv_parse_log_rec(): Note those tablespaces whose size is being changed
by the redo log, by invoking fil_space_set_recv_size().
innobase_init(): Correct an error message, and do not require a larger
innodb_buffer_pool_size when starting up with a smaller innodb_page_size.
innobase_start_or_create_for_mysql(): Allow startup with any initial
size of the ibdata1 file if the autoextend attribute is set. Require
the minimum size of fixed-size system tablespaces to be 640 pages,
not 10 megabytes. Implement innodb_data_file_size_debug.
open_or_create_data_files(): Round the system tablespace size down
to pages, not to full megabytes, (Our test truncates the system
tablespace to more than 800 pages with innodb_page_size=4k.
InnoDB should not imagine that it was truncated to 768 pages
and then overwrite good pages in the tablespace.)
fil_flush_low(): Refactored from fil_flush().
fil_space_extend_must_retry(): Refactored from
fil_extend_space_to_desired_size().
fil_mutex_enter_and_prepare_for_io(): Extend the tablespace if
fil_space_set_recv_size() was called.
The test case has been successfully run with all the
innodb_page_size values 4k, 8k, 16k, 32k, 64k.
2016-12-28 12:05:43 +02:00
|
|
|
#endif /* UNIV_DEBUG */
|
2016-08-12 11:17:45 +03:00
|
|
|
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
/** whether some background threads that create redo log have been started */
|
|
|
|
static bool srv_started_redo;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
/** At a shutdown this value climbs from SRV_SHUTDOWN_NONE to
|
|
|
|
SRV_SHUTDOWN_CLEANUP and then to SRV_SHUTDOWN_LAST_PHASE, and so on */
|
2016-08-12 11:17:45 +03:00
|
|
|
enum srv_shutdown_t srv_shutdown_state = SRV_SHUTDOWN_NONE;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
/** Name of srv_monitor_file */
|
|
|
|
static char* srv_monitor_file_name;
|
2019-10-29 22:37:12 +01:00
|
|
|
std::unique_ptr<tpool::timer> srv_master_timer;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
/** */
|
|
|
|
#define SRV_MAX_N_PENDING_SYNC_IOS 100
|
|
|
|
|
|
|
|
#ifdef UNIV_PFS_THREAD
|
|
|
|
/* Keys to register InnoDB threads with performance schema */
|
2019-10-29 22:37:12 +01:00
|
|
|
mysql_pfs_key_t thread_pool_thread_key;
|
2014-02-26 19:11:54 +01:00
|
|
|
#endif /* UNIV_PFS_THREAD */
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
#ifdef HAVE_PSI_STAGE_INTERFACE
|
|
|
|
/** Array of all InnoDB stage events for monitoring activities via
|
|
|
|
performance schema. */
|
|
|
|
static PSI_stage_info* srv_stages[] =
|
2014-02-26 19:11:54 +01:00
|
|
|
{
|
2016-08-12 11:17:45 +03:00
|
|
|
&srv_stage_alter_table_end,
|
|
|
|
&srv_stage_alter_table_insert,
|
|
|
|
&srv_stage_alter_table_log_index,
|
|
|
|
&srv_stage_alter_table_log_table,
|
|
|
|
&srv_stage_alter_table_merge_sort,
|
|
|
|
&srv_stage_alter_table_read_pk_internal_sort,
|
|
|
|
&srv_stage_buffer_pool_load,
|
|
|
|
};
|
|
|
|
#endif /* HAVE_PSI_STAGE_INTERFACE */
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
/*********************************************************************//**
|
|
|
|
Check if a file can be opened in read-write mode.
|
2016-08-12 11:17:45 +03:00
|
|
|
@return true if it doesn't exist or can be opened in rw mode. */
|
2014-02-26 19:11:54 +01:00
|
|
|
static
|
|
|
|
bool
|
|
|
|
srv_file_check_mode(
|
|
|
|
/*================*/
|
|
|
|
const char* name) /*!< in: filename to check */
|
|
|
|
{
|
|
|
|
os_file_stat_t stat;
|
|
|
|
|
|
|
|
memset(&stat, 0x0, sizeof(stat));
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
dberr_t err = os_file_get_status(
|
|
|
|
name, &stat, true, srv_read_only_mode);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
if (err == DB_FAIL) {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::error() << "os_file_get_status() failed on '" << name
|
|
|
|
<< "'. Can't determine file permissions.";
|
2014-02-26 19:11:54 +01:00
|
|
|
return(false);
|
|
|
|
|
|
|
|
} else if (err == DB_SUCCESS) {
|
|
|
|
|
|
|
|
/* Note: stat.rw_perm is only valid of files */
|
|
|
|
|
2015-06-16 10:57:05 +02:00
|
|
|
if (stat.type == OS_FILE_TYPE_FILE) {
|
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
if (!stat.rw_perm) {
|
2016-08-12 11:17:45 +03:00
|
|
|
const char* mode = srv_read_only_mode
|
|
|
|
? "read" : "read-write";
|
|
|
|
ib::error() << name << " can't be opened in "
|
|
|
|
<< mode << " mode.";
|
2014-02-26 19:11:54 +01:00
|
|
|
return(false);
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
/* Not a regular file, bail out. */
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::error() << "'" << name << "' not a regular file.";
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
return(false);
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
|
|
|
|
/* This is OK. If the file create fails on RO media, there
|
|
|
|
is nothing we can do. */
|
|
|
|
|
|
|
|
ut_a(err == DB_NOT_FOUND);
|
|
|
|
}
|
|
|
|
|
|
|
|
return(true);
|
|
|
|
}
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
/** Initial number of the redo log file */
|
|
|
|
static const char INIT_LOG_FILE0[]= "101";
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
/** Creates log file.
|
2020-10-26 15:59:30 +02:00
|
|
|
@param[in] create_new_db whether the database is being initialized
|
|
|
|
@param[in] lsn FIL_PAGE_FILE_FLUSH_LSN value
|
|
|
|
@param[out] logfile0 name of the log file
|
2016-08-12 11:17:45 +03:00
|
|
|
@return DB_SUCCESS or error code */
|
2020-10-26 15:59:30 +02:00
|
|
|
static dberr_t create_log_file(bool create_new_db, lsn_t lsn,
|
|
|
|
std::string& logfile0)
|
2014-02-26 19:11:54 +01:00
|
|
|
{
|
|
|
|
if (srv_read_only_mode) {
|
2020-01-12 02:05:28 +07:00
|
|
|
ib::error() << "Cannot create log file in read-only mode";
|
|
|
|
return DB_READ_ONLY;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
/* Crashing after deleting the first file should be
|
|
|
|
recoverable. The buffer pool was clean, and we can simply
|
2020-01-12 02:05:28 +07:00
|
|
|
create log file from the scratch. */
|
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_6", delete_log_file("0");
|
|
|
|
return DB_ERROR;);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-03-07 14:47:15 +02:00
|
|
|
for (size_t i = 0; i < 102; i++) {
|
|
|
|
delete_log_file(std::to_string(i).c_str());
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2017-02-16 09:18:46 +02:00
|
|
|
DBUG_PRINT("ib_log", ("After innodb_log_abort_6"));
|
2020-06-08 13:14:27 +03:00
|
|
|
DBUG_ASSERT(!buf_pool.any_io_pending());
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_7", return DB_ERROR;);
|
2017-02-16 09:18:46 +02:00
|
|
|
DBUG_PRINT("ib_log", ("After innodb_log_abort_7"));
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
logfile0 = get_log_file_path(LOG_FILE_NAME_PREFIX)
|
|
|
|
.append(INIT_LOG_FILE0);
|
2019-11-25 22:32:24 +07:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
bool ret;
|
|
|
|
pfs_os_file_t file = os_file_create(
|
|
|
|
innodb_log_file_key, logfile0.c_str(),
|
|
|
|
OS_FILE_CREATE|OS_FILE_ON_ERROR_NO_EXIT, OS_FILE_NORMAL,
|
|
|
|
OS_LOG_FILE, srv_read_only_mode, &ret);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
if (!ret) {
|
|
|
|
ib::error() << "Cannot create " << logfile0;
|
|
|
|
return DB_ERROR;
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
ib::info() << "Setting log file " << logfile0 << " size to "
|
|
|
|
<< srv_log_file_size << " bytes";
|
2019-11-25 22:32:24 +07:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
ret = os_file_set_size(logfile0.c_str(), file, srv_log_file_size);
|
|
|
|
if (!ret) {
|
|
|
|
os_file_close(file);
|
|
|
|
ib::error() << "Cannot set log file " << logfile0
|
|
|
|
<< " size to " << srv_log_file_size << " bytes";
|
|
|
|
return DB_ERROR;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
ret = os_file_close(file);
|
|
|
|
ut_a(ret);
|
2019-11-25 22:32:24 +07:00
|
|
|
|
2017-01-30 17:00:51 +02:00
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_8", return(DB_ERROR););
|
2017-02-16 09:18:46 +02:00
|
|
|
DBUG_PRINT("ib_log", ("After innodb_log_abort_8"));
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
/* We did not create the first log file initially as LOG_FILE_NAME, so
|
|
|
|
that crash recovery cannot find it until it has been completed and
|
|
|
|
renamed. */
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
log_sys.log.create();
|
Reduce the granularity of innodb_log_file_size
In Mariabackup, we would want the backed-up redo log file size to be
a multiple of 512 bytes, or OS_FILE_LOG_BLOCK_SIZE. However, at startup,
InnoDB would be picky, requiring the file size to be a multiple of
innodb_page_size.
Furthermore, InnoDB would require the parameter to be a multiple of
one megabyte, while the minimum granularity is 512 bytes. Because
the data-file-oriented fil_io() API is being used for writing the
InnoDB redo log, writes will for now require innodb_log_file_size to
be a multiple of the maximum innodb_page_size (65536 bytes).
To complicate matters, InnoDB startup divided srv_log_file_size by
UNIV_PAGE_SIZE, so that initially, the unit was bytes, and later it
was innodb_page_size. We will simplify this and keep srv_log_file_size
in bytes at all times.
innobase_log_file_size: Remove. Remove some obsolete checks against
overflow on 32-bit systems. srv_log_file_size is always 64 bits, and
the maximum size 512GiB in multiples of innodb_page_size always fits
in ulint (which is 32 or 64 bits). 512GiB would be 8,388,608*64KiB or
134,217,728*4KiB.
log_init(): Remove the parameter file_size that was always passed as
srv_log_file_size.
log_set_capacity(): Add a parameter for passing the requested file size.
srv_log_file_size_requested: Declare static in srv0start.cc.
create_log_file(), create_log_files(),
innobase_start_or_create_for_mysql(): Invoke fil_node_create()
with srv_log_file_size expressed in multiples of innodb_page_size.
innobase_start_or_create_for_mysql(): Require the redo log file sizes
to be multiples of 512 bytes.
2017-06-05 22:47:20 +03:00
|
|
|
if (!log_set_capacity(srv_log_file_size_requested)) {
|
2020-01-12 02:05:28 +07:00
|
|
|
return DB_ERROR;
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
log_sys.log.open_file(logfile0);
|
2020-10-26 15:59:30 +02:00
|
|
|
if (!fil_system.sys_space->open(create_new_db)) {
|
|
|
|
return DB_ERROR;
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
/* Create a log checkpoint. */
|
2020-10-26 16:43:52 +02:00
|
|
|
mysql_mutex_lock(&log_sys.mutex);
|
2018-04-27 10:06:14 +03:00
|
|
|
if (log_sys.is_encrypted() && !log_crypt_init()) {
|
2019-07-01 18:50:44 +05:30
|
|
|
return DB_ERROR;
|
2017-02-10 12:11:42 +02:00
|
|
|
}
|
2016-08-12 11:17:45 +03:00
|
|
|
ut_d(recv_no_log_write = false);
|
2020-03-05 11:45:28 +02:00
|
|
|
lsn = ut_uint64_align_up(lsn, OS_FILE_LOG_BLOCK_SIZE);
|
|
|
|
log_sys.set_lsn(lsn + LOG_BLOCK_HDR_SIZE);
|
|
|
|
log_sys.log.set_lsn(lsn);
|
2019-05-03 20:14:09 +03:00
|
|
|
log_sys.log.set_lsn_offset(LOG_FILE_HDR_SIZE);
|
2019-05-03 20:02:11 +03:00
|
|
|
|
2019-05-03 20:14:09 +03:00
|
|
|
log_sys.buf_next_to_write = 0;
|
2020-03-05 11:45:28 +02:00
|
|
|
log_sys.write_lsn = lsn;
|
2019-05-03 20:02:11 +03:00
|
|
|
|
2019-05-03 20:14:09 +03:00
|
|
|
log_sys.next_checkpoint_no = 0;
|
|
|
|
log_sys.last_checkpoint_lsn = 0;
|
2019-05-03 20:02:11 +03:00
|
|
|
|
2019-05-03 20:14:09 +03:00
|
|
|
memset(log_sys.buf, 0, srv_log_buffer_size);
|
2020-03-05 11:45:28 +02:00
|
|
|
log_block_init(log_sys.buf, lsn);
|
2019-05-03 20:14:09 +03:00
|
|
|
log_block_set_first_rec_group(log_sys.buf, LOG_BLOCK_HDR_SIZE);
|
2020-04-28 00:40:13 +03:00
|
|
|
memset(log_sys.flush_buf, 0, srv_log_buffer_size);
|
2019-05-03 20:02:11 +03:00
|
|
|
|
2019-05-03 20:14:09 +03:00
|
|
|
log_sys.buf_free = LOG_BLOCK_HDR_SIZE;
|
2019-05-03 20:02:11 +03:00
|
|
|
|
2020-03-15 21:53:25 +03:00
|
|
|
log_sys.log.write_header_durable(lsn);
|
|
|
|
|
2020-10-26 16:43:52 +02:00
|
|
|
mysql_mutex_unlock(&log_sys.mutex);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2019-10-09 18:47:14 +03:00
|
|
|
log_make_checkpoint();
|
MDEV-23855: Improve InnoDB log checkpoint performance
After MDEV-15053, MDEV-22871, MDEV-23399 shifted the scalability
bottleneck, log checkpoints became a new bottleneck.
If innodb_io_capacity is set low or innodb_max_dirty_pct_lwm is
set high and the workload fits in the buffer pool, the page cleaner
thread will perform very little flushing. When we reach the capacity
of the circular redo log file ib_logfile0 and must initiate a checkpoint,
some 'furious flushing' will be necessary. (If innodb_flush_sync=OFF,
then flushing would continue at the innodb_io_capacity rate, and
writers would be throttled.)
We have the best chance of advancing the checkpoint LSN immediately
after a page flush batch has been completed. Hence, it is best to
perform checkpoints after every batch in the page cleaner thread,
attempting to run once per second.
By initiating high-priority flushing in the page cleaner as early
as possible, we aim to make the throughput more stable.
The function buf_flush_wait_flushed() used to sleep for 10ms, hoping
that the page cleaner thread would do something during that time.
The observed end result was that a large number of threads that call
log_free_check() would end up sleeping while nothing useful is happening.
We will revise the design so that in the default innodb_flush_sync=ON
mode, buf_flush_wait_flushed() will wake up the page cleaner thread
to perform the necessary flushing, and it will wait for a signal from
the page cleaner thread.
If innodb_io_capacity is set to a low value (causing the page cleaner to
throttle its work), a write workload would initially perform well, until
the capacity of the circular ib_logfile0 is reached and log_free_check()
will trigger checkpoints. At that point, the extra waiting in
buf_flush_wait_flushed() will start reducing throughput.
The page cleaner thread will also initiate log checkpoints after each
buf_flush_lists() call, because that is the best point of time for
the checkpoint LSN to advance by the maximum amount.
Even in 'furious flushing' mode we invoke buf_flush_lists() with
innodb_io_capacity_max pages at a time, and at the start of each
batch (in the log_flush() callback function that runs in a separate
task) we will invoke os_aio_wait_until_no_pending_writes(). This
tweak allows the checkpoint to advance in smaller steps and
significantly reduces the maximum latency. On an Intel Optane 960
NVMe SSD on Linux, it reduced from 4.6 seconds to 74 milliseconds.
On Microsoft Windows with a slower SSD, it reduced from more than
180 seconds to 0.6 seconds.
We will make innodb_adaptive_flushing=OFF simply flush innodb_io_capacity
per second whenever the dirty proportion of buffer pool pages exceeds
innodb_max_dirty_pages_pct_lwm. For innodb_adaptive_flushing=ON we try
to make page_cleaner_flush_pages_recommendation() more consistent and
predictable: if we are below innodb_adaptive_flushing_lwm, let us flush
pages according to the return value of af_get_pct_for_dirty().
innodb_max_dirty_pages_pct_lwm: Revert the change of the default value
that was made in MDEV-23399. The value innodb_max_dirty_pages_pct_lwm=0
guarantees that a shutdown of an idle server will be fast. Users might
be surprised if normal shutdown suddenly became slower when upgrading
within a GA release series.
innodb_checkpoint_usec: Remove. The master task will no longer perform
periodic log checkpoints. It is the duty of the page cleaner thread.
log_sys.max_modified_age: Remove. The current span of the
buf_pool.flush_list expressed in LSN only matters for adaptive
flushing (outside the 'furious flushing' condition).
For the correctness of checkpoints, the only thing that matters is
the checkpoint age (log_sys.lsn - log_sys.last_checkpoint_lsn).
This run-time constant was also reported as log_max_modified_age_sync.
log_sys.max_checkpoint_age_async: Remove. This does not serve any
purpose, because the checkpoints will now be triggered by the page
cleaner thread. We will retain the log_sys.max_checkpoint_age limit
for engaging 'furious flushing'.
page_cleaner.slot: Remove. It turns out that
page_cleaner_slot.flush_list_time was duplicating
page_cleaner.slot.flush_time and page_cleaner.slot.flush_list_pass
was duplicating page_cleaner.flush_pass.
Likewise, there were some redundant monitor counters, because the
page cleaner thread no longer performs any buf_pool.LRU flushing, and
because there only is one buf_flush_page_cleaner thread.
buf_flush_sync_lsn: Protect writes by buf_pool.flush_list_mutex.
buf_pool_t::get_oldest_modification(): Add a parameter to specify the
return value when no persistent data pages are dirty. Require the
caller to hold buf_pool.flush_list_mutex.
log_buf_pool_get_oldest_modification(): Take the fall-back LSN
as a parameter. All callers will also invoke log_sys.get_lsn().
log_preflush_pool_modified_pages(): Replaced with buf_flush_wait_flushed().
buf_flush_wait_flushed(): Implement two limits. If not enough buffer pool
has been flushed, signal the page cleaner (unless innodb_flush_sync=OFF)
and wait for the page cleaner to complete. If the page cleaner
thread is not running (which can be the case durign shutdown),
initiate the flush and wait for it directly.
buf_flush_ahead(): If innodb_flush_sync=ON (the default),
submit a new buf_flush_sync_lsn target for the page cleaner
but do not wait for the flushing to finish.
log_get_capacity(), log_get_max_modified_age_async(): Remove, to make
it easier to see that af_get_pct_for_lsn() is not acquiring any mutexes.
page_cleaner_flush_pages_recommendation(): Protect all access to
buf_pool.flush_list with buf_pool.flush_list_mutex. Previously there
were some race conditions in the calculation.
buf_flush_sync_for_checkpoint(): New function to process
buf_flush_sync_lsn in the page cleaner thread. At the end of
each batch, we try to wake up any blocked buf_flush_wait_flushed().
If everything up to buf_flush_sync_lsn has been flushed, we will
reset buf_flush_sync_lsn=0. The page cleaner thread will keep
'furious flushing' until the limit is reached. Any threads that
are waiting in buf_flush_wait_flushed() will be able to resume
as soon as their own limit has been satisfied.
buf_flush_page_cleaner: Prioritize buf_flush_sync_lsn and do not
sleep as long as it is set. Do not update any page_cleaner statistics
for this special mode of operation. In the normal mode
(buf_flush_sync_lsn is not set for innodb_flush_sync=ON),
try to wake up once per second. No longer check whether
srv_inc_activity_count() has been called. After each batch,
try to perform a log checkpoint, because the best chances for
the checkpoint LSN to advance by the maximum amount are upon
completing a flushing batch.
log_t: Move buf_free, max_buf_free possibly to the same cache line
with log_sys.mutex.
log_margin_checkpoint_age(): Simplify the logic, and replace
a 0.1-second sleep with a call to buf_flush_wait_flushed() to
initiate flushing. Moved to the same compilation unit
with the only caller.
log_close(): Clean up the calculations. (Should be no functional
change.) Return whether flush-ahead is needed. Moved to the same
compilation unit with the only caller.
mtr_t::finish_write(): Return whether flush-ahead is needed.
mtr_t::commit(): Invoke buf_flush_ahead() when needed. Let us avoid
external calls in mtr_t::commit() and make the logic easier to follow
by having related code in a single compilation unit. Also, we will
invoke srv_stats.log_write_requests.inc() only once per
mini-transaction commit, while not holding mutexes.
log_checkpoint_margin(): Only care about log_sys.max_checkpoint_age.
Upon reaching log_sys.max_checkpoint_age where we must wait to prevent
the log from getting corrupted, let us wait for at most 1MiB of LSN
at a time, before rechecking the condition. This should allow writers
to proceed even if the redo log capacity has been reached and
'furious flushing' is in progress. We no longer care about
log_sys.max_modified_age_sync or log_sys.max_modified_age_async.
The log_sys.max_modified_age_sync could be a relic from the time when
there was a srv_master_thread that wrote dirty pages to data files.
Also, we no longer have any log_sys.max_checkpoint_age_async limit,
because log checkpoints will now be triggered by the page cleaner
thread upon completing buf_flush_lists().
log_set_capacity(): Simplify the calculations of the limit
(no functional change).
log_checkpoint_low(): Split from log_checkpoint(). Moved to the
same compilation unit with the caller.
log_make_checkpoint(): Only wait for everything to be flushed until
the current LSN.
create_log_file(): After checkpoint, invoke log_write_up_to()
to ensure that the FILE_CHECKPOINT record has been written.
This avoids ut_ad(!srv_log_file_created) in create_log_file_rename().
srv_start(): Do not call recv_recovery_from_checkpoint_start()
if the log has just been created. Set fil_system.space_id_reuse_warned
before dict_boot() has been executed, and clear it after recovery
has finished.
dict_boot(): Initialize fil_system.max_assigned_id.
srv_check_activity(): Remove. The activity count is counting transaction
commits and therefore mostly interesting for the purge of history.
BtrBulk::insert(): Do not explicitly wake up the page cleaner,
but do invoke srv_inc_activity_count(), because that counter is
still being used in buf_load_throttle_if_needed() for some
heuristics. (It might be cleaner to execute buf_load() in the
page cleaner thread!)
Reviewed by: Vladislav Vaintroub
2020-10-26 16:35:47 +02:00
|
|
|
log_write_up_to(LSN_MAX, true);
|
2019-05-03 20:02:11 +03:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
return DB_SUCCESS;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2017-01-30 17:00:51 +02:00
|
|
|
/** Rename the first redo log file.
|
|
|
|
@param[in] lsn FIL_PAGE_FILE_FLUSH_LSN value
|
|
|
|
@param[in,out] logfile0 name of the first log file
|
|
|
|
@return error code
|
|
|
|
@retval DB_SUCCESS on successful operation */
|
2020-11-25 11:28:26 +02:00
|
|
|
MY_ATTRIBUTE((warn_unused_result))
|
2020-01-12 02:05:28 +07:00
|
|
|
static dberr_t create_log_file_rename(lsn_t lsn, std::string &logfile0)
|
2014-02-26 19:11:54 +01:00
|
|
|
{
|
2020-01-12 02:05:28 +07:00
|
|
|
ut_ad(!srv_log_file_created);
|
|
|
|
ut_d(srv_log_file_created= true);
|
2017-06-28 11:58:43 +03:00
|
|
|
|
2019-11-25 22:32:24 +07:00
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_9", return (DB_ERROR););
|
|
|
|
DBUG_PRINT("ib_log", ("After innodb_log_abort_9"));
|
2017-01-30 17:00:51 +02:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
/* Rename the first log file, now that a log checkpoint has been created. */
|
|
|
|
auto new_name = get_log_file_path();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
ib::info() << "Renaming log file " << logfile0 << " to " << new_name;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-10-26 16:43:52 +02:00
|
|
|
mysql_mutex_lock(&log_sys.mutex);
|
2020-01-12 02:05:28 +07:00
|
|
|
ut_ad(logfile0.size() == 2 + new_name.size());
|
|
|
|
logfile0= new_name;
|
|
|
|
dberr_t err= log_sys.log.rename(std::move(new_name));
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-10-26 16:43:52 +02:00
|
|
|
mysql_mutex_unlock(&log_sys.mutex);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2019-11-25 22:32:24 +07:00
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_10", err= DB_ERROR;);
|
2017-01-30 17:00:51 +02:00
|
|
|
|
2019-11-25 22:32:24 +07:00
|
|
|
if (err == DB_SUCCESS)
|
2020-01-12 02:05:28 +07:00
|
|
|
ib::info() << "New log file created, LSN=" << lsn;
|
2017-01-30 17:00:51 +02:00
|
|
|
|
2020-02-01 23:54:57 +08:00
|
|
|
return err;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2019-12-04 15:00:57 +02:00
|
|
|
/** Create an undo tablespace file
|
|
|
|
@param[in] name file name
|
2016-08-12 11:17:45 +03:00
|
|
|
@return DB_SUCCESS or error code */
|
2019-12-04 15:00:57 +02:00
|
|
|
static dberr_t srv_undo_tablespace_create(const char* name)
|
2014-02-26 19:11:54 +01:00
|
|
|
{
|
2017-05-15 17:17:16 +03:00
|
|
|
pfs_os_file_t fh;
|
2016-08-12 11:17:45 +03:00
|
|
|
bool ret;
|
|
|
|
dberr_t err = DB_SUCCESS;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
os_file_create_subdirs_if_needed(name);
|
|
|
|
|
|
|
|
fh = os_file_create(
|
|
|
|
innodb_data_file_key,
|
|
|
|
name,
|
|
|
|
srv_read_only_mode ? OS_FILE_OPEN : OS_FILE_CREATE,
|
|
|
|
OS_FILE_NORMAL, OS_DATA_FILE, srv_read_only_mode, &ret);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2019-11-13 18:14:44 +01:00
|
|
|
if (!ret) {
|
2016-08-12 11:17:45 +03:00
|
|
|
if (os_file_get_last_error(false) != OS_FILE_ALREADY_EXISTS
|
|
|
|
#ifdef UNIV_AIX
|
|
|
|
/* AIX 5.1 after security patch ML7 may have
|
|
|
|
errno set to 0 here, which causes our function
|
|
|
|
to return 100; work around that AIX problem */
|
|
|
|
&& os_file_get_last_error(false) != 100
|
|
|
|
#endif /* UNIV_AIX */
|
|
|
|
) {
|
|
|
|
ib::error() << "Can't create UNDO tablespace "
|
|
|
|
<< name;
|
|
|
|
}
|
|
|
|
err = DB_ERROR;
|
2019-11-13 18:14:44 +01:00
|
|
|
} else if (srv_read_only_mode) {
|
|
|
|
ib::info() << name << " opened in read-only mode";
|
2016-08-12 11:17:45 +03:00
|
|
|
} else {
|
|
|
|
/* We created the data file and now write it full of zeros */
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Data file " << name << " did not exist: new to"
|
|
|
|
" be created";
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Setting file " << name << " size to "
|
2019-12-04 15:00:57 +02:00
|
|
|
<< (SRV_UNDO_TABLESPACE_SIZE_IN_PAGES >> (20 - srv_page_size_shift)) << " MB";
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Database physically writes the file full: "
|
|
|
|
<< "wait...";
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2019-12-04 15:00:57 +02:00
|
|
|
if (!os_file_set_size(name, fh, os_offset_t
|
|
|
|
{SRV_UNDO_TABLESPACE_SIZE_IN_PAGES}
|
|
|
|
<< srv_page_size_shift)) {
|
|
|
|
ib::error() << "Unable to allocate " << name;
|
2016-08-12 11:17:45 +03:00
|
|
|
err = DB_ERROR;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
os_file_close(fh);
|
|
|
|
}
|
|
|
|
|
|
|
|
return(err);
|
|
|
|
}
|
2018-11-27 14:49:20 +02:00
|
|
|
|
2019-12-04 15:00:57 +02:00
|
|
|
/* Validate the number of undo opened undo tablespace and user given
|
|
|
|
undo tablespace
|
|
|
|
@return DB_SUCCESS if it is valid */
|
|
|
|
static dberr_t srv_validate_undo_tablespaces()
|
2016-08-12 11:17:45 +03:00
|
|
|
{
|
2019-12-04 15:00:57 +02:00
|
|
|
/* If the user says that there are fewer than what we find we
|
|
|
|
tolerate that discrepancy but not the inverse. Because there could
|
|
|
|
be unused undo tablespaces for future use. */
|
|
|
|
|
|
|
|
if (srv_undo_tablespaces > srv_undo_tablespaces_open)
|
|
|
|
{
|
|
|
|
ib::error() << "Expected to open innodb_undo_tablespaces="
|
|
|
|
<< srv_undo_tablespaces
|
|
|
|
<< " but was able to find only "
|
|
|
|
<< srv_undo_tablespaces_open;
|
|
|
|
|
|
|
|
return DB_ERROR;
|
|
|
|
}
|
|
|
|
else if (srv_undo_tablespaces_open > 0)
|
|
|
|
{
|
|
|
|
ib::info() << "Opened " << srv_undo_tablespaces_open
|
|
|
|
<< " undo tablespaces";
|
|
|
|
|
|
|
|
if (srv_undo_tablespaces == 0)
|
|
|
|
ib::warn() << "innodb_undo_tablespaces=0 disables"
|
|
|
|
" dedicated undo log tablespaces";
|
|
|
|
}
|
|
|
|
return DB_SUCCESS;
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2019-12-04 15:00:57 +02:00
|
|
|
/** @return the number of active undo tablespaces (except system tablespace) */
|
|
|
|
static ulint trx_rseg_get_n_undo_tablespaces()
|
|
|
|
{
|
|
|
|
std::set<uint32_t> space_ids;
|
|
|
|
mtr_t mtr;
|
|
|
|
mtr.start();
|
|
|
|
|
|
|
|
if (const buf_block_t *sys_header= trx_sysf_get(&mtr, false))
|
|
|
|
for (ulint rseg_id= 0; rseg_id < TRX_SYS_N_RSEGS; rseg_id++)
|
|
|
|
if (trx_sysf_rseg_get_page_no(sys_header, rseg_id) != FIL_NULL)
|
|
|
|
if (uint32_t space= trx_sysf_rseg_get_space(sys_header, rseg_id))
|
|
|
|
space_ids.insert(space);
|
|
|
|
mtr.commit();
|
|
|
|
return space_ids.size();
|
|
|
|
}
|
2018-11-27 14:49:20 +02:00
|
|
|
|
2019-12-04 15:00:57 +02:00
|
|
|
/** Open an undo tablespace.
|
|
|
|
@param[in] create whether undo tablespaces are being created
|
|
|
|
@param[in] name tablespace file name
|
|
|
|
@param[in] i undo tablespace count
|
|
|
|
@return undo tablespace identifier
|
|
|
|
@retval 0 on failure */
|
|
|
|
static ulint srv_undo_tablespace_open(bool create, const char* name, ulint i)
|
|
|
|
{
|
|
|
|
bool success;
|
|
|
|
char undo_name[sizeof "innodb_undo000"];
|
|
|
|
ulint space_id= 0;
|
|
|
|
ulint fsp_flags= 0;
|
|
|
|
|
|
|
|
if (create)
|
|
|
|
{
|
|
|
|
space_id= srv_undo_space_id_start + i;
|
|
|
|
snprintf(undo_name, sizeof(undo_name),
|
|
|
|
"innodb_undo%03u", static_cast<unsigned>(space_id));
|
|
|
|
switch (srv_checksum_algorithm) {
|
|
|
|
case SRV_CHECKSUM_ALGORITHM_FULL_CRC32:
|
|
|
|
case SRV_CHECKSUM_ALGORITHM_STRICT_FULL_CRC32:
|
|
|
|
fsp_flags= FSP_FLAGS_FCRC32_MASK_MARKER | FSP_FLAGS_FCRC32_PAGE_SSIZE();
|
|
|
|
break;
|
|
|
|
default:
|
|
|
|
fsp_flags= FSP_FLAGS_PAGE_SSIZE();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-12-04 20:01:04 +02:00
|
|
|
pfs_os_file_t fh= os_file_create(innodb_data_file_key, name, OS_FILE_OPEN |
|
|
|
|
OS_FILE_ON_ERROR_NO_EXIT |
|
|
|
|
OS_FILE_ON_ERROR_SILENT,
|
|
|
|
OS_FILE_AIO, OS_DATA_FILE,
|
|
|
|
srv_read_only_mode, &success);
|
2019-12-04 15:00:57 +02:00
|
|
|
|
|
|
|
if (!success)
|
|
|
|
return 0;
|
|
|
|
|
|
|
|
os_offset_t size= os_file_get_size(fh);
|
|
|
|
ut_a(size != os_offset_t(-1));
|
|
|
|
|
|
|
|
if (!create)
|
|
|
|
{
|
|
|
|
page_t *page= static_cast<byte*>(aligned_malloc(srv_page_size,
|
|
|
|
srv_page_size));
|
|
|
|
dberr_t err= os_file_read(IORequestRead, fh, page, 0, srv_page_size);
|
|
|
|
if (err != DB_SUCCESS)
|
|
|
|
{
|
|
|
|
err_exit:
|
|
|
|
ib::error() << "Unable to read first page of file " << name;
|
|
|
|
aligned_free(page);
|
|
|
|
return err;
|
|
|
|
}
|
|
|
|
|
2019-12-04 20:01:04 +02:00
|
|
|
uint32_t id= mach_read_from_4(FIL_PAGE_SPACE_ID + page);
|
2019-11-25 22:32:24 +07:00
|
|
|
if (id == 0 || id >= SRV_SPACE_ID_UPPER_BOUND ||
|
2020-01-21 22:22:48 +08:00
|
|
|
memcmp_aligned<2>(FIL_PAGE_SPACE_ID + page,
|
2019-12-04 20:01:04 +02:00
|
|
|
FSP_HEADER_OFFSET + FSP_SPACE_ID + page, 4))
|
|
|
|
{
|
|
|
|
ib::error() << "Inconsistent tablespace ID in file " << name;
|
|
|
|
err= DB_CORRUPTION;
|
|
|
|
goto err_exit;
|
|
|
|
}
|
|
|
|
|
|
|
|
fsp_flags= mach_read_from_4(FSP_HEADER_OFFSET + FSP_SPACE_FLAGS + page);
|
|
|
|
if (buf_page_is_corrupted(false, page, fsp_flags))
|
2019-12-04 15:00:57 +02:00
|
|
|
{
|
2019-12-04 20:01:04 +02:00
|
|
|
ib::error() << "Checksum mismatch in the first page of file " << name;
|
2019-12-04 15:00:57 +02:00
|
|
|
err= DB_CORRUPTION;
|
|
|
|
goto err_exit;
|
|
|
|
}
|
|
|
|
|
|
|
|
space_id= id;
|
|
|
|
snprintf(undo_name, sizeof undo_name, "innodb_undo%03u", id);
|
|
|
|
aligned_free(page);
|
|
|
|
}
|
|
|
|
|
|
|
|
/* Load the tablespace into InnoDB's internal data structures. */
|
|
|
|
|
|
|
|
/* We set the biggest space id to the undo tablespace
|
|
|
|
because InnoDB hasn't opened any other tablespace apart
|
|
|
|
from the system tablespace. */
|
|
|
|
|
|
|
|
fil_set_max_space_id_if_bigger(space_id);
|
|
|
|
|
2020-10-26 15:59:30 +02:00
|
|
|
fil_space_t *space= fil_space_t::create(undo_name, space_id, fsp_flags,
|
|
|
|
FIL_TYPE_TABLESPACE, NULL);
|
2019-12-04 15:00:57 +02:00
|
|
|
ut_a(fil_validate());
|
|
|
|
ut_a(space);
|
|
|
|
|
|
|
|
fil_node_t *file= space->add(name, fh, 0, false, true);
|
2020-12-04 19:02:58 +02:00
|
|
|
mysql_mutex_lock(&fil_system.mutex);
|
2019-12-04 15:00:57 +02:00
|
|
|
|
|
|
|
if (create)
|
|
|
|
{
|
2020-10-26 15:59:30 +02:00
|
|
|
space->set_sizes(SRV_UNDO_TABLESPACE_SIZE_IN_PAGES);
|
2020-10-15 16:28:19 +03:00
|
|
|
space->size= file->size= uint32_t(size >> srv_page_size_shift);
|
2019-12-04 15:00:57 +02:00
|
|
|
}
|
2020-10-26 15:59:30 +02:00
|
|
|
else if (!file->read_page0())
|
2019-12-04 15:00:57 +02:00
|
|
|
{
|
2020-10-26 15:59:30 +02:00
|
|
|
os_file_close(file->handle);
|
|
|
|
file->handle= OS_FILE_CLOSED;
|
|
|
|
ut_a(fil_system.n_open > 0);
|
|
|
|
fil_system.n_open--;
|
2019-12-04 15:00:57 +02:00
|
|
|
}
|
|
|
|
|
2020-12-04 19:02:58 +02:00
|
|
|
mysql_mutex_unlock(&fil_system.mutex);
|
2019-12-04 15:00:57 +02:00
|
|
|
return space_id;
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
2015-06-16 10:57:05 +02:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
/** Check if undo tablespaces and redo log files exist before creating a
|
|
|
|
new system tablespace
|
|
|
|
@retval DB_SUCCESS if all undo and redo logs are not found
|
|
|
|
@retval DB_ERROR if any undo and redo logs are found */
|
|
|
|
static
|
|
|
|
dberr_t
|
|
|
|
srv_check_undo_redo_logs_exists()
|
|
|
|
{
|
|
|
|
bool ret;
|
2018-04-15 15:29:55 +01:00
|
|
|
pfs_os_file_t fh;
|
2016-08-12 11:17:45 +03:00
|
|
|
char name[OS_FILE_MAX_PATH];
|
2015-06-16 10:57:05 +02:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
/* Check if any undo tablespaces exist */
|
|
|
|
for (ulint i = 1; i <= srv_undo_tablespaces; ++i) {
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2017-11-11 23:07:24 +02:00
|
|
|
snprintf(
|
2016-08-12 11:17:45 +03:00
|
|
|
name, sizeof(name),
|
2017-04-21 05:51:27 +03:00
|
|
|
"%s%cundo%03zu",
|
2016-08-12 11:17:45 +03:00
|
|
|
srv_undo_dir, OS_PATH_SEPARATOR,
|
|
|
|
i);
|
|
|
|
|
|
|
|
fh = os_file_create(
|
|
|
|
innodb_data_file_key, name,
|
|
|
|
OS_FILE_OPEN_RETRY
|
|
|
|
| OS_FILE_ON_ERROR_NO_EXIT
|
|
|
|
| OS_FILE_ON_ERROR_SILENT,
|
|
|
|
OS_FILE_NORMAL,
|
|
|
|
OS_DATA_FILE,
|
|
|
|
srv_read_only_mode,
|
|
|
|
&ret);
|
|
|
|
|
|
|
|
if (ret) {
|
|
|
|
os_file_close(fh);
|
|
|
|
ib::error()
|
|
|
|
<< "undo tablespace '" << name << "' exists."
|
|
|
|
" Creating system tablespace with existing undo"
|
|
|
|
" tablespaces is not supported. Please delete"
|
|
|
|
" all undo tablespaces before creating new"
|
|
|
|
" system tablespace.";
|
|
|
|
return(DB_ERROR);
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
/* Check if redo log file exists */
|
|
|
|
auto logfilename = get_log_file_path();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
fh = os_file_create(innodb_log_file_key, logfilename.c_str(),
|
|
|
|
OS_FILE_OPEN_RETRY | OS_FILE_ON_ERROR_NO_EXIT
|
|
|
|
| OS_FILE_ON_ERROR_SILENT,
|
|
|
|
OS_FILE_NORMAL, OS_LOG_FILE, srv_read_only_mode,
|
|
|
|
&ret);
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
if (ret) {
|
|
|
|
os_file_close(fh);
|
|
|
|
ib::error() << "redo log file '" << logfilename
|
|
|
|
<< "' exists. Creating system tablespace with"
|
|
|
|
" existing redo log file is not recommended."
|
|
|
|
" Please delete redo log file before"
|
|
|
|
" creating new system tablespace.";
|
|
|
|
return DB_ERROR;
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
return(DB_SUCCESS);
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2019-12-04 15:00:57 +02:00
|
|
|
static dberr_t srv_all_undo_tablespaces_open(bool create_new_db, ulint n_undo)
|
|
|
|
{
|
|
|
|
/* Open all the undo tablespaces that are currently in use. If we
|
|
|
|
fail to open any of these it is a fatal error. The tablespace ids
|
|
|
|
should be contiguous. It is a fatal error because they are required
|
|
|
|
for recovery and are referenced by the UNDO logs (a.k.a RBS). */
|
|
|
|
|
|
|
|
ulint prev_id= create_new_db ? srv_undo_space_id_start - 1 : 0;
|
|
|
|
|
|
|
|
for (ulint i= 0; i < n_undo; ++i)
|
|
|
|
{
|
|
|
|
char name[OS_FILE_MAX_PATH];
|
|
|
|
snprintf(name, sizeof name, "%s%cundo%03zu", srv_undo_dir,
|
|
|
|
OS_PATH_SEPARATOR, i + 1);
|
|
|
|
ulint space_id= srv_undo_tablespace_open(create_new_db, name, i);
|
|
|
|
if (!space_id)
|
|
|
|
{
|
|
|
|
if (!create_new_db)
|
|
|
|
break;
|
|
|
|
ib::error() << "Unable to open create tablespace '" << name << "'.";
|
|
|
|
return DB_ERROR;
|
|
|
|
}
|
|
|
|
|
|
|
|
/* Should be no gaps in undo tablespace ids. */
|
|
|
|
ut_a(!i || prev_id + 1 == space_id);
|
|
|
|
|
|
|
|
prev_id= space_id;
|
|
|
|
|
|
|
|
/* Note the first undo tablespace id in case of
|
|
|
|
no active undo tablespace. */
|
|
|
|
if (0 == srv_undo_tablespaces_open++)
|
|
|
|
srv_undo_space_id_start= space_id;
|
|
|
|
}
|
|
|
|
|
|
|
|
/* Open any extra unused undo tablespaces. These must be contiguous.
|
|
|
|
We stop at the first failure. These are undo tablespaces that are
|
|
|
|
not in use and therefore not required by recovery. We only check
|
|
|
|
that there are no gaps. */
|
|
|
|
|
|
|
|
for (ulint i= prev_id + 1; i < srv_undo_space_id_start + TRX_SYS_N_RSEGS;
|
|
|
|
++i)
|
|
|
|
{
|
|
|
|
char name[OS_FILE_MAX_PATH];
|
|
|
|
snprintf(name, sizeof(name),
|
|
|
|
"%s%cundo%03zu", srv_undo_dir, OS_PATH_SEPARATOR, i);
|
|
|
|
if (!srv_undo_tablespace_open(create_new_db, name, i))
|
|
|
|
break;
|
|
|
|
++srv_undo_tablespaces_open;
|
|
|
|
}
|
|
|
|
|
|
|
|
return srv_validate_undo_tablespaces();
|
|
|
|
}
|
|
|
|
|
MDEV-12289 Keep 128 persistent rollback segments for compatibility and performance
InnoDB divides the allocation of undo logs into rollback segments.
The DB_ROLL_PTR system column of clustered indexes can address up to
128 rollback segments (TRX_SYS_N_RSEGS). Originally, InnoDB only
created one rollback segment. In MySQL 5.5 or in the InnoDB Plugin
for MySQL 5.1, all 128 rollback segments were created.
MySQL 5.7 hard-codes the rollback segment IDs 1..32 for temporary undo logs.
On upgrade, unless a slow shutdown (innodb_fast_shutdown=0)
was performed on the old server instance, these rollback segments
could be in use by transactions that are in XA PREPARE state or
transactions that were left behind by a server kill followed by a
normal shutdown immediately after restart.
Persistent tables cannot refer to temporary undo logs or vice versa.
Therefore, we should keep two distinct sets of rollback segments:
one for persistent tables and another for temporary tables. In this way,
all 128 rollback segments will be available for both types of tables,
which could improve performance. Also, MariaDB 10.2 will remain more
compatible than MySQL 5.7 with data files from earlier versions of
MySQL or MariaDB.
trx_sys_t::temp_rsegs[TRX_SYS_N_RSEGS]: A new array of temporary
rollback segments. The trx_sys_t::rseg_array[TRX_SYS_N_RSEGS] will
be solely for persistent undo logs.
srv_tmp_undo_logs. Remove. Use the constant TRX_SYS_N_RSEGS.
srv_available_undo_logs: Change the type to ulong.
trx_rseg_get_on_id(): Remove. Instead, let the callers refer to
trx_sys directly.
trx_rseg_create(), trx_sysf_rseg_find_free(): Remove unneeded parameters.
These functions only deal with persistent undo logs.
trx_temp_rseg_create(): New function, to create all temporary rollback
segments at server startup.
trx_rseg_t::is_persistent(): Determine if the rollback segment is for
persistent tables.
trx_sys_is_noredo_rseg_slot(): Remove. The callers must know based on
context (such as table handle) whether the DB_ROLL_PTR is referring to
a persistent undo log.
trx_sys_create_rsegs(): Remove all parameters, which were always passed
as global variables. Instead, modify the global variables directly.
enum trx_rseg_type_t: Remove.
trx_t::get_temp_rseg(): A method to ensure that a temporary
rollback segment has been assigned for the transaction.
trx_t::assign_temp_rseg(): Replaces trx_assign_rseg().
trx_purge_free_segment(), trx_purge_truncate_rseg_history():
Remove the redundant variable noredo=false.
Temporary undo logs are discarded immediately at transaction commit
or rollback, not lazily by purge.
trx_purge_mark_undo_for_truncate(): Remove references to the
temporary rollback segments.
trx_purge_mark_undo_for_truncate(): Remove a check for temporary
rollback segments. Only the dedicated persistent undo log tablespaces
can be truncated.
trx_undo_get_undo_rec_low(), trx_undo_get_undo_rec(): Add the
parameter is_temp.
trx_rseg_mem_restore(): Split from trx_rseg_mem_create().
Initialize the undo log and the rollback segment from the file
data structures.
trx_sysf_get_n_rseg_slots(): Renamed from
trx_sysf_used_slots_for_redo_rseg(). Count the persistent
rollback segment headers that have been initialized.
trx_sys_close(): Also free trx_sys->temp_rsegs[].
get_next_redo_rseg(): Merged to trx_assign_rseg_low().
trx_assign_rseg_low(): Remove the parameters and access the
global variables directly. Revert to simple round-robin, now that
the whole trx_sys->rseg_array[] is for persistent undo log again.
get_next_noredo_rseg(): Moved to trx_t::assign_temp_rseg().
srv_undo_tablespaces_init(): Remove some parameters and use the
global variables directly. Clarify some error messages.
Adjust the test innodb.log_file. Apparently, before these changes,
InnoDB somehow ignored missing dedicated undo tablespace files that
are pointed by the TRX_SYS header page, possibly losing part of
essential transaction system state.
2017-03-30 13:11:34 +03:00
|
|
|
/** Open the configured number of dedicated undo tablespaces.
|
|
|
|
@param[in] create_new_db whether the database is being initialized
|
2016-08-12 11:17:45 +03:00
|
|
|
@return DB_SUCCESS or error code */
|
2014-02-26 19:11:54 +01:00
|
|
|
dberr_t
|
MDEV-12289 Keep 128 persistent rollback segments for compatibility and performance
InnoDB divides the allocation of undo logs into rollback segments.
The DB_ROLL_PTR system column of clustered indexes can address up to
128 rollback segments (TRX_SYS_N_RSEGS). Originally, InnoDB only
created one rollback segment. In MySQL 5.5 or in the InnoDB Plugin
for MySQL 5.1, all 128 rollback segments were created.
MySQL 5.7 hard-codes the rollback segment IDs 1..32 for temporary undo logs.
On upgrade, unless a slow shutdown (innodb_fast_shutdown=0)
was performed on the old server instance, these rollback segments
could be in use by transactions that are in XA PREPARE state or
transactions that were left behind by a server kill followed by a
normal shutdown immediately after restart.
Persistent tables cannot refer to temporary undo logs or vice versa.
Therefore, we should keep two distinct sets of rollback segments:
one for persistent tables and another for temporary tables. In this way,
all 128 rollback segments will be available for both types of tables,
which could improve performance. Also, MariaDB 10.2 will remain more
compatible than MySQL 5.7 with data files from earlier versions of
MySQL or MariaDB.
trx_sys_t::temp_rsegs[TRX_SYS_N_RSEGS]: A new array of temporary
rollback segments. The trx_sys_t::rseg_array[TRX_SYS_N_RSEGS] will
be solely for persistent undo logs.
srv_tmp_undo_logs. Remove. Use the constant TRX_SYS_N_RSEGS.
srv_available_undo_logs: Change the type to ulong.
trx_rseg_get_on_id(): Remove. Instead, let the callers refer to
trx_sys directly.
trx_rseg_create(), trx_sysf_rseg_find_free(): Remove unneeded parameters.
These functions only deal with persistent undo logs.
trx_temp_rseg_create(): New function, to create all temporary rollback
segments at server startup.
trx_rseg_t::is_persistent(): Determine if the rollback segment is for
persistent tables.
trx_sys_is_noredo_rseg_slot(): Remove. The callers must know based on
context (such as table handle) whether the DB_ROLL_PTR is referring to
a persistent undo log.
trx_sys_create_rsegs(): Remove all parameters, which were always passed
as global variables. Instead, modify the global variables directly.
enum trx_rseg_type_t: Remove.
trx_t::get_temp_rseg(): A method to ensure that a temporary
rollback segment has been assigned for the transaction.
trx_t::assign_temp_rseg(): Replaces trx_assign_rseg().
trx_purge_free_segment(), trx_purge_truncate_rseg_history():
Remove the redundant variable noredo=false.
Temporary undo logs are discarded immediately at transaction commit
or rollback, not lazily by purge.
trx_purge_mark_undo_for_truncate(): Remove references to the
temporary rollback segments.
trx_purge_mark_undo_for_truncate(): Remove a check for temporary
rollback segments. Only the dedicated persistent undo log tablespaces
can be truncated.
trx_undo_get_undo_rec_low(), trx_undo_get_undo_rec(): Add the
parameter is_temp.
trx_rseg_mem_restore(): Split from trx_rseg_mem_create().
Initialize the undo log and the rollback segment from the file
data structures.
trx_sysf_get_n_rseg_slots(): Renamed from
trx_sysf_used_slots_for_redo_rseg(). Count the persistent
rollback segment headers that have been initialized.
trx_sys_close(): Also free trx_sys->temp_rsegs[].
get_next_redo_rseg(): Merged to trx_assign_rseg_low().
trx_assign_rseg_low(): Remove the parameters and access the
global variables directly. Revert to simple round-robin, now that
the whole trx_sys->rseg_array[] is for persistent undo log again.
get_next_noredo_rseg(): Moved to trx_t::assign_temp_rseg().
srv_undo_tablespaces_init(): Remove some parameters and use the
global variables directly. Clarify some error messages.
Adjust the test innodb.log_file. Apparently, before these changes,
InnoDB somehow ignored missing dedicated undo tablespace files that
are pointed by the TRX_SYS header page, possibly losing part of
essential transaction system state.
2017-03-30 13:11:34 +03:00
|
|
|
srv_undo_tablespaces_init(bool create_new_db)
|
2014-02-26 19:11:54 +01:00
|
|
|
{
|
2019-12-04 15:00:57 +02:00
|
|
|
srv_undo_tablespaces_open= 0;
|
|
|
|
|
|
|
|
ut_a(srv_undo_tablespaces <= TRX_SYS_N_RSEGS);
|
|
|
|
ut_a(!create_new_db || srv_operation == SRV_OPERATION_NORMAL);
|
|
|
|
|
|
|
|
if (srv_undo_tablespaces == 1)
|
|
|
|
srv_undo_tablespaces= 0;
|
|
|
|
|
|
|
|
/* Create the undo spaces only if we are creating a new
|
|
|
|
instance. We don't allow creating of new undo tablespaces
|
|
|
|
in an existing instance (yet). */
|
|
|
|
if (create_new_db)
|
|
|
|
{
|
|
|
|
srv_undo_space_id_start= 1;
|
|
|
|
DBUG_EXECUTE_IF("innodb_undo_upgrade", srv_undo_space_id_start= 3;);
|
|
|
|
|
|
|
|
for (ulint i= 0; i < srv_undo_tablespaces; ++i)
|
|
|
|
{
|
|
|
|
char name[OS_FILE_MAX_PATH];
|
|
|
|
snprintf(name, sizeof name, "%s%cundo%03zu",
|
|
|
|
srv_undo_dir, OS_PATH_SEPARATOR, i + 1);
|
|
|
|
if (dberr_t err= srv_undo_tablespace_create(name))
|
|
|
|
{
|
|
|
|
ib::error() << "Could not create undo tablespace '" << name << "'.";
|
|
|
|
return err;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/* Get the tablespace ids of all the undo segments excluding
|
|
|
|
the system tablespace (0). If we are creating a new instance then
|
|
|
|
we build the undo_tablespace_ids ourselves since they don't
|
|
|
|
already exist. */
|
|
|
|
srv_undo_tablespaces_active= srv_undo_tablespaces;
|
|
|
|
|
|
|
|
ulint n_undo= (create_new_db || srv_operation == SRV_OPERATION_BACKUP ||
|
|
|
|
srv_operation == SRV_OPERATION_RESTORE_DELTA)
|
|
|
|
? srv_undo_tablespaces : TRX_SYS_N_RSEGS;
|
|
|
|
|
|
|
|
if (dberr_t err= srv_all_undo_tablespaces_open(create_new_db, n_undo))
|
|
|
|
return err;
|
|
|
|
|
|
|
|
/* Initialize srv_undo_space_id_start=0 when there are no
|
|
|
|
dedicated undo tablespaces. */
|
|
|
|
if (srv_undo_tablespaces_open == 0)
|
|
|
|
srv_undo_space_id_start= 0;
|
|
|
|
|
|
|
|
if (create_new_db)
|
|
|
|
{
|
|
|
|
mtr_t mtr;
|
|
|
|
for (ulint i= 0; i < srv_undo_tablespaces; ++i)
|
|
|
|
{
|
|
|
|
mtr.start();
|
|
|
|
fsp_header_init(fil_space_get(srv_undo_space_id_start + i),
|
|
|
|
SRV_UNDO_TABLESPACE_SIZE_IN_PAGES, &mtr);
|
|
|
|
mtr.commit();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return DB_SUCCESS;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
/** Create the temporary file tablespace.
|
|
|
|
@param[in] create_new_db whether we are creating a new database
|
|
|
|
@return DB_SUCCESS or error code. */
|
|
|
|
static
|
|
|
|
dberr_t
|
2016-12-16 16:36:54 +02:00
|
|
|
srv_open_tmp_tablespace(bool create_new_db)
|
2016-08-12 11:17:45 +03:00
|
|
|
{
|
|
|
|
ulint sum_of_new_sizes;
|
|
|
|
|
|
|
|
/* Will try to remove if there is existing file left-over by last
|
|
|
|
unclean shutdown */
|
2016-12-16 16:36:54 +02:00
|
|
|
srv_tmp_space.set_sanity_check_status(true);
|
|
|
|
srv_tmp_space.delete_files();
|
|
|
|
srv_tmp_space.set_ignore_read_only(true);
|
2016-08-12 11:17:45 +03:00
|
|
|
|
|
|
|
ib::info() << "Creating shared tablespace for temporary tables";
|
|
|
|
|
|
|
|
bool create_new_temp_space;
|
|
|
|
|
2016-12-16 16:36:54 +02:00
|
|
|
srv_tmp_space.set_space_id(SRV_TMP_SPACE_ID);
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2016-12-16 16:36:54 +02:00
|
|
|
dberr_t err = srv_tmp_space.check_file_spec(
|
|
|
|
&create_new_temp_space, 12 * 1024 * 1024);
|
2016-08-12 11:17:45 +03:00
|
|
|
|
|
|
|
if (err == DB_FAIL) {
|
2018-03-28 09:29:14 +03:00
|
|
|
ib::error() << "The innodb_temporary"
|
|
|
|
" data file must be writable!";
|
2016-08-12 11:17:45 +03:00
|
|
|
err = DB_ERROR;
|
|
|
|
} else if (err != DB_SUCCESS) {
|
2018-03-28 09:29:14 +03:00
|
|
|
ib::error() << "Could not create the shared innodb_temporary.";
|
2016-12-16 16:36:54 +02:00
|
|
|
} else if ((err = srv_tmp_space.open_or_create(
|
2016-08-12 11:17:45 +03:00
|
|
|
true, create_new_db, &sum_of_new_sizes, NULL))
|
|
|
|
!= DB_SUCCESS) {
|
2018-03-28 09:29:14 +03:00
|
|
|
ib::error() << "Unable to create the shared innodb_temporary";
|
2020-10-26 15:59:30 +02:00
|
|
|
} else if (fil_system.temp_space->open(true)) {
|
2018-03-28 09:29:14 +03:00
|
|
|
/* Initialize the header page */
|
|
|
|
mtr_t mtr;
|
|
|
|
mtr.start();
|
|
|
|
mtr.set_log_mode(MTR_LOG_NO_REDO);
|
2018-03-22 14:17:43 +02:00
|
|
|
fsp_header_init(fil_system.temp_space,
|
2018-03-28 09:29:14 +03:00
|
|
|
srv_tmp_space.get_sum_of_sizes(),
|
|
|
|
&mtr);
|
|
|
|
mtr.commit();
|
2016-08-12 11:17:45 +03:00
|
|
|
} else {
|
2018-03-28 09:29:14 +03:00
|
|
|
/* This file was just opened in the code above! */
|
|
|
|
ib::error() << "The innodb_temporary"
|
|
|
|
" data file cannot be re-opened"
|
|
|
|
" after check_file_spec() succeeded!";
|
|
|
|
err = DB_ERROR;
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
return(err);
|
|
|
|
}
|
|
|
|
|
2020-10-26 15:04:24 +02:00
|
|
|
/** Shutdown background threads, except the page cleaner. */
|
|
|
|
static void srv_shutdown_threads()
|
2016-08-12 11:17:45 +03:00
|
|
|
{
|
2017-06-29 11:31:01 +03:00
|
|
|
ut_ad(!srv_undo_sources);
|
2016-08-12 11:17:45 +03:00
|
|
|
srv_shutdown_state = SRV_SHUTDOWN_EXIT_THREADS;
|
2020-12-04 16:18:04 +02:00
|
|
|
ut_d(srv_master_thread_enable());
|
2019-10-29 22:37:12 +01:00
|
|
|
srv_master_timer.reset();
|
|
|
|
|
|
|
|
if (purge_sys.enabled()) {
|
|
|
|
srv_purge_shutdown();
|
|
|
|
}
|
|
|
|
|
2020-10-26 15:04:24 +02:00
|
|
|
if (srv_n_fil_crypt_threads) {
|
|
|
|
fil_crypt_set_thread_cnt(0);
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
#ifdef UNIV_DEBUG
|
|
|
|
# define srv_init_abort(_db_err) \
|
|
|
|
srv_init_abort_low(create_new_db, __FILE__, __LINE__, _db_err)
|
|
|
|
#else
|
|
|
|
# define srv_init_abort(_db_err) \
|
|
|
|
srv_init_abort_low(create_new_db, _db_err)
|
|
|
|
#endif /* UNIV_DEBUG */
|
|
|
|
|
|
|
|
/** Innobase start-up aborted. Perform cleanup actions.
|
|
|
|
@param[in] create_new_db TRUE if new db is being created
|
|
|
|
@param[in] file File name
|
|
|
|
@param[in] line Line number
|
|
|
|
@param[in] err Reason for aborting InnoDB startup
|
|
|
|
@return DB_SUCCESS or error code. */
|
2017-01-25 15:11:46 +02:00
|
|
|
MY_ATTRIBUTE((warn_unused_result, nonnull))
|
2016-08-12 11:17:45 +03:00
|
|
|
static
|
|
|
|
dberr_t
|
|
|
|
srv_init_abort_low(
|
|
|
|
bool create_new_db,
|
|
|
|
#ifdef UNIV_DEBUG
|
|
|
|
const char* file,
|
2017-03-01 08:27:39 +02:00
|
|
|
unsigned line,
|
2016-08-12 11:17:45 +03:00
|
|
|
#endif /* UNIV_DEBUG */
|
|
|
|
dberr_t err)
|
|
|
|
{
|
MDEV-15053 Reduce buf_pool_t::mutex contention
User-visible changes: The INFORMATION_SCHEMA views INNODB_BUFFER_PAGE
and INNODB_BUFFER_PAGE_LRU will report a dummy value FLUSH_TYPE=0
and will no longer report the PAGE_STATE value READY_FOR_USE.
We will remove some fields from buf_page_t and move much code to
member functions of buf_pool_t and buf_page_t, so that the access
rules of data members can be enforced consistently.
Evicting or adding pages in buf_pool.LRU will remain covered by
buf_pool.mutex.
Evicting or adding pages in buf_pool.page_hash will remain
covered by both buf_pool.mutex and the buf_pool.page_hash X-latch.
After this fix, buf_pool.page_hash lookups can entirely
avoid acquiring buf_pool.mutex, only relying on
buf_pool.hash_lock_get() S-latch.
Similarly, buf_flush_check_neighbors() can will rely solely on
buf_pool.mutex, no buf_pool.page_hash latch at all.
The buf_pool.mutex is rather contended in I/O heavy benchmarks,
especially when the workload does not fit in the buffer pool.
The first attempt to alleviate the contention was the
buf_pool_t::mutex split in
commit 4ed7082eefe56b3e97e0edefb3df76dd7ef5e858
which introduced buf_block_t::mutex, which we are now removing.
Later, multiple instances of buf_pool_t were introduced
in commit c18084f71b02ea707c6461353e6cfc15d7553bc6
and recently removed by us in
commit 1a6f708ec594ac0ae2dd30db926ab07b100fa24b (MDEV-15058).
UNIV_BUF_DEBUG: Remove. This option to enable some buffer pool
related debugging in otherwise non-debug builds has not been used
for years. Instead, we have been using UNIV_DEBUG, which is enabled
in CMAKE_BUILD_TYPE=Debug.
buf_block_t::mutex, buf_pool_t::zip_mutex: Remove. We can mainly rely on
std::atomic and the buf_pool.page_hash latches, and in some cases
depend on buf_pool.mutex or buf_pool.flush_list_mutex just like before.
We must always release buf_block_t::lock before invoking
unfix() or io_unfix(), to prevent a glitch where a block that was
added to the buf_pool.free list would apper X-latched. See
commit c5883debd6ef440a037011c11873b396923e93c5 how this glitch
was finally caught in a debug environment.
We move some buf_pool_t::page_hash specific code from the
ha and hash modules to buf_pool, for improved readability.
buf_pool_t::close(): Assert that all blocks are clean, except
on aborted startup or crash-like shutdown.
buf_pool_t::validate(): No longer attempt to validate
n_flush[] against the number of BUF_IO_WRITE fixed blocks,
because buf_page_t::flush_type no longer exists.
buf_pool_t::watch_set(): Replaces buf_pool_watch_set().
Reduce mutex contention by separating the buf_pool.watch[]
allocation and the insert into buf_pool.page_hash.
buf_pool_t::page_hash_lock<bool exclusive>(): Acquire a
buf_pool.page_hash latch.
Replaces and extends buf_page_hash_lock_s_confirm()
and buf_page_hash_lock_x_confirm().
buf_pool_t::READ_AHEAD_PAGES: Renamed from BUF_READ_AHEAD_PAGES.
buf_pool_t::curr_size, old_size, read_ahead_area, n_pend_reads:
Use Atomic_counter.
buf_pool_t::running_out(): Replaces buf_LRU_buf_pool_running_out().
buf_pool_t::LRU_remove(): Remove a block from the LRU list
and return its predecessor. Incorporates buf_LRU_adjust_hp(),
which was removed.
buf_page_get_gen(): Remove a redundant call of fsp_is_system_temporary(),
for mode == BUF_GET_IF_IN_POOL_OR_WATCH, which is only used by
BTR_DELETE_OP (purge), which is never invoked on temporary tables.
buf_free_from_unzip_LRU_list_batch(): Avoid redundant assignments.
buf_LRU_free_from_unzip_LRU_list(): Simplify the loop condition.
buf_LRU_free_page(): Clarify the function comment.
buf_flush_check_neighbor(), buf_flush_check_neighbors():
Rewrite the construction of the page hash range. We will hold
the buf_pool.mutex for up to buf_pool.read_ahead_area (at most 64)
consecutive lookups of buf_pool.page_hash.
buf_flush_page_and_try_neighbors(): Remove.
Merge to its only callers, and remove redundant operations in
buf_flush_LRU_list_batch().
buf_read_ahead_random(), buf_read_ahead_linear(): Rewrite.
Do not acquire buf_pool.mutex, and iterate directly with page_id_t.
ut_2_power_up(): Remove. my_round_up_to_next_power() is inlined
and avoids any loops.
fil_page_get_prev(), fil_page_get_next(), fil_addr_is_null(): Remove.
buf_flush_page(): Add a fil_space_t* parameter. Minimize the
buf_pool.mutex hold time. buf_pool.n_flush[] is no longer updated
atomically with the io_fix, and we will protect most buf_block_t
fields with buf_block_t::lock. The function
buf_flush_write_block_low() is removed and merged here.
buf_page_init_for_read(): Use static linkage. Initialize the newly
allocated block and acquire the exclusive buf_block_t::lock while not
holding any mutex.
IORequest::IORequest(): Remove the body. We only need to invoke
set_punch_hole() in buf_flush_page() and nowhere else.
buf_page_t::flush_type: Remove. Replaced by IORequest::flush_type.
This field is only used during a fil_io() call.
That function already takes IORequest as a parameter, so we had
better introduce for the rarely changing field.
buf_block_t::init(): Replaces buf_page_init().
buf_page_t::init(): Replaces buf_page_init_low().
buf_block_t::initialise(): Initialise many fields, but
keep the buf_page_t::state(). Both buf_pool_t::validate() and
buf_page_optimistic_get() requires that buf_page_t::in_file()
be protected atomically with buf_page_t::in_page_hash
and buf_page_t::in_LRU_list.
buf_page_optimistic_get(): Now that buf_block_t::mutex
no longer exists, we must check buf_page_t::io_fix()
after acquiring the buf_pool.page_hash lock, to detect
whether buf_page_init_for_read() has been initiated.
We will also check the io_fix() before acquiring hash_lock
in order to avoid unnecessary computation.
The field buf_block_t::modify_clock (protected by buf_block_t::lock)
allows buf_page_optimistic_get() to validate the block.
buf_page_t::real_size: Remove. It was only used while flushing
pages of page_compressed tables.
buf_page_encrypt(): Add an output parameter that allows us ot eliminate
buf_page_t::real_size. Replace a condition with debug assertion.
buf_page_should_punch_hole(): Remove.
buf_dblwr_t::add_to_batch(): Replaces buf_dblwr_add_to_batch().
Add the parameter size (to replace buf_page_t::real_size).
buf_dblwr_t::write_single_page(): Replaces buf_dblwr_write_single_page().
Add the parameter size (to replace buf_page_t::real_size).
fil_system_t::detach(): Replaces fil_space_detach().
Ensure that fil_validate() will not be violated even if
fil_system.mutex is released and reacquired.
fil_node_t::complete_io(): Renamed from fil_node_complete_io().
fil_node_t::close_to_free(): Replaces fil_node_close_to_free().
Avoid invoking fil_node_t::close() because fil_system.n_open
has already been decremented in fil_space_t::detach().
BUF_BLOCK_READY_FOR_USE: Remove. Directly use BUF_BLOCK_MEMORY.
BUF_BLOCK_ZIP_DIRTY: Remove. Directly use BUF_BLOCK_ZIP_PAGE,
and distinguish dirty pages by buf_page_t::oldest_modification().
BUF_BLOCK_POOL_WATCH: Remove. Use BUF_BLOCK_NOT_USED instead.
This state was only being used for buf_page_t that are in
buf_pool.watch.
buf_pool_t::watch[]: Remove pointer indirection.
buf_page_t::in_flush_list: Remove. It was set if and only if
buf_page_t::oldest_modification() is nonzero.
buf_page_decrypt_after_read(), buf_corrupt_page_release(),
buf_page_check_corrupt(): Change the const fil_space_t* parameter
to const fil_node_t& so that we can report the correct file name.
buf_page_monitor(): Declare as an ATTRIBUTE_COLD global function.
buf_page_io_complete(): Split to buf_page_read_complete() and
buf_page_write_complete().
buf_dblwr_t::in_use: Remove.
buf_dblwr_t::buf_block_array: Add IORequest::flush_t.
buf_dblwr_sync_datafiles(): Remove. It was a useless wrapper of
os_aio_wait_until_no_pending_writes().
buf_flush_write_complete(): Declare static, not global.
Add the parameter IORequest::flush_t.
buf_flush_freed_page(): Simplify the code.
recv_sys_t::flush_lru: Renamed from flush_type and changed to bool.
fil_read(), fil_write(): Replaced with direct use of fil_io().
fil_buffering_disabled(): Remove. Check srv_file_flush_method directly.
fil_mutex_enter_and_prepare_for_io(): Return the resolved
fil_space_t* to avoid a duplicated lookup in the caller.
fil_report_invalid_page_access(): Clean up the parameters.
fil_io(): Return fil_io_t, which comprises fil_node_t and error code.
Always invoke fil_space_t::acquire_for_io() and let either the
sync=true caller or fil_aio_callback() invoke
fil_space_t::release_for_io().
fil_aio_callback(): Rewrite to replace buf_page_io_complete().
fil_check_pending_operations(): Remove a parameter, and remove some
redundant lookups.
fil_node_close_to_free(): Wait for n_pending==0. Because we no longer
do an extra lookup of the tablespace between fil_io() and the
completion of the operation, we must give fil_node_t::complete_io() a
chance to decrement the counter.
fil_close_tablespace(): Remove unused parameter trx, and document
that this is only invoked during the error handling of IMPORT TABLESPACE.
row_import_discard_changes(): Merged with the only caller,
row_import_cleanup(). Do not lock up the data dictionary while
invoking fil_close_tablespace().
logs_empty_and_mark_files_at_shutdown(): Do not invoke
fil_close_all_files(), to avoid a !needs_flush assertion failure
on fil_node_t::close().
innodb_shutdown(): Invoke os_aio_free() before fil_close_all_files().
fil_close_all_files(): Invoke fil_flush_file_spaces()
to ensure proper durability.
thread_pool::unbind(): Fix a crash that would occur on Windows
after srv_thread_pool->disable_aio() and os_file_close().
This fix was submitted by Vladislav Vaintroub.
Thanks to Matthias Leich and Axel Schwenke for extensive testing,
Vladislav Vaintroub for helpful comments, and Eugene Kosov for a review.
2020-06-05 12:35:46 +03:00
|
|
|
ut_ad(srv_is_being_started);
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
if (create_new_db) {
|
2017-06-08 12:45:08 +03:00
|
|
|
ib::error() << "Database creation was aborted"
|
2016-08-12 11:17:45 +03:00
|
|
|
#ifdef UNIV_DEBUG
|
|
|
|
" at " << innobase_basename(file) << "[" << line << "]"
|
|
|
|
#endif /* UNIV_DEBUG */
|
2020-06-04 10:24:10 +03:00
|
|
|
" with error " << err << ". You may need"
|
2016-08-12 11:17:45 +03:00
|
|
|
" to delete the ibdata1 file before trying to start"
|
|
|
|
" up again.";
|
|
|
|
} else {
|
|
|
|
ib::error() << "Plugin initialization aborted"
|
|
|
|
#ifdef UNIV_DEBUG
|
|
|
|
" at " << innobase_basename(file) << "[" << line << "]"
|
|
|
|
#endif /* UNIV_DEBUG */
|
2020-06-04 10:24:10 +03:00
|
|
|
" with error " << err;
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
|
|
|
|
2018-01-30 13:39:48 +02:00
|
|
|
srv_shutdown_bg_undo_sources();
|
2020-10-26 15:04:24 +02:00
|
|
|
srv_shutdown_threads();
|
2016-08-12 11:17:45 +03:00
|
|
|
return(err);
|
|
|
|
}
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
/** Prepare to delete the redo log file. Flush the dirty pages from all the
|
2016-08-12 11:17:45 +03:00
|
|
|
buffer pools. Flush the redo log buffer to the redo log file.
|
2020-01-12 02:05:28 +07:00
|
|
|
@param[in] old_exists old redo log file exists
|
2016-08-12 11:17:45 +03:00
|
|
|
@return lsn upto which data pages have been flushed. */
|
2020-01-12 02:05:28 +07:00
|
|
|
static lsn_t srv_prepare_to_delete_redo_log_file(bool old_exists)
|
2016-08-12 11:17:45 +03:00
|
|
|
{
|
2020-01-12 02:05:28 +07:00
|
|
|
DBUG_ENTER("srv_prepare_to_delete_redo_log_file");
|
2017-02-16 09:18:46 +02:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
lsn_t flushed_lsn;
|
|
|
|
ulint count = 0;
|
|
|
|
|
2018-09-11 21:31:47 +03:00
|
|
|
if (log_sys.log.subformat != 2) {
|
MDEV-14717: Prevent crash-downgrade to earlier MariaDB 10.2
A crash-downgrade of a RENAME (or TRUNCATE or table-rebuilding
ALTER TABLE or OPTIMIZE TABLE) operation to an earlier 10.2 version
would trigger a debug assertion failure during rollback,
in trx_roll_pop_top_rec_of_trx(). In a non-debug build, the
TRX_UNDO_RENAME_TABLE record would be misinterpreted as an
update_undo log record, and typically the file name would be
interpreted as DB_TRX_ID,DB_ROLL_PTR,PRIMARY KEY. If a matching
record would be found, row_undo_mod() would hit ut_error in
switch (node->rec_type). Typically, ut_a(table2 == NULL) would
fail when opening the table from SQL.
Because of this, we prevent a crash-downgrade to earlier MariaDB 10.2
versions by changing the InnoDB redo log format identifier to the
10.3 identifier, and by introducing a subformat identifier so that
10.2 can continue to refuse crash-downgrade from 10.3 or later.
After a clean shutdown, a downgrade to MariaDB 10.2.13 or later would
still be possible thanks to MDEV-14909. A downgrade to older 10.2
versions is only possible after removing the log files (not recommended).
LOG_HEADER_FORMAT_CURRENT: Change to 103 (originally the 10.3 format).
log_group_t: Add subformat. For 10.2, we will use subformat 1,
and will refuse crash recovery from any other subformat of the
10.3 format, that is, a genuine 10.3 redo log.
recv_find_max_checkpoint(): Allow startup after clean shutdown
from a future LOG_HEADER_FORMAT_10_4 (unencrypted only).
We cannot handle the encrypted 10.4 redo log block format,
which was introduced in MDEV-12041. Allow crash recovery from
the original 10.2 format as well as the new format.
In Mariabackup --backup, do not allow any startup from 10.3 or 10.4
redo logs.
recv_recovery_from_checkpoint_start(): Skip redo log apply for
clean 10.3 redo log, but not for the new 10.2 redo log
(10.3 format, subformat 1).
srv_prepare_to_delete_redo_log_files(): On format or subformat
mismatch, set srv_log_file_size = 0, so that we will display the
correct message.
innobase_start_or_create_for_mysql(): Check for format or subformat
mismatch.
xtrabackup_backup_func(): Remove debug assertions that were made
redundant by the code changes in recv_find_max_checkpoint().
2018-09-07 17:24:31 +03:00
|
|
|
srv_log_file_size = 0;
|
|
|
|
}
|
|
|
|
|
2020-06-08 13:14:27 +03:00
|
|
|
for (;;) {
|
2016-08-12 11:17:45 +03:00
|
|
|
/* Clean the buffer pool. */
|
2020-02-12 14:45:21 +02:00
|
|
|
buf_flush_sync();
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2017-02-16 09:18:46 +02:00
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_1", DBUG_RETURN(0););
|
|
|
|
DBUG_PRINT("ib_log", ("After innodb_log_abort_1"));
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2020-10-26 16:43:52 +02:00
|
|
|
mysql_mutex_lock(&log_sys.mutex);
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2020-03-05 11:45:28 +02:00
|
|
|
fil_names_clear(log_sys.get_lsn(), false);
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2020-03-05 11:45:28 +02:00
|
|
|
flushed_lsn = log_sys.get_lsn();
|
2016-08-12 11:17:45 +03:00
|
|
|
|
|
|
|
{
|
2017-02-10 12:11:42 +02:00
|
|
|
ib::info info;
|
2018-01-03 15:22:52 +02:00
|
|
|
if (srv_log_file_size == 0
|
2019-10-28 14:54:28 +02:00
|
|
|
|| (log_sys.log.format & ~log_t::FORMAT_ENCRYPTED)
|
MDEV-12353: Change the redo log encoding
log_t::FORMAT_10_5: physical redo log format tag
log_phys_t: Buffered records in the physical format.
The log record bytes will follow the last data field,
making use of alignment padding that would otherwise be wasted.
If there are multiple records for the same page, also those
may be appended to an existing log_phys_t object if the memory
is available.
In the physical format, the first byte of a record identifies the
record and its length (up to 15 bytes). For longer records, the
immediately following bytes will encode the remaining length
in a variable-length encoding. Usually, a variable-length-encoded
page identifier will follow, followed by optional payload, whose
length is included in the initially encoded total record length.
When a mini-transaction is updating multiple fields in a page,
it can avoid repeating the tablespace identifier and page number
by setting the same_page flag (most significant bit) in the first
byte of the log record. The byte offset of the record will be
relative to where the previous record for that page ended.
Until MDEV-14425 introduces a separate file-level log for
redo log checkpoints and file operations, we will write the
file-level records in the page-level redo log file.
The record FILE_CHECKPOINT (which replaces MLOG_CHECKPOINT)
will be removed in MDEV-14425, and one sequential scan of the
page recovery log will suffice.
Compared to MLOG_FILE_CREATE2, FILE_CREATE will not include any flags.
If the information is needed, it can be parsed from WRITE records that
modify FSP_SPACE_FLAGS.
MLOG_ZIP_WRITE_STRING: Remove. The record was only introduced temporarily
as part of this work, before being replaced with WRITE (along with
MLOG_WRITE_STRING, MLOG_1BYTE, MLOG_nBYTES).
mtr_buf_t::empty(): Check if the buffer is empty.
mtr_t::m_n_log_recs: Remove. It suffices to check if m_log is empty.
mtr_t::m_last, mtr_t::m_last_offset: End of the latest m_log record,
for the same_page encoding.
page_recv_t::last_offset: Reflects mtr_t::m_last_offset.
Valid values for last_offset during recovery should be 0 or above 8.
(The first 8 bytes of a page are the checksum and the page number,
and neither are ever updated directly by log records.)
Internally, the special value 1 indicates that the same_page form
will not be allowed for the subsequent record.
mtr_t::page_create(): Take the block descriptor as parameter,
so that it can be compared to mtr_t::m_last. The INIT_INDEX_PAGE
record will always followed by a subtype byte, because same_page
records must be longer than 1 byte.
trx_undo_page_init(): Combine the writes in WRITE record.
trx_undo_header_create(): Write 4 bytes using a special MEMSET
record that includes 1 bytes of length and 2 bytes of payload.
flst_write_addr(): Define as a static function. Combine the writes.
flst_zero_both(): Replaces two flst_zero_addr() calls.
flst_init(): Do not inline the function.
fsp_free_seg_inode(): Zerofill the whole inode.
fsp_apply_init_file_page(): Initialize FIL_PAGE_PREV,FIL_PAGE_NEXT
to FIL_NULL when using the physical format.
btr_create(): Assert !page_has_siblings() because fsp_apply_init_file_page()
must have been invoked.
fil_ibd_create(): Do not write FILE_MODIFY after FILE_CREATE.
fil_names_dirty_and_write(): Remove the parameter mtr.
Write the records using a separate mini-transaction object,
because any FILE_ records must be at the start of a mini-transaction log.
recv_recover_page(): Add a fil_space_t* parameter.
After applying log to the a ROW_FORMAT=COMPRESSED page,
invoke buf_zip_decompress() to restore the uncompressed page.
buf_page_io_complete(): Remove the temporary hack to discard the
uncompressed page of a ROW_FORMAT=COMPRESSED page.
page_zip_write_header(): Remove. Use mtr_t::write() or
mtr_t::memset() instead, and update the compressed page frame
separately.
trx_undo_header_add_space_for_xid(): Remove.
trx_undo_seg_create(): Perform the changes that were previously
made by trx_undo_header_add_space_for_xid().
btr_reset_instant(): New function: Reset the table to MariaDB 10.2
or 10.3 format when rolling back an instant ALTER TABLE operation.
page_rec_find_owner_rec(): Merge with the only callers.
page_cur_insert_rec_low(): Combine writes by using a local buffer.
MEMMOVE data from the preceding record whenever feasible
(copying at least 3 bytes).
page_cur_insert_rec_zip(): Combine writes to page header fields.
PageBulk::insertPage(): Issue MEMMOVE records to copy a matching
part from the preceding record.
PageBulk::finishPage(): Combine the writes to the page header
and to the sparse page directory slots.
mtr_t::write(): Only log the least significant (last) bytes
of multi-byte fields that actually differ.
For updating FSP_SIZE, we must always write all 4 bytes to the
redo log, so that the fil_space_set_recv_size() logic in
recv_sys_t::parse() will work.
mtr_t::memcpy(), mtr_t::zmemcpy(): Take a pointer argument
instead of a numeric offset to the page frame. Only log the
last bytes of multi-byte fields that actually differ.
In fil_space_crypt_t::write_page0(), we must log also any
unchanged bytes, so that recovery will recognize the record
and invoke fil_crypt_parse().
Future work:
MDEV-21724 Optimize page_cur_insert_rec_low() redo logging
MDEV-21725 Optimize btr_page_reorganize_low() redo logging
MDEV-21727 Optimize redo logging for ROW_FORMAT=COMPRESSED
2020-02-13 19:12:17 +02:00
|
|
|
!= log_t::FORMAT_10_5) {
|
2017-02-10 12:11:42 +02:00
|
|
|
info << "Upgrading redo log: ";
|
2020-01-12 02:05:28 +07:00
|
|
|
} else if (!old_exists
|
2017-02-10 12:11:42 +02:00
|
|
|
|| srv_log_file_size
|
|
|
|
!= srv_log_file_size_requested) {
|
|
|
|
if (srv_encrypt_log
|
2018-04-27 10:06:14 +03:00
|
|
|
== (my_bool)log_sys.is_encrypted()) {
|
2017-02-10 12:11:42 +02:00
|
|
|
info << (srv_encrypt_log
|
|
|
|
? "Resizing encrypted"
|
|
|
|
: "Resizing");
|
|
|
|
} else if (srv_encrypt_log) {
|
|
|
|
info << "Encrypting and resizing";
|
|
|
|
} else {
|
|
|
|
info << "Removing encryption"
|
|
|
|
" and resizing";
|
|
|
|
}
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
info << " redo log from " << srv_log_file_size
|
|
|
|
<< " to ";
|
2017-02-10 12:11:42 +02:00
|
|
|
} else if (srv_encrypt_log) {
|
|
|
|
info << "Encrypting redo log: ";
|
2016-08-12 11:17:45 +03:00
|
|
|
} else {
|
2017-02-10 12:11:42 +02:00
|
|
|
info << "Removing redo log encryption: ";
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
2017-02-10 12:11:42 +02:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
info << srv_log_file_size_requested
|
Reduce the granularity of innodb_log_file_size
In Mariabackup, we would want the backed-up redo log file size to be
a multiple of 512 bytes, or OS_FILE_LOG_BLOCK_SIZE. However, at startup,
InnoDB would be picky, requiring the file size to be a multiple of
innodb_page_size.
Furthermore, InnoDB would require the parameter to be a multiple of
one megabyte, while the minimum granularity is 512 bytes. Because
the data-file-oriented fil_io() API is being used for writing the
InnoDB redo log, writes will for now require innodb_log_file_size to
be a multiple of the maximum innodb_page_size (65536 bytes).
To complicate matters, InnoDB startup divided srv_log_file_size by
UNIV_PAGE_SIZE, so that initially, the unit was bytes, and later it
was innodb_page_size. We will simplify this and keep srv_log_file_size
in bytes at all times.
innobase_log_file_size: Remove. Remove some obsolete checks against
overflow on 32-bit systems. srv_log_file_size is always 64 bits, and
the maximum size 512GiB in multiples of innodb_page_size always fits
in ulint (which is 32 or 64 bits). 512GiB would be 8,388,608*64KiB or
134,217,728*4KiB.
log_init(): Remove the parameter file_size that was always passed as
srv_log_file_size.
log_set_capacity(): Add a parameter for passing the requested file size.
srv_log_file_size_requested: Declare static in srv0start.cc.
create_log_file(), create_log_files(),
innobase_start_or_create_for_mysql(): Invoke fil_node_create()
with srv_log_file_size expressed in multiples of innodb_page_size.
innobase_start_or_create_for_mysql(): Require the redo log file sizes
to be multiples of 512 bytes.
2017-06-05 22:47:20 +03:00
|
|
|
<< " bytes; LSN=" << flushed_lsn;
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
|
|
|
|
2020-10-26 16:43:52 +02:00
|
|
|
mysql_mutex_unlock(&log_sys.mutex);
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2020-03-05 11:45:28 +02:00
|
|
|
if (flushed_lsn != log_sys.get_flushed_lsn()) {
|
2020-02-07 22:12:35 +01:00
|
|
|
log_write_up_to(flushed_lsn, false);
|
2020-04-07 12:22:57 +03:00
|
|
|
log_sys.log.flush();
|
2020-02-07 22:12:35 +01:00
|
|
|
}
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2021-03-30 09:58:24 +03:00
|
|
|
ut_ad(flushed_lsn == log_sys.get_lsn());
|
2016-08-12 11:17:45 +03:00
|
|
|
|
|
|
|
/* Check if the buffer pools are clean. If not
|
|
|
|
retry till it is clean. */
|
2020-06-08 13:14:27 +03:00
|
|
|
if (ulint pending_io = buf_pool.io_pending()) {
|
2016-08-12 11:17:45 +03:00
|
|
|
count++;
|
|
|
|
/* Print a message every 60 seconds if we
|
|
|
|
are waiting to clean the buffer pools */
|
|
|
|
if (srv_print_verbose_log && count > 600) {
|
|
|
|
ib::info() << "Waiting for "
|
|
|
|
<< pending_io << " buffer "
|
|
|
|
<< "page I/Os to complete";
|
|
|
|
count = 0;
|
|
|
|
}
|
2020-06-08 13:14:27 +03:00
|
|
|
|
2021-03-16 16:09:41 +03:00
|
|
|
std::this_thread::sleep_for(
|
|
|
|
std::chrono::milliseconds(100));
|
2020-06-08 13:14:27 +03:00
|
|
|
continue;
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
|
|
|
|
2020-06-08 13:14:27 +03:00
|
|
|
break;
|
|
|
|
}
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2017-02-16 09:18:46 +02:00
|
|
|
DBUG_RETURN(flushed_lsn);
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
/** Tries to locate LOG_FILE_NAME and check it's size, etc
|
|
|
|
@param[out] log_file_found returns true here if correct file was found
|
|
|
|
@return dberr_t with DB_SUCCESS or some error */
|
|
|
|
static dberr_t find_and_check_log_file(bool &log_file_found)
|
|
|
|
{
|
|
|
|
log_file_found= false;
|
|
|
|
|
|
|
|
auto logfile0= get_log_file_path();
|
|
|
|
os_file_stat_t stat_info;
|
|
|
|
const dberr_t err= os_file_get_status(logfile0.c_str(), &stat_info, false,
|
|
|
|
srv_read_only_mode);
|
|
|
|
|
|
|
|
auto is_operation_restore= []() -> bool {
|
|
|
|
return srv_operation == SRV_OPERATION_RESTORE ||
|
|
|
|
srv_operation == SRV_OPERATION_RESTORE_EXPORT;
|
|
|
|
};
|
|
|
|
|
|
|
|
if (err == DB_NOT_FOUND)
|
|
|
|
{
|
|
|
|
if (is_operation_restore())
|
|
|
|
return DB_NOT_FOUND;
|
|
|
|
|
|
|
|
return DB_SUCCESS;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (stat_info.type != OS_FILE_TYPE_FILE)
|
|
|
|
return DB_SUCCESS;
|
|
|
|
|
|
|
|
if (!srv_file_check_mode(logfile0.c_str()))
|
|
|
|
return DB_ERROR;
|
|
|
|
|
|
|
|
const os_offset_t size= stat_info.size;
|
|
|
|
ut_a(size != (os_offset_t) -1);
|
|
|
|
|
|
|
|
if (size % OS_FILE_LOG_BLOCK_SIZE)
|
|
|
|
{
|
|
|
|
ib::error() << "Log file " << logfile0 << " size " << size
|
|
|
|
<< " is not a multiple of " << OS_FILE_LOG_BLOCK_SIZE
|
|
|
|
<< " bytes";
|
|
|
|
return DB_ERROR;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (size == 0 && is_operation_restore())
|
|
|
|
{
|
|
|
|
/* Tolerate an empty LOG_FILE_NAME from a previous run of
|
|
|
|
mariabackup --prepare. */
|
|
|
|
return DB_NOT_FOUND;
|
|
|
|
}
|
|
|
|
/* The first log file must consist of at least the following 512-byte pages:
|
|
|
|
header, checkpoint page 1, empty, checkpoint page 2, redo log page(s).
|
|
|
|
|
|
|
|
Mariabackup --prepare would create an empty LOG_FILE_NAME. Tolerate it. */
|
|
|
|
if (size != 0 && size <= OS_FILE_LOG_BLOCK_SIZE * 4)
|
|
|
|
{
|
|
|
|
ib::error() << "Log file " << logfile0 << " size " << size
|
|
|
|
<< " is too small";
|
|
|
|
return DB_ERROR;
|
|
|
|
}
|
|
|
|
srv_log_file_size= size;
|
|
|
|
|
|
|
|
log_file_found= true;
|
|
|
|
return DB_SUCCESS;
|
|
|
|
}
|
|
|
|
|
2021-03-17 10:03:06 +03:00
|
|
|
static tpool::task_group rollback_all_recovered_group(1);
|
|
|
|
static tpool::task rollback_all_recovered_task(trx_rollback_all_recovered,
|
|
|
|
nullptr,
|
|
|
|
&rollback_all_recovered_group);
|
|
|
|
|
2018-04-29 09:41:42 +03:00
|
|
|
/** Start InnoDB.
|
|
|
|
@param[in] create_new_db whether to create a new database
|
2016-08-12 11:17:45 +03:00
|
|
|
@return DB_SUCCESS or error code */
|
2018-04-29 09:41:42 +03:00
|
|
|
dberr_t srv_start(bool create_new_db)
|
2014-02-26 19:11:54 +01:00
|
|
|
{
|
2016-08-12 11:17:45 +03:00
|
|
|
lsn_t flushed_lsn;
|
2017-01-31 19:43:03 +02:00
|
|
|
dberr_t err = DB_SUCCESS;
|
2020-01-12 02:05:28 +07:00
|
|
|
bool srv_log_file_found = true;
|
2014-02-26 19:11:54 +01:00
|
|
|
mtr_t mtr;
|
|
|
|
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
ut_ad(srv_operation == SRV_OPERATION_NORMAL
|
2017-10-10 10:28:54 +03:00
|
|
|
|| srv_operation == SRV_OPERATION_RESTORE
|
|
|
|
|| srv_operation == SRV_OPERATION_RESTORE_EXPORT);
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
|
2017-02-01 16:03:57 +05:30
|
|
|
if (srv_force_recovery == SRV_FORCE_NO_LOG_REDO) {
|
|
|
|
srv_read_only_mode = true;
|
|
|
|
}
|
|
|
|
|
2015-10-09 17:21:46 +02:00
|
|
|
high_level_read_only = srv_read_only_mode
|
MDEV-19514 Defer change buffer merge until pages are requested
We will remove the InnoDB background operation of merging buffered
changes to secondary index leaf pages. Changes will only be merged as a
result of an operation that accesses a secondary index leaf page,
such as a SQL statement that performs a lookup via that index,
or is modifying the index. Also ROLLBACK and some background operations,
such as purging the history of committed transactions, or computing
index cardinality statistics, can cause change buffer merge.
Encryption key rotation will not perform change buffer merge.
The motivation of this change is to simplify the I/O logic and to
allow crash recovery to happen in the background (MDEV-14481).
We also hope that this will reduce the number of "mystery" crashes
due to corrupted data. Because change buffer merge will typically
take place as a result of executing SQL statements, there should be
a clearer connection between the crash and the SQL statements that
were executed when the server crashed.
In many cases, a slight performance improvement was observed.
This is joint work with Thirunarayanan Balathandayuthapani
and was tested by Axel Schwenke and Matthias Leich.
The InnoDB monitor counter innodb_ibuf_merge_usec will be removed.
On slow shutdown (innodb_fast_shutdown=0), we will continue to
merge all buffered changes (and purge all undo log history).
Two InnoDB configuration parameters will be changed as follows:
innodb_disable_background_merge: Removed.
This parameter existed only in debug builds.
All change buffer merges will use synchronous reads.
innodb_force_recovery will be changed as follows:
* innodb_force_recovery=4 will be the same as innodb_force_recovery=3
(the change buffer merge cannot be disabled; it can only happen as
a result of an operation that accesses a secondary index leaf page).
The option used to be capable of corrupting secondary index leaf pages.
Now that capability is removed, and innodb_force_recovery=4 becomes 'safe'.
* innodb_force_recovery=5 (which essentially hard-wires
SET GLOBAL TRANSACTION ISOLATION LEVEL READ UNCOMMITTED)
becomes safe to use. Bogus data can be returned to SQL, but
persistent InnoDB data files will not be corrupted further.
* innodb_force_recovery=6 (ignore the redo log files)
will be the only option that can potentially cause
persistent corruption of InnoDB data files.
Code changes:
buf_page_t::ibuf_exist: New flag, to indicate whether buffered
changes exist for a buffer pool page. Pages with pending changes
can be returned by buf_page_get_gen(). Previously, the changes
were always merged inside buf_page_get_gen() if needed.
ibuf_page_exists(const buf_page_t&): Check if a buffered changes
exist for an X-latched or read-fixed page.
buf_page_get_gen(): Add the parameter allow_ibuf_merge=false.
All callers that know that they may be accessing a secondary index
leaf page must pass this parameter as allow_ibuf_merge=true,
unless it does not matter for that caller whether all buffered
changes have been applied. Assert that whenever allow_ibuf_merge
holds, the page actually is a leaf page. Attempt change buffer
merge only to secondary B-tree index leaf pages.
btr_block_get(): Add parameter 'bool merge'.
All callers of btr_block_get() should know whether the page could be
a secondary index leaf page. If it is not, we should avoid consulting
the change buffer bitmap to even consider a merge. This is the main
interface to requesting index pages from the buffer pool.
ibuf_merge_or_delete_for_page(), recv_recover_page(): Replace
buf_page_get_known_nowait() with much simpler logic, because
it is now guaranteed that that the block is x-latched or read-fixed.
mlog_init_t::mark_ibuf_exist(): Renamed from mlog_init_t::ibuf_merge().
On crash recovery, we will no longer merge any buffered changes
for the pages that we read into the buffer pool during the last batch
of applying log records.
buf_page_get_gen_known_nowait(), BUF_MAKE_YOUNG, BUF_KEEP_OLD: Remove.
btr_search_guess_on_hash(): Merge buf_page_get_gen_known_nowait()
to its only remaining caller.
buf_page_make_young_if_needed(): Define as an inline function.
Add the parameter buf_pool.
buf_page_peek_if_young(), buf_page_peek_if_too_old(): Add the
parameter buf_pool.
fil_space_validate_for_mtr_commit(): Remove a bogus comment
about background merge of the change buffer.
btr_cur_open_at_rnd_pos_func(), btr_cur_search_to_nth_level_func(),
btr_cur_open_at_index_side_func(): Use narrower data types and scopes.
ibuf_read_merge_pages(): Replaces buf_read_ibuf_merge_pages().
Merge the change buffer by invoking buf_page_get_gen().
2019-10-11 17:28:15 +03:00
|
|
|
|| srv_force_recovery > SRV_FORCE_NO_IBUF_MERGE
|
2018-03-29 12:55:24 +03:00
|
|
|
|| srv_sys_space.created_new_raw();
|
2014-02-26 19:23:04 +01:00
|
|
|
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
srv_started_redo = false;
|
2017-02-10 17:01:45 +01:00
|
|
|
|
2017-01-26 14:05:00 +02:00
|
|
|
compile_time_assert(sizeof(ulint) == sizeof(void*));
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
#ifdef UNIV_DEBUG
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "!!!!!!!! UNIV_DEBUG switched on !!!!!!!!!";
|
2014-02-26 19:11:54 +01:00
|
|
|
#endif
|
|
|
|
|
|
|
|
#ifdef UNIV_IBUF_DEBUG
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "!!!!!!!! UNIV_IBUF_DEBUG switched on !!!!!!!!!";
|
2014-02-26 19:11:54 +01:00
|
|
|
#endif
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Compressed tables use zlib " ZLIB_VERSION
|
2014-02-26 19:11:54 +01:00
|
|
|
#ifdef UNIV_ZIP_DEBUG
|
|
|
|
" with validation"
|
|
|
|
#endif /* UNIV_ZIP_DEBUG */
|
2016-08-12 11:17:45 +03:00
|
|
|
;
|
2014-02-26 19:11:54 +01:00
|
|
|
#ifdef UNIV_ZIP_COPY
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "and extra copying";
|
2014-02-26 19:11:54 +01:00
|
|
|
#endif /* UNIV_ZIP_COPY */
|
|
|
|
|
|
|
|
/* Since InnoDB does not currently clean up all its internal data
|
|
|
|
structures in MySQL Embedded Server Library server_end(), we
|
|
|
|
print an error message if someone tries to start up InnoDB a
|
2016-08-12 11:17:45 +03:00
|
|
|
second time during the process lifetime. */
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
if (srv_start_has_been_called) {
|
|
|
|
ib::error() << "Startup called second time"
|
|
|
|
" during the process lifetime."
|
|
|
|
" In the MySQL Embedded Server Library"
|
|
|
|
" you cannot call server_init() more than"
|
|
|
|
" once during the process lifetime.";
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2017-04-25 09:26:01 +03:00
|
|
|
srv_start_has_been_called = true;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
srv_is_being_started = true;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
/* Register performance schema stages before any real work has been
|
|
|
|
started which may need to be instrumented. */
|
MDEV-21907: InnoDB: Enable -Wconversion on clang and GCC
The -Wconversion in GCC seems to be stricter than in clang.
GCC at least since version 4.4.7 issues truncation warnings for
assignments to bitfields, while clang 10 appears to only issue
warnings when the sizes in bytes rounded to the nearest integer
powers of 2 are different.
Before GCC 10.0.0, -Wconversion required more casts and would not
allow some operations, such as x<<=1 or x+=1 on a data type that
is narrower than int.
GCC 5 (but not GCC 4, GCC 6, or any later version) is complaining
about x|=y even when x and y are compatible types that are narrower
than int. Hence, we must rewrite some x|=y as
x=static_cast<byte>(x|y) or similar, or we must disable -Wconversion.
In GCC 6 and later, the warning for assigning wider to bitfields
that are narrower than 8, 16, or 32 bits can be suppressed by
applying a bitwise & with the exact bitmask of the bitfield.
For older GCC, we must disable -Wconversion for GCC 4 or 5 in such
cases.
The bitwise negation operator appears to promote short integers
to a wider type, and hence we must add explicit truncation casts
around them. Microsoft Visual C does not allow a static_cast to
truncate a constant, such as static_cast<byte>(1) truncating int.
Hence, we will use the constructor-style cast byte(~1) for such cases.
This has been tested at least with GCC 4.8.5, 5.4.0, 7.4.0, 9.2.1, 10.0.0,
clang 9.0.1, 10.0.0, and MSVC 14.22.27905 (Microsoft Visual Studio 2019)
on 64-bit and 32-bit targets (IA-32, AMD64, POWER 8, POWER 9, ARMv8).
2020-03-12 19:46:41 +02:00
|
|
|
mysql_stage_register("innodb", srv_stages,
|
|
|
|
static_cast<int>(UT_ARR_SIZE(srv_stages)));
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-12-04 18:07:25 +02:00
|
|
|
srv_max_n_threads =
|
|
|
|
1 /* dict_stats_thread */
|
|
|
|
+ 1 /* fts_optimize_thread */
|
|
|
|
+ 128 /* safety margin */
|
|
|
|
+ max_connections;
|
2014-02-26 19:23:04 +01:00
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
srv_boot();
|
|
|
|
|
2020-09-17 16:07:37 +02:00
|
|
|
ib::info() << my_crc32c_implementation();
|
2016-12-01 12:35:59 +11:00
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
if (!srv_read_only_mode) {
|
2020-12-04 19:02:58 +02:00
|
|
|
mysql_mutex_init(srv_monitor_file_mutex_key,
|
|
|
|
&srv_monitor_file_mutex, nullptr);
|
|
|
|
mysql_mutex_init(srv_misc_tmpfile_mutex_key,
|
|
|
|
&srv_misc_tmpfile_mutex, nullptr);
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-12-04 19:02:58 +02:00
|
|
|
if (!srv_read_only_mode) {
|
2014-02-26 19:11:54 +01:00
|
|
|
if (srv_innodb_status) {
|
|
|
|
|
|
|
|
srv_monitor_file_name = static_cast<char*>(
|
2016-08-12 11:17:45 +03:00
|
|
|
ut_malloc_nokey(
|
2014-02-26 19:11:54 +01:00
|
|
|
strlen(fil_path_to_mysql_datadir)
|
|
|
|
+ 20 + sizeof "/innodb_status."));
|
|
|
|
|
2016-09-06 09:43:16 +03:00
|
|
|
sprintf(srv_monitor_file_name,
|
|
|
|
"%s/innodb_status." ULINTPF,
|
2014-02-26 19:11:54 +01:00
|
|
|
fil_path_to_mysql_datadir,
|
2020-09-03 16:40:28 +03:00
|
|
|
static_cast<ulint>
|
|
|
|
(IF_WIN(GetCurrentProcessId(), getpid())));
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2019-05-23 12:55:03 +04:00
|
|
|
srv_monitor_file = my_fopen(srv_monitor_file_name,
|
|
|
|
O_RDWR|O_TRUNC|O_CREAT,
|
|
|
|
MYF(MY_WME));
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
if (!srv_monitor_file) {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::error() << "Unable to create "
|
|
|
|
<< srv_monitor_file_name << ": "
|
|
|
|
<< strerror(errno);
|
2017-03-08 22:36:10 +02:00
|
|
|
if (err == DB_SUCCESS) {
|
|
|
|
err = DB_ERROR;
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
} else {
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
srv_monitor_file_name = NULL;
|
2018-03-15 13:47:28 +11:00
|
|
|
srv_monitor_file = os_file_create_tmpfile();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2017-03-08 22:36:10 +02:00
|
|
|
if (!srv_monitor_file && err == DB_SUCCESS) {
|
|
|
|
err = DB_ERROR;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-03-15 13:47:28 +11:00
|
|
|
srv_misc_tmpfile = os_file_create_tmpfile();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2017-03-08 22:36:10 +02:00
|
|
|
if (!srv_misc_tmpfile && err == DB_SUCCESS) {
|
|
|
|
err = DB_ERROR;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-03-08 22:36:10 +02:00
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return(srv_init_abort(err));
|
|
|
|
}
|
|
|
|
|
2020-12-14 15:27:03 +02:00
|
|
|
srv_n_file_io_threads = srv_n_read_io_threads + srv_n_write_io_threads;
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
if (!srv_read_only_mode) {
|
|
|
|
/* Add the log and ibuf IO threads. */
|
|
|
|
srv_n_file_io_threads += 2;
|
|
|
|
} else {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Disabling background log and ibuf IO write"
|
|
|
|
<< " threads.";
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
ut_a(srv_n_file_io_threads <= SRV_MAX_N_IO_THREADS);
|
|
|
|
|
2020-12-14 15:27:03 +02:00
|
|
|
if (os_aio_init()) {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::error() << "Cannot initialize AIO sub-system";
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
return(srv_init_abort(DB_ERROR));
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2020-12-14 15:27:03 +02:00
|
|
|
#ifdef LINUX_NATIVE_AIO
|
|
|
|
if (srv_use_native_aio) {
|
|
|
|
ib::info() << "Using Linux native AIO";
|
|
|
|
}
|
|
|
|
#endif
|
2021-03-15 11:30:17 +02:00
|
|
|
#ifdef HAVE_URING
|
|
|
|
if (srv_use_native_aio) {
|
|
|
|
ib::info() << "Using liburing";
|
|
|
|
}
|
|
|
|
#endif
|
2020-12-14 15:27:03 +02:00
|
|
|
|
2018-03-28 09:00:06 +03:00
|
|
|
fil_system.create(srv_file_per_table ? 50000 : 5000);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Initializing buffer pool, total size = "
|
2020-02-12 14:45:21 +02:00
|
|
|
<< srv_buf_pool_size
|
|
|
|
<< ", chunk size = " << srv_buf_pool_chunk_unit;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-03-18 21:48:00 +02:00
|
|
|
if (buf_pool.create()) {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::error() << "Cannot allocate memory for the buffer pool";
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
return(srv_init_abort(DB_ERROR));
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Completed initialization of buffer pool";
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
#ifdef UNIV_DEBUG
|
|
|
|
/* We have observed deadlocks with a 5MB buffer pool but
|
|
|
|
the actual lower limit could very well be a little higher. */
|
|
|
|
|
|
|
|
if (srv_buf_pool_size <= 5 * 1024 * 1024) {
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Small buffer pool size ("
|
|
|
|
<< srv_buf_pool_size / 1024 / 1024
|
|
|
|
<< "M), the flst_validate() debug function can cause a"
|
|
|
|
<< " deadlock if the buffer pool fills up.";
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
#endif /* UNIV_DEBUG */
|
|
|
|
|
2018-04-27 10:06:14 +03:00
|
|
|
log_sys.create();
|
2019-05-24 16:19:38 +03:00
|
|
|
recv_sys.create();
|
2018-02-22 20:46:42 +04:00
|
|
|
lock_sys.create(srv_lock_table_size);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
|
2016-12-05 21:04:30 +02:00
|
|
|
if (!srv_read_only_mode) {
|
|
|
|
buf_flush_page_cleaner_init();
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
ut_ad(buf_page_cleaner_is_active);
|
2016-12-05 21:04:30 +02:00
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
srv_startup_is_before_trx_rollback_phase = !create_new_db;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
/* Check if undo tablespaces and redo log files exist before creating
|
|
|
|
a new system tablespace */
|
|
|
|
if (create_new_db) {
|
|
|
|
err = srv_check_undo_redo_logs_exists();
|
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return(srv_init_abort(DB_ERROR));
|
|
|
|
}
|
2019-05-24 16:19:38 +03:00
|
|
|
recv_sys.debug_free();
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
/* Open or create the data files. */
|
|
|
|
ulint sum_of_new_sizes;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
err = srv_sys_space.open_or_create(
|
|
|
|
false, create_new_db, &sum_of_new_sizes, &flushed_lsn);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
switch (err) {
|
|
|
|
case DB_SUCCESS:
|
|
|
|
break;
|
|
|
|
case DB_CANNOT_OPEN_FILE:
|
|
|
|
ib::error()
|
|
|
|
<< "Could not open or create the system tablespace. If"
|
|
|
|
" you tried to add new data files to the system"
|
|
|
|
" tablespace, and it failed here, you should now"
|
|
|
|
" edit innodb_data_file_path in my.cnf back to what"
|
|
|
|
" it was, and remove the new ibdata files InnoDB"
|
|
|
|
" created in this failed attempt. InnoDB only wrote"
|
|
|
|
" those files full of zeros, but did not yet use"
|
|
|
|
" them in any way. But be careful: do not remove"
|
|
|
|
" old data files which contain your precious data!";
|
|
|
|
/* fall through */
|
|
|
|
default:
|
|
|
|
/* Other errors might come from Datafile::validate_first_page() */
|
|
|
|
return(srv_init_abort(err));
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
srv_log_file_size_requested = srv_log_file_size;
|
|
|
|
|
2019-07-01 18:50:44 +05:30
|
|
|
if (innodb_encrypt_temporary_tables && !log_crypt_init()) {
|
2019-06-27 16:23:03 +05:30
|
|
|
return srv_init_abort(DB_ERROR);
|
|
|
|
}
|
|
|
|
|
2019-11-25 22:32:24 +07:00
|
|
|
std::string logfile0;
|
MDEV-23855: Improve InnoDB log checkpoint performance
After MDEV-15053, MDEV-22871, MDEV-23399 shifted the scalability
bottleneck, log checkpoints became a new bottleneck.
If innodb_io_capacity is set low or innodb_max_dirty_pct_lwm is
set high and the workload fits in the buffer pool, the page cleaner
thread will perform very little flushing. When we reach the capacity
of the circular redo log file ib_logfile0 and must initiate a checkpoint,
some 'furious flushing' will be necessary. (If innodb_flush_sync=OFF,
then flushing would continue at the innodb_io_capacity rate, and
writers would be throttled.)
We have the best chance of advancing the checkpoint LSN immediately
after a page flush batch has been completed. Hence, it is best to
perform checkpoints after every batch in the page cleaner thread,
attempting to run once per second.
By initiating high-priority flushing in the page cleaner as early
as possible, we aim to make the throughput more stable.
The function buf_flush_wait_flushed() used to sleep for 10ms, hoping
that the page cleaner thread would do something during that time.
The observed end result was that a large number of threads that call
log_free_check() would end up sleeping while nothing useful is happening.
We will revise the design so that in the default innodb_flush_sync=ON
mode, buf_flush_wait_flushed() will wake up the page cleaner thread
to perform the necessary flushing, and it will wait for a signal from
the page cleaner thread.
If innodb_io_capacity is set to a low value (causing the page cleaner to
throttle its work), a write workload would initially perform well, until
the capacity of the circular ib_logfile0 is reached and log_free_check()
will trigger checkpoints. At that point, the extra waiting in
buf_flush_wait_flushed() will start reducing throughput.
The page cleaner thread will also initiate log checkpoints after each
buf_flush_lists() call, because that is the best point of time for
the checkpoint LSN to advance by the maximum amount.
Even in 'furious flushing' mode we invoke buf_flush_lists() with
innodb_io_capacity_max pages at a time, and at the start of each
batch (in the log_flush() callback function that runs in a separate
task) we will invoke os_aio_wait_until_no_pending_writes(). This
tweak allows the checkpoint to advance in smaller steps and
significantly reduces the maximum latency. On an Intel Optane 960
NVMe SSD on Linux, it reduced from 4.6 seconds to 74 milliseconds.
On Microsoft Windows with a slower SSD, it reduced from more than
180 seconds to 0.6 seconds.
We will make innodb_adaptive_flushing=OFF simply flush innodb_io_capacity
per second whenever the dirty proportion of buffer pool pages exceeds
innodb_max_dirty_pages_pct_lwm. For innodb_adaptive_flushing=ON we try
to make page_cleaner_flush_pages_recommendation() more consistent and
predictable: if we are below innodb_adaptive_flushing_lwm, let us flush
pages according to the return value of af_get_pct_for_dirty().
innodb_max_dirty_pages_pct_lwm: Revert the change of the default value
that was made in MDEV-23399. The value innodb_max_dirty_pages_pct_lwm=0
guarantees that a shutdown of an idle server will be fast. Users might
be surprised if normal shutdown suddenly became slower when upgrading
within a GA release series.
innodb_checkpoint_usec: Remove. The master task will no longer perform
periodic log checkpoints. It is the duty of the page cleaner thread.
log_sys.max_modified_age: Remove. The current span of the
buf_pool.flush_list expressed in LSN only matters for adaptive
flushing (outside the 'furious flushing' condition).
For the correctness of checkpoints, the only thing that matters is
the checkpoint age (log_sys.lsn - log_sys.last_checkpoint_lsn).
This run-time constant was also reported as log_max_modified_age_sync.
log_sys.max_checkpoint_age_async: Remove. This does not serve any
purpose, because the checkpoints will now be triggered by the page
cleaner thread. We will retain the log_sys.max_checkpoint_age limit
for engaging 'furious flushing'.
page_cleaner.slot: Remove. It turns out that
page_cleaner_slot.flush_list_time was duplicating
page_cleaner.slot.flush_time and page_cleaner.slot.flush_list_pass
was duplicating page_cleaner.flush_pass.
Likewise, there were some redundant monitor counters, because the
page cleaner thread no longer performs any buf_pool.LRU flushing, and
because there only is one buf_flush_page_cleaner thread.
buf_flush_sync_lsn: Protect writes by buf_pool.flush_list_mutex.
buf_pool_t::get_oldest_modification(): Add a parameter to specify the
return value when no persistent data pages are dirty. Require the
caller to hold buf_pool.flush_list_mutex.
log_buf_pool_get_oldest_modification(): Take the fall-back LSN
as a parameter. All callers will also invoke log_sys.get_lsn().
log_preflush_pool_modified_pages(): Replaced with buf_flush_wait_flushed().
buf_flush_wait_flushed(): Implement two limits. If not enough buffer pool
has been flushed, signal the page cleaner (unless innodb_flush_sync=OFF)
and wait for the page cleaner to complete. If the page cleaner
thread is not running (which can be the case durign shutdown),
initiate the flush and wait for it directly.
buf_flush_ahead(): If innodb_flush_sync=ON (the default),
submit a new buf_flush_sync_lsn target for the page cleaner
but do not wait for the flushing to finish.
log_get_capacity(), log_get_max_modified_age_async(): Remove, to make
it easier to see that af_get_pct_for_lsn() is not acquiring any mutexes.
page_cleaner_flush_pages_recommendation(): Protect all access to
buf_pool.flush_list with buf_pool.flush_list_mutex. Previously there
were some race conditions in the calculation.
buf_flush_sync_for_checkpoint(): New function to process
buf_flush_sync_lsn in the page cleaner thread. At the end of
each batch, we try to wake up any blocked buf_flush_wait_flushed().
If everything up to buf_flush_sync_lsn has been flushed, we will
reset buf_flush_sync_lsn=0. The page cleaner thread will keep
'furious flushing' until the limit is reached. Any threads that
are waiting in buf_flush_wait_flushed() will be able to resume
as soon as their own limit has been satisfied.
buf_flush_page_cleaner: Prioritize buf_flush_sync_lsn and do not
sleep as long as it is set. Do not update any page_cleaner statistics
for this special mode of operation. In the normal mode
(buf_flush_sync_lsn is not set for innodb_flush_sync=ON),
try to wake up once per second. No longer check whether
srv_inc_activity_count() has been called. After each batch,
try to perform a log checkpoint, because the best chances for
the checkpoint LSN to advance by the maximum amount are upon
completing a flushing batch.
log_t: Move buf_free, max_buf_free possibly to the same cache line
with log_sys.mutex.
log_margin_checkpoint_age(): Simplify the logic, and replace
a 0.1-second sleep with a call to buf_flush_wait_flushed() to
initiate flushing. Moved to the same compilation unit
with the only caller.
log_close(): Clean up the calculations. (Should be no functional
change.) Return whether flush-ahead is needed. Moved to the same
compilation unit with the only caller.
mtr_t::finish_write(): Return whether flush-ahead is needed.
mtr_t::commit(): Invoke buf_flush_ahead() when needed. Let us avoid
external calls in mtr_t::commit() and make the logic easier to follow
by having related code in a single compilation unit. Also, we will
invoke srv_stats.log_write_requests.inc() only once per
mini-transaction commit, while not holding mutexes.
log_checkpoint_margin(): Only care about log_sys.max_checkpoint_age.
Upon reaching log_sys.max_checkpoint_age where we must wait to prevent
the log from getting corrupted, let us wait for at most 1MiB of LSN
at a time, before rechecking the condition. This should allow writers
to proceed even if the redo log capacity has been reached and
'furious flushing' is in progress. We no longer care about
log_sys.max_modified_age_sync or log_sys.max_modified_age_async.
The log_sys.max_modified_age_sync could be a relic from the time when
there was a srv_master_thread that wrote dirty pages to data files.
Also, we no longer have any log_sys.max_checkpoint_age_async limit,
because log checkpoints will now be triggered by the page cleaner
thread upon completing buf_flush_lists().
log_set_capacity(): Simplify the calculations of the limit
(no functional change).
log_checkpoint_low(): Split from log_checkpoint(). Moved to the
same compilation unit with the caller.
log_make_checkpoint(): Only wait for everything to be flushed until
the current LSN.
create_log_file(): After checkpoint, invoke log_write_up_to()
to ensure that the FILE_CHECKPOINT record has been written.
This avoids ut_ad(!srv_log_file_created) in create_log_file_rename().
srv_start(): Do not call recv_recovery_from_checkpoint_start()
if the log has just been created. Set fil_system.space_id_reuse_warned
before dict_boot() has been executed, and clear it after recovery
has finished.
dict_boot(): Initialize fil_system.max_assigned_id.
srv_check_activity(): Remove. The activity count is counting transaction
commits and therefore mostly interesting for the purge of history.
BtrBulk::insert(): Do not explicitly wake up the page cleaner,
but do invoke srv_inc_activity_count(), because that counter is
still being used in buf_load_throttle_if_needed() for some
heuristics. (It might be cleaner to execute buf_load() in the
page cleaner thread!)
Reviewed by: Vladislav Vaintroub
2020-10-26 16:35:47 +02:00
|
|
|
bool create_new_log = create_new_db;
|
2014-02-26 19:11:54 +01:00
|
|
|
if (create_new_db) {
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
flushed_lsn = log_sys.get_lsn();
|
|
|
|
log_sys.set_flushed_lsn(flushed_lsn);
|
2020-02-12 14:45:21 +02:00
|
|
|
buf_flush_sync();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-10-26 15:59:30 +02:00
|
|
|
err = create_log_file(true, flushed_lsn, logfile0);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
if (err != DB_SUCCESS) {
|
2016-08-12 11:17:45 +03:00
|
|
|
return(srv_init_abort(err));
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
} else {
|
MDEV-17433 Allow InnoDB start up with empty ib_logfile0 from mariabackup --prepare
A prepared backup from Mariabackup does not really need to contain any
redo log file, because all log will have been applied to the data files.
When the user copies a prepared backup to a data directory (overwriting
existing files), it could happen that the data directory already contained
redo log files from the past. mariabackup --copy-back) would delete the
old redo log files, but a user’s own copying script might not do that.
To prevent corruption caused by mixing an old redo log file with data
files from a backup, starting with MDEV-13311, Mariabackup would create
a zero-length ib_logfile0 that would prevent startup.
Actually, there is no need to prevent InnoDB from starting up when a
single zero-length file ib_logfile0 is present. Only if there exist
multiple data files of different lengths, then we should refuse to
start up due to inconsistency. A single zero-length ib_logfile0 should
be treated as if the log files were missing: create new log files
according to the configuration.
open_log_file(): Remove. There is no need to open the log files
at this point, because os_file_get_status() already determined
the size of the file.
innobase_start_or_create_for_mysql(): Move the creation of new
log files a little later, not when finding out that the first log
file does not exist, but after finding out that it does not exist
or it exists as a zero-length file.
2018-10-11 22:48:19 +03:00
|
|
|
srv_log_file_size = 0;
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
bool log_file_found;
|
|
|
|
if (dberr_t err = find_and_check_log_file(log_file_found)) {
|
2014-02-26 19:11:54 +01:00
|
|
|
if (err == DB_NOT_FOUND) {
|
2020-01-12 02:05:28 +07:00
|
|
|
return DB_SUCCESS;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
2020-01-12 02:05:28 +07:00
|
|
|
return srv_init_abort(err);
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
MDEV-23855: Improve InnoDB log checkpoint performance
After MDEV-15053, MDEV-22871, MDEV-23399 shifted the scalability
bottleneck, log checkpoints became a new bottleneck.
If innodb_io_capacity is set low or innodb_max_dirty_pct_lwm is
set high and the workload fits in the buffer pool, the page cleaner
thread will perform very little flushing. When we reach the capacity
of the circular redo log file ib_logfile0 and must initiate a checkpoint,
some 'furious flushing' will be necessary. (If innodb_flush_sync=OFF,
then flushing would continue at the innodb_io_capacity rate, and
writers would be throttled.)
We have the best chance of advancing the checkpoint LSN immediately
after a page flush batch has been completed. Hence, it is best to
perform checkpoints after every batch in the page cleaner thread,
attempting to run once per second.
By initiating high-priority flushing in the page cleaner as early
as possible, we aim to make the throughput more stable.
The function buf_flush_wait_flushed() used to sleep for 10ms, hoping
that the page cleaner thread would do something during that time.
The observed end result was that a large number of threads that call
log_free_check() would end up sleeping while nothing useful is happening.
We will revise the design so that in the default innodb_flush_sync=ON
mode, buf_flush_wait_flushed() will wake up the page cleaner thread
to perform the necessary flushing, and it will wait for a signal from
the page cleaner thread.
If innodb_io_capacity is set to a low value (causing the page cleaner to
throttle its work), a write workload would initially perform well, until
the capacity of the circular ib_logfile0 is reached and log_free_check()
will trigger checkpoints. At that point, the extra waiting in
buf_flush_wait_flushed() will start reducing throughput.
The page cleaner thread will also initiate log checkpoints after each
buf_flush_lists() call, because that is the best point of time for
the checkpoint LSN to advance by the maximum amount.
Even in 'furious flushing' mode we invoke buf_flush_lists() with
innodb_io_capacity_max pages at a time, and at the start of each
batch (in the log_flush() callback function that runs in a separate
task) we will invoke os_aio_wait_until_no_pending_writes(). This
tweak allows the checkpoint to advance in smaller steps and
significantly reduces the maximum latency. On an Intel Optane 960
NVMe SSD on Linux, it reduced from 4.6 seconds to 74 milliseconds.
On Microsoft Windows with a slower SSD, it reduced from more than
180 seconds to 0.6 seconds.
We will make innodb_adaptive_flushing=OFF simply flush innodb_io_capacity
per second whenever the dirty proportion of buffer pool pages exceeds
innodb_max_dirty_pages_pct_lwm. For innodb_adaptive_flushing=ON we try
to make page_cleaner_flush_pages_recommendation() more consistent and
predictable: if we are below innodb_adaptive_flushing_lwm, let us flush
pages according to the return value of af_get_pct_for_dirty().
innodb_max_dirty_pages_pct_lwm: Revert the change of the default value
that was made in MDEV-23399. The value innodb_max_dirty_pages_pct_lwm=0
guarantees that a shutdown of an idle server will be fast. Users might
be surprised if normal shutdown suddenly became slower when upgrading
within a GA release series.
innodb_checkpoint_usec: Remove. The master task will no longer perform
periodic log checkpoints. It is the duty of the page cleaner thread.
log_sys.max_modified_age: Remove. The current span of the
buf_pool.flush_list expressed in LSN only matters for adaptive
flushing (outside the 'furious flushing' condition).
For the correctness of checkpoints, the only thing that matters is
the checkpoint age (log_sys.lsn - log_sys.last_checkpoint_lsn).
This run-time constant was also reported as log_max_modified_age_sync.
log_sys.max_checkpoint_age_async: Remove. This does not serve any
purpose, because the checkpoints will now be triggered by the page
cleaner thread. We will retain the log_sys.max_checkpoint_age limit
for engaging 'furious flushing'.
page_cleaner.slot: Remove. It turns out that
page_cleaner_slot.flush_list_time was duplicating
page_cleaner.slot.flush_time and page_cleaner.slot.flush_list_pass
was duplicating page_cleaner.flush_pass.
Likewise, there were some redundant monitor counters, because the
page cleaner thread no longer performs any buf_pool.LRU flushing, and
because there only is one buf_flush_page_cleaner thread.
buf_flush_sync_lsn: Protect writes by buf_pool.flush_list_mutex.
buf_pool_t::get_oldest_modification(): Add a parameter to specify the
return value when no persistent data pages are dirty. Require the
caller to hold buf_pool.flush_list_mutex.
log_buf_pool_get_oldest_modification(): Take the fall-back LSN
as a parameter. All callers will also invoke log_sys.get_lsn().
log_preflush_pool_modified_pages(): Replaced with buf_flush_wait_flushed().
buf_flush_wait_flushed(): Implement two limits. If not enough buffer pool
has been flushed, signal the page cleaner (unless innodb_flush_sync=OFF)
and wait for the page cleaner to complete. If the page cleaner
thread is not running (which can be the case durign shutdown),
initiate the flush and wait for it directly.
buf_flush_ahead(): If innodb_flush_sync=ON (the default),
submit a new buf_flush_sync_lsn target for the page cleaner
but do not wait for the flushing to finish.
log_get_capacity(), log_get_max_modified_age_async(): Remove, to make
it easier to see that af_get_pct_for_lsn() is not acquiring any mutexes.
page_cleaner_flush_pages_recommendation(): Protect all access to
buf_pool.flush_list with buf_pool.flush_list_mutex. Previously there
were some race conditions in the calculation.
buf_flush_sync_for_checkpoint(): New function to process
buf_flush_sync_lsn in the page cleaner thread. At the end of
each batch, we try to wake up any blocked buf_flush_wait_flushed().
If everything up to buf_flush_sync_lsn has been flushed, we will
reset buf_flush_sync_lsn=0. The page cleaner thread will keep
'furious flushing' until the limit is reached. Any threads that
are waiting in buf_flush_wait_flushed() will be able to resume
as soon as their own limit has been satisfied.
buf_flush_page_cleaner: Prioritize buf_flush_sync_lsn and do not
sleep as long as it is set. Do not update any page_cleaner statistics
for this special mode of operation. In the normal mode
(buf_flush_sync_lsn is not set for innodb_flush_sync=ON),
try to wake up once per second. No longer check whether
srv_inc_activity_count() has been called. After each batch,
try to perform a log checkpoint, because the best chances for
the checkpoint LSN to advance by the maximum amount are upon
completing a flushing batch.
log_t: Move buf_free, max_buf_free possibly to the same cache line
with log_sys.mutex.
log_margin_checkpoint_age(): Simplify the logic, and replace
a 0.1-second sleep with a call to buf_flush_wait_flushed() to
initiate flushing. Moved to the same compilation unit
with the only caller.
log_close(): Clean up the calculations. (Should be no functional
change.) Return whether flush-ahead is needed. Moved to the same
compilation unit with the only caller.
mtr_t::finish_write(): Return whether flush-ahead is needed.
mtr_t::commit(): Invoke buf_flush_ahead() when needed. Let us avoid
external calls in mtr_t::commit() and make the logic easier to follow
by having related code in a single compilation unit. Also, we will
invoke srv_stats.log_write_requests.inc() only once per
mini-transaction commit, while not holding mutexes.
log_checkpoint_margin(): Only care about log_sys.max_checkpoint_age.
Upon reaching log_sys.max_checkpoint_age where we must wait to prevent
the log from getting corrupted, let us wait for at most 1MiB of LSN
at a time, before rechecking the condition. This should allow writers
to proceed even if the redo log capacity has been reached and
'furious flushing' is in progress. We no longer care about
log_sys.max_modified_age_sync or log_sys.max_modified_age_async.
The log_sys.max_modified_age_sync could be a relic from the time when
there was a srv_master_thread that wrote dirty pages to data files.
Also, we no longer have any log_sys.max_checkpoint_age_async limit,
because log checkpoints will now be triggered by the page cleaner
thread upon completing buf_flush_lists().
log_set_capacity(): Simplify the calculations of the limit
(no functional change).
log_checkpoint_low(): Split from log_checkpoint(). Moved to the
same compilation unit with the caller.
log_make_checkpoint(): Only wait for everything to be flushed until
the current LSN.
create_log_file(): After checkpoint, invoke log_write_up_to()
to ensure that the FILE_CHECKPOINT record has been written.
This avoids ut_ad(!srv_log_file_created) in create_log_file_rename().
srv_start(): Do not call recv_recovery_from_checkpoint_start()
if the log has just been created. Set fil_system.space_id_reuse_warned
before dict_boot() has been executed, and clear it after recovery
has finished.
dict_boot(): Initialize fil_system.max_assigned_id.
srv_check_activity(): Remove. The activity count is counting transaction
commits and therefore mostly interesting for the purge of history.
BtrBulk::insert(): Do not explicitly wake up the page cleaner,
but do invoke srv_inc_activity_count(), because that counter is
still being used in buf_load_throttle_if_needed() for some
heuristics. (It might be cleaner to execute buf_load() in the
page cleaner thread!)
Reviewed by: Vladislav Vaintroub
2020-10-26 16:35:47 +02:00
|
|
|
create_new_log = srv_log_file_size == 0;
|
|
|
|
if (create_new_log) {
|
MDEV-17433 Allow InnoDB start up with empty ib_logfile0 from mariabackup --prepare
A prepared backup from Mariabackup does not really need to contain any
redo log file, because all log will have been applied to the data files.
When the user copies a prepared backup to a data directory (overwriting
existing files), it could happen that the data directory already contained
redo log files from the past. mariabackup --copy-back) would delete the
old redo log files, but a user’s own copying script might not do that.
To prevent corruption caused by mixing an old redo log file with data
files from a backup, starting with MDEV-13311, Mariabackup would create
a zero-length ib_logfile0 that would prevent startup.
Actually, there is no need to prevent InnoDB from starting up when a
single zero-length file ib_logfile0 is present. Only if there exist
multiple data files of different lengths, then we should refuse to
start up due to inconsistency. A single zero-length ib_logfile0 should
be treated as if the log files were missing: create new log files
according to the configuration.
open_log_file(): Remove. There is no need to open the log files
at this point, because os_file_get_status() already determined
the size of the file.
innobase_start_or_create_for_mysql(): Move the creation of new
log files a little later, not when finding out that the first log
file does not exist, but after finding out that it does not exist
or it exists as a zero-length file.
2018-10-11 22:48:19 +03:00
|
|
|
if (flushed_lsn < lsn_t(1000)) {
|
|
|
|
ib::error()
|
2020-01-12 02:05:28 +07:00
|
|
|
<< "Cannot create log file because"
|
MDEV-17433 Allow InnoDB start up with empty ib_logfile0 from mariabackup --prepare
A prepared backup from Mariabackup does not really need to contain any
redo log file, because all log will have been applied to the data files.
When the user copies a prepared backup to a data directory (overwriting
existing files), it could happen that the data directory already contained
redo log files from the past. mariabackup --copy-back) would delete the
old redo log files, but a user’s own copying script might not do that.
To prevent corruption caused by mixing an old redo log file with data
files from a backup, starting with MDEV-13311, Mariabackup would create
a zero-length ib_logfile0 that would prevent startup.
Actually, there is no need to prevent InnoDB from starting up when a
single zero-length file ib_logfile0 is present. Only if there exist
multiple data files of different lengths, then we should refuse to
start up due to inconsistency. A single zero-length ib_logfile0 should
be treated as if the log files were missing: create new log files
according to the configuration.
open_log_file(): Remove. There is no need to open the log files
at this point, because os_file_get_status() already determined
the size of the file.
innobase_start_or_create_for_mysql(): Move the creation of new
log files a little later, not when finding out that the first log
file does not exist, but after finding out that it does not exist
or it exists as a zero-length file.
2018-10-11 22:48:19 +03:00
|
|
|
" data files are corrupt or the"
|
|
|
|
" database was not shut down cleanly"
|
|
|
|
" after creating the data files.";
|
|
|
|
return srv_init_abort(DB_ERROR);
|
|
|
|
}
|
|
|
|
|
|
|
|
srv_log_file_size = srv_log_file_size_requested;
|
|
|
|
|
2020-10-26 15:59:30 +02:00
|
|
|
err = create_log_file(false, flushed_lsn, logfile0);
|
MDEV-17433 Allow InnoDB start up with empty ib_logfile0 from mariabackup --prepare
A prepared backup from Mariabackup does not really need to contain any
redo log file, because all log will have been applied to the data files.
When the user copies a prepared backup to a data directory (overwriting
existing files), it could happen that the data directory already contained
redo log files from the past. mariabackup --copy-back) would delete the
old redo log files, but a user’s own copying script might not do that.
To prevent corruption caused by mixing an old redo log file with data
files from a backup, starting with MDEV-13311, Mariabackup would create
a zero-length ib_logfile0 that would prevent startup.
Actually, there is no need to prevent InnoDB from starting up when a
single zero-length file ib_logfile0 is present. Only if there exist
multiple data files of different lengths, then we should refuse to
start up due to inconsistency. A single zero-length ib_logfile0 should
be treated as if the log files were missing: create new log files
according to the configuration.
open_log_file(): Remove. There is no need to open the log files
at this point, because os_file_get_status() already determined
the size of the file.
innobase_start_or_create_for_mysql(): Move the creation of new
log files a little later, not when finding out that the first log
file does not exist, but after finding out that it does not exist
or it exists as a zero-length file.
2018-10-11 22:48:19 +03:00
|
|
|
|
|
|
|
if (err == DB_SUCCESS) {
|
2020-01-12 02:05:28 +07:00
|
|
|
err = create_log_file_rename(flushed_lsn,
|
|
|
|
logfile0);
|
MDEV-17433 Allow InnoDB start up with empty ib_logfile0 from mariabackup --prepare
A prepared backup from Mariabackup does not really need to contain any
redo log file, because all log will have been applied to the data files.
When the user copies a prepared backup to a data directory (overwriting
existing files), it could happen that the data directory already contained
redo log files from the past. mariabackup --copy-back) would delete the
old redo log files, but a user’s own copying script might not do that.
To prevent corruption caused by mixing an old redo log file with data
files from a backup, starting with MDEV-13311, Mariabackup would create
a zero-length ib_logfile0 that would prevent startup.
Actually, there is no need to prevent InnoDB from starting up when a
single zero-length file ib_logfile0 is present. Only if there exist
multiple data files of different lengths, then we should refuse to
start up due to inconsistency. A single zero-length ib_logfile0 should
be treated as if the log files were missing: create new log files
according to the configuration.
open_log_file(): Remove. There is no need to open the log files
at this point, because os_file_get_status() already determined
the size of the file.
innobase_start_or_create_for_mysql(): Move the creation of new
log files a little later, not when finding out that the first log
file does not exist, but after finding out that it does not exist
or it exists as a zero-length file.
2018-10-11 22:48:19 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return(srv_init_abort(err));
|
|
|
|
}
|
|
|
|
|
|
|
|
/* Suppress the message about
|
|
|
|
crash recovery. */
|
2021-03-30 09:58:24 +03:00
|
|
|
flushed_lsn = log_sys.get_lsn();
|
2020-01-12 02:05:28 +07:00
|
|
|
goto file_checked;
|
MDEV-17433 Allow InnoDB start up with empty ib_logfile0 from mariabackup --prepare
A prepared backup from Mariabackup does not really need to contain any
redo log file, because all log will have been applied to the data files.
When the user copies a prepared backup to a data directory (overwriting
existing files), it could happen that the data directory already contained
redo log files from the past. mariabackup --copy-back) would delete the
old redo log files, but a user’s own copying script might not do that.
To prevent corruption caused by mixing an old redo log file with data
files from a backup, starting with MDEV-13311, Mariabackup would create
a zero-length ib_logfile0 that would prevent startup.
Actually, there is no need to prevent InnoDB from starting up when a
single zero-length file ib_logfile0 is present. Only if there exist
multiple data files of different lengths, then we should refuse to
start up due to inconsistency. A single zero-length ib_logfile0 should
be treated as if the log files were missing: create new log files
according to the configuration.
open_log_file(): Remove. There is no need to open the log files
at this point, because os_file_get_status() already determined
the size of the file.
innobase_start_or_create_for_mysql(): Move the creation of new
log files a little later, not when finding out that the first log
file does not exist, but after finding out that it does not exist
or it exists as a zero-length file.
2018-10-11 22:48:19 +03:00
|
|
|
}
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
srv_log_file_found = log_file_found;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
log_sys.log.open_file(get_log_file_path());
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
log_sys.log.create();
|
2017-05-26 14:04:19 +03:00
|
|
|
|
Reduce the granularity of innodb_log_file_size
In Mariabackup, we would want the backed-up redo log file size to be
a multiple of 512 bytes, or OS_FILE_LOG_BLOCK_SIZE. However, at startup,
InnoDB would be picky, requiring the file size to be a multiple of
innodb_page_size.
Furthermore, InnoDB would require the parameter to be a multiple of
one megabyte, while the minimum granularity is 512 bytes. Because
the data-file-oriented fil_io() API is being used for writing the
InnoDB redo log, writes will for now require innodb_log_file_size to
be a multiple of the maximum innodb_page_size (65536 bytes).
To complicate matters, InnoDB startup divided srv_log_file_size by
UNIV_PAGE_SIZE, so that initially, the unit was bytes, and later it
was innodb_page_size. We will simplify this and keep srv_log_file_size
in bytes at all times.
innobase_log_file_size: Remove. Remove some obsolete checks against
overflow on 32-bit systems. srv_log_file_size is always 64 bits, and
the maximum size 512GiB in multiples of innodb_page_size always fits
in ulint (which is 32 or 64 bits). 512GiB would be 8,388,608*64KiB or
134,217,728*4KiB.
log_init(): Remove the parameter file_size that was always passed as
srv_log_file_size.
log_set_capacity(): Add a parameter for passing the requested file size.
srv_log_file_size_requested: Declare static in srv0start.cc.
create_log_file(), create_log_files(),
innobase_start_or_create_for_mysql(): Invoke fil_node_create()
with srv_log_file_size expressed in multiples of innodb_page_size.
innobase_start_or_create_for_mysql(): Require the redo log file sizes
to be multiples of 512 bytes.
2017-06-05 22:47:20 +03:00
|
|
|
if (!log_set_capacity(srv_log_file_size_requested)) {
|
2016-08-12 11:17:45 +03:00
|
|
|
return(srv_init_abort(DB_ERROR));
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
file_checked:
|
|
|
|
/* Open log file and data files in the systemtablespace: we keep
|
|
|
|
them open until database shutdown */
|
2018-03-28 09:29:14 +03:00
|
|
|
ut_d(fil_system.sys_space->recv_size = srv_sys_space_size_debug);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-10-26 15:59:30 +02:00
|
|
|
err = fil_system.sys_space->open(create_new_db)
|
|
|
|
? srv_undo_tablespaces_init(create_new_db)
|
|
|
|
: DB_ERROR;
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
/* If the force recovery is set very high then we carry on regardless
|
|
|
|
of all errors. Basically this is fingers crossed mode. */
|
|
|
|
|
|
|
|
if (err != DB_SUCCESS
|
|
|
|
&& srv_force_recovery < SRV_FORCE_NO_UNDO_LOG_SCAN) {
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
return(srv_init_abort(err));
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
/* Initialize objects used by dict stats gathering thread, which
|
|
|
|
can also be used by recovery if it tries to drop some table */
|
|
|
|
if (!srv_read_only_mode) {
|
2019-10-29 22:37:12 +01:00
|
|
|
dict_stats_init();
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2017-12-22 16:15:41 +02:00
|
|
|
trx_sys.create();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
if (create_new_db) {
|
2019-11-13 18:14:44 +01:00
|
|
|
ut_ad(!srv_read_only_mode);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
mtr_start(&mtr);
|
2018-03-22 14:17:43 +02:00
|
|
|
ut_ad(fil_system.sys_space->id == 0);
|
2017-06-06 14:59:42 +03:00
|
|
|
compile_time_assert(TRX_SYS_SPACE == 0);
|
|
|
|
compile_time_assert(IBUF_SPACE_ID == 0);
|
2020-10-15 16:28:19 +03:00
|
|
|
fsp_header_init(fil_system.sys_space,
|
|
|
|
uint32_t(sum_of_new_sizes), &mtr);
|
2017-06-06 14:59:42 +03:00
|
|
|
|
|
|
|
ulint ibuf_root = btr_create(
|
MDEV-12266: Change dict_table_t::space to fil_space_t*
InnoDB always keeps all tablespaces in the fil_system cache.
The fil_system.LRU is only for closing file handles; the
fil_space_t and fil_node_t for all data files will remain
in main memory. Between startup to shutdown, they can only be
created and removed by DDL statements. Therefore, we can
let dict_table_t::space point directly to the fil_space_t.
dict_table_t::space_id: A numeric tablespace ID for the corner cases
where we do not have a tablespace. The most prominent examples are
ALTER TABLE...DISCARD TABLESPACE or a missing or corrupted file.
There are a few functional differences; most notably:
(1) DROP TABLE will delete matching .ibd and .cfg files,
even if they were not attached to the data dictionary.
(2) Some error messages will report file names instead of numeric IDs.
There still are many functions that use numeric tablespace IDs instead
of fil_space_t*, and many functions could be converted to fil_space_t
member functions. Also, Tablespace and Datafile should be merged with
fil_space_t and fil_node_t. page_id_t and buf_page_get_gen() could use
fil_space_t& instead of a numeric ID, and after moving to a single
buffer pool (MDEV-15058), buf_pool_t::page_hash could be moved to
fil_space_t::page_hash.
FilSpace: Remove. Only few calls to fil_space_acquire() will remain,
and gradually they should be removed.
mtr_t::set_named_space_id(ulint): Renamed from set_named_space(),
to prevent accidental calls to this slower function. Very few
callers remain.
fseg_create(), fsp_reserve_free_extents(): Take fil_space_t*
as a parameter instead of a space_id.
fil_space_t::rename(): Wrapper for fil_rename_tablespace_check(),
fil_name_write_rename(), fil_rename_tablespace(). Mariabackup
passes the parameter log=false; InnoDB passes log=true.
dict_mem_table_create(): Take fil_space_t* instead of space_id
as parameter.
dict_process_sys_tables_rec_and_mtr_commit(): Replace the parameter
'status' with 'bool cached'.
dict_get_and_save_data_dir_path(): Avoid copying the fil_node_t::name.
fil_ibd_open(): Return the tablespace.
fil_space_t::set_imported(): Replaces fil_space_set_imported().
truncate_t: Change many member function parameters to fil_space_t*,
and remove page_size parameters.
row_truncate_prepare(): Merge to its only caller.
row_drop_table_from_cache(): Assert that the table is persistent.
dict_create_sys_indexes_tuple(): Write SYS_INDEXES.SPACE=FIL_NULL
if the tablespace has been discarded.
row_import_update_discarded_flag(): Remove a constant parameter.
2018-03-27 16:31:10 +03:00
|
|
|
DICT_CLUSTERED | DICT_IBUF, fil_system.sys_space,
|
2020-02-20 22:00:43 +02:00
|
|
|
DICT_IBUF_ID_MIN, nullptr, &mtr);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
mtr_commit(&mtr);
|
|
|
|
|
2017-06-08 12:45:08 +03:00
|
|
|
if (ibuf_root == FIL_NULL) {
|
2016-08-12 11:17:45 +03:00
|
|
|
return(srv_init_abort(DB_ERROR));
|
|
|
|
}
|
|
|
|
|
2017-06-08 12:45:08 +03:00
|
|
|
ut_ad(ibuf_root == IBUF_TREE_ROOT_PAGE_NO);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
/* To maintain backward compatibility we create only
|
|
|
|
the first rollback segment before the double write buffer.
|
|
|
|
All the remaining rollback segments will be created later,
|
|
|
|
after the double write buffer has been created. */
|
|
|
|
trx_sys_create_sys_pages();
|
2018-01-30 14:16:09 +02:00
|
|
|
trx_lists_init_at_db_start();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
err = dict_create();
|
|
|
|
|
|
|
|
if (err != DB_SUCCESS) {
|
2016-08-12 11:17:45 +03:00
|
|
|
return(srv_init_abort(err));
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2020-02-12 14:45:21 +02:00
|
|
|
buf_flush_sync();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2021-03-30 09:58:24 +03:00
|
|
|
flushed_lsn = log_sys.get_lsn();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-09-06 09:43:16 +03:00
|
|
|
err = fil_write_flushed_lsn(flushed_lsn);
|
|
|
|
|
2017-01-30 17:00:51 +02:00
|
|
|
if (err == DB_SUCCESS) {
|
2020-01-12 02:05:28 +07:00
|
|
|
err = create_log_file_rename(flushed_lsn, logfile0);
|
2017-01-30 17:00:51 +02:00
|
|
|
}
|
|
|
|
|
2016-09-06 09:43:16 +03:00
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return(srv_init_abort(err));
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
} else {
|
MDEV-23855: Improve InnoDB log checkpoint performance
After MDEV-15053, MDEV-22871, MDEV-23399 shifted the scalability
bottleneck, log checkpoints became a new bottleneck.
If innodb_io_capacity is set low or innodb_max_dirty_pct_lwm is
set high and the workload fits in the buffer pool, the page cleaner
thread will perform very little flushing. When we reach the capacity
of the circular redo log file ib_logfile0 and must initiate a checkpoint,
some 'furious flushing' will be necessary. (If innodb_flush_sync=OFF,
then flushing would continue at the innodb_io_capacity rate, and
writers would be throttled.)
We have the best chance of advancing the checkpoint LSN immediately
after a page flush batch has been completed. Hence, it is best to
perform checkpoints after every batch in the page cleaner thread,
attempting to run once per second.
By initiating high-priority flushing in the page cleaner as early
as possible, we aim to make the throughput more stable.
The function buf_flush_wait_flushed() used to sleep for 10ms, hoping
that the page cleaner thread would do something during that time.
The observed end result was that a large number of threads that call
log_free_check() would end up sleeping while nothing useful is happening.
We will revise the design so that in the default innodb_flush_sync=ON
mode, buf_flush_wait_flushed() will wake up the page cleaner thread
to perform the necessary flushing, and it will wait for a signal from
the page cleaner thread.
If innodb_io_capacity is set to a low value (causing the page cleaner to
throttle its work), a write workload would initially perform well, until
the capacity of the circular ib_logfile0 is reached and log_free_check()
will trigger checkpoints. At that point, the extra waiting in
buf_flush_wait_flushed() will start reducing throughput.
The page cleaner thread will also initiate log checkpoints after each
buf_flush_lists() call, because that is the best point of time for
the checkpoint LSN to advance by the maximum amount.
Even in 'furious flushing' mode we invoke buf_flush_lists() with
innodb_io_capacity_max pages at a time, and at the start of each
batch (in the log_flush() callback function that runs in a separate
task) we will invoke os_aio_wait_until_no_pending_writes(). This
tweak allows the checkpoint to advance in smaller steps and
significantly reduces the maximum latency. On an Intel Optane 960
NVMe SSD on Linux, it reduced from 4.6 seconds to 74 milliseconds.
On Microsoft Windows with a slower SSD, it reduced from more than
180 seconds to 0.6 seconds.
We will make innodb_adaptive_flushing=OFF simply flush innodb_io_capacity
per second whenever the dirty proportion of buffer pool pages exceeds
innodb_max_dirty_pages_pct_lwm. For innodb_adaptive_flushing=ON we try
to make page_cleaner_flush_pages_recommendation() more consistent and
predictable: if we are below innodb_adaptive_flushing_lwm, let us flush
pages according to the return value of af_get_pct_for_dirty().
innodb_max_dirty_pages_pct_lwm: Revert the change of the default value
that was made in MDEV-23399. The value innodb_max_dirty_pages_pct_lwm=0
guarantees that a shutdown of an idle server will be fast. Users might
be surprised if normal shutdown suddenly became slower when upgrading
within a GA release series.
innodb_checkpoint_usec: Remove. The master task will no longer perform
periodic log checkpoints. It is the duty of the page cleaner thread.
log_sys.max_modified_age: Remove. The current span of the
buf_pool.flush_list expressed in LSN only matters for adaptive
flushing (outside the 'furious flushing' condition).
For the correctness of checkpoints, the only thing that matters is
the checkpoint age (log_sys.lsn - log_sys.last_checkpoint_lsn).
This run-time constant was also reported as log_max_modified_age_sync.
log_sys.max_checkpoint_age_async: Remove. This does not serve any
purpose, because the checkpoints will now be triggered by the page
cleaner thread. We will retain the log_sys.max_checkpoint_age limit
for engaging 'furious flushing'.
page_cleaner.slot: Remove. It turns out that
page_cleaner_slot.flush_list_time was duplicating
page_cleaner.slot.flush_time and page_cleaner.slot.flush_list_pass
was duplicating page_cleaner.flush_pass.
Likewise, there were some redundant monitor counters, because the
page cleaner thread no longer performs any buf_pool.LRU flushing, and
because there only is one buf_flush_page_cleaner thread.
buf_flush_sync_lsn: Protect writes by buf_pool.flush_list_mutex.
buf_pool_t::get_oldest_modification(): Add a parameter to specify the
return value when no persistent data pages are dirty. Require the
caller to hold buf_pool.flush_list_mutex.
log_buf_pool_get_oldest_modification(): Take the fall-back LSN
as a parameter. All callers will also invoke log_sys.get_lsn().
log_preflush_pool_modified_pages(): Replaced with buf_flush_wait_flushed().
buf_flush_wait_flushed(): Implement two limits. If not enough buffer pool
has been flushed, signal the page cleaner (unless innodb_flush_sync=OFF)
and wait for the page cleaner to complete. If the page cleaner
thread is not running (which can be the case durign shutdown),
initiate the flush and wait for it directly.
buf_flush_ahead(): If innodb_flush_sync=ON (the default),
submit a new buf_flush_sync_lsn target for the page cleaner
but do not wait for the flushing to finish.
log_get_capacity(), log_get_max_modified_age_async(): Remove, to make
it easier to see that af_get_pct_for_lsn() is not acquiring any mutexes.
page_cleaner_flush_pages_recommendation(): Protect all access to
buf_pool.flush_list with buf_pool.flush_list_mutex. Previously there
were some race conditions in the calculation.
buf_flush_sync_for_checkpoint(): New function to process
buf_flush_sync_lsn in the page cleaner thread. At the end of
each batch, we try to wake up any blocked buf_flush_wait_flushed().
If everything up to buf_flush_sync_lsn has been flushed, we will
reset buf_flush_sync_lsn=0. The page cleaner thread will keep
'furious flushing' until the limit is reached. Any threads that
are waiting in buf_flush_wait_flushed() will be able to resume
as soon as their own limit has been satisfied.
buf_flush_page_cleaner: Prioritize buf_flush_sync_lsn and do not
sleep as long as it is set. Do not update any page_cleaner statistics
for this special mode of operation. In the normal mode
(buf_flush_sync_lsn is not set for innodb_flush_sync=ON),
try to wake up once per second. No longer check whether
srv_inc_activity_count() has been called. After each batch,
try to perform a log checkpoint, because the best chances for
the checkpoint LSN to advance by the maximum amount are upon
completing a flushing batch.
log_t: Move buf_free, max_buf_free possibly to the same cache line
with log_sys.mutex.
log_margin_checkpoint_age(): Simplify the logic, and replace
a 0.1-second sleep with a call to buf_flush_wait_flushed() to
initiate flushing. Moved to the same compilation unit
with the only caller.
log_close(): Clean up the calculations. (Should be no functional
change.) Return whether flush-ahead is needed. Moved to the same
compilation unit with the only caller.
mtr_t::finish_write(): Return whether flush-ahead is needed.
mtr_t::commit(): Invoke buf_flush_ahead() when needed. Let us avoid
external calls in mtr_t::commit() and make the logic easier to follow
by having related code in a single compilation unit. Also, we will
invoke srv_stats.log_write_requests.inc() only once per
mini-transaction commit, while not holding mutexes.
log_checkpoint_margin(): Only care about log_sys.max_checkpoint_age.
Upon reaching log_sys.max_checkpoint_age where we must wait to prevent
the log from getting corrupted, let us wait for at most 1MiB of LSN
at a time, before rechecking the condition. This should allow writers
to proceed even if the redo log capacity has been reached and
'furious flushing' is in progress. We no longer care about
log_sys.max_modified_age_sync or log_sys.max_modified_age_async.
The log_sys.max_modified_age_sync could be a relic from the time when
there was a srv_master_thread that wrote dirty pages to data files.
Also, we no longer have any log_sys.max_checkpoint_age_async limit,
because log checkpoints will now be triggered by the page cleaner
thread upon completing buf_flush_lists().
log_set_capacity(): Simplify the calculations of the limit
(no functional change).
log_checkpoint_low(): Split from log_checkpoint(). Moved to the
same compilation unit with the caller.
log_make_checkpoint(): Only wait for everything to be flushed until
the current LSN.
create_log_file(): After checkpoint, invoke log_write_up_to()
to ensure that the FILE_CHECKPOINT record has been written.
This avoids ut_ad(!srv_log_file_created) in create_log_file_rename().
srv_start(): Do not call recv_recovery_from_checkpoint_start()
if the log has just been created. Set fil_system.space_id_reuse_warned
before dict_boot() has been executed, and clear it after recovery
has finished.
dict_boot(): Initialize fil_system.max_assigned_id.
srv_check_activity(): Remove. The activity count is counting transaction
commits and therefore mostly interesting for the purge of history.
BtrBulk::insert(): Do not explicitly wake up the page cleaner,
but do invoke srv_inc_activity_count(), because that counter is
still being used in buf_load_throttle_if_needed() for some
heuristics. (It might be cleaner to execute buf_load() in the
page cleaner thread!)
Reviewed by: Vladislav Vaintroub
2020-10-26 16:35:47 +02:00
|
|
|
/* Suppress warnings in fil_space_t::create() for files
|
|
|
|
that are being read before dict_boot() has recovered
|
|
|
|
DICT_HDR_MAX_SPACE_ID. */
|
|
|
|
fil_system.space_id_reuse_warned = true;
|
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
/* We always try to do a recovery, even if the database had
|
|
|
|
been shut down normally: this is the normal startup path */
|
|
|
|
|
MDEV-23855: Improve InnoDB log checkpoint performance
After MDEV-15053, MDEV-22871, MDEV-23399 shifted the scalability
bottleneck, log checkpoints became a new bottleneck.
If innodb_io_capacity is set low or innodb_max_dirty_pct_lwm is
set high and the workload fits in the buffer pool, the page cleaner
thread will perform very little flushing. When we reach the capacity
of the circular redo log file ib_logfile0 and must initiate a checkpoint,
some 'furious flushing' will be necessary. (If innodb_flush_sync=OFF,
then flushing would continue at the innodb_io_capacity rate, and
writers would be throttled.)
We have the best chance of advancing the checkpoint LSN immediately
after a page flush batch has been completed. Hence, it is best to
perform checkpoints after every batch in the page cleaner thread,
attempting to run once per second.
By initiating high-priority flushing in the page cleaner as early
as possible, we aim to make the throughput more stable.
The function buf_flush_wait_flushed() used to sleep for 10ms, hoping
that the page cleaner thread would do something during that time.
The observed end result was that a large number of threads that call
log_free_check() would end up sleeping while nothing useful is happening.
We will revise the design so that in the default innodb_flush_sync=ON
mode, buf_flush_wait_flushed() will wake up the page cleaner thread
to perform the necessary flushing, and it will wait for a signal from
the page cleaner thread.
If innodb_io_capacity is set to a low value (causing the page cleaner to
throttle its work), a write workload would initially perform well, until
the capacity of the circular ib_logfile0 is reached and log_free_check()
will trigger checkpoints. At that point, the extra waiting in
buf_flush_wait_flushed() will start reducing throughput.
The page cleaner thread will also initiate log checkpoints after each
buf_flush_lists() call, because that is the best point of time for
the checkpoint LSN to advance by the maximum amount.
Even in 'furious flushing' mode we invoke buf_flush_lists() with
innodb_io_capacity_max pages at a time, and at the start of each
batch (in the log_flush() callback function that runs in a separate
task) we will invoke os_aio_wait_until_no_pending_writes(). This
tweak allows the checkpoint to advance in smaller steps and
significantly reduces the maximum latency. On an Intel Optane 960
NVMe SSD on Linux, it reduced from 4.6 seconds to 74 milliseconds.
On Microsoft Windows with a slower SSD, it reduced from more than
180 seconds to 0.6 seconds.
We will make innodb_adaptive_flushing=OFF simply flush innodb_io_capacity
per second whenever the dirty proportion of buffer pool pages exceeds
innodb_max_dirty_pages_pct_lwm. For innodb_adaptive_flushing=ON we try
to make page_cleaner_flush_pages_recommendation() more consistent and
predictable: if we are below innodb_adaptive_flushing_lwm, let us flush
pages according to the return value of af_get_pct_for_dirty().
innodb_max_dirty_pages_pct_lwm: Revert the change of the default value
that was made in MDEV-23399. The value innodb_max_dirty_pages_pct_lwm=0
guarantees that a shutdown of an idle server will be fast. Users might
be surprised if normal shutdown suddenly became slower when upgrading
within a GA release series.
innodb_checkpoint_usec: Remove. The master task will no longer perform
periodic log checkpoints. It is the duty of the page cleaner thread.
log_sys.max_modified_age: Remove. The current span of the
buf_pool.flush_list expressed in LSN only matters for adaptive
flushing (outside the 'furious flushing' condition).
For the correctness of checkpoints, the only thing that matters is
the checkpoint age (log_sys.lsn - log_sys.last_checkpoint_lsn).
This run-time constant was also reported as log_max_modified_age_sync.
log_sys.max_checkpoint_age_async: Remove. This does not serve any
purpose, because the checkpoints will now be triggered by the page
cleaner thread. We will retain the log_sys.max_checkpoint_age limit
for engaging 'furious flushing'.
page_cleaner.slot: Remove. It turns out that
page_cleaner_slot.flush_list_time was duplicating
page_cleaner.slot.flush_time and page_cleaner.slot.flush_list_pass
was duplicating page_cleaner.flush_pass.
Likewise, there were some redundant monitor counters, because the
page cleaner thread no longer performs any buf_pool.LRU flushing, and
because there only is one buf_flush_page_cleaner thread.
buf_flush_sync_lsn: Protect writes by buf_pool.flush_list_mutex.
buf_pool_t::get_oldest_modification(): Add a parameter to specify the
return value when no persistent data pages are dirty. Require the
caller to hold buf_pool.flush_list_mutex.
log_buf_pool_get_oldest_modification(): Take the fall-back LSN
as a parameter. All callers will also invoke log_sys.get_lsn().
log_preflush_pool_modified_pages(): Replaced with buf_flush_wait_flushed().
buf_flush_wait_flushed(): Implement two limits. If not enough buffer pool
has been flushed, signal the page cleaner (unless innodb_flush_sync=OFF)
and wait for the page cleaner to complete. If the page cleaner
thread is not running (which can be the case durign shutdown),
initiate the flush and wait for it directly.
buf_flush_ahead(): If innodb_flush_sync=ON (the default),
submit a new buf_flush_sync_lsn target for the page cleaner
but do not wait for the flushing to finish.
log_get_capacity(), log_get_max_modified_age_async(): Remove, to make
it easier to see that af_get_pct_for_lsn() is not acquiring any mutexes.
page_cleaner_flush_pages_recommendation(): Protect all access to
buf_pool.flush_list with buf_pool.flush_list_mutex. Previously there
were some race conditions in the calculation.
buf_flush_sync_for_checkpoint(): New function to process
buf_flush_sync_lsn in the page cleaner thread. At the end of
each batch, we try to wake up any blocked buf_flush_wait_flushed().
If everything up to buf_flush_sync_lsn has been flushed, we will
reset buf_flush_sync_lsn=0. The page cleaner thread will keep
'furious flushing' until the limit is reached. Any threads that
are waiting in buf_flush_wait_flushed() will be able to resume
as soon as their own limit has been satisfied.
buf_flush_page_cleaner: Prioritize buf_flush_sync_lsn and do not
sleep as long as it is set. Do not update any page_cleaner statistics
for this special mode of operation. In the normal mode
(buf_flush_sync_lsn is not set for innodb_flush_sync=ON),
try to wake up once per second. No longer check whether
srv_inc_activity_count() has been called. After each batch,
try to perform a log checkpoint, because the best chances for
the checkpoint LSN to advance by the maximum amount are upon
completing a flushing batch.
log_t: Move buf_free, max_buf_free possibly to the same cache line
with log_sys.mutex.
log_margin_checkpoint_age(): Simplify the logic, and replace
a 0.1-second sleep with a call to buf_flush_wait_flushed() to
initiate flushing. Moved to the same compilation unit
with the only caller.
log_close(): Clean up the calculations. (Should be no functional
change.) Return whether flush-ahead is needed. Moved to the same
compilation unit with the only caller.
mtr_t::finish_write(): Return whether flush-ahead is needed.
mtr_t::commit(): Invoke buf_flush_ahead() when needed. Let us avoid
external calls in mtr_t::commit() and make the logic easier to follow
by having related code in a single compilation unit. Also, we will
invoke srv_stats.log_write_requests.inc() only once per
mini-transaction commit, while not holding mutexes.
log_checkpoint_margin(): Only care about log_sys.max_checkpoint_age.
Upon reaching log_sys.max_checkpoint_age where we must wait to prevent
the log from getting corrupted, let us wait for at most 1MiB of LSN
at a time, before rechecking the condition. This should allow writers
to proceed even if the redo log capacity has been reached and
'furious flushing' is in progress. We no longer care about
log_sys.max_modified_age_sync or log_sys.max_modified_age_async.
The log_sys.max_modified_age_sync could be a relic from the time when
there was a srv_master_thread that wrote dirty pages to data files.
Also, we no longer have any log_sys.max_checkpoint_age_async limit,
because log checkpoints will now be triggered by the page cleaner
thread upon completing buf_flush_lists().
log_set_capacity(): Simplify the calculations of the limit
(no functional change).
log_checkpoint_low(): Split from log_checkpoint(). Moved to the
same compilation unit with the caller.
log_make_checkpoint(): Only wait for everything to be flushed until
the current LSN.
create_log_file(): After checkpoint, invoke log_write_up_to()
to ensure that the FILE_CHECKPOINT record has been written.
This avoids ut_ad(!srv_log_file_created) in create_log_file_rename().
srv_start(): Do not call recv_recovery_from_checkpoint_start()
if the log has just been created. Set fil_system.space_id_reuse_warned
before dict_boot() has been executed, and clear it after recovery
has finished.
dict_boot(): Initialize fil_system.max_assigned_id.
srv_check_activity(): Remove. The activity count is counting transaction
commits and therefore mostly interesting for the purge of history.
BtrBulk::insert(): Do not explicitly wake up the page cleaner,
but do invoke srv_inc_activity_count(), because that counter is
still being used in buf_load_throttle_if_needed() for some
heuristics. (It might be cleaner to execute buf_load() in the
page cleaner thread!)
Reviewed by: Vladislav Vaintroub
2020-10-26 16:35:47 +02:00
|
|
|
err = create_new_log
|
|
|
|
? DB_SUCCESS
|
|
|
|
: recv_recovery_from_checkpoint_start(flushed_lsn);
|
2020-01-12 02:05:28 +07:00
|
|
|
recv_sys.close_files();
|
|
|
|
|
2019-05-24 16:19:38 +03:00
|
|
|
recv_sys.dblwr.pages.clear();
|
2016-08-12 11:17:45 +03:00
|
|
|
|
2017-01-26 14:05:00 +02:00
|
|
|
if (err != DB_SUCCESS) {
|
2016-12-30 08:56:13 +02:00
|
|
|
return(srv_init_abort(err));
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2018-01-16 13:57:30 +02:00
|
|
|
switch (srv_operation) {
|
|
|
|
case SRV_OPERATION_NORMAL:
|
|
|
|
case SRV_OPERATION_RESTORE_EXPORT:
|
|
|
|
/* Initialize the change buffer. */
|
|
|
|
err = dict_boot();
|
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return(srv_init_abort(err));
|
|
|
|
}
|
2018-02-20 15:10:03 +02:00
|
|
|
/* fall through */
|
|
|
|
case SRV_OPERATION_RESTORE:
|
2018-01-16 13:57:30 +02:00
|
|
|
/* This must precede
|
|
|
|
recv_apply_hashed_log_recs(true). */
|
2019-12-04 15:00:57 +02:00
|
|
|
srv_undo_tablespaces_active
|
|
|
|
= trx_rseg_get_n_undo_tablespaces();
|
|
|
|
err = srv_validate_undo_tablespaces();
|
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return srv_init_abort(err);
|
|
|
|
}
|
2020-10-16 14:38:07 +03:00
|
|
|
if (srv_operation == SRV_OPERATION_RESTORE) {
|
|
|
|
break;
|
|
|
|
}
|
2018-01-30 14:16:09 +02:00
|
|
|
trx_lists_init_at_db_start();
|
2018-01-16 13:57:30 +02:00
|
|
|
break;
|
|
|
|
case SRV_OPERATION_RESTORE_DELTA:
|
|
|
|
case SRV_OPERATION_BACKUP:
|
2020-03-10 20:05:17 +02:00
|
|
|
ut_ad("wrong mariabackup mode" == 0);
|
2018-01-16 13:57:30 +02:00
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
if (srv_force_recovery < SRV_FORCE_NO_LOG_REDO) {
|
|
|
|
/* Apply the hashed log records to the
|
|
|
|
respective file pages, for the last batch of
|
|
|
|
recv_group_scan_log_recs(). */
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-03-30 18:45:09 +03:00
|
|
|
recv_sys.apply(true);
|
MDEV-12253: Buffer pool blocks are accessed after they have been freed
Problem was that bpage was referenced after it was already freed
from LRU. Fixed by adding a new variable encrypted that is
passed down to buf_page_check_corrupt() and used in
buf_page_get_gen() to stop processing page read.
This patch should also address following test failures and
bugs:
MDEV-12419: IMPORT should not look up tablespace in
PageConverter::validate(). This is now removed.
MDEV-10099: encryption.innodb_onlinealter_encryption fails
sporadically in buildbot
MDEV-11420: encryption.innodb_encryption-page-compression
failed in buildbot
MDEV-11222: encryption.encrypt_and_grep failed in buildbot on P8
Removed dict_table_t::is_encrypted and dict_table_t::ibd_file_missing
and replaced these with dict_table_t::file_unreadable. Table
ibd file is missing if fil_get_space(space_id) returns NULL
and encrypted if not. Removed dict_table_t::is_corrupted field.
Ported FilSpace class from 10.2 and using that on buf_page_check_corrupt(),
buf_page_decrypt_after_read(), buf_page_encrypt_before_write(),
buf_dblwr_process(), buf_read_page(), dict_stats_save_defrag_stats().
Added test cases when enrypted page could be read while doing
redo log crash recovery. Also added test case for row compressed
blobs.
btr_cur_open_at_index_side_func(),
btr_cur_open_at_rnd_pos_func(): Avoid referencing block that is
NULL.
buf_page_get_zip(): Issue error if page read fails.
buf_page_get_gen(): Use dberr_t for error detection and
do not reference bpage after we hare freed it.
buf_mark_space_corrupt(): remove bpage from LRU also when
it is encrypted.
buf_page_check_corrupt(): @return DB_SUCCESS if page has
been read and is not corrupted,
DB_PAGE_CORRUPTED if page based on checksum check is corrupted,
DB_DECRYPTION_FAILED if page post encryption checksum matches but
after decryption normal page checksum does not match. In read
case only DB_SUCCESS is possible.
buf_page_io_complete(): use dberr_t for error handling.
buf_flush_write_block_low(),
buf_read_ahead_random(),
buf_read_page_async(),
buf_read_ahead_linear(),
buf_read_ibuf_merge_pages(),
buf_read_recv_pages(),
fil_aio_wait():
Issue error if page read fails.
btr_pcur_move_to_next_page(): Do not reference page if it is
NULL.
Introduced dict_table_t::is_readable() and dict_index_t::is_readable()
that will return true if tablespace exists and pages read from
tablespace are not corrupted or page decryption failed.
Removed buf_page_t::key_version. After page decryption the
key version is not removed from page frame. For unencrypted
pages, old key_version is removed at buf_page_encrypt_before_write()
dict_stats_update_transient_for_index(),
dict_stats_update_transient()
Do not continue if table decryption failed or table
is corrupted.
dict0stats.cc: Introduced a dict_stats_report_error function
to avoid code duplication.
fil_parse_write_crypt_data():
Check that key read from redo log entry is found from
encryption plugin and if it is not, refuse to start.
PageConverter::validate(): Removed access to fil_space_t as
tablespace is not available during import.
Fixed error code on innodb.innodb test.
Merged test cased innodb-bad-key-change5 and innodb-bad-key-shutdown
to innodb-bad-key-change2. Removed innodb-bad-key-change5 test.
Decreased unnecessary complexity on some long lasting tests.
Removed fil_inc_pending_ops(), fil_decr_pending_ops(),
fil_get_first_space(), fil_get_next_space(),
fil_get_first_space_safe(), fil_get_next_space_safe()
functions.
fil_space_verify_crypt_checksum(): Fixed bug found using ASAN
where FIL_PAGE_END_LSN_OLD_CHECKSUM field was incorrectly
accessed from row compressed tables. Fixed out of page frame
bug for row compressed tables in
fil_space_verify_crypt_checksum() found using ASAN. Incorrect
function was called for compressed table.
Added new tests for discard, rename table and drop (we should allow them
even when page decryption fails). Alter table rename is not allowed.
Added test for restart with innodb-force-recovery=1 when page read on
redo-recovery cant be decrypted. Added test for corrupted table where
both page data and FIL_PAGE_FILE_FLUSH_LSN_OR_KEY_VERSION is corrupted.
Adjusted the test case innodb_bug14147491 so that it does not anymore
expect crash. Instead table is just mostly not usable.
fil0fil.h: fil_space_acquire_low is not visible function
and fil_space_acquire and fil_space_acquire_silent are
inline functions. FilSpace class uses fil_space_acquire_low
directly.
recv_apply_hashed_log_recs() does not return anything.
2017-04-26 15:19:16 +03:00
|
|
|
|
2020-12-04 16:18:04 +02:00
|
|
|
if (recv_sys.is_corrupt_log()
|
|
|
|
|| recv_sys.is_corrupt_fs()) {
|
2017-07-05 22:09:28 +03:00
|
|
|
return(srv_init_abort(DB_CORRUPTION));
|
MDEV-12253: Buffer pool blocks are accessed after they have been freed
Problem was that bpage was referenced after it was already freed
from LRU. Fixed by adding a new variable encrypted that is
passed down to buf_page_check_corrupt() and used in
buf_page_get_gen() to stop processing page read.
This patch should also address following test failures and
bugs:
MDEV-12419: IMPORT should not look up tablespace in
PageConverter::validate(). This is now removed.
MDEV-10099: encryption.innodb_onlinealter_encryption fails
sporadically in buildbot
MDEV-11420: encryption.innodb_encryption-page-compression
failed in buildbot
MDEV-11222: encryption.encrypt_and_grep failed in buildbot on P8
Removed dict_table_t::is_encrypted and dict_table_t::ibd_file_missing
and replaced these with dict_table_t::file_unreadable. Table
ibd file is missing if fil_get_space(space_id) returns NULL
and encrypted if not. Removed dict_table_t::is_corrupted field.
Ported FilSpace class from 10.2 and using that on buf_page_check_corrupt(),
buf_page_decrypt_after_read(), buf_page_encrypt_before_write(),
buf_dblwr_process(), buf_read_page(), dict_stats_save_defrag_stats().
Added test cases when enrypted page could be read while doing
redo log crash recovery. Also added test case for row compressed
blobs.
btr_cur_open_at_index_side_func(),
btr_cur_open_at_rnd_pos_func(): Avoid referencing block that is
NULL.
buf_page_get_zip(): Issue error if page read fails.
buf_page_get_gen(): Use dberr_t for error detection and
do not reference bpage after we hare freed it.
buf_mark_space_corrupt(): remove bpage from LRU also when
it is encrypted.
buf_page_check_corrupt(): @return DB_SUCCESS if page has
been read and is not corrupted,
DB_PAGE_CORRUPTED if page based on checksum check is corrupted,
DB_DECRYPTION_FAILED if page post encryption checksum matches but
after decryption normal page checksum does not match. In read
case only DB_SUCCESS is possible.
buf_page_io_complete(): use dberr_t for error handling.
buf_flush_write_block_low(),
buf_read_ahead_random(),
buf_read_page_async(),
buf_read_ahead_linear(),
buf_read_ibuf_merge_pages(),
buf_read_recv_pages(),
fil_aio_wait():
Issue error if page read fails.
btr_pcur_move_to_next_page(): Do not reference page if it is
NULL.
Introduced dict_table_t::is_readable() and dict_index_t::is_readable()
that will return true if tablespace exists and pages read from
tablespace are not corrupted or page decryption failed.
Removed buf_page_t::key_version. After page decryption the
key version is not removed from page frame. For unencrypted
pages, old key_version is removed at buf_page_encrypt_before_write()
dict_stats_update_transient_for_index(),
dict_stats_update_transient()
Do not continue if table decryption failed or table
is corrupted.
dict0stats.cc: Introduced a dict_stats_report_error function
to avoid code duplication.
fil_parse_write_crypt_data():
Check that key read from redo log entry is found from
encryption plugin and if it is not, refuse to start.
PageConverter::validate(): Removed access to fil_space_t as
tablespace is not available during import.
Fixed error code on innodb.innodb test.
Merged test cased innodb-bad-key-change5 and innodb-bad-key-shutdown
to innodb-bad-key-change2. Removed innodb-bad-key-change5 test.
Decreased unnecessary complexity on some long lasting tests.
Removed fil_inc_pending_ops(), fil_decr_pending_ops(),
fil_get_first_space(), fil_get_next_space(),
fil_get_first_space_safe(), fil_get_next_space_safe()
functions.
fil_space_verify_crypt_checksum(): Fixed bug found using ASAN
where FIL_PAGE_END_LSN_OLD_CHECKSUM field was incorrectly
accessed from row compressed tables. Fixed out of page frame
bug for row compressed tables in
fil_space_verify_crypt_checksum() found using ASAN. Incorrect
function was called for compressed table.
Added new tests for discard, rename table and drop (we should allow them
even when page decryption fails). Alter table rename is not allowed.
Added test for restart with innodb-force-recovery=1 when page read on
redo-recovery cant be decrypted. Added test for corrupted table where
both page data and FIL_PAGE_FILE_FLUSH_LSN_OR_KEY_VERSION is corrupted.
Adjusted the test case innodb_bug14147491 so that it does not anymore
expect crash. Instead table is just mostly not usable.
fil0fil.h: fil_space_acquire_low is not visible function
and fil_space_acquire and fil_space_acquire_silent are
inline functions. FilSpace class uses fil_space_acquire_low
directly.
recv_apply_hashed_log_recs() does not return anything.
2017-04-26 15:19:16 +03:00
|
|
|
}
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
DBUG_PRINT("ib_log", ("apply completed"));
|
|
|
|
|
|
|
|
if (recv_needed_recovery) {
|
|
|
|
trx_sys_print_mysql_binlog_offset();
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
MDEV-23855: Improve InnoDB log checkpoint performance
After MDEV-15053, MDEV-22871, MDEV-23399 shifted the scalability
bottleneck, log checkpoints became a new bottleneck.
If innodb_io_capacity is set low or innodb_max_dirty_pct_lwm is
set high and the workload fits in the buffer pool, the page cleaner
thread will perform very little flushing. When we reach the capacity
of the circular redo log file ib_logfile0 and must initiate a checkpoint,
some 'furious flushing' will be necessary. (If innodb_flush_sync=OFF,
then flushing would continue at the innodb_io_capacity rate, and
writers would be throttled.)
We have the best chance of advancing the checkpoint LSN immediately
after a page flush batch has been completed. Hence, it is best to
perform checkpoints after every batch in the page cleaner thread,
attempting to run once per second.
By initiating high-priority flushing in the page cleaner as early
as possible, we aim to make the throughput more stable.
The function buf_flush_wait_flushed() used to sleep for 10ms, hoping
that the page cleaner thread would do something during that time.
The observed end result was that a large number of threads that call
log_free_check() would end up sleeping while nothing useful is happening.
We will revise the design so that in the default innodb_flush_sync=ON
mode, buf_flush_wait_flushed() will wake up the page cleaner thread
to perform the necessary flushing, and it will wait for a signal from
the page cleaner thread.
If innodb_io_capacity is set to a low value (causing the page cleaner to
throttle its work), a write workload would initially perform well, until
the capacity of the circular ib_logfile0 is reached and log_free_check()
will trigger checkpoints. At that point, the extra waiting in
buf_flush_wait_flushed() will start reducing throughput.
The page cleaner thread will also initiate log checkpoints after each
buf_flush_lists() call, because that is the best point of time for
the checkpoint LSN to advance by the maximum amount.
Even in 'furious flushing' mode we invoke buf_flush_lists() with
innodb_io_capacity_max pages at a time, and at the start of each
batch (in the log_flush() callback function that runs in a separate
task) we will invoke os_aio_wait_until_no_pending_writes(). This
tweak allows the checkpoint to advance in smaller steps and
significantly reduces the maximum latency. On an Intel Optane 960
NVMe SSD on Linux, it reduced from 4.6 seconds to 74 milliseconds.
On Microsoft Windows with a slower SSD, it reduced from more than
180 seconds to 0.6 seconds.
We will make innodb_adaptive_flushing=OFF simply flush innodb_io_capacity
per second whenever the dirty proportion of buffer pool pages exceeds
innodb_max_dirty_pages_pct_lwm. For innodb_adaptive_flushing=ON we try
to make page_cleaner_flush_pages_recommendation() more consistent and
predictable: if we are below innodb_adaptive_flushing_lwm, let us flush
pages according to the return value of af_get_pct_for_dirty().
innodb_max_dirty_pages_pct_lwm: Revert the change of the default value
that was made in MDEV-23399. The value innodb_max_dirty_pages_pct_lwm=0
guarantees that a shutdown of an idle server will be fast. Users might
be surprised if normal shutdown suddenly became slower when upgrading
within a GA release series.
innodb_checkpoint_usec: Remove. The master task will no longer perform
periodic log checkpoints. It is the duty of the page cleaner thread.
log_sys.max_modified_age: Remove. The current span of the
buf_pool.flush_list expressed in LSN only matters for adaptive
flushing (outside the 'furious flushing' condition).
For the correctness of checkpoints, the only thing that matters is
the checkpoint age (log_sys.lsn - log_sys.last_checkpoint_lsn).
This run-time constant was also reported as log_max_modified_age_sync.
log_sys.max_checkpoint_age_async: Remove. This does not serve any
purpose, because the checkpoints will now be triggered by the page
cleaner thread. We will retain the log_sys.max_checkpoint_age limit
for engaging 'furious flushing'.
page_cleaner.slot: Remove. It turns out that
page_cleaner_slot.flush_list_time was duplicating
page_cleaner.slot.flush_time and page_cleaner.slot.flush_list_pass
was duplicating page_cleaner.flush_pass.
Likewise, there were some redundant monitor counters, because the
page cleaner thread no longer performs any buf_pool.LRU flushing, and
because there only is one buf_flush_page_cleaner thread.
buf_flush_sync_lsn: Protect writes by buf_pool.flush_list_mutex.
buf_pool_t::get_oldest_modification(): Add a parameter to specify the
return value when no persistent data pages are dirty. Require the
caller to hold buf_pool.flush_list_mutex.
log_buf_pool_get_oldest_modification(): Take the fall-back LSN
as a parameter. All callers will also invoke log_sys.get_lsn().
log_preflush_pool_modified_pages(): Replaced with buf_flush_wait_flushed().
buf_flush_wait_flushed(): Implement two limits. If not enough buffer pool
has been flushed, signal the page cleaner (unless innodb_flush_sync=OFF)
and wait for the page cleaner to complete. If the page cleaner
thread is not running (which can be the case durign shutdown),
initiate the flush and wait for it directly.
buf_flush_ahead(): If innodb_flush_sync=ON (the default),
submit a new buf_flush_sync_lsn target for the page cleaner
but do not wait for the flushing to finish.
log_get_capacity(), log_get_max_modified_age_async(): Remove, to make
it easier to see that af_get_pct_for_lsn() is not acquiring any mutexes.
page_cleaner_flush_pages_recommendation(): Protect all access to
buf_pool.flush_list with buf_pool.flush_list_mutex. Previously there
were some race conditions in the calculation.
buf_flush_sync_for_checkpoint(): New function to process
buf_flush_sync_lsn in the page cleaner thread. At the end of
each batch, we try to wake up any blocked buf_flush_wait_flushed().
If everything up to buf_flush_sync_lsn has been flushed, we will
reset buf_flush_sync_lsn=0. The page cleaner thread will keep
'furious flushing' until the limit is reached. Any threads that
are waiting in buf_flush_wait_flushed() will be able to resume
as soon as their own limit has been satisfied.
buf_flush_page_cleaner: Prioritize buf_flush_sync_lsn and do not
sleep as long as it is set. Do not update any page_cleaner statistics
for this special mode of operation. In the normal mode
(buf_flush_sync_lsn is not set for innodb_flush_sync=ON),
try to wake up once per second. No longer check whether
srv_inc_activity_count() has been called. After each batch,
try to perform a log checkpoint, because the best chances for
the checkpoint LSN to advance by the maximum amount are upon
completing a flushing batch.
log_t: Move buf_free, max_buf_free possibly to the same cache line
with log_sys.mutex.
log_margin_checkpoint_age(): Simplify the logic, and replace
a 0.1-second sleep with a call to buf_flush_wait_flushed() to
initiate flushing. Moved to the same compilation unit
with the only caller.
log_close(): Clean up the calculations. (Should be no functional
change.) Return whether flush-ahead is needed. Moved to the same
compilation unit with the only caller.
mtr_t::finish_write(): Return whether flush-ahead is needed.
mtr_t::commit(): Invoke buf_flush_ahead() when needed. Let us avoid
external calls in mtr_t::commit() and make the logic easier to follow
by having related code in a single compilation unit. Also, we will
invoke srv_stats.log_write_requests.inc() only once per
mini-transaction commit, while not holding mutexes.
log_checkpoint_margin(): Only care about log_sys.max_checkpoint_age.
Upon reaching log_sys.max_checkpoint_age where we must wait to prevent
the log from getting corrupted, let us wait for at most 1MiB of LSN
at a time, before rechecking the condition. This should allow writers
to proceed even if the redo log capacity has been reached and
'furious flushing' is in progress. We no longer care about
log_sys.max_modified_age_sync or log_sys.max_modified_age_async.
The log_sys.max_modified_age_sync could be a relic from the time when
there was a srv_master_thread that wrote dirty pages to data files.
Also, we no longer have any log_sys.max_checkpoint_age_async limit,
because log checkpoints will now be triggered by the page cleaner
thread upon completing buf_flush_lists().
log_set_capacity(): Simplify the calculations of the limit
(no functional change).
log_checkpoint_low(): Split from log_checkpoint(). Moved to the
same compilation unit with the caller.
log_make_checkpoint(): Only wait for everything to be flushed until
the current LSN.
create_log_file(): After checkpoint, invoke log_write_up_to()
to ensure that the FILE_CHECKPOINT record has been written.
This avoids ut_ad(!srv_log_file_created) in create_log_file_rename().
srv_start(): Do not call recv_recovery_from_checkpoint_start()
if the log has just been created. Set fil_system.space_id_reuse_warned
before dict_boot() has been executed, and clear it after recovery
has finished.
dict_boot(): Initialize fil_system.max_assigned_id.
srv_check_activity(): Remove. The activity count is counting transaction
commits and therefore mostly interesting for the purge of history.
BtrBulk::insert(): Do not explicitly wake up the page cleaner,
but do invoke srv_inc_activity_count(), because that counter is
still being used in buf_load_throttle_if_needed() for some
heuristics. (It might be cleaner to execute buf_load() in the
page cleaner thread!)
Reviewed by: Vladislav Vaintroub
2020-10-26 16:35:47 +02:00
|
|
|
fil_system.space_id_reuse_warned = false;
|
|
|
|
|
2017-01-26 14:05:00 +02:00
|
|
|
if (!srv_read_only_mode) {
|
|
|
|
const ulint flags = FSP_FLAGS_PAGE_SSIZE();
|
|
|
|
for (ulint id = 0; id <= srv_undo_tablespaces; id++) {
|
2018-03-27 11:49:57 +03:00
|
|
|
if (fil_space_t* space = fil_space_get(id)) {
|
|
|
|
fsp_flags_try_adjust(space, flags);
|
2017-01-26 14:05:00 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (sum_of_new_sizes > 0) {
|
|
|
|
/* New data file(s) were added */
|
|
|
|
mtr.start();
|
2020-11-20 12:30:55 +02:00
|
|
|
mtr.x_lock_space(fil_system.sys_space);
|
2018-03-28 09:29:14 +03:00
|
|
|
buf_block_t* block = buf_page_get(
|
2019-02-06 19:50:11 +02:00
|
|
|
page_id_t(0, 0), 0,
|
2018-03-28 09:29:14 +03:00
|
|
|
RW_SX_LATCH, &mtr);
|
|
|
|
ulint size = mach_read_from_4(
|
|
|
|
FSP_HEADER_OFFSET + FSP_SIZE
|
|
|
|
+ block->frame);
|
|
|
|
ut_ad(size == fil_system.sys_space
|
|
|
|
->size_in_header);
|
|
|
|
size += sum_of_new_sizes;
|
2019-12-03 10:19:45 +02:00
|
|
|
mtr.write<4>(*block,
|
|
|
|
FSP_HEADER_OFFSET + FSP_SIZE
|
|
|
|
+ block->frame, size);
|
2020-10-15 16:28:19 +03:00
|
|
|
fil_system.sys_space->size_in_header
|
|
|
|
= uint32_t(size);
|
2017-01-26 14:05:00 +02:00
|
|
|
mtr.commit();
|
|
|
|
/* Immediately write the log record about
|
|
|
|
increased tablespace size to disk, so that it
|
|
|
|
is durable even if mysqld would crash
|
|
|
|
quickly */
|
|
|
|
log_buffer_flush_to_disk();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-03-28 09:29:14 +03:00
|
|
|
#ifdef UNIV_DEBUG
|
|
|
|
{
|
|
|
|
mtr.start();
|
2019-02-06 19:50:11 +02:00
|
|
|
buf_block_t* block = buf_page_get(page_id_t(0, 0), 0,
|
2018-03-28 09:29:14 +03:00
|
|
|
RW_S_LATCH, &mtr);
|
|
|
|
ut_ad(mach_read_from_4(FSP_SIZE + FSP_HEADER_OFFSET
|
|
|
|
+ block->frame)
|
|
|
|
== fil_system.sys_space->size_in_header);
|
|
|
|
mtr.commit();
|
|
|
|
}
|
|
|
|
#endif
|
2017-01-26 14:05:00 +02:00
|
|
|
const ulint tablespace_size_in_header
|
2018-03-28 09:29:14 +03:00
|
|
|
= fil_system.sys_space->size_in_header;
|
2017-01-26 14:05:00 +02:00
|
|
|
const ulint sum_of_data_file_sizes
|
|
|
|
= srv_sys_space.get_sum_of_sizes();
|
|
|
|
/* Compare the system tablespace file size to what is
|
2017-06-08 12:45:08 +03:00
|
|
|
stored in FSP_SIZE. In srv_sys_space.open_or_create()
|
2017-01-26 14:05:00 +02:00
|
|
|
we already checked that the file sizes match the
|
|
|
|
innodb_data_file_path specification. */
|
|
|
|
if (srv_read_only_mode
|
|
|
|
|| sum_of_data_file_sizes == tablespace_size_in_header) {
|
|
|
|
/* Do not complain about the size. */
|
|
|
|
} else if (!srv_sys_space.can_auto_extend_last_file()
|
|
|
|
|| sum_of_data_file_sizes
|
|
|
|
< tablespace_size_in_header) {
|
|
|
|
ib::error() << "Tablespace size stored in header is "
|
|
|
|
<< tablespace_size_in_header
|
|
|
|
<< " pages, but the sum of data file sizes is "
|
|
|
|
<< sum_of_data_file_sizes << " pages";
|
|
|
|
|
|
|
|
if (srv_force_recovery == 0
|
|
|
|
&& sum_of_data_file_sizes
|
|
|
|
< tablespace_size_in_header) {
|
|
|
|
ib::error() <<
|
|
|
|
"Cannot start InnoDB. The tail of"
|
|
|
|
" the system tablespace is"
|
|
|
|
" missing. Have you edited"
|
|
|
|
" innodb_data_file_path in my.cnf"
|
|
|
|
" in an inappropriate way, removing"
|
|
|
|
" data files from there?"
|
|
|
|
" You can set innodb_force_recovery=1"
|
|
|
|
" in my.cnf to force"
|
|
|
|
" a startup if you are trying to"
|
|
|
|
" recover a badly corrupt database.";
|
|
|
|
|
|
|
|
return(srv_init_abort(DB_ERROR));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
recv_sys.debug_free();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2017-10-10 10:28:54 +03:00
|
|
|
if (srv_operation == SRV_OPERATION_RESTORE
|
|
|
|
|| srv_operation == SRV_OPERATION_RESTORE_EXPORT) {
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
/* After applying the redo log from
|
|
|
|
SRV_OPERATION_BACKUP, flush the changes
|
2017-10-10 10:28:54 +03:00
|
|
|
to the data files and truncate or delete the log.
|
|
|
|
Unless --export is specified, no further change to
|
|
|
|
InnoDB files is needed. */
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
ut_ad(!srv_force_recovery);
|
|
|
|
ut_ad(recv_no_log_write);
|
2021-03-30 09:58:24 +03:00
|
|
|
err = fil_write_flushed_lsn(log_sys.get_lsn());
|
2020-06-08 13:14:27 +03:00
|
|
|
DBUG_ASSERT(!buf_pool.any_io_pending());
|
2020-01-12 02:05:28 +07:00
|
|
|
log_sys.log.close_file();
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
if (err == DB_SUCCESS) {
|
2017-10-10 10:28:54 +03:00
|
|
|
bool trunc = srv_operation
|
|
|
|
== SRV_OPERATION_RESTORE;
|
2020-01-12 02:05:28 +07:00
|
|
|
if (!trunc) {
|
|
|
|
delete_log_file("0");
|
|
|
|
} else {
|
|
|
|
auto logfile0 = get_log_file_path();
|
2017-10-10 10:28:54 +03:00
|
|
|
/* Truncate the first log file. */
|
2020-01-12 02:05:28 +07:00
|
|
|
fclose(fopen(logfile0.c_str(), "w"));
|
2017-10-10 10:28:54 +03:00
|
|
|
}
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
}
|
|
|
|
return(err);
|
|
|
|
}
|
|
|
|
|
2017-02-10 12:11:42 +02:00
|
|
|
/* Upgrade or resize or rebuild the redo logs before
|
|
|
|
generating any dirty pages, so that the old redo log
|
2020-01-12 02:05:28 +07:00
|
|
|
file will not be written to. */
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
if (srv_force_recovery == SRV_FORCE_NO_LOG_REDO) {
|
|
|
|
/* Completely ignore the redo log. */
|
|
|
|
} else if (srv_read_only_mode) {
|
|
|
|
/* Leave the redo log alone. */
|
|
|
|
} else if (srv_log_file_size_requested == srv_log_file_size
|
2020-01-12 02:05:28 +07:00
|
|
|
&& srv_log_file_found
|
2018-04-27 10:06:14 +03:00
|
|
|
&& log_sys.log.format
|
2018-01-03 15:22:52 +02:00
|
|
|
== (srv_encrypt_log
|
MDEV-12353: Change the redo log encoding
log_t::FORMAT_10_5: physical redo log format tag
log_phys_t: Buffered records in the physical format.
The log record bytes will follow the last data field,
making use of alignment padding that would otherwise be wasted.
If there are multiple records for the same page, also those
may be appended to an existing log_phys_t object if the memory
is available.
In the physical format, the first byte of a record identifies the
record and its length (up to 15 bytes). For longer records, the
immediately following bytes will encode the remaining length
in a variable-length encoding. Usually, a variable-length-encoded
page identifier will follow, followed by optional payload, whose
length is included in the initially encoded total record length.
When a mini-transaction is updating multiple fields in a page,
it can avoid repeating the tablespace identifier and page number
by setting the same_page flag (most significant bit) in the first
byte of the log record. The byte offset of the record will be
relative to where the previous record for that page ended.
Until MDEV-14425 introduces a separate file-level log for
redo log checkpoints and file operations, we will write the
file-level records in the page-level redo log file.
The record FILE_CHECKPOINT (which replaces MLOG_CHECKPOINT)
will be removed in MDEV-14425, and one sequential scan of the
page recovery log will suffice.
Compared to MLOG_FILE_CREATE2, FILE_CREATE will not include any flags.
If the information is needed, it can be parsed from WRITE records that
modify FSP_SPACE_FLAGS.
MLOG_ZIP_WRITE_STRING: Remove. The record was only introduced temporarily
as part of this work, before being replaced with WRITE (along with
MLOG_WRITE_STRING, MLOG_1BYTE, MLOG_nBYTES).
mtr_buf_t::empty(): Check if the buffer is empty.
mtr_t::m_n_log_recs: Remove. It suffices to check if m_log is empty.
mtr_t::m_last, mtr_t::m_last_offset: End of the latest m_log record,
for the same_page encoding.
page_recv_t::last_offset: Reflects mtr_t::m_last_offset.
Valid values for last_offset during recovery should be 0 or above 8.
(The first 8 bytes of a page are the checksum and the page number,
and neither are ever updated directly by log records.)
Internally, the special value 1 indicates that the same_page form
will not be allowed for the subsequent record.
mtr_t::page_create(): Take the block descriptor as parameter,
so that it can be compared to mtr_t::m_last. The INIT_INDEX_PAGE
record will always followed by a subtype byte, because same_page
records must be longer than 1 byte.
trx_undo_page_init(): Combine the writes in WRITE record.
trx_undo_header_create(): Write 4 bytes using a special MEMSET
record that includes 1 bytes of length and 2 bytes of payload.
flst_write_addr(): Define as a static function. Combine the writes.
flst_zero_both(): Replaces two flst_zero_addr() calls.
flst_init(): Do not inline the function.
fsp_free_seg_inode(): Zerofill the whole inode.
fsp_apply_init_file_page(): Initialize FIL_PAGE_PREV,FIL_PAGE_NEXT
to FIL_NULL when using the physical format.
btr_create(): Assert !page_has_siblings() because fsp_apply_init_file_page()
must have been invoked.
fil_ibd_create(): Do not write FILE_MODIFY after FILE_CREATE.
fil_names_dirty_and_write(): Remove the parameter mtr.
Write the records using a separate mini-transaction object,
because any FILE_ records must be at the start of a mini-transaction log.
recv_recover_page(): Add a fil_space_t* parameter.
After applying log to the a ROW_FORMAT=COMPRESSED page,
invoke buf_zip_decompress() to restore the uncompressed page.
buf_page_io_complete(): Remove the temporary hack to discard the
uncompressed page of a ROW_FORMAT=COMPRESSED page.
page_zip_write_header(): Remove. Use mtr_t::write() or
mtr_t::memset() instead, and update the compressed page frame
separately.
trx_undo_header_add_space_for_xid(): Remove.
trx_undo_seg_create(): Perform the changes that were previously
made by trx_undo_header_add_space_for_xid().
btr_reset_instant(): New function: Reset the table to MariaDB 10.2
or 10.3 format when rolling back an instant ALTER TABLE operation.
page_rec_find_owner_rec(): Merge with the only callers.
page_cur_insert_rec_low(): Combine writes by using a local buffer.
MEMMOVE data from the preceding record whenever feasible
(copying at least 3 bytes).
page_cur_insert_rec_zip(): Combine writes to page header fields.
PageBulk::insertPage(): Issue MEMMOVE records to copy a matching
part from the preceding record.
PageBulk::finishPage(): Combine the writes to the page header
and to the sparse page directory slots.
mtr_t::write(): Only log the least significant (last) bytes
of multi-byte fields that actually differ.
For updating FSP_SIZE, we must always write all 4 bytes to the
redo log, so that the fil_space_set_recv_size() logic in
recv_sys_t::parse() will work.
mtr_t::memcpy(), mtr_t::zmemcpy(): Take a pointer argument
instead of a numeric offset to the page frame. Only log the
last bytes of multi-byte fields that actually differ.
In fil_space_crypt_t::write_page0(), we must log also any
unchanged bytes, so that recovery will recognize the record
and invoke fil_crypt_parse().
Future work:
MDEV-21724 Optimize page_cur_insert_rec_low() redo logging
MDEV-21725 Optimize btr_page_reorganize_low() redo logging
MDEV-21727 Optimize redo logging for ROW_FORMAT=COMPRESSED
2020-02-13 19:12:17 +02:00
|
|
|
? log_t::FORMAT_ENC_10_5
|
|
|
|
: log_t::FORMAT_10_5)
|
2018-09-07 22:15:06 +03:00
|
|
|
&& log_sys.log.subformat == 2) {
|
MDEV-14717: Prevent crash-downgrade to earlier MariaDB 10.2
A crash-downgrade of a RENAME (or TRUNCATE or table-rebuilding
ALTER TABLE or OPTIMIZE TABLE) operation to an earlier 10.2 version
would trigger a debug assertion failure during rollback,
in trx_roll_pop_top_rec_of_trx(). In a non-debug build, the
TRX_UNDO_RENAME_TABLE record would be misinterpreted as an
update_undo log record, and typically the file name would be
interpreted as DB_TRX_ID,DB_ROLL_PTR,PRIMARY KEY. If a matching
record would be found, row_undo_mod() would hit ut_error in
switch (node->rec_type). Typically, ut_a(table2 == NULL) would
fail when opening the table from SQL.
Because of this, we prevent a crash-downgrade to earlier MariaDB 10.2
versions by changing the InnoDB redo log format identifier to the
10.3 identifier, and by introducing a subformat identifier so that
10.2 can continue to refuse crash-downgrade from 10.3 or later.
After a clean shutdown, a downgrade to MariaDB 10.2.13 or later would
still be possible thanks to MDEV-14909. A downgrade to older 10.2
versions is only possible after removing the log files (not recommended).
LOG_HEADER_FORMAT_CURRENT: Change to 103 (originally the 10.3 format).
log_group_t: Add subformat. For 10.2, we will use subformat 1,
and will refuse crash recovery from any other subformat of the
10.3 format, that is, a genuine 10.3 redo log.
recv_find_max_checkpoint(): Allow startup after clean shutdown
from a future LOG_HEADER_FORMAT_10_4 (unencrypted only).
We cannot handle the encrypted 10.4 redo log block format,
which was introduced in MDEV-12041. Allow crash recovery from
the original 10.2 format as well as the new format.
In Mariabackup --backup, do not allow any startup from 10.3 or 10.4
redo logs.
recv_recovery_from_checkpoint_start(): Skip redo log apply for
clean 10.3 redo log, but not for the new 10.2 redo log
(10.3 format, subformat 1).
srv_prepare_to_delete_redo_log_files(): On format or subformat
mismatch, set srv_log_file_size = 0, so that we will display the
correct message.
innobase_start_or_create_for_mysql(): Check for format or subformat
mismatch.
xtrabackup_backup_func(): Remove debug assertions that were made
redundant by the code changes in recv_find_max_checkpoint().
2018-09-07 17:24:31 +03:00
|
|
|
/* No need to add or remove encryption,
|
|
|
|
upgrade, downgrade, or resize. */
|
2017-02-10 12:11:42 +02:00
|
|
|
} else {
|
2020-01-12 02:05:28 +07:00
|
|
|
/* Prepare to delete the old redo log file */
|
|
|
|
flushed_lsn = srv_prepare_to_delete_redo_log_file(
|
|
|
|
srv_log_file_found);
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_1",
|
|
|
|
return(srv_init_abort(DB_ERROR)););
|
|
|
|
/* Prohibit redo log writes from any other
|
|
|
|
threads until creating a log checkpoint at the
|
2020-01-12 02:05:28 +07:00
|
|
|
end of create_log_file(). */
|
2017-02-10 12:11:42 +02:00
|
|
|
ut_d(recv_no_log_write = true);
|
2020-06-08 13:14:27 +03:00
|
|
|
DBUG_ASSERT(!buf_pool.any_io_pending());
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_3",
|
|
|
|
return(srv_init_abort(DB_ERROR)););
|
2017-02-16 09:18:46 +02:00
|
|
|
DBUG_PRINT("ib_log", ("After innodb_log_abort_3"));
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
/* Stamp the LSN to the data files. */
|
|
|
|
err = fil_write_flushed_lsn(flushed_lsn);
|
|
|
|
|
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_4", err = DB_ERROR;);
|
2017-02-16 09:18:46 +02:00
|
|
|
DBUG_PRINT("ib_log", ("After innodb_log_abort_4"));
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return(srv_init_abort(err));
|
|
|
|
}
|
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
/* Close the redo log file, so that we can replace it */
|
|
|
|
log_sys.log.close_file();
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
DBUG_EXECUTE_IF("innodb_log_abort_5",
|
|
|
|
return(srv_init_abort(DB_ERROR)););
|
2017-02-16 09:18:46 +02:00
|
|
|
DBUG_PRINT("ib_log", ("After innodb_log_abort_5"));
|
2017-02-10 12:11:42 +02:00
|
|
|
|
2020-01-12 02:05:28 +07:00
|
|
|
ib::info()
|
|
|
|
<< "Starting to delete and rewrite log file.";
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
srv_log_file_size = srv_log_file_size_requested;
|
|
|
|
|
2020-10-26 15:59:30 +02:00
|
|
|
err = create_log_file(false, flushed_lsn, logfile0);
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
if (err == DB_SUCCESS) {
|
2020-01-12 02:05:28 +07:00
|
|
|
err = create_log_file_rename(flushed_lsn,
|
|
|
|
logfile0);
|
2017-02-10 12:11:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return(srv_init_abort(err));
|
|
|
|
}
|
|
|
|
}
|
2019-05-23 17:34:08 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
ut_ad(err == DB_SUCCESS);
|
|
|
|
ut_a(sum_of_new_sizes != ULINT_UNDEFINED);
|
|
|
|
|
|
|
|
/* Create the doublewrite buffer to a new tablespace */
|
|
|
|
if (!srv_read_only_mode && srv_force_recovery < SRV_FORCE_NO_TRX_UNDO
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
&& !buf_dblwr.create()) {
|
2019-05-23 17:34:08 +03:00
|
|
|
return(srv_init_abort(DB_ERROR));
|
|
|
|
}
|
|
|
|
|
|
|
|
/* Here the double write buffer has already been created and so
|
|
|
|
any new rollback segments will be allocated after the double
|
|
|
|
write buffer. The default segment should already exist.
|
|
|
|
We create the new segments only if it's a new database or
|
|
|
|
the database was shutdown cleanly. */
|
2017-02-10 12:11:42 +02:00
|
|
|
|
2019-05-23 17:34:08 +03:00
|
|
|
/* Note: When creating the extra rollback segments during an upgrade
|
|
|
|
we violate the latching order, even if the change buffer is empty.
|
2020-12-04 19:02:58 +02:00
|
|
|
It cannot create a deadlock because we are still
|
2019-05-23 17:34:08 +03:00
|
|
|
running in single threaded mode essentially. Only the IO threads
|
|
|
|
should be running at this stage. */
|
|
|
|
|
|
|
|
if (!trx_sys_create_rsegs()) {
|
|
|
|
return(srv_init_abort(DB_ERROR));
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!create_new_db) {
|
2019-11-28 16:35:20 +02:00
|
|
|
ut_ad(high_level_read_only
|
|
|
|
|| srv_force_recovery <= SRV_FORCE_NO_IBUF_MERGE);
|
2017-02-10 12:11:42 +02:00
|
|
|
|
|
|
|
/* Validate a few system page types that were left
|
2019-05-23 17:34:08 +03:00
|
|
|
uninitialized before MySQL or MariaDB 5.5. */
|
2019-11-28 16:22:53 +02:00
|
|
|
if (!high_level_read_only
|
|
|
|
&& !fil_system.sys_space->full_crc32()) {
|
2017-02-10 12:11:42 +02:00
|
|
|
buf_block_t* block;
|
|
|
|
mtr.start();
|
|
|
|
/* Bitmap page types will be reset in
|
|
|
|
buf_dblwr_check_block() without redo logging. */
|
|
|
|
block = buf_page_get(
|
|
|
|
page_id_t(IBUF_SPACE_ID,
|
|
|
|
FSP_IBUF_HEADER_PAGE_NO),
|
2019-02-06 19:50:11 +02:00
|
|
|
0, RW_X_LATCH, &mtr);
|
2018-10-18 18:23:12 +03:00
|
|
|
fil_block_check_type(*block, FIL_PAGE_TYPE_SYS, &mtr);
|
2017-02-10 12:11:42 +02:00
|
|
|
/* Already MySQL 3.23.53 initialized
|
|
|
|
FSP_IBUF_TREE_ROOT_PAGE_NO to
|
|
|
|
FIL_PAGE_INDEX. No need to reset that one. */
|
|
|
|
block = buf_page_get(
|
|
|
|
page_id_t(TRX_SYS_SPACE, TRX_SYS_PAGE_NO),
|
2019-02-06 19:50:11 +02:00
|
|
|
0, RW_X_LATCH, &mtr);
|
2018-10-18 18:23:12 +03:00
|
|
|
fil_block_check_type(*block, FIL_PAGE_TYPE_TRX_SYS,
|
2017-02-10 12:11:42 +02:00
|
|
|
&mtr);
|
|
|
|
block = buf_page_get(
|
|
|
|
page_id_t(TRX_SYS_SPACE,
|
|
|
|
FSP_FIRST_RSEG_PAGE_NO),
|
2019-02-06 19:50:11 +02:00
|
|
|
0, RW_X_LATCH, &mtr);
|
2018-10-18 18:23:12 +03:00
|
|
|
fil_block_check_type(*block, FIL_PAGE_TYPE_SYS, &mtr);
|
2017-02-10 12:11:42 +02:00
|
|
|
block = buf_page_get(
|
|
|
|
page_id_t(TRX_SYS_SPACE, FSP_DICT_HDR_PAGE_NO),
|
2019-02-06 19:50:11 +02:00
|
|
|
0, RW_X_LATCH, &mtr);
|
2018-10-18 18:23:12 +03:00
|
|
|
fil_block_check_type(*block, FIL_PAGE_TYPE_SYS, &mtr);
|
2017-02-10 12:11:42 +02:00
|
|
|
mtr.commit();
|
|
|
|
}
|
|
|
|
|
2019-11-28 16:35:20 +02:00
|
|
|
/* Roll back any recovered data dictionary
|
|
|
|
transactions, so that the data dictionary tables will
|
|
|
|
be free of any locks. The data dictionary latch
|
|
|
|
should guarantee that there is at most one data
|
|
|
|
dictionary transaction active at a time. */
|
|
|
|
if (!high_level_read_only
|
|
|
|
&& srv_force_recovery < SRV_FORCE_NO_TRX_UNDO) {
|
2019-08-23 08:06:17 +03:00
|
|
|
/* If the following call is ever removed, the
|
|
|
|
first-time ha_innobase::open() must hold (or
|
|
|
|
acquire and release) a table lock that
|
|
|
|
conflicts with trx_resurrect_table_locks(), to
|
2019-11-28 16:35:20 +02:00
|
|
|
ensure that any recovered incomplete ALTER
|
|
|
|
TABLE will have been rolled back. Otherwise,
|
|
|
|
dict_table_t::instant could be cleared by
|
|
|
|
rollback invoking
|
|
|
|
dict_index_t::clear_instant_alter() while open
|
|
|
|
table handles exist in client connections. */
|
2017-12-21 15:45:40 +04:00
|
|
|
trx_rollback_recovered(false);
|
2017-02-10 12:11:42 +02:00
|
|
|
}
|
|
|
|
|
MDEV-19514 Defer change buffer merge until pages are requested
We will remove the InnoDB background operation of merging buffered
changes to secondary index leaf pages. Changes will only be merged as a
result of an operation that accesses a secondary index leaf page,
such as a SQL statement that performs a lookup via that index,
or is modifying the index. Also ROLLBACK and some background operations,
such as purging the history of committed transactions, or computing
index cardinality statistics, can cause change buffer merge.
Encryption key rotation will not perform change buffer merge.
The motivation of this change is to simplify the I/O logic and to
allow crash recovery to happen in the background (MDEV-14481).
We also hope that this will reduce the number of "mystery" crashes
due to corrupted data. Because change buffer merge will typically
take place as a result of executing SQL statements, there should be
a clearer connection between the crash and the SQL statements that
were executed when the server crashed.
In many cases, a slight performance improvement was observed.
This is joint work with Thirunarayanan Balathandayuthapani
and was tested by Axel Schwenke and Matthias Leich.
The InnoDB monitor counter innodb_ibuf_merge_usec will be removed.
On slow shutdown (innodb_fast_shutdown=0), we will continue to
merge all buffered changes (and purge all undo log history).
Two InnoDB configuration parameters will be changed as follows:
innodb_disable_background_merge: Removed.
This parameter existed only in debug builds.
All change buffer merges will use synchronous reads.
innodb_force_recovery will be changed as follows:
* innodb_force_recovery=4 will be the same as innodb_force_recovery=3
(the change buffer merge cannot be disabled; it can only happen as
a result of an operation that accesses a secondary index leaf page).
The option used to be capable of corrupting secondary index leaf pages.
Now that capability is removed, and innodb_force_recovery=4 becomes 'safe'.
* innodb_force_recovery=5 (which essentially hard-wires
SET GLOBAL TRANSACTION ISOLATION LEVEL READ UNCOMMITTED)
becomes safe to use. Bogus data can be returned to SQL, but
persistent InnoDB data files will not be corrupted further.
* innodb_force_recovery=6 (ignore the redo log files)
will be the only option that can potentially cause
persistent corruption of InnoDB data files.
Code changes:
buf_page_t::ibuf_exist: New flag, to indicate whether buffered
changes exist for a buffer pool page. Pages with pending changes
can be returned by buf_page_get_gen(). Previously, the changes
were always merged inside buf_page_get_gen() if needed.
ibuf_page_exists(const buf_page_t&): Check if a buffered changes
exist for an X-latched or read-fixed page.
buf_page_get_gen(): Add the parameter allow_ibuf_merge=false.
All callers that know that they may be accessing a secondary index
leaf page must pass this parameter as allow_ibuf_merge=true,
unless it does not matter for that caller whether all buffered
changes have been applied. Assert that whenever allow_ibuf_merge
holds, the page actually is a leaf page. Attempt change buffer
merge only to secondary B-tree index leaf pages.
btr_block_get(): Add parameter 'bool merge'.
All callers of btr_block_get() should know whether the page could be
a secondary index leaf page. If it is not, we should avoid consulting
the change buffer bitmap to even consider a merge. This is the main
interface to requesting index pages from the buffer pool.
ibuf_merge_or_delete_for_page(), recv_recover_page(): Replace
buf_page_get_known_nowait() with much simpler logic, because
it is now guaranteed that that the block is x-latched or read-fixed.
mlog_init_t::mark_ibuf_exist(): Renamed from mlog_init_t::ibuf_merge().
On crash recovery, we will no longer merge any buffered changes
for the pages that we read into the buffer pool during the last batch
of applying log records.
buf_page_get_gen_known_nowait(), BUF_MAKE_YOUNG, BUF_KEEP_OLD: Remove.
btr_search_guess_on_hash(): Merge buf_page_get_gen_known_nowait()
to its only remaining caller.
buf_page_make_young_if_needed(): Define as an inline function.
Add the parameter buf_pool.
buf_page_peek_if_young(), buf_page_peek_if_too_old(): Add the
parameter buf_pool.
fil_space_validate_for_mtr_commit(): Remove a bogus comment
about background merge of the change buffer.
btr_cur_open_at_rnd_pos_func(), btr_cur_search_to_nth_level_func(),
btr_cur_open_at_index_side_func(): Use narrower data types and scopes.
ibuf_read_merge_pages(): Replaces buf_read_ibuf_merge_pages().
Merge the change buffer by invoking buf_page_get_gen().
2019-10-11 17:28:15 +03:00
|
|
|
if (srv_force_recovery <= SRV_FORCE_NO_IBUF_MERGE) {
|
2014-02-26 19:11:54 +01:00
|
|
|
/* The following call is necessary for the insert
|
|
|
|
buffer to work with multiple tablespaces. We must
|
|
|
|
know the mapping between space id's and .ibd file
|
|
|
|
names.
|
|
|
|
|
|
|
|
In a crash recovery, we check that the info in data
|
|
|
|
dictionary is consistent with what we already know
|
2016-08-12 11:17:45 +03:00
|
|
|
about space id's from the calls to fil_ibd_load().
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
In a normal startup, we create the space objects for
|
|
|
|
every table in the InnoDB data dictionary that has
|
|
|
|
an .ibd file.
|
|
|
|
|
2019-04-03 16:10:20 +03:00
|
|
|
We also determine the maximum tablespace id used. */
|
|
|
|
dict_check_tablespaces_and_store_max_id();
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
2014-08-25 13:35:33 +03:00
|
|
|
|
2019-05-23 17:34:08 +03:00
|
|
|
if (srv_force_recovery < SRV_FORCE_NO_TRX_UNDO
|
|
|
|
&& !srv_read_only_mode) {
|
|
|
|
/* Drop partially created indexes. */
|
|
|
|
row_merge_drop_temp_indexes();
|
|
|
|
/* Drop garbage tables. */
|
|
|
|
row_mysql_drop_garbage_tables();
|
|
|
|
|
|
|
|
/* Drop any auxiliary tables that were not
|
|
|
|
dropped when the parent table was
|
|
|
|
dropped. This can happen if the parent table
|
|
|
|
was dropped but the server crashed before the
|
|
|
|
auxiliary tables were dropped. */
|
|
|
|
fts_drop_orphaned_tables();
|
|
|
|
|
|
|
|
/* Rollback incomplete non-DDL transactions */
|
|
|
|
trx_rollback_is_active = true;
|
2021-03-17 10:03:06 +03:00
|
|
|
srv_thread_pool->submit_task(&rollback_all_recovered_task);
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
srv_startup_is_before_trx_rollback_phase = false;
|
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
if (!srv_read_only_mode) {
|
2018-11-20 15:19:32 +03:00
|
|
|
DBUG_EXECUTE_IF("innodb_skip_monitors", goto skip_monitors;);
|
2019-10-29 22:37:12 +01:00
|
|
|
/* Create the task which warns of long semaphore waits */
|
2020-11-25 16:54:00 +02:00
|
|
|
srv_start_periodic_timer(srv_monitor_timer, srv_monitor_task,
|
|
|
|
SRV_MONITOR_INTERVAL);
|
2019-10-29 22:37:12 +01:00
|
|
|
|
2018-11-20 15:19:32 +03:00
|
|
|
#ifndef DBUG_OFF
|
|
|
|
skip_monitors:
|
|
|
|
#endif
|
2018-02-22 09:49:50 +02:00
|
|
|
ut_ad(srv_force_recovery >= SRV_FORCE_NO_UNDO_LOG_SCAN
|
2018-05-15 14:39:50 +03:00
|
|
|
|| !purge_sys.enabled());
|
MDEV-12288 Reset DB_TRX_ID when the history is removed, to speed up MVCC
Let InnoDB purge reset DB_TRX_ID,DB_ROLL_PTR when the history is removed.
[TODO: It appears that the resetting is not taking place as often as
it could be. We should test that a simple INSERT should eventually
cause row_purge_reset_trx_id() to be invoked unless DROP TABLE is
invoked soon enough.]
The InnoDB clustered index record system columns DB_TRX_ID,DB_ROLL_PTR
are used by multi-versioning. After the history is no longer needed, these
columns can safely be reset to 0 and 1<<55 (to indicate a fresh insert).
When a reader sees 0 in the DB_TRX_ID column, it can instantly determine
that the record is present the read view. There is no need to acquire
the transaction system mutex to check if the transaction exists, because
writes can never be conducted by a transaction whose ID is 0.
The persistent InnoDB undo log used to be split into two parts:
insert_undo and update_undo. The insert_undo log was discarded at
transaction commit or rollback, and the update_undo log was processed
by the purge subsystem. As part of this change, we will only generate
a single undo log for new transactions, and the purge subsystem will
reset the DB_TRX_ID whenever a clustered index record is touched.
That is, all persistent undo log will be preserved at transaction commit
or rollback, to be removed by purge.
The InnoDB redo log format is changed in two ways:
We remove the redo log record type MLOG_UNDO_HDR_REUSE, and
we introduce the MLOG_ZIP_WRITE_TRX_ID record for updating the
DB_TRX_ID,DB_ROLL_PTR in a ROW_FORMAT=COMPRESSED table.
This is also changing the format of persistent InnoDB data files:
undo log and clustered index leaf page records. It will still be
possible via import and export to exchange data files with earlier
versions of MariaDB. The change to clustered index leaf page records
is simple: we allow DB_TRX_ID to be 0.
When it comes to the undo log, we must be able to upgrade from earlier
MariaDB versions after a clean shutdown (no redo log to apply).
While it would be nice to perform a slow shutdown (innodb_fast_shutdown=0)
before an upgrade, to empty the undo logs, we cannot assume that this
has been done. So, separate insert_undo log may exist for recovered
uncommitted transactions. These transactions may be automatically
rolled back, or they may be in XA PREPARE state, in which case InnoDB
will preserve the transaction until an explicit XA COMMIT or XA ROLLBACK.
Upgrade has been tested by starting up MariaDB 10.2 with
./mysql-test-run --manual-gdb innodb.read_only_recovery
and then starting up this patched server with
and without --innodb-read-only.
trx_undo_ptr_t::undo: Renamed from update_undo.
trx_undo_ptr_t::old_insert: Renamed from insert_undo.
trx_rseg_t::undo_list: Renamed from update_undo_list.
trx_rseg_t::undo_cached: Merged from update_undo_cached
and insert_undo_cached.
trx_rseg_t::old_insert_list: Renamed from insert_undo_list.
row_purge_reset_trx_id(): New function to reset the columns.
This will be called for all undo processing in purge
that does not remove the clustered index record.
trx_undo_update_rec_get_update(): Allow trx_id=0 when copying the
old DB_TRX_ID of the record to the undo log.
ReadView::changes_visible(): Allow id==0. (Return true for it.
This is what speeds up the MVCC.)
row_vers_impl_x_locked_low(), row_vers_build_for_semi_consistent_read():
Implement a fast path for DB_TRX_ID=0.
Always initialize the TRX_UNDO_PAGE_TYPE to 0. Remove undo->type.
MLOG_UNDO_HDR_REUSE: Remove. This changes the redo log format!
innobase_start_or_create_for_mysql(): Set srv_undo_sources before
starting any transactions.
The parsing of the MLOG_ZIP_WRITE_TRX_ID record was successfully
tested by running the following:
./mtr --parallel=auto --mysqld=--debug=d,ib_log innodb_zip.bug56680
grep MLOG_ZIP_WRITE_TRX_ID var/*/log/mysqld.1.err
2017-07-07 13:08:16 +03:00
|
|
|
|
|
|
|
if (srv_force_recovery < SRV_FORCE_NO_BACKGROUND) {
|
|
|
|
srv_undo_sources = true;
|
2019-10-29 22:37:12 +01:00
|
|
|
/* Create the dict stats gathering task */
|
|
|
|
dict_stats_start();
|
MDEV-12288 Reset DB_TRX_ID when the history is removed, to speed up MVCC
Let InnoDB purge reset DB_TRX_ID,DB_ROLL_PTR when the history is removed.
[TODO: It appears that the resetting is not taking place as often as
it could be. We should test that a simple INSERT should eventually
cause row_purge_reset_trx_id() to be invoked unless DROP TABLE is
invoked soon enough.]
The InnoDB clustered index record system columns DB_TRX_ID,DB_ROLL_PTR
are used by multi-versioning. After the history is no longer needed, these
columns can safely be reset to 0 and 1<<55 (to indicate a fresh insert).
When a reader sees 0 in the DB_TRX_ID column, it can instantly determine
that the record is present the read view. There is no need to acquire
the transaction system mutex to check if the transaction exists, because
writes can never be conducted by a transaction whose ID is 0.
The persistent InnoDB undo log used to be split into two parts:
insert_undo and update_undo. The insert_undo log was discarded at
transaction commit or rollback, and the update_undo log was processed
by the purge subsystem. As part of this change, we will only generate
a single undo log for new transactions, and the purge subsystem will
reset the DB_TRX_ID whenever a clustered index record is touched.
That is, all persistent undo log will be preserved at transaction commit
or rollback, to be removed by purge.
The InnoDB redo log format is changed in two ways:
We remove the redo log record type MLOG_UNDO_HDR_REUSE, and
we introduce the MLOG_ZIP_WRITE_TRX_ID record for updating the
DB_TRX_ID,DB_ROLL_PTR in a ROW_FORMAT=COMPRESSED table.
This is also changing the format of persistent InnoDB data files:
undo log and clustered index leaf page records. It will still be
possible via import and export to exchange data files with earlier
versions of MariaDB. The change to clustered index leaf page records
is simple: we allow DB_TRX_ID to be 0.
When it comes to the undo log, we must be able to upgrade from earlier
MariaDB versions after a clean shutdown (no redo log to apply).
While it would be nice to perform a slow shutdown (innodb_fast_shutdown=0)
before an upgrade, to empty the undo logs, we cannot assume that this
has been done. So, separate insert_undo log may exist for recovered
uncommitted transactions. These transactions may be automatically
rolled back, or they may be in XA PREPARE state, in which case InnoDB
will preserve the transaction until an explicit XA COMMIT or XA ROLLBACK.
Upgrade has been tested by starting up MariaDB 10.2 with
./mysql-test-run --manual-gdb innodb.read_only_recovery
and then starting up this patched server with
and without --innodb-read-only.
trx_undo_ptr_t::undo: Renamed from update_undo.
trx_undo_ptr_t::old_insert: Renamed from insert_undo.
trx_rseg_t::undo_list: Renamed from update_undo_list.
trx_rseg_t::undo_cached: Merged from update_undo_cached
and insert_undo_cached.
trx_rseg_t::old_insert_list: Renamed from insert_undo_list.
row_purge_reset_trx_id(): New function to reset the columns.
This will be called for all undo processing in purge
that does not remove the clustered index record.
trx_undo_update_rec_get_update(): Allow trx_id=0 when copying the
old DB_TRX_ID of the record to the undo log.
ReadView::changes_visible(): Allow id==0. (Return true for it.
This is what speeds up the MVCC.)
row_vers_impl_x_locked_low(), row_vers_build_for_semi_consistent_read():
Implement a fast path for DB_TRX_ID=0.
Always initialize the TRX_UNDO_PAGE_TYPE to 0. Remove undo->type.
MLOG_UNDO_HDR_REUSE: Remove. This changes the redo log format!
innobase_start_or_create_for_mysql(): Set srv_undo_sources before
starting any transactions.
The parsing of the MLOG_ZIP_WRITE_TRX_ID record was successfully
tested by running the following:
./mtr --parallel=auto --mysqld=--debug=d,ib_log innodb_zip.bug56680
grep MLOG_ZIP_WRITE_TRX_ID var/*/log/mysqld.1.err
2017-07-07 13:08:16 +03:00
|
|
|
/* Create the thread that will optimize the
|
|
|
|
FULLTEXT search index subsystem. */
|
|
|
|
fts_optimize_init();
|
|
|
|
}
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
/* Create the SYS_FOREIGN and SYS_FOREIGN_COLS system tables */
|
|
|
|
err = dict_create_or_check_foreign_constraint_tables();
|
2017-02-01 15:47:33 +02:00
|
|
|
if (err == DB_SUCCESS) {
|
MDEV-22343 Remove SYS_TABLESPACES and SYS_DATAFILES
The InnoDB internal tables SYS_TABLESPACES and SYS_DATAFILES as well as the
INFORMATION_SCHEMA views INNODB_SYS_TABLESPACES and INNODB_SYS_DATAFILES
were introduced in MySQL 5.6 for no good reason in
mysql/mysql-server/commit/e9255a22ef16d612a8076bc0b34002bc5a784627
when the InnoDB support for the DATA DIRECTORY attribute was introduced.
The file system should be the authoritative source of information on files.
Storing information about file system paths in the file system (symlinks,
or even the .isl files that were unfortunately chosen as the solution) is
sufficient. If information is additionally stored in some hidden tables
inside the InnoDB system tablespace, everything unnecessarily becomes
more complicated, because more copies of data mean more opportunity
for the copies to be out of sync, and because modifying the data in
the system tablespace in the desired way might not be possible at all
without modifying the InnoDB source code. So, the copy in the system
tablespace basically is a redundant, non-authoritative source of
information.
We will stop creating or accessing the system tables SYS_TABLESPACES
and SYS_DATAFILES.
We will also remove the view
INFORMATION_SCHEMA.INNODB_SYS_DATAFILES along with SYS_DATAFILES.
The view
INFORMATION_SCHEMA.INNODB_SYS_TABLESPACES will be repurposed
to directly reflect fil_system.space_list. The column
PAGE_SIZE, which would always contain the value of
the GLOBAL read-only variable innodb_page_size, is
removed. The column ZIP_PAGE_SIZE, which would actually
contain the physical page size of a page, is renamed to
PAGE_SIZE. Finally, a new column FILENAME is added, as a
replacement of SYS_DATAFILES.PATH.
This will also
address MDEV-21801 (files that were created before upgrading
to MySQL 5.6 or MariaDB 10.0 or later were never registered
in SYS_TABLESPACES or SYS_DATAFILES) and
MDEV-21801 (information about the system tablespace is not stored
in SYS_TABLESPACES or SYS_DATAFILES).
2020-11-11 11:02:27 +02:00
|
|
|
err = dict_create_or_check_sys_virtual();
|
2016-08-12 11:17:45 +03:00
|
|
|
}
|
2017-02-01 15:47:33 +02:00
|
|
|
switch (err) {
|
|
|
|
case DB_SUCCESS:
|
|
|
|
break;
|
|
|
|
case DB_READ_ONLY:
|
|
|
|
if (srv_force_recovery >= SRV_FORCE_NO_TRX_UNDO) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
ib::error() << "Cannot create system tables in read-only mode";
|
|
|
|
/* fall through */
|
|
|
|
default:
|
2016-08-12 11:17:45 +03:00
|
|
|
return(srv_init_abort(err));
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
if (!srv_read_only_mode && srv_operation == SRV_OPERATION_NORMAL) {
|
2017-05-10 15:13:11 +03:00
|
|
|
/* Initialize the innodb_temporary tablespace and keep
|
|
|
|
it open until shutdown. */
|
|
|
|
err = srv_open_tmp_tablespace(create_new_db);
|
|
|
|
|
|
|
|
if (err != DB_SUCCESS) {
|
|
|
|
return(srv_init_abort(err));
|
|
|
|
}
|
|
|
|
|
|
|
|
trx_temp_rseg_create();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2018-05-09 16:52:45 +03:00
|
|
|
if (srv_force_recovery < SRV_FORCE_NO_BACKGROUND) {
|
2019-10-29 22:37:12 +01:00
|
|
|
srv_start_periodic_timer(srv_master_timer, srv_master_callback, 1000);
|
2018-05-09 16:52:45 +03:00
|
|
|
}
|
2017-06-21 12:22:02 +03:00
|
|
|
}
|
MDEV-13039 innodb_fast_shutdown=0 may fail to purge all undo log
When a slow shutdown is performed soon after spawning some work for
background threads that can create or commit transactions, it is possible
that new transactions are started or committed after the purge has finished.
This is violating the specification of innodb_fast_shutdown=0, namely that
the purge must be completed. (None of the history of the recent transactions
would be purged.)
Also, it is possible that the purge threads would exit in slow shutdown
while there exist active transactions, such as recovered incomplete
transactions that are being rolled back. Thus, the slow shutdown could
fail to purge some undo log that becomes purgeable after the transaction
commit or rollback.
srv_undo_sources: A flag that indicates if undo log can be generated
or the persistent, whether by background threads or by user SQL.
Even when this flag is clear, active transactions that already exist
in the system may be committed or rolled back.
innodb_shutdown(): Renamed from innobase_shutdown_for_mysql().
Do not return an error code; the operation never fails.
Clear the srv_undo_sources flag, and also ensure that the background
DROP TABLE queue is empty.
srv_purge_should_exit(): Do not allow the purge to exit if
srv_undo_sources are active or the background DROP TABLE queue is not
empty, or in slow shutdown, if any active transactions exist
(and are being rolled back).
srv_purge_coordinator_thread(): Remove some previous workarounds
for this bug.
innobase_start_or_create_for_mysql(): Set buf_page_cleaner_is_active
and srv_dict_stats_thread_active directly. Set srv_undo_sources before
starting the purge subsystem, to prevent immediate shutdown of the purge.
Create dict_stats_thread and fts_optimize_thread immediately
after setting srv_undo_sources, so that shutdown can use this flag to
determine if these subsystems were started.
dict_stats_shutdown(): Shut down dict_stats_thread. Backported from 10.2.
srv_shutdown_table_bg_threads(): Remove (unused).
2017-06-08 15:43:06 +03:00
|
|
|
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
if (!srv_read_only_mode && srv_operation == SRV_OPERATION_NORMAL
|
2017-06-21 12:22:02 +03:00
|
|
|
&& srv_force_recovery < SRV_FORCE_NO_BACKGROUND) {
|
2020-06-03 16:19:13 +02:00
|
|
|
srv_init_purge_tasks();
|
2019-10-29 22:37:12 +01:00
|
|
|
purge_sys.coordinator_startup();
|
|
|
|
srv_wake_purge_thread_if_not_active();
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2017-12-13 23:14:15 +02:00
|
|
|
srv_is_being_started = false;
|
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
if (srv_print_verbose_log) {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << INNODB_VERSION_STR
|
MDEV-15132 Avoid accessing the TRX_SYS page
InnoDB maintains an internal persistent sequence of transaction
identifiers. This sequence is used for assigning both transaction
start identifiers (DB_TRX_ID=trx->id) and end identifiers (trx->no)
as well as end identifiers for the mysql.transaction_registry table
that was introduced in MDEV-12894.
TRX_SYS_TRX_ID_WRITE_MARGIN: Remove. After this many updates of
the sequence we used to update the TRX_SYS page. We can avoid accessing
the TRX_SYS page if we modify the InnoDB startup so that resurrecting
the sequence from other pages of the transaction system.
TRX_SYS_TRX_ID_STORE: Deprecate. The field only exists for the purpose
of upgrading from an earlier version of MySQL or MariaDB.
Starting with this fix, MariaDB will rely on the fields
TRX_UNDO_TRX_ID, TRX_UNDO_TRX_NO in the undo log header page of
each non-committed transaction, and on the new field
TRX_RSEG_MAX_TRX_ID in rollback segment header pages.
Because of this change, setting innodb_force_recovery=5 or 6 may cause
the system to recover with trx_sys.get_max_trx_id()==0. We must adjust
checks for invalid DB_TRX_ID and PAGE_MAX_TRX_ID accordingly.
We will change the startup and shutdown messages to display the
trx_sys.get_max_trx_id() in addition to the log sequence number.
trx_sys_t::flush_max_trx_id(): Remove.
trx_undo_mem_create_at_db_start(), trx_undo_lists_init():
Add an output parameter max_trx_id, to be updated from
TRX_UNDO_TRX_ID, TRX_UNDO_TRX_NO.
TRX_RSEG_MAX_TRX_ID: New field, for persisting
trx_sys.get_max_trx_id() at the time of the latest transaction commit.
Startup is not reading the undo log pages of committed transactions.
We want to avoid additional page accesses on startup, as well as
trouble when all undo logs have been emptied.
On startup, we will simply determine the maximum value from all pages
that are being read anyway.
TRX_RSEG_FORMAT: Redefined from TRX_RSEG_MAX_SIZE.
Old versions of InnoDB wrote uninitialized garbage to unused data fields.
Because of this, we cannot simply introduce a new field in the
rollback segment pages and expect it to be always zero, like it would
if the database was created by a recent enough InnoDB version.
Luckily, it looks like the field TRX_RSEG_MAX_SIZE was always written
as 0xfffffffe. We will indicate a new subformat of the page by writing
0 to this field. This has the nice side effect that after a downgrade
to older versions of InnoDB, transactions should fail to allocate any
undo log, that is, writes will be blocked. So, there is no problem of
getting corrupted transaction identifiers after downgrading.
trx_rseg_t::max_size: Remove.
trx_rseg_header_create(): Remove the parameter max_size=ULINT_MAX.
trx_purge_add_undo_to_history(): Update TRX_RSEG_MAX_SIZE
(and TRX_RSEG_FORMAT if needed). This is invoked on transaction commit.
trx_rseg_mem_restore(): If TRX_RSEG_FORMAT contains 0,
read TRX_RSEG_MAX_SIZE.
trx_rseg_array_init(): Invoke trx_sys.init_max_trx_id(max_trx_id + 1)
where max_trx_id was the maximum that was encountered in the rollback
segment pages and the undo log pages of recovered active, XA PREPARE,
or some committed transactions. (See trx_purge_add_undo_to_history()
which invokes trx_rsegf_set_nth_undo(..., FIL_NULL, ...);
not all committed transactions will be immediately detached from the
rollback segment header.)
2018-01-31 10:24:19 +02:00
|
|
|
<< " started; log sequence number "
|
2020-03-02 14:34:52 +02:00
|
|
|
<< recv_sys.recovered_lsn
|
MDEV-15132 Avoid accessing the TRX_SYS page
InnoDB maintains an internal persistent sequence of transaction
identifiers. This sequence is used for assigning both transaction
start identifiers (DB_TRX_ID=trx->id) and end identifiers (trx->no)
as well as end identifiers for the mysql.transaction_registry table
that was introduced in MDEV-12894.
TRX_SYS_TRX_ID_WRITE_MARGIN: Remove. After this many updates of
the sequence we used to update the TRX_SYS page. We can avoid accessing
the TRX_SYS page if we modify the InnoDB startup so that resurrecting
the sequence from other pages of the transaction system.
TRX_SYS_TRX_ID_STORE: Deprecate. The field only exists for the purpose
of upgrading from an earlier version of MySQL or MariaDB.
Starting with this fix, MariaDB will rely on the fields
TRX_UNDO_TRX_ID, TRX_UNDO_TRX_NO in the undo log header page of
each non-committed transaction, and on the new field
TRX_RSEG_MAX_TRX_ID in rollback segment header pages.
Because of this change, setting innodb_force_recovery=5 or 6 may cause
the system to recover with trx_sys.get_max_trx_id()==0. We must adjust
checks for invalid DB_TRX_ID and PAGE_MAX_TRX_ID accordingly.
We will change the startup and shutdown messages to display the
trx_sys.get_max_trx_id() in addition to the log sequence number.
trx_sys_t::flush_max_trx_id(): Remove.
trx_undo_mem_create_at_db_start(), trx_undo_lists_init():
Add an output parameter max_trx_id, to be updated from
TRX_UNDO_TRX_ID, TRX_UNDO_TRX_NO.
TRX_RSEG_MAX_TRX_ID: New field, for persisting
trx_sys.get_max_trx_id() at the time of the latest transaction commit.
Startup is not reading the undo log pages of committed transactions.
We want to avoid additional page accesses on startup, as well as
trouble when all undo logs have been emptied.
On startup, we will simply determine the maximum value from all pages
that are being read anyway.
TRX_RSEG_FORMAT: Redefined from TRX_RSEG_MAX_SIZE.
Old versions of InnoDB wrote uninitialized garbage to unused data fields.
Because of this, we cannot simply introduce a new field in the
rollback segment pages and expect it to be always zero, like it would
if the database was created by a recent enough InnoDB version.
Luckily, it looks like the field TRX_RSEG_MAX_SIZE was always written
as 0xfffffffe. We will indicate a new subformat of the page by writing
0 to this field. This has the nice side effect that after a downgrade
to older versions of InnoDB, transactions should fail to allocate any
undo log, that is, writes will be blocked. So, there is no problem of
getting corrupted transaction identifiers after downgrading.
trx_rseg_t::max_size: Remove.
trx_rseg_header_create(): Remove the parameter max_size=ULINT_MAX.
trx_purge_add_undo_to_history(): Update TRX_RSEG_MAX_SIZE
(and TRX_RSEG_FORMAT if needed). This is invoked on transaction commit.
trx_rseg_mem_restore(): If TRX_RSEG_FORMAT contains 0,
read TRX_RSEG_MAX_SIZE.
trx_rseg_array_init(): Invoke trx_sys.init_max_trx_id(max_trx_id + 1)
where max_trx_id was the maximum that was encountered in the rollback
segment pages and the undo log pages of recovered active, XA PREPARE,
or some committed transactions. (See trx_purge_add_undo_to_history()
which invokes trx_rsegf_set_nth_undo(..., FIL_NULL, ...);
not all committed transactions will be immediately detached from the
rollback segment header.)
2018-01-31 10:24:19 +02:00
|
|
|
<< "; transaction id " << trx_sys.get_max_trx_id();
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
if (srv_force_recovery > 0) {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "!!! innodb_force_recovery is set to "
|
|
|
|
<< srv_force_recovery << " !!!";
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
if (srv_force_recovery == 0) {
|
2020-03-31 14:37:11 +03:00
|
|
|
/* In the change buffer we may have even bigger tablespace
|
2014-02-26 19:11:54 +01:00
|
|
|
id's, because we may have dropped those tablespaces, but
|
2020-03-31 14:37:11 +03:00
|
|
|
the buffered records have not been cleaned yet. */
|
2014-02-26 19:11:54 +01:00
|
|
|
ibuf_update_max_tablespace_id();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!srv_read_only_mode) {
|
2016-08-12 11:17:45 +03:00
|
|
|
if (create_new_db) {
|
|
|
|
srv_buffer_pool_load_at_startup = FALSE;
|
|
|
|
}
|
|
|
|
|
2015-07-14 16:05:29 -04:00
|
|
|
#ifdef WITH_WSREP
|
|
|
|
/*
|
|
|
|
Create the dump/load thread only when not running with
|
|
|
|
--wsrep-recover.
|
|
|
|
*/
|
2019-01-23 15:30:00 +04:00
|
|
|
if (!get_wsrep_recovery()) {
|
2015-07-14 16:05:29 -04:00
|
|
|
#endif /* WITH_WSREP */
|
2017-08-31 08:27:59 +03:00
|
|
|
|
2019-10-29 22:37:12 +01:00
|
|
|
/* Start buffer pool dump/load task */
|
|
|
|
buf_load_at_startup();
|
MDEV-11638 Encryption causes race conditions in InnoDB shutdown
InnoDB shutdown failed to properly take fil_crypt_thread() into account.
The encryption threads were signalled to shut down together with other
non-critical tasks. This could be much too early in case of slow shutdown,
which could need minutes to complete the purge. Furthermore, InnoDB
failed to wait for the fil_crypt_thread() to actually exit before
proceeding to the final steps of shutdown, causing the race conditions.
Furthermore, the log_scrub_thread() was shut down way too early.
Also it should remain until the SRV_SHUTDOWN_FLUSH_PHASE.
fil_crypt_threads_end(): Remove. This would cause the threads to
be terminated way too early.
srv_buf_dump_thread_active, srv_dict_stats_thread_active,
lock_sys->timeout_thread_active, log_scrub_thread_active,
srv_monitor_active, srv_error_monitor_active: Remove a race condition
between startup and shutdown, by setting these in the startup thread
that creates threads, not in each created thread. In this way, once the
flag is cleared, it will remain cleared during shutdown.
srv_n_fil_crypt_threads_started, fil_crypt_threads_event: Declare in
global rather than static scope.
log_scrub_event, srv_log_scrub_thread_active, log_scrub_thread():
Declare in static rather than global scope. Let these be created by
log_init() and freed by log_shutdown().
rotate_thread_t::should_shutdown(): Do not shut down before the
SRV_SHUTDOWN_FLUSH_PHASE.
srv_any_background_threads_are_active(): Remove. These checks now
exist in logs_empty_and_mark_files_at_shutdown().
logs_empty_and_mark_files_at_shutdown(): Shut down the threads in
the proper order. Keep fil_crypt_thread() and log_scrub_thread() alive
until SRV_SHUTDOWN_FLUSH_PHASE, and check that they actually terminate.
2017-01-04 18:43:32 +02:00
|
|
|
|
2015-07-14 16:05:29 -04:00
|
|
|
#ifdef WITH_WSREP
|
|
|
|
} else {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::warn() <<
|
2015-07-14 16:05:29 -04:00
|
|
|
"Skipping buffer pool dump/restore during "
|
2016-08-12 11:17:45 +03:00
|
|
|
"wsrep recovery.";
|
2015-07-14 16:05:29 -04:00
|
|
|
}
|
|
|
|
#endif /* WITH_WSREP */
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2017-01-31 19:43:03 +02:00
|
|
|
/* Create thread(s) that handles key rotation. This is
|
|
|
|
needed already here as log_preflush_pool_modified_pages
|
|
|
|
will flush dirty pages and that might need e.g.
|
2020-12-04 16:18:04 +02:00
|
|
|
fil_crypt_threads_cond. */
|
2017-01-31 19:43:03 +02:00
|
|
|
fil_crypt_threads_init();
|
|
|
|
|
|
|
|
/* Initialize online defragmentation. */
|
|
|
|
btr_defragment_init();
|
|
|
|
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
srv_started_redo = true;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
return(DB_SUCCESS);
|
|
|
|
}
|
|
|
|
|
2017-06-08 15:43:06 +03:00
|
|
|
/** Shut down background threads that can generate undo log. */
|
2018-04-29 09:41:42 +03:00
|
|
|
void srv_shutdown_bg_undo_sources()
|
2016-12-14 17:46:58 +01:00
|
|
|
{
|
2020-10-26 15:04:24 +02:00
|
|
|
srv_shutdown_state = SRV_SHUTDOWN_INITIATED;
|
|
|
|
|
2020-12-04 16:18:04 +02:00
|
|
|
ut_d(srv_master_thread_enable());
|
|
|
|
|
MDEV-13039 innodb_fast_shutdown=0 may fail to purge all undo log
When a slow shutdown is performed soon after spawning some work for
background threads that can create or commit transactions, it is possible
that new transactions are started or committed after the purge has finished.
This is violating the specification of innodb_fast_shutdown=0, namely that
the purge must be completed. (None of the history of the recent transactions
would be purged.)
Also, it is possible that the purge threads would exit in slow shutdown
while there exist active transactions, such as recovered incomplete
transactions that are being rolled back. Thus, the slow shutdown could
fail to purge some undo log that becomes purgeable after the transaction
commit or rollback.
srv_undo_sources: A flag that indicates if undo log can be generated
or the persistent, whether by background threads or by user SQL.
Even when this flag is clear, active transactions that already exist
in the system may be committed or rolled back.
innodb_shutdown(): Renamed from innobase_shutdown_for_mysql().
Do not return an error code; the operation never fails.
Clear the srv_undo_sources flag, and also ensure that the background
DROP TABLE queue is empty.
srv_purge_should_exit(): Do not allow the purge to exit if
srv_undo_sources are active or the background DROP TABLE queue is not
empty, or in slow shutdown, if any active transactions exist
(and are being rolled back).
srv_purge_coordinator_thread(): Remove some previous workarounds
for this bug.
innobase_start_or_create_for_mysql(): Set buf_page_cleaner_is_active
and srv_dict_stats_thread_active directly. Set srv_undo_sources before
starting the purge subsystem, to prevent immediate shutdown of the purge.
Create dict_stats_thread and fts_optimize_thread immediately
after setting srv_undo_sources, so that shutdown can use this flag to
determine if these subsystems were started.
dict_stats_shutdown(): Shut down dict_stats_thread. Backported from 10.2.
srv_shutdown_table_bg_threads(): Remove (unused).
2017-06-08 15:43:06 +03:00
|
|
|
if (srv_undo_sources) {
|
2017-01-31 19:43:03 +02:00
|
|
|
ut_ad(!srv_read_only_mode);
|
|
|
|
fts_optimize_shutdown();
|
|
|
|
dict_stats_shutdown();
|
MDEV-13039 innodb_fast_shutdown=0 may fail to purge all undo log
When a slow shutdown is performed soon after spawning some work for
background threads that can create or commit transactions, it is possible
that new transactions are started or committed after the purge has finished.
This is violating the specification of innodb_fast_shutdown=0, namely that
the purge must be completed. (None of the history of the recent transactions
would be purged.)
Also, it is possible that the purge threads would exit in slow shutdown
while there exist active transactions, such as recovered incomplete
transactions that are being rolled back. Thus, the slow shutdown could
fail to purge some undo log that becomes purgeable after the transaction
commit or rollback.
srv_undo_sources: A flag that indicates if undo log can be generated
or the persistent, whether by background threads or by user SQL.
Even when this flag is clear, active transactions that already exist
in the system may be committed or rolled back.
innodb_shutdown(): Renamed from innobase_shutdown_for_mysql().
Do not return an error code; the operation never fails.
Clear the srv_undo_sources flag, and also ensure that the background
DROP TABLE queue is empty.
srv_purge_should_exit(): Do not allow the purge to exit if
srv_undo_sources are active or the background DROP TABLE queue is not
empty, or in slow shutdown, if any active transactions exist
(and are being rolled back).
srv_purge_coordinator_thread(): Remove some previous workarounds
for this bug.
innobase_start_or_create_for_mysql(): Set buf_page_cleaner_is_active
and srv_dict_stats_thread_active directly. Set srv_undo_sources before
starting the purge subsystem, to prevent immediate shutdown of the purge.
Create dict_stats_thread and fts_optimize_thread immediately
after setting srv_undo_sources, so that shutdown can use this flag to
determine if these subsystems were started.
dict_stats_shutdown(): Shut down dict_stats_thread. Backported from 10.2.
srv_shutdown_table_bg_threads(): Remove (unused).
2017-06-08 15:43:06 +03:00
|
|
|
while (row_get_background_drop_list_len_low()) {
|
2020-07-23 16:23:20 +05:30
|
|
|
srv_inc_activity_count();
|
2021-03-16 23:34:55 +03:00
|
|
|
std::this_thread::yield();
|
MDEV-13039 innodb_fast_shutdown=0 may fail to purge all undo log
When a slow shutdown is performed soon after spawning some work for
background threads that can create or commit transactions, it is possible
that new transactions are started or committed after the purge has finished.
This is violating the specification of innodb_fast_shutdown=0, namely that
the purge must be completed. (None of the history of the recent transactions
would be purged.)
Also, it is possible that the purge threads would exit in slow shutdown
while there exist active transactions, such as recovered incomplete
transactions that are being rolled back. Thus, the slow shutdown could
fail to purge some undo log that becomes purgeable after the transaction
commit or rollback.
srv_undo_sources: A flag that indicates if undo log can be generated
or the persistent, whether by background threads or by user SQL.
Even when this flag is clear, active transactions that already exist
in the system may be committed or rolled back.
innodb_shutdown(): Renamed from innobase_shutdown_for_mysql().
Do not return an error code; the operation never fails.
Clear the srv_undo_sources flag, and also ensure that the background
DROP TABLE queue is empty.
srv_purge_should_exit(): Do not allow the purge to exit if
srv_undo_sources are active or the background DROP TABLE queue is not
empty, or in slow shutdown, if any active transactions exist
(and are being rolled back).
srv_purge_coordinator_thread(): Remove some previous workarounds
for this bug.
innobase_start_or_create_for_mysql(): Set buf_page_cleaner_is_active
and srv_dict_stats_thread_active directly. Set srv_undo_sources before
starting the purge subsystem, to prevent immediate shutdown of the purge.
Create dict_stats_thread and fts_optimize_thread immediately
after setting srv_undo_sources, so that shutdown can use this flag to
determine if these subsystems were started.
dict_stats_shutdown(): Shut down dict_stats_thread. Backported from 10.2.
srv_shutdown_table_bg_threads(): Remove (unused).
2017-06-08 15:43:06 +03:00
|
|
|
}
|
|
|
|
srv_undo_sources = false;
|
2017-01-31 19:43:03 +02:00
|
|
|
}
|
2016-12-14 17:46:58 +01:00
|
|
|
}
|
|
|
|
|
2019-10-29 22:37:12 +01:00
|
|
|
/**
|
2019-11-13 18:14:44 +01:00
|
|
|
Shutdown purge to make sure that there is no possibility that we call any
|
|
|
|
plugin code (e.g., audit) inside virtual column computation.
|
2019-10-29 22:37:12 +01:00
|
|
|
*/
|
|
|
|
void innodb_preshutdown()
|
|
|
|
{
|
2019-11-13 18:14:44 +01:00
|
|
|
static bool first_time= true;
|
|
|
|
if (!first_time)
|
|
|
|
return;
|
|
|
|
first_time= false;
|
|
|
|
|
|
|
|
if (srv_read_only_mode)
|
|
|
|
return;
|
|
|
|
if (!srv_fast_shutdown && srv_operation == SRV_OPERATION_NORMAL)
|
2020-06-05 19:04:04 +03:00
|
|
|
{
|
|
|
|
/* Because a slow shutdown must empty the change buffer, we had
|
|
|
|
better prevent any further changes from being buffered. */
|
|
|
|
innodb_change_buffering= 0;
|
|
|
|
|
2020-10-09 12:47:58 +03:00
|
|
|
if (trx_sys.is_initialised())
|
|
|
|
while (trx_sys.any_active_transactions())
|
2021-03-16 16:09:41 +03:00
|
|
|
std::this_thread::sleep_for(std::chrono::milliseconds(1));
|
2020-06-05 19:04:04 +03:00
|
|
|
}
|
2019-11-13 18:14:44 +01:00
|
|
|
srv_shutdown_bg_undo_sources();
|
|
|
|
srv_purge_shutdown();
|
2020-10-26 15:04:24 +02:00
|
|
|
|
|
|
|
if (srv_n_fil_crypt_threads)
|
|
|
|
fil_crypt_set_thread_cnt(0);
|
2019-10-29 22:37:12 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-01-31 19:43:03 +02:00
|
|
|
/** Shut down InnoDB. */
|
2018-04-29 09:41:42 +03:00
|
|
|
void innodb_shutdown()
|
2014-02-26 19:11:54 +01:00
|
|
|
{
|
2019-10-29 22:37:12 +01:00
|
|
|
innodb_preshutdown();
|
2017-06-08 15:43:06 +03:00
|
|
|
ut_ad(!srv_undo_sources);
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
switch (srv_operation) {
|
|
|
|
case SRV_OPERATION_BACKUP:
|
Follow-up fix to MDEV-12988 backup fails if innodb_undo_tablespaces>0
The fix broke mariabackup --prepare --incremental.
The restore of an incremental backup starts up (parts of) InnoDB twice.
First, all data files are discovered for applying .delta files. Then,
after the .delta files have been applied, InnoDB will be restarted
more completely, so that the redo log records will be applied via the
buffer pool.
During the first startup, the buffer pool is not initialized, and thus
trx_rseg_get_n_undo_tablespaces() must not be invoked. The apply of
the .delta files will currently assume that the --innodb-undo-tablespaces
option correctly specifies the number of undo tablespace files, just
like --backup does.
The second InnoDB startup of --prepare for applying the redo log will
properly invoke trx_rseg_get_n_undo_tablespaces().
enum srv_operation_mode: Add SRV_OPERATION_RESTORE_DELTA for
distinguishing the apply of .delta files from SRV_OPERATION_RESTORE.
srv_undo_tablespaces_init(): In mariabackup --prepare --incremental,
in the initial SRV_OPERATION_RESTORE_DELTA phase, do not invoke
trx_rseg_get_n_undo_tablespaces() because the buffer pool or the
redo logs are not available. Instead, blindly rely on the parameter
--innodb-undo-tablespaces.
2017-08-18 08:52:41 +03:00
|
|
|
case SRV_OPERATION_RESTORE_DELTA:
|
2020-10-26 15:04:24 +02:00
|
|
|
break;
|
|
|
|
case SRV_OPERATION_RESTORE:
|
2017-10-10 10:28:54 +03:00
|
|
|
case SRV_OPERATION_RESTORE_EXPORT:
|
2020-10-26 15:04:24 +02:00
|
|
|
srv_shutdown_state = SRV_SHUTDOWN_CLEANUP;
|
|
|
|
if (!buf_page_cleaner_is_active) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
mysql_mutex_lock(&buf_pool.flush_list_mutex);
|
|
|
|
while (buf_page_cleaner_is_active) {
|
2021-02-07 12:19:24 +02:00
|
|
|
pthread_cond_signal(&buf_pool.do_flush_list);
|
|
|
|
my_cond_wait(&buf_pool.done_flush_list,
|
|
|
|
&buf_pool.flush_list_mutex.m_mutex);
|
2020-10-26 15:04:24 +02:00
|
|
|
}
|
|
|
|
mysql_mutex_unlock(&buf_pool.flush_list_mutex);
|
MDEV-12548 Initial implementation of Mariabackup for MariaDB 10.2
InnoDB I/O and buffer pool interfaces and the redo log format
have been changed between MariaDB 10.1 and 10.2, and the backup
code has to be adjusted accordingly.
The code has been simplified, and many memory leaks have been fixed.
Instead of the file name xtrabackup_logfile, the file name ib_logfile0
is being used for the copy of the redo log. Unnecessary InnoDB startup and
shutdown and some unnecessary threads have been removed.
Some help was provided by Vladislav Vaintroub.
Parameters have been cleaned up and aligned with those of MariaDB 10.2.
The --dbug option has been added, so that in debug builds,
--dbug=d,ib_log can be specified to enable diagnostic messages
for processing redo log entries.
By default, innodb_doublewrite=OFF, so that --prepare works faster.
If more crash-safety for --prepare is needed, double buffering
can be enabled.
The parameter innodb_log_checksums=OFF can be used to ignore redo log
checksums in --backup.
Some messages have been cleaned up.
Unless --export is specified, Mariabackup will not deal with undo log.
The InnoDB mini-transaction redo log is not only about user-level
transactions; it is actually about mini-transactions. To avoid confusion,
call it the redo log, not transaction log.
We disable any undo log processing in --prepare.
Because MariaDB 10.2 supports indexed virtual columns, the
undo log processing would need to be able to evaluate virtual column
expressions. To reduce the amount of code dependencies, we will not
process any undo log in prepare.
This means that the --export option must be disabled for now.
This also means that the following options are redundant
and have been removed:
xtrabackup --apply-log-only
innobackupex --redo-only
In addition to disabling any undo log processing, we will disable any
further changes to data pages during --prepare, including the change
buffer merge. This means that restoring incremental backups should
reliably work even when change buffering is being used on the server.
Because of this, preparing a backup will not generate any further
redo log, and the redo log file can be safely deleted. (If the
--export option is enabled in the future, it must generate redo log
when processing undo logs and buffered changes.)
In --prepare, we cannot easily know if a partial backup was used,
especially when restoring a series of incremental backups. So, we
simply warn about any missing files, and ignore the redo log for them.
FIXME: Enable the --export option.
FIXME: Improve the handling of the MLOG_INDEX_LOAD record, and write
a test that initiates a backup while an ALGORITHM=INPLACE operation
is creating indexes or rebuilding a table. An error should be detected
when preparing the backup.
FIXME: In --incremental --prepare, xtrabackup_apply_delta() should
ensure that if FSP_SIZE is modified, the file size will be adjusted
accordingly.
2017-06-30 10:49:37 +03:00
|
|
|
break;
|
|
|
|
case SRV_OPERATION_NORMAL:
|
|
|
|
/* Shut down the persistent files. */
|
|
|
|
logs_empty_and_mark_files_at_shutdown();
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
MDEV-15053 Reduce buf_pool_t::mutex contention
User-visible changes: The INFORMATION_SCHEMA views INNODB_BUFFER_PAGE
and INNODB_BUFFER_PAGE_LRU will report a dummy value FLUSH_TYPE=0
and will no longer report the PAGE_STATE value READY_FOR_USE.
We will remove some fields from buf_page_t and move much code to
member functions of buf_pool_t and buf_page_t, so that the access
rules of data members can be enforced consistently.
Evicting or adding pages in buf_pool.LRU will remain covered by
buf_pool.mutex.
Evicting or adding pages in buf_pool.page_hash will remain
covered by both buf_pool.mutex and the buf_pool.page_hash X-latch.
After this fix, buf_pool.page_hash lookups can entirely
avoid acquiring buf_pool.mutex, only relying on
buf_pool.hash_lock_get() S-latch.
Similarly, buf_flush_check_neighbors() can will rely solely on
buf_pool.mutex, no buf_pool.page_hash latch at all.
The buf_pool.mutex is rather contended in I/O heavy benchmarks,
especially when the workload does not fit in the buffer pool.
The first attempt to alleviate the contention was the
buf_pool_t::mutex split in
commit 4ed7082eefe56b3e97e0edefb3df76dd7ef5e858
which introduced buf_block_t::mutex, which we are now removing.
Later, multiple instances of buf_pool_t were introduced
in commit c18084f71b02ea707c6461353e6cfc15d7553bc6
and recently removed by us in
commit 1a6f708ec594ac0ae2dd30db926ab07b100fa24b (MDEV-15058).
UNIV_BUF_DEBUG: Remove. This option to enable some buffer pool
related debugging in otherwise non-debug builds has not been used
for years. Instead, we have been using UNIV_DEBUG, which is enabled
in CMAKE_BUILD_TYPE=Debug.
buf_block_t::mutex, buf_pool_t::zip_mutex: Remove. We can mainly rely on
std::atomic and the buf_pool.page_hash latches, and in some cases
depend on buf_pool.mutex or buf_pool.flush_list_mutex just like before.
We must always release buf_block_t::lock before invoking
unfix() or io_unfix(), to prevent a glitch where a block that was
added to the buf_pool.free list would apper X-latched. See
commit c5883debd6ef440a037011c11873b396923e93c5 how this glitch
was finally caught in a debug environment.
We move some buf_pool_t::page_hash specific code from the
ha and hash modules to buf_pool, for improved readability.
buf_pool_t::close(): Assert that all blocks are clean, except
on aborted startup or crash-like shutdown.
buf_pool_t::validate(): No longer attempt to validate
n_flush[] against the number of BUF_IO_WRITE fixed blocks,
because buf_page_t::flush_type no longer exists.
buf_pool_t::watch_set(): Replaces buf_pool_watch_set().
Reduce mutex contention by separating the buf_pool.watch[]
allocation and the insert into buf_pool.page_hash.
buf_pool_t::page_hash_lock<bool exclusive>(): Acquire a
buf_pool.page_hash latch.
Replaces and extends buf_page_hash_lock_s_confirm()
and buf_page_hash_lock_x_confirm().
buf_pool_t::READ_AHEAD_PAGES: Renamed from BUF_READ_AHEAD_PAGES.
buf_pool_t::curr_size, old_size, read_ahead_area, n_pend_reads:
Use Atomic_counter.
buf_pool_t::running_out(): Replaces buf_LRU_buf_pool_running_out().
buf_pool_t::LRU_remove(): Remove a block from the LRU list
and return its predecessor. Incorporates buf_LRU_adjust_hp(),
which was removed.
buf_page_get_gen(): Remove a redundant call of fsp_is_system_temporary(),
for mode == BUF_GET_IF_IN_POOL_OR_WATCH, which is only used by
BTR_DELETE_OP (purge), which is never invoked on temporary tables.
buf_free_from_unzip_LRU_list_batch(): Avoid redundant assignments.
buf_LRU_free_from_unzip_LRU_list(): Simplify the loop condition.
buf_LRU_free_page(): Clarify the function comment.
buf_flush_check_neighbor(), buf_flush_check_neighbors():
Rewrite the construction of the page hash range. We will hold
the buf_pool.mutex for up to buf_pool.read_ahead_area (at most 64)
consecutive lookups of buf_pool.page_hash.
buf_flush_page_and_try_neighbors(): Remove.
Merge to its only callers, and remove redundant operations in
buf_flush_LRU_list_batch().
buf_read_ahead_random(), buf_read_ahead_linear(): Rewrite.
Do not acquire buf_pool.mutex, and iterate directly with page_id_t.
ut_2_power_up(): Remove. my_round_up_to_next_power() is inlined
and avoids any loops.
fil_page_get_prev(), fil_page_get_next(), fil_addr_is_null(): Remove.
buf_flush_page(): Add a fil_space_t* parameter. Minimize the
buf_pool.mutex hold time. buf_pool.n_flush[] is no longer updated
atomically with the io_fix, and we will protect most buf_block_t
fields with buf_block_t::lock. The function
buf_flush_write_block_low() is removed and merged here.
buf_page_init_for_read(): Use static linkage. Initialize the newly
allocated block and acquire the exclusive buf_block_t::lock while not
holding any mutex.
IORequest::IORequest(): Remove the body. We only need to invoke
set_punch_hole() in buf_flush_page() and nowhere else.
buf_page_t::flush_type: Remove. Replaced by IORequest::flush_type.
This field is only used during a fil_io() call.
That function already takes IORequest as a parameter, so we had
better introduce for the rarely changing field.
buf_block_t::init(): Replaces buf_page_init().
buf_page_t::init(): Replaces buf_page_init_low().
buf_block_t::initialise(): Initialise many fields, but
keep the buf_page_t::state(). Both buf_pool_t::validate() and
buf_page_optimistic_get() requires that buf_page_t::in_file()
be protected atomically with buf_page_t::in_page_hash
and buf_page_t::in_LRU_list.
buf_page_optimistic_get(): Now that buf_block_t::mutex
no longer exists, we must check buf_page_t::io_fix()
after acquiring the buf_pool.page_hash lock, to detect
whether buf_page_init_for_read() has been initiated.
We will also check the io_fix() before acquiring hash_lock
in order to avoid unnecessary computation.
The field buf_block_t::modify_clock (protected by buf_block_t::lock)
allows buf_page_optimistic_get() to validate the block.
buf_page_t::real_size: Remove. It was only used while flushing
pages of page_compressed tables.
buf_page_encrypt(): Add an output parameter that allows us ot eliminate
buf_page_t::real_size. Replace a condition with debug assertion.
buf_page_should_punch_hole(): Remove.
buf_dblwr_t::add_to_batch(): Replaces buf_dblwr_add_to_batch().
Add the parameter size (to replace buf_page_t::real_size).
buf_dblwr_t::write_single_page(): Replaces buf_dblwr_write_single_page().
Add the parameter size (to replace buf_page_t::real_size).
fil_system_t::detach(): Replaces fil_space_detach().
Ensure that fil_validate() will not be violated even if
fil_system.mutex is released and reacquired.
fil_node_t::complete_io(): Renamed from fil_node_complete_io().
fil_node_t::close_to_free(): Replaces fil_node_close_to_free().
Avoid invoking fil_node_t::close() because fil_system.n_open
has already been decremented in fil_space_t::detach().
BUF_BLOCK_READY_FOR_USE: Remove. Directly use BUF_BLOCK_MEMORY.
BUF_BLOCK_ZIP_DIRTY: Remove. Directly use BUF_BLOCK_ZIP_PAGE,
and distinguish dirty pages by buf_page_t::oldest_modification().
BUF_BLOCK_POOL_WATCH: Remove. Use BUF_BLOCK_NOT_USED instead.
This state was only being used for buf_page_t that are in
buf_pool.watch.
buf_pool_t::watch[]: Remove pointer indirection.
buf_page_t::in_flush_list: Remove. It was set if and only if
buf_page_t::oldest_modification() is nonzero.
buf_page_decrypt_after_read(), buf_corrupt_page_release(),
buf_page_check_corrupt(): Change the const fil_space_t* parameter
to const fil_node_t& so that we can report the correct file name.
buf_page_monitor(): Declare as an ATTRIBUTE_COLD global function.
buf_page_io_complete(): Split to buf_page_read_complete() and
buf_page_write_complete().
buf_dblwr_t::in_use: Remove.
buf_dblwr_t::buf_block_array: Add IORequest::flush_t.
buf_dblwr_sync_datafiles(): Remove. It was a useless wrapper of
os_aio_wait_until_no_pending_writes().
buf_flush_write_complete(): Declare static, not global.
Add the parameter IORequest::flush_t.
buf_flush_freed_page(): Simplify the code.
recv_sys_t::flush_lru: Renamed from flush_type and changed to bool.
fil_read(), fil_write(): Replaced with direct use of fil_io().
fil_buffering_disabled(): Remove. Check srv_file_flush_method directly.
fil_mutex_enter_and_prepare_for_io(): Return the resolved
fil_space_t* to avoid a duplicated lookup in the caller.
fil_report_invalid_page_access(): Clean up the parameters.
fil_io(): Return fil_io_t, which comprises fil_node_t and error code.
Always invoke fil_space_t::acquire_for_io() and let either the
sync=true caller or fil_aio_callback() invoke
fil_space_t::release_for_io().
fil_aio_callback(): Rewrite to replace buf_page_io_complete().
fil_check_pending_operations(): Remove a parameter, and remove some
redundant lookups.
fil_node_close_to_free(): Wait for n_pending==0. Because we no longer
do an extra lookup of the tablespace between fil_io() and the
completion of the operation, we must give fil_node_t::complete_io() a
chance to decrement the counter.
fil_close_tablespace(): Remove unused parameter trx, and document
that this is only invoked during the error handling of IMPORT TABLESPACE.
row_import_discard_changes(): Merged with the only caller,
row_import_cleanup(). Do not lock up the data dictionary while
invoking fil_close_tablespace().
logs_empty_and_mark_files_at_shutdown(): Do not invoke
fil_close_all_files(), to avoid a !needs_flush assertion failure
on fil_node_t::close().
innodb_shutdown(): Invoke os_aio_free() before fil_close_all_files().
fil_close_all_files(): Invoke fil_flush_file_spaces()
to ensure proper durability.
thread_pool::unbind(): Fix a crash that would occur on Windows
after srv_thread_pool->disable_aio() and os_file_close().
This fix was submitted by Vladislav Vaintroub.
Thanks to Matthias Leich and Axel Schwenke for extensive testing,
Vladislav Vaintroub for helpful comments, and Eugene Kosov for a review.
2020-06-05 12:35:46 +03:00
|
|
|
os_aio_free();
|
2020-10-26 16:04:12 +02:00
|
|
|
fil_space_t::close_all();
|
2017-06-29 23:10:46 +03:00
|
|
|
/* Exit any remaining threads. */
|
2020-10-26 15:04:24 +02:00
|
|
|
ut_ad(!buf_page_cleaner_is_active);
|
|
|
|
srv_shutdown_threads();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
if (srv_monitor_file) {
|
2019-05-23 12:55:03 +04:00
|
|
|
my_fclose(srv_monitor_file, MYF(MY_WME));
|
2014-02-26 19:11:54 +01:00
|
|
|
srv_monitor_file = 0;
|
|
|
|
if (srv_monitor_file_name) {
|
|
|
|
unlink(srv_monitor_file_name);
|
2016-08-12 11:17:45 +03:00
|
|
|
ut_free(srv_monitor_file_name);
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (srv_misc_tmpfile) {
|
2019-05-23 12:55:03 +04:00
|
|
|
my_fclose(srv_misc_tmpfile, MYF(MY_WME));
|
2014-02-26 19:11:54 +01:00
|
|
|
srv_misc_tmpfile = 0;
|
|
|
|
}
|
|
|
|
|
2019-05-17 14:32:53 +03:00
|
|
|
ut_ad(dict_sys.is_initialised() || !srv_was_started);
|
2018-01-17 19:43:08 +04:00
|
|
|
ut_ad(trx_sys.is_initialised() || !srv_was_started);
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
ut_ad(buf_dblwr.is_initialised() || !srv_was_started
|
|
|
|
|| srv_read_only_mode
|
2017-02-01 15:47:33 +02:00
|
|
|
|| srv_force_recovery >= SRV_FORCE_NO_TRX_UNDO);
|
2018-02-22 20:46:42 +04:00
|
|
|
ut_ad(lock_sys.is_initialised() || !srv_was_started);
|
2018-04-27 10:06:14 +03:00
|
|
|
ut_ad(log_sys.is_initialised() || !srv_was_started);
|
2019-07-03 16:05:34 +03:00
|
|
|
ut_ad(ibuf.index || !srv_was_started);
|
2017-01-31 19:43:03 +02:00
|
|
|
|
2019-10-29 22:37:12 +01:00
|
|
|
dict_stats_deinit();
|
2014-12-22 16:53:17 +02:00
|
|
|
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
if (srv_started_redo) {
|
2017-01-31 19:43:03 +02:00
|
|
|
ut_ad(!srv_read_only_mode);
|
|
|
|
/* srv_shutdown_bg_undo_sources() already invoked
|
|
|
|
fts_optimize_shutdown(); dict_stats_shutdown(); */
|
|
|
|
|
|
|
|
fil_crypt_threads_cleanup();
|
2017-05-26 22:44:16 +03:00
|
|
|
btr_defragment_shutdown();
|
2017-01-31 19:43:03 +02:00
|
|
|
}
|
2015-12-20 12:33:58 +01:00
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
/* This must be disabled before closing the buffer pool
|
|
|
|
and closing the data dictionary. */
|
|
|
|
|
2017-02-23 23:05:12 +02:00
|
|
|
#ifdef BTR_CUR_HASH_ADAPT
|
2019-05-17 14:32:53 +03:00
|
|
|
if (dict_sys.is_initialised()) {
|
2020-05-27 13:03:06 +05:30
|
|
|
btr_search_disable();
|
2017-01-31 19:43:03 +02:00
|
|
|
}
|
2017-02-23 23:05:12 +02:00
|
|
|
#endif /* BTR_CUR_HASH_ADAPT */
|
2019-07-03 16:05:34 +03:00
|
|
|
ibuf_close();
|
2018-04-27 10:06:14 +03:00
|
|
|
log_sys.close();
|
2018-02-22 09:30:41 +02:00
|
|
|
purge_sys.close();
|
2018-03-30 00:33:58 +04:00
|
|
|
trx_sys.close();
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
buf_dblwr.close();
|
2018-02-22 20:46:42 +04:00
|
|
|
lock_sys.close();
|
2016-08-12 11:17:45 +03:00
|
|
|
trx_pool_close();
|
|
|
|
|
2014-02-26 19:11:54 +01:00
|
|
|
if (!srv_read_only_mode) {
|
2020-12-04 19:02:58 +02:00
|
|
|
mysql_mutex_destroy(&srv_monitor_file_mutex);
|
|
|
|
mysql_mutex_destroy(&srv_misc_tmpfile_mutex);
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2019-05-17 14:32:53 +03:00
|
|
|
dict_sys.close();
|
2019-11-14 16:58:58 +02:00
|
|
|
btr_search_sys_free();
|
2014-02-26 19:11:54 +01:00
|
|
|
row_mysql_close();
|
|
|
|
srv_free();
|
2018-03-28 09:00:06 +03:00
|
|
|
fil_system.close();
|
2014-02-26 19:11:54 +01:00
|
|
|
pars_lexer_close();
|
2019-05-24 16:19:38 +03:00
|
|
|
recv_sys.close();
|
2017-06-29 22:24:48 +03:00
|
|
|
|
2020-03-18 21:48:00 +02:00
|
|
|
ut_ad(buf_pool.is_initialised() || !srv_was_started);
|
|
|
|
buf_pool.close();
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2020-12-04 11:46:47 +02:00
|
|
|
srv_sys_space.shutdown();
|
|
|
|
if (srv_tmp_space.get_sanity_check_status()) {
|
2021-02-01 13:17:17 +02:00
|
|
|
if (fil_system.temp_space) {
|
|
|
|
fil_system.temp_space->close();
|
|
|
|
}
|
2020-12-04 11:46:47 +02:00
|
|
|
srv_tmp_space.delete_files();
|
|
|
|
}
|
|
|
|
srv_tmp_space.shutdown();
|
|
|
|
|
2017-01-31 19:43:03 +02:00
|
|
|
if (srv_was_started && srv_print_verbose_log) {
|
2016-08-12 11:17:45 +03:00
|
|
|
ib::info() << "Shutdown completed; log sequence number "
|
MDEV-15132 Avoid accessing the TRX_SYS page
InnoDB maintains an internal persistent sequence of transaction
identifiers. This sequence is used for assigning both transaction
start identifiers (DB_TRX_ID=trx->id) and end identifiers (trx->no)
as well as end identifiers for the mysql.transaction_registry table
that was introduced in MDEV-12894.
TRX_SYS_TRX_ID_WRITE_MARGIN: Remove. After this many updates of
the sequence we used to update the TRX_SYS page. We can avoid accessing
the TRX_SYS page if we modify the InnoDB startup so that resurrecting
the sequence from other pages of the transaction system.
TRX_SYS_TRX_ID_STORE: Deprecate. The field only exists for the purpose
of upgrading from an earlier version of MySQL or MariaDB.
Starting with this fix, MariaDB will rely on the fields
TRX_UNDO_TRX_ID, TRX_UNDO_TRX_NO in the undo log header page of
each non-committed transaction, and on the new field
TRX_RSEG_MAX_TRX_ID in rollback segment header pages.
Because of this change, setting innodb_force_recovery=5 or 6 may cause
the system to recover with trx_sys.get_max_trx_id()==0. We must adjust
checks for invalid DB_TRX_ID and PAGE_MAX_TRX_ID accordingly.
We will change the startup and shutdown messages to display the
trx_sys.get_max_trx_id() in addition to the log sequence number.
trx_sys_t::flush_max_trx_id(): Remove.
trx_undo_mem_create_at_db_start(), trx_undo_lists_init():
Add an output parameter max_trx_id, to be updated from
TRX_UNDO_TRX_ID, TRX_UNDO_TRX_NO.
TRX_RSEG_MAX_TRX_ID: New field, for persisting
trx_sys.get_max_trx_id() at the time of the latest transaction commit.
Startup is not reading the undo log pages of committed transactions.
We want to avoid additional page accesses on startup, as well as
trouble when all undo logs have been emptied.
On startup, we will simply determine the maximum value from all pages
that are being read anyway.
TRX_RSEG_FORMAT: Redefined from TRX_RSEG_MAX_SIZE.
Old versions of InnoDB wrote uninitialized garbage to unused data fields.
Because of this, we cannot simply introduce a new field in the
rollback segment pages and expect it to be always zero, like it would
if the database was created by a recent enough InnoDB version.
Luckily, it looks like the field TRX_RSEG_MAX_SIZE was always written
as 0xfffffffe. We will indicate a new subformat of the page by writing
0 to this field. This has the nice side effect that after a downgrade
to older versions of InnoDB, transactions should fail to allocate any
undo log, that is, writes will be blocked. So, there is no problem of
getting corrupted transaction identifiers after downgrading.
trx_rseg_t::max_size: Remove.
trx_rseg_header_create(): Remove the parameter max_size=ULINT_MAX.
trx_purge_add_undo_to_history(): Update TRX_RSEG_MAX_SIZE
(and TRX_RSEG_FORMAT if needed). This is invoked on transaction commit.
trx_rseg_mem_restore(): If TRX_RSEG_FORMAT contains 0,
read TRX_RSEG_MAX_SIZE.
trx_rseg_array_init(): Invoke trx_sys.init_max_trx_id(max_trx_id + 1)
where max_trx_id was the maximum that was encountered in the rollback
segment pages and the undo log pages of recovered active, XA PREPARE,
or some committed transactions. (See trx_purge_add_undo_to_history()
which invokes trx_rsegf_set_nth_undo(..., FIL_NULL, ...);
not all committed transactions will be immediately detached from the
rollback segment header.)
2018-01-31 10:24:19 +02:00
|
|
|
<< srv_shutdown_lsn
|
|
|
|
<< "; transaction id " << trx_sys.get_max_trx_id();
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
2019-11-14 16:58:58 +02:00
|
|
|
srv_thread_pool_end();
|
MDEV-23399: Performance regression with write workloads
The buffer pool refactoring in MDEV-15053 and MDEV-22871 shifted
the performance bottleneck to the page flushing.
The configuration parameters will be changed as follows:
innodb_lru_flush_size=32 (new: how many pages to flush on LRU eviction)
innodb_lru_scan_depth=1536 (old: 1024)
innodb_max_dirty_pages_pct=90 (old: 75)
innodb_max_dirty_pages_pct_lwm=75 (old: 0)
Note: The parameter innodb_lru_scan_depth will only affect LRU
eviction of buffer pool pages when a new page is being allocated. The
page cleaner thread will no longer evict any pages. It used to
guarantee that some pages will remain free in the buffer pool. Now, we
perform that eviction 'on demand' in buf_LRU_get_free_block().
The parameter innodb_lru_scan_depth(srv_LRU_scan_depth) is used as follows:
* When the buffer pool is being shrunk in buf_pool_t::withdraw_blocks()
* As a buf_pool.free limit in buf_LRU_list_batch() for terminating
the flushing that is initiated e.g., by buf_LRU_get_free_block()
The parameter also used to serve as an initial limit for unzip_LRU
eviction (evicting uncompressed page frames while retaining
ROW_FORMAT=COMPRESSED pages), but now we will use a hard-coded limit
of 100 or unlimited for invoking buf_LRU_scan_and_free_block().
The status variables will be changed as follows:
innodb_buffer_pool_pages_flushed: This includes also the count of
innodb_buffer_pool_pages_LRU_flushed and should work reliably,
updated one by one in buf_flush_page() to give more real-time
statistics. The function buf_flush_stats(), which we are removing,
was not called in every code path. For both counters, we will use
regular variables that are incremented in a critical section of
buf_pool.mutex. Note that show_innodb_vars() directly links to the
variables, and reads of the counters will *not* be protected by
buf_pool.mutex, so you cannot get a consistent snapshot of both variables.
The following INFORMATION_SCHEMA.INNODB_METRICS counters will be
removed, because the page cleaner no longer deals with writing or
evicting least recently used pages, and because the single-page writes
have been removed:
* buffer_LRU_batch_flush_avg_time_slot
* buffer_LRU_batch_flush_avg_time_thread
* buffer_LRU_batch_flush_avg_time_est
* buffer_LRU_batch_flush_avg_pass
* buffer_LRU_single_flush_scanned
* buffer_LRU_single_flush_num_scan
* buffer_LRU_single_flush_scanned_per_call
When moving to a single buffer pool instance in MDEV-15058, we missed
some opportunity to simplify the buf_flush_page_cleaner thread. It was
unnecessarily using a mutex and some complex data structures, even
though we always have a single page cleaner thread.
Furthermore, the buf_flush_page_cleaner thread had separate 'recovery'
and 'shutdown' modes where it was waiting to be triggered by some
other thread, adding unnecessary latency and potential for hangs in
relatively rarely executed startup or shutdown code.
The page cleaner was also running two kinds of batches in an
interleaved fashion: "LRU flush" (writing out some least recently used
pages and evicting them on write completion) and the normal batches
that aim to increase the MIN(oldest_modification) in the buffer pool,
to help the log checkpoint advance.
The buf_pool.flush_list flushing was being blocked by
buf_block_t::lock for no good reason. Furthermore, if the FIL_PAGE_LSN
of a page is ahead of log_sys.get_flushed_lsn(), that is, what has
been persistently written to the redo log, we would trigger a log
flush and then resume the page flushing. This would unnecessarily
limit the performance of the page cleaner thread and trigger the
infamous messages "InnoDB: page_cleaner: 1000ms intended loop took 4450ms.
The settings might not be optimal" that were suppressed in
commit d1ab89037a518fcffbc50c24e4bd94e4ec33aed0 unless log_warnings>2.
Our revised algorithm will make log_sys.get_flushed_lsn() advance at
the start of buf_flush_lists(), and then execute a 'best effort' to
write out all pages. The flush batches will skip pages that were modified
since the log was written, or are are currently exclusively locked.
The MDEV-13670 message "page_cleaner: 1000ms intended loop took" message
will be removed, because by design, the buf_flush_page_cleaner() should
not be blocked during a batch for extended periods of time.
We will remove the single-page flushing altogether. Related to this,
the debug parameter innodb_doublewrite_batch_size will be removed,
because all of the doublewrite buffer will be used for flushing
batches. If a page needs to be evicted from the buffer pool and all
100 least recently used pages in the buffer pool have unflushed
changes, buf_LRU_get_free_block() will execute buf_flush_lists() to
write out and evict innodb_lru_flush_size pages. At most one thread
will execute buf_flush_lists() in buf_LRU_get_free_block(); other
threads will wait for that LRU flushing batch to finish.
To improve concurrency, we will replace the InnoDB ib_mutex_t and
os_event_t native mutexes and condition variables in this area of code.
Most notably, this means that the buffer pool mutex (buf_pool.mutex)
is no longer instrumented via any InnoDB interfaces. It will continue
to be instrumented via PERFORMANCE_SCHEMA.
For now, both buf_pool.flush_list_mutex and buf_pool.mutex will be
declared with MY_MUTEX_INIT_FAST (PTHREAD_MUTEX_ADAPTIVE_NP). The critical
sections of buf_pool.flush_list_mutex should be shorter than those for
buf_pool.mutex, because in the worst case, they cover a linear scan of
buf_pool.flush_list, while the worst case of a critical section of
buf_pool.mutex covers a linear scan of the potentially much longer
buf_pool.LRU list.
mysql_mutex_is_owner(), safe_mutex_is_owner(): New predicate, usable
with SAFE_MUTEX. Some InnoDB debug assertions need this predicate
instead of mysql_mutex_assert_owner() or mysql_mutex_assert_not_owner().
buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
Replaces buf_pool_t::init_flush[] and buf_pool_t::n_flush[].
The number of active flush operations.
buf_pool_t::mutex, buf_pool_t::flush_list_mutex: Use mysql_mutex_t
instead of ib_mutex_t, to have native mutexes with PERFORMANCE_SCHEMA
and SAFE_MUTEX instrumentation.
buf_pool_t::done_flush_LRU: Condition variable for !n_flush_LRU.
buf_pool_t::done_flush_list: Condition variable for !n_flush_list.
buf_pool_t::do_flush_list: Condition variable to wake up the
buf_flush_page_cleaner when a log checkpoint needs to be written
or the server is being shut down. Replaces buf_flush_event.
We will keep using timed waits (the page cleaner thread will wake
_at least_ once per second), because the calculations for
innodb_adaptive_flushing depend on fixed time intervals.
buf_dblwr: Allocate statically, and move all code to member functions.
Use a native mutex and condition variable. Remove code to deal with
single-page flushing.
buf_dblwr_check_block(): Make the check debug-only. We were spending
a significant amount of execution time in page_simple_validate_new().
flush_counters_t::unzip_LRU_evicted: Remove.
IORequest: Make more members const. FIXME: m_fil_node should be removed.
buf_flush_sync_lsn: Protect by std::atomic, not page_cleaner.mutex
(which we are removing).
page_cleaner_slot_t, page_cleaner_t: Remove many redundant members.
pc_request_flush_slot(): Replaces pc_request() and pc_flush_slot().
recv_writer_thread: Remove. Recovery works just fine without it, if we
simply invoke buf_flush_sync() at the end of each batch in
recv_sys_t::apply().
recv_recovery_from_checkpoint_finish(): Remove. We can simply call
recv_sys.debug_free() directly.
srv_started_redo: Replaces srv_start_state.
SRV_SHUTDOWN_FLUSH_PHASE: Remove. logs_empty_and_mark_files_at_shutdown()
can communicate with the normal page cleaner loop via the new function
flush_buffer_pool().
buf_flush_remove(): Assert that the calling thread is holding
buf_pool.flush_list_mutex. This removes unnecessary mutex operations
from buf_flush_remove_pages() and buf_flush_dirty_pages(),
which replace buf_LRU_flush_or_remove_pages().
buf_flush_lists(): Renamed from buf_flush_batch(), with simplified
interface. Return the number of flushed pages. Clarified comments and
renamed min_n to max_n. Identify LRU batch by lsn=0. Merge all the functions
buf_flush_start(), buf_flush_batch(), buf_flush_end() directly to this
function, which was their only caller, and remove 2 unnecessary
buf_pool.mutex release/re-acquisition that we used to perform around
the buf_flush_batch() call. At the start, if not all log has been
durably written, wait for a background task to do it, or start a new
task to do it. This allows the log write to run concurrently with our
page flushing batch. Any pages that were skipped due to too recent
FIL_PAGE_LSN or due to them being latched by a writer should be flushed
during the next batch, unless there are further modifications to those
pages. It is possible that a page that we must flush due to small
oldest_modification also carries a recent FIL_PAGE_LSN or is being
constantly modified. In the worst case, all writers would then end up
waiting in log_free_check() to allow the flushing and the checkpoint
to complete.
buf_do_flush_list_batch(): Clarify comments, and rename min_n to max_n.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_flush_space(): Auxiliary function to look up a tablespace for
page flushing.
buf_flush_page(): Defer the computation of space->full_crc32(). Never
call log_write_up_to(), but instead skip persistent pages whose latest
modification (FIL_PAGE_LSN) is newer than the redo log. Also skip
pages on which we cannot acquire a shared latch without waiting.
buf_flush_try_neighbors(): Do not bother checking buf_fix_count
because buf_flush_page() will no longer wait for the page latch.
Take the tablespace as a parameter, and only execute this function
when innodb_flush_neighbors>0. Avoid repeated calls of page_id_t::fold().
buf_flush_relocate_on_flush_list(): Declare as cold, and push down
a condition from the callers.
buf_flush_check_neighbor(): Take id.fold() as a parameter.
buf_flush_sync(): Ensure that the buf_pool.flush_list is empty,
because the flushing batch will skip pages whose modifications have
not yet been written to the log or were latched for modification.
buf_free_from_unzip_LRU_list_batch(): Remove redundant local variables.
buf_flush_LRU_list_batch(): Let the caller buf_do_LRU_batch() initialize
the counters, and report n->evicted.
Cache the last looked up tablespace. If neighbor flushing is not applicable,
invoke buf_flush_page() directly, avoiding a page lookup in between.
buf_do_LRU_batch(): Return the number of pages flushed.
buf_LRU_free_page(): Only release and re-acquire buf_pool.mutex if
adaptive hash index entries are pointing to the block.
buf_LRU_get_free_block(): Do not wake up the page cleaner, because it
will no longer perform any useful work for us, and we do not want it
to compete for I/O while buf_flush_lists(innodb_lru_flush_size, 0)
writes out and evicts at most innodb_lru_flush_size pages. (The
function buf_do_LRU_batch() may complete after writing fewer pages if
more than innodb_lru_scan_depth pages end up in buf_pool.free list.)
Eliminate some mutex release-acquire cycles, and wait for the LRU
flush batch to complete before rescanning.
buf_LRU_check_size_of_non_data_objects(): Simplify the code.
buf_page_write_complete(): Remove the parameter evict, and always
evict pages that were part of an LRU flush.
buf_page_create(): Take a pre-allocated page as a parameter.
buf_pool_t::free_block(): Free a pre-allocated block.
recv_sys_t::recover_low(), recv_sys_t::apply(): Preallocate the block
while not holding recv_sys.mutex. During page allocation, we may
initiate a page flush, which in turn may initiate a log flush, which
would require acquiring log_sys.mutex, which should always be acquired
before recv_sys.mutex in order to avoid deadlocks. Therefore, we must
not be holding recv_sys.mutex while allocating a buffer pool block.
BtrBulk::logFreeCheck(): Skip a redundant condition.
row_undo_step(): Do not invoke srv_inc_activity_count() for every row
that is being rolled back. It should suffice to invoke the function in
trx_flush_log_if_needed() during trx_t::commit_in_memory() when the
rollback completes.
sync_check_enable(): Remove. We will enable innodb_sync_debug from the
very beginning.
Reviewed by: Vladislav Vaintroub
2020-10-15 12:10:42 +03:00
|
|
|
srv_started_redo = false;
|
2017-04-25 09:26:01 +03:00
|
|
|
srv_was_started = false;
|
|
|
|
srv_start_has_been_called = false;
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
/** Get the meta-data filename from the table name for a
|
|
|
|
single-table tablespace.
|
|
|
|
@param[in] table table object
|
|
|
|
@param[out] filename filename
|
|
|
|
@param[in] max_len filename max length */
|
2014-02-26 19:11:54 +01:00
|
|
|
void
|
|
|
|
srv_get_meta_data_filename(
|
2016-09-06 09:43:16 +03:00
|
|
|
dict_table_t* table,
|
|
|
|
char* filename,
|
|
|
|
ulint max_len)
|
2014-02-26 19:11:54 +01:00
|
|
|
{
|
2016-08-12 11:17:45 +03:00
|
|
|
ulint len;
|
|
|
|
char* path;
|
|
|
|
|
|
|
|
/* Make sure the data_dir_path is set. */
|
|
|
|
dict_get_and_save_data_dir_path(table, false);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
|
|
|
if (DICT_TF_HAS_DATA_DIR(table->flags)) {
|
|
|
|
ut_a(table->data_dir_path);
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
path = fil_make_filepath(
|
|
|
|
table->data_dir_path, table->name.m_name, CFG, true);
|
2014-02-26 19:11:54 +01:00
|
|
|
} else {
|
2016-08-12 11:17:45 +03:00
|
|
|
path = fil_make_filepath(NULL, table->name.m_name, CFG, false);
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
ut_a(path);
|
2019-10-30 07:31:39 +02:00
|
|
|
len = strlen(path);
|
2014-02-26 19:11:54 +01:00
|
|
|
ut_a(max_len >= len);
|
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
strcpy(filename, path);
|
2014-02-26 19:11:54 +01:00
|
|
|
|
2016-08-12 11:17:45 +03:00
|
|
|
ut_free(path);
|
2014-02-26 19:11:54 +01:00
|
|
|
}
|