thd_destructor_proxy(): Ensure that purge actually exits,
like the logic should have been ever since MDEV-14080.
srv_purge_shutdown(): A new function to wait for the
purge coordinator to exit. Before exiting, the
purge coordinator will ensure that all purge workers have exited.
srv_purge_coordinator_thread(): Wait for all purge worker threads
to actually exit. An analysis of a core dump of a hung 10.3 server
revealed that one srv_worker_thread did not exit, even though the
purge coordinator had exited. This caused kill_server_thread and
mysqld_main to wait indefinitely. The main InnoDB shutdown was
never called, because unireg_end() was never called.
Thanks to Sergey Vojtovich for feedback and many ideas.
purge_state_t: Remove. The states are replaced with
purge_sys_t::enabled() and purge_sys_t::paused() as follows:
PURGE_STATE_INIT, PURGE_STATE_EXIT, PURGE_STATE_DISABLED: !enabled().
PURGE_STATE_RUN, PURGE_STATE_STOP: paused() distinguishes these.
purge_sys_t::m_paused: Renamed from purge_sys_t::n_stop.
Protected by atomic memory access only, not purge_sys_t::latch.
purge_sys_t::m_enabled: An atomically updated Boolean that
replaces purge_sys_t::state.
purge_sys_t:🏃 Remove, because it duplicates
srv_sys.n_threads_active[SRV_PURGE].
purge_sys_t::running(): Accessor for srv_sys.n_threads_active[SRV_PURGE].
purge_sys_t::stop(): Renamed from trx_purge_stop().
purge_sys_t::resume(): Renamed from trx_purge_run().
Do not acquire latch; solely rely on atomics.
purge_sys_t::is_initialised(), purge_sys_t::m_initialised: Remove.
purge_sys_t::create(), purge_sys_t::close(): Instead of invoking
is_initialised(), check whether event is NULL.
purge_sys_t::event: Move before latch, so that fields that are
protected by latch can reside on the same cache line with latch.
srv_start_wait_for_purge_to_start(): Merge to the only caller srv_start().
trx_purge_stop(): Release purge_sys->latch before attempting to
wake up the purge threads, so that they can actually wake up.
This is a regression of commit a13a636c74
which attempted to fix MDEV-11802 by ensuring that srv_purge_wakeup()
will actually wait until all purge threads wake up.
Due to the purge_sys->latch, the threads might never wake up,
because some purge threads could end up waiting for purge_sys->latch
in trx_undo_prev_version_build() while holding dict_operation_lock
in shared mode. This in turn would block any DDL operations, the
InnoDB master thread, and InnoDB shutdown.
Problem:
=======
InnoDB master thread encounters the shutdown state as SRV_SHUTDOWN_FLUSH_PHASE
when innodb_force_recovery >=2 and slow scheduling of master thread during
shutdown.
Fix:
====
There is no need for master thread itself if innodb_force_recovery >=2.
Don't create the master thread if innodb_force_recovery >= 2
Remove unused InnoDB function parameters and functions.
i_s_sys_virtual_fill_table(): Do not allocate heap memory.
mtr_is_block_fix(): Replace with mtr_memo_contains().
mtr_is_page_fix(): Replace with mtr_memo_contains_page().
Implement innodb_flush_method as an enum parameter in Mariabackup,
instead of ignoring the option and hard-wiring it to a default value.
xb0xb.h: Remove. Only xtrabackup.cc refers to the enum parameters.
innodb_flush_method_names[], innodb_flush_method_typelib[]:
Define as non-static, so that mariabackup can share the definitions.
srv_file_flush_method: Change the type to ulong, to match the
assignment in init_one_value() and handle_options() in mariabackup.
There is only one log_sys and only one log_sys.log.
log_t::files::create(): Replaces log_init().
log_t::files::close(): Replaces log_group_close(), log_group_close_all().
fil_close_log_files(): if (free) log_sys.log_close();
The callers that passed free=true used to call log_group_close_all().
log_header_read(): Replaces log_group_header_read().
log_t::files::file_header_bufs_ptr: Use a single allocation.
log_t::files::file_header_bufs[]: Statically allocate the pointers.
log_t::files::set_fields(): Replaces log_group_set_fields().
log_t::files::calc_lsn_offset(): Replaces log_group_calc_lsn_offset().
Simplify the computation by using fewer variables.
log_t::files::read_log_seg(): Replaces log_group_read_log_seg().
log_sys_t::complete_checkpoint(): Replaces log_io_complete_checkpoint().
fil_aio_wait(): Move the logic from log_io_complete().
There is only one redo log subsystem in InnoDB. Allocate the object
statically, to avoid unnecessary dereferencing of the pointer.
log_t::create(): Renamed from log_sys_init().
log_t::close(): Renamed from log_shutdown().
log_t::checkpoint_buf_ptr: Remove. Allocate log_t::checkpoint_buf
statically.
Bind more InnoDB parameters directly to MYSQL_SYSVAR and
remove "shadow variables".
innodb_change_buffering: Declare as ENUM, not STRING.
innodb_flush_method: Declare as ENUM, not STRING.
innodb_log_buffer_size: Bind directly to srv_log_buffer_size,
without rounding it to a multiple of innodb_page_size.
LOG_BUFFER_SIZE: Remove.
SysTablespace::normalize_size(): Renamed from normalize().
innodb_init_params(): A new function to initialize and validate
InnoDB startup parameters.
innodb_init(): Renamed from innobase_init(). Invoke innodb_init_params()
before actually trying to start up InnoDB.
srv_start(bool): Renamed from innobase_start_or_create_for_mysql().
Added the input parameter create_new_db.
SRV_ALL_O_DIRECT_FSYNC: Define only for _WIN32.
xb_normalize_init_values(): Merge to innodb_init_param().
MariaDB uses HAVE_LZO, not HAVE_LZO1X (which was never defined).
Also, the variable srv_lzo_disabled was never defined or read
(only declared and assigned to, in unreachable code).
file IO, rather than int.
On Windows, it is suboptimal to depend on C runtime, as it has limited
number of file descriptors. This change eliminates
os_file_read_no_error_handling_int_fd(), os_file_write_int_fd(),
OS_FILE_FROM_FD() macro.
srv_purge_should_exit(): Remove the parameter n_purged.
If we happened to have n_purged==0 while some transaction was still
active, and then that transaction was added to the history list,
we were prematurely stopping the purge. It is more appropriate to
first check for trx_sys.any_active_transactions() == 0
(this count can only decrease during shutdown) and then for
trx_sys.history_size() == 0 (that count typically decreases, but
can increase when any remaining active transactions are committed
or rolled back).
innodb.dml_purge: Remove a server restart, and explicitly wait for
purge, and use FLUSH TABLE FOR EXPORT to read the file contents.
This will make the test run faster, easier to debug, and also
allow it to run with --embedded. This might also help repeat
MDEV-11802 better. The issue MDEV-13603 remains will remain tested
by innodb.table_flags.
purge_sys_t::n_submitted: Document that it is only accessed by
srv_purge_coordinator_thread.
purge_sys_t::n_completed: Exclusively use my_atomic access.
srv_task_execute(): Simplify the code.
srv_purge_coordinator_thread(): Test the cheaper condition first.
trx_purge(): Atomically access purge_sys.n_completed.
Remove some code duplication.
trx_purge_wait_for_workers_to_complete(): Atomically access
purge_sys.n_completed. Remove an unnecessary local variable.
trx_purge_stop(): Remove a redundant assignment.
buf_flush_remove(): Disable the output for now, because we
certainly do not want this after every page flush on shutdown.
It must be rate-limited somehow. There already is a timeout
extension for waiting the page cleaner to exit in
logs_empty_and_mark_files_at_shutdown().
log_write_up_to(): Use correct format.
srv_purge_should_exit(): Move the timeout extension to the
appropriate place, from one of the callers.
Use systemd EXTEND_TIMEOUT_USEC to advise systemd of progress
Move towards progress measures rather than pure time based measures.
Progress reporting at numberious shutdown/startup locations incuding:
* For innodb_fast_shutdown=0 trx_roll_must_shutdown() for rolling back incomplete transactions.
* For merging the change buffer (in srv_shutdown(bool ibuf_merge))
* For purging history, srv_do_purge
Thanks Marko for feedback and suggestions.
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.
Add fil_system_t::sys_space, fil_system_t::temp_space.
These will replace lookups for TRX_SYS_SPACE or SRV_TMP_SPACE_ID.
mtr_t::m_undo_space, mtr_t::m_sys_space: Remove.
mtr_t::set_sys_modified(): Remove.
fil_space_get_type(), fil_space_get_n_reserved_extents(): Remove.
fsp_header_get_tablespace_size(), fsp_header_inc_size():
Merge to the only caller, innobase_start_or_create_for_mysql().
ha_innobase::check_if_supported_inplace_alter(): Only check for
high_level_read_only. Do not unnecessarily refuse
ALTER TABLE...ALGORITHM=INPLACE if innodb_force_recovery was
specified as 1, 2, or 3.
innobase_start_or_create_for_mysql(): Block all writes from SQL
if the system tablespace was initialized with 'newraw'.
There is only one lock_sys. Allocate it statically in order to avoid
dereferencing a pointer whenever accessing it. Also, align some
members to their own cache line in order to avoid false sharing.
lock_sys_t::create(): The deferred constructor.
lock_sys_t::close(): The early destructor.
There is only one purge_sys. Allocate it statically in order to avoid
dereferencing a pointer whenever accessing it. Also, align some
members to their own cache line in order to avoid false sharing.
purge_sys_t::create(): The deferred constructor.
purge_sys_t::close(): The early destructor.
undo::Truncate::create(): The deferred constructor.
Because purge_sys.undo_trunc is constructed before the start-up
parameters are parsed, the normal constructor would copy a
wrong value of srv_purge_rseg_truncate_frequency.
TrxUndoRsegsIterator: Do not forward-declare an inline constructor,
because the static construction of purge_sys.rseg_iter would not have
access to it.
trx_purge(): Remove the parameter limit or batch_size, which is
always passed as srv_purge_batch_size.
trx_purge_attach_undo_recs(): Remove the parameters purge_sys, batch_size.
Refer to srv_purge_batch_size.
trx_purge_wait_for_workers_to_complete(): Remove the parameter purge_sys.
Add innodb debug system variable, innodb_buffer_pool_load_pages_abort, to test
the behaviour of innodb_buffer_pool_load_incomplete.
(innodb_buufer_pool_dump_abort_loads.test)
This is based on a prototype by
Thirunarayanan Balathandayuthapani <thiru@mariadb.com>.
Binlog and Galera write-set replication information was written into
TRX_SYS page on each commit. Instead of writing to the TRX_SYS during
normal operation, InnoDB can make use of rollback segment header pages,
which are already being written to during a commit.
The following list of fields in rollback segment header page are added:
TRX_RSEG_BINLOG_OFFSET
TRX_RSEG_BINLOG_NAME (NUL-terminated; empty name = not present)
TRX_RSEG_WSREP_XID_FORMAT (0=not present; 1=present)
TRX_RSEG_WSREP_XID_GTRID
TRX_RSEG_WSREP_XID_BQUAL
TRX_RSEG_WSREP_XID_DATA
trx_sys_t: Introduce the fields
recovered_binlog_filename, recovered_binlog_offset, recovered_wsrep_xid.
To facilitate upgrade from older mysql or mariaDB versions, we will read
the information in TRX_SYS page. It will be overridden by the
information that we find in rollback segment header pages.
Mariabackup --prepare will read the metadata from the rollback
segment header pages via trx_rseg_array_init(). It will still
not read any undo log pages or recover any transactions.
trx_sys_t::rseg_history_len: Make private, and clarify the
documentation.
trx_sys_t::history_size(): Read rseg_history_len.
trx_sys_t::history_insert(), trx_sys_t::history_remove(),
trx_sys_t::history_add(): Update rseg_history_len.
Handle string length as size_t, consistently (almost always:))
Change function prototypes to accept size_t, where in the past
ulong or uint were used. change local/member variables to size_t
when appropriate.
This fix excludes rocksdb, spider,spider, sphinx and connect for now.
The merge omitted some InnoDB and XtraDB conflict resolutions,
most notably, failing to merge the fix of MDEV-12173.
ibuf_merge_or_delete_for_page(), lock_rec_block_validate():
Invoke fil_space_acquire_silent() instead of fil_space_acquire().
This fixes MDEV-12173.
wsrep_debug, wsrep_trx_is_aborting(): Removed unused declarations.
_fil_io(): Remove. Instead, declare default parameters for the XtraDB
fil_io().
buf_read_page_low(): Declare default parameters, and clean up some
callers.
os_aio(): Correct the macro that is defined when !UNIV_PFS_IO.
With trx_sys_t::rw_trx_ids removal, MVCC snapshot overhead became
slightly higher. That is instead of copying an array we now have to
iterate LF_HASH. All this done under trx_sys.mutex protection.
This patch moves MVCC snapshot out of trx_sys.mutex.
Clean-ups:
Removed MVCC: doesn't make too much sense to keep it in a separate class
anymore.
Refactored ReadView so that it now calls register()/deregister() routines
(it was vice versa before).
ReadView doesn't have friends anymore. :(
Even less trx_sys.mutex references.
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.)
trx_rseg_mem_restore(): Update the max_trx_id from the undo log pages.
trx_sys_init_at_db_start(): Remove; merge with trx_lists_init_at_db_start().
trx_undo_lists_init(): Move to the only calling module, trx0rseg.cc.
trx_undo_mem_create_at_db_start(): Declare globally. Return the number
of pages.
trx_sysf_t: Remove.
trx_sysf_get(): Return the TRX_SYS page, not a pointer within it.
trx_sysf_rseg_get_space(), trx_sysf_rseg_get_page_no():
Remove a parameter, and merge the declaration and definition.
Take the TRX_SYS page as a parameter.
TRX_SYS_N_RSEGS: Correct the comment.
trx_sysf_rseg_find_free(), trx_sys_update_mysql_binlog_offset(),
trx_sys_update_wsrep_checkpoint(): Take the TRX_SYS page as a parameter.
trx_rseg_header_create(): Add a parameter for the TRX_SYS page.
trx_sysf_rseg_set_space(), trx_sysf_rseg_set_page_no(): Remove;
merge to the only caller, trx_rseg_header_create().
srv_init_abort_low(): Call srv_shutdown_bg_undo_sources() so that if
startup aborts while creating InnoDB system tables, the shutdown will
proceed correctly.
There is only one transaction system object in InnoDB.
Allocate the storage for it at link time, not at runtime.
lock_rec_fetch_page(): Use the correct fetch mode BUF_GET.
Pages may never be deallocated from a tablespace while
record locks are pointing to them.
Reduce divergence between trx_sys_t::rw_trx_hash and trx_sys_t::rw_trx_list
by not adding recovered COMMITTED transactions to trx_sys_t::rw_trx_list.
Such transactions are discarded immediately without creating trx object.
This also required to split rollback and cleanup phases of recovery. To
reflect these updates the following renames happened:
trx_rollback_or_clean_all_recovered() -> trx_rollback_all_recovered()
trx_rollback_or_clean_is_active -> trx_rollback_is_active
trx_rollback_or_clean_recovered() -> trx_rollback_recovered()
trx_cleanup_at_db_startup() -> trx_cleanup_recovered()
Also removed a hack from lock_trx_release_locks(). Instead let recovery
rollback thread to skip committed XA transactions.
innobase_start_or_create_for_mysql(): Only start the data dictionary
and transaction subsystems in normal server startup and during
mariabackup --export.
recv_log_recover_10_3(): Determine if a log from MariaDB 10.3 is clean.
recv_find_max_checkpoint(): Allow startup with a clean 10.3 redo log.
srv_prepare_to_delete_redo_log_files(): When starting up with a 10.3 log,
display a "Downgrading redo log" message instead of "Upgrading".
While the redo log format was changed in MariaDB 10.3.2 and 10.3.3
due to MDEV-12288 and MDEV-11369, it should be technically possible
to upgrade from a crashed MariaDB 10.2 instance.
On a related note, it should be possible for Mariabackup 10.3
to create a backup from a running MariaDB Server 10.2.
mlog_id_t: Put back the 10.2 specific redo log record types
MLOG_UNDO_INSERT, MLOG_UNDO_ERASE_END, MLOG_UNDO_INIT,
MLOG_UNDO_HDR_REUSE.
trx_undo_parse_add_undo_rec(): Parse or apply MLOG_UNDO_INSERT.
trx_undo_erase_page_end(): Apply MLOG_UNDO_ERASE_END.
trx_undo_parse_page_init(): Parse or apply MLOG_UNDO_INIT.
trx_undo_parse_page_header_reuse(): Parse or apply MLOG_UNDO_HDR_REUSE.
recv_log_recover_10_2(): Remove. Always parse the redo log from 10.2.
recv_find_max_checkpoint(), recv_recovery_from_checkpoint_start():
Always parse the redo log from MariaDB 10.2.
recv_parse_or_apply_log_rec_body(): Parse or apply
MLOG_UNDO_INSERT, MLOG_UNDO_ERASE_END, MLOG_UNDO_INIT.
srv_prepare_to_delete_redo_log_files(),
innobase_start_or_create_for_mysql(): Upgrade from a previous (supported)
redo log format.
row_quiesce_table_start(), row_quiesce_table_complete():
Use the more appropriate predicate srv_undo_sources for skipping
purge control. (This change alone is insufficient; it is possible
that this predicate will change during the call to trx_purge_stop()
or trx_purge_run().)
trx_purge_stop(), trx_purge_run(): Tolerate PURGE_STATE_EXIT.
It is very well possible to initiate shutdown soon after the statement
FLUSH TABLES FOR EXPORT has been submitted to execution.
srv_purge_coordinator_thread(): Ensure that the wait for purge_sys->event
in trx_purge_stop() will terminate when the coordinator thread exits.
ha_print_info(): Remove.
srv_printf_innodb_monitor(): Do not acquire btr_search_latches[]
Add the equivalent functionality that was part of the non-debug
version of ha_print_info().
Starting with MySQL 5.7 (or MariaDB 10.2.2) InnoDB no longer contains
the "table monitor" or "tablespace monitor". The conditions on
srv_print_innodb_tablespace_monitor, srv_print_innodb_table_monitor
never hold. So, the code was dead.
Also, remove a bogus reference to dict_print(), which used to implement
the InnoDB table monitor.
srv_purge_wakeup(): If thd_destructor_proxy has initiated the first
step of shutdown, ensure that all purge threads terminate.
logs_empty_and_mark_files_at_shutdown(): Add a debug assertion.
(The purge threads should have been shut down already before this step.)
Also, MDEV-14317 When ALTER TABLE is aborted, do not write garbage pages to data files
As pointed out by Shaohua Wang, the merge of MDEV-13328 from
MariaDB 10.1 (based on MySQL 5.6) to 10.2 (based on 5.7)
was performed incorrectly.
Let us always pass a non-NULL FlushObserver* when writing
to data files is desired.
FlushObserver::is_partial_flush(): Check if this is a bulk-load
(partial flush of the tablespace).
FlushObserver::is_interrupted(): Check for interrupt status.
buf_LRU_flush_or_remove_pages(): Instead of trx_t*, take
FlushObserver* as a parameter.
buf_flush_or_remove_pages(): Remove the parameters flush, trx.
If observer!=NULL, write out the data pages. Use the new predicate
observer->is_partial() to distinguish a partial tablespace flush
(after bulk-loading) from a full tablespace flush (export).
Return a bool (whether all pages were removed from the flush_list).
buf_flush_dirty_pages(): Remove the parameter trx.
Replace all references in InnoDB and XtraDB error log messages
to bugs.mysql.com with references to https://jira.mariadb.org/.
The original merge
commit 4274d0bf57
was accidentally reverted by the subsequent merge
commit 3b35d745c3
New test cases
innodb-page-cleaners
Modified test cases
innodb_page_cleaners_basic
New function buf_flush_set_page_cleaner_thread_cnt
Increase or decrease the amount of page cleaner worker threads.
In case of increase this function creates based on current
abount and requested amount how many new threads should be
created. In case of decrease this function sets up the
requested amount of threads and uses is_requested event
to signal workers. Then we wait until all new treads
are started, old threads that should exit signal
is_finished or shutdown has marked that page cleaner
should finish.
buf_flush_page_cleaner_worker
Store current thread id and thread_no and then signal
event is_finished. If number of used page cleaner threads
decrease we shut down those threads that have thread_no
greater or equal than number of page configured page
cleaners - 1 (note that there will be always page cleaner
coordinator). Before exiting we signal is_finished.
New function innodb_page_cleaners_threads_update
Update function for innodb-page-cleaners system variable.
innobase_start_or_create_for_mysql
If more than one page cleaner threads is configured
we use new function buf_flush_set_page_cleaner_thread_cnt
to set up the requested threads (-1 coordinator).