Commit graph

154 commits

Author SHA1 Message Date
Vladislav Vaintroub
fc2ec25733 MDEV-26166 replace log_write_up_to(LSN_MAX,...) with log_buffer_flush_to_disk()
Also, remove comparison lsn > flush/write lsn, prior to calling
log_write_up_to. The checks and early returns are part of this function.
2021-07-16 18:44:58 +02:00
Marko Mäkelä
e329dc8d86 MDEV-25948 fixup: Demote a warning to a note
buf_dblwr_t::recover(): Issue a note, not a warning, about
pages whose FIL_PAGE_LSN is in the future. This was supposed to be
part of commit 762bcb81b5 (MDEV-25948)
but had been accidentally omitted.
2021-06-24 18:51:05 +03:00
Marko Mäkelä
762bcb81b5 MDEV-25948 Remove log_flush_task
Vladislav Vaintroub suggested that invoking log_flush_up_to()
for every page could perform better than invoking a log write
between buf_pool.flush_list batches, like we started doing in
commit 3a9a3be1c6 (MDEV-23855).
This could depend on the sequence in which pages are being
modified. The buf_pool.flush_list is ordered by
oldest_modification, while the FIL_PAGE_LSN of the pages is
theoretically independent of that. In the pathological case,
we will wait for a log write before writing each individual page.

It turns out that we can defer the call to log_flush_up_to()
until just before submitting the page write. If the doublewrite
buffer is being used, we can submit a write batch of "future" pages
to the doublewrite buffer, and only wait for the log write right
before we are writing an already doublewritten page.
The next doublewrite batch will not be initiated before the last
page write from the current batch has completed.

When a future version introduces asynchronous writes if the log,
we could initiate a write at the start of a flushing batch, to
reduce waiting further.
2021-06-23 19:06:52 +03:00
Marko Mäkelä
6dfd44c828 MDEV-25954: Trim os_aio_wait_until_no_pending_writes()
It turns out that we had some unnecessary waits for no outstanding
write requests to exist. They were basically working around a
bug that was fixed in MDEV-25953.

On write completion callback, blocks will be marked clean.
So, it is sufficient to consult buf_pool.flush_list to determine
which writes have not been completed yet.

On FLUSH TABLES...FOR EXPORT we must still wait for all pending
asynchronous writes to complete, because buf_flush_file_space()
would merely guarantee that writes will have been initiated.
2021-06-23 19:06:49 +03:00
Marko Mäkelä
4f4a4cf9eb MDEV-23399 fixup: Use plain pthread_cond
The condition variables that were introduced in
commit 7cffb5f6e8 (MDEV-23399)
are never instrumented with PERFORMANCE_SCHEMA.
Let us avoid the storage overhead and dead code.
2021-02-07 12:19:24 +02:00
Marko Mäkelä
8677c14e65 MDEV-24391 heap-use-after-free in fil_space_t::flush_low()
We observed a race condition that involved two threads
executing fil_flush_file_spaces() and one thread
executing fil_delete_tablespace(). After one of the
fil_flush_file_spaces() observed that
space.needs_flush_not_stopping() is set and was
releasing the fil_system.mutex, the other fil_flush_file_spaces()
would complete the execution of fil_space_t::flush_low() on
the same tablespace. Then, fil_delete_tablespace() would
destroy the object, because the value of fil_space_t::n_pending
did not prevent that. Finally, the fil_flush_file_spaces() would
resume execution and invoke fil_space_t::flush_low() on the freed
object.

This race condition was introduced in
commit 118e258aaa of MDEV-23855.

fil_space_t::flush(): Add a template parameter that indicates
whether the caller is holding a reference to prevent the
tablespace from being freed.

buf_dblwr_t::flush_buffered_writes_completed(),
row_quiesce_table_start(): Acquire a reference for the duration
of the fil_space_t::flush_low() operation. It should be impossible
for the object to be freed in these code paths, but we want to
satisfy the debug assertions.

fil_space_t::flush_low(): Do not increment or decrement the
reference count, but instead assert that the caller is holding
a reference.

fil_space_extend_must_retry(), fil_flush_file_spaces():
Acquire a reference before releasing fil_system.mutex.
This is what will fix the race condition.
2020-12-11 09:05:26 +02:00
Marko Mäkelä
83591a23d6 MDEV-24350 buf_dblwr unnecessarily uses memory-intensive srv_stats counters
The counters in srv_stats use std::atomic and multiple cache lines per
counter. This is an overkill in a case where a critical section already
exists in the code. A regular variable will work just fine, with much
smaller memory bus impact.
2020-12-04 17:52:23 +02:00
Marko Mäkelä
6d3356c12e MDEV-24053 MSAN use-of-uninitialized-value in tpool::simulated_aio::simulated_aio_callback()
Starting with commit ef3f71fa74
MemorySanitizer would complain that we are writing uninitialized
data via the doublewrite buffer.

buf_dblwr_t::add_to_batch(): Zero out any unused part of the
doublewrite buffer, for PAGE_COMPRESSED and ROW_FORMAT=COMPRESSED
tables.

Reviewed by: Eugene Kosov
2020-10-29 15:55:07 +02:00
Marko Mäkelä
a5a2ef079c MDEV-23855: Implement asynchronous doublewrite
Synchronous writes and calls to fdatasync(), fsync() or
FlushFileBuffers() would ruin performance. So, let us
submit asynchronous writes for the doublewrite buffer.
We submit a single request for the likely case that the
two doublewrite buffers are contiquous in the system tablespace.

buf_dblwr_t::flush_buffered_writes_completed(): The completion callback
of buf_dblwr_t::flush_buffered_writes().

os_aio_wait_until_no_pending_writes(): Also wait for doublewrite batches.

buf_dblwr_t::element::space: Remove. We can simply use
element::request.node->space instead.

Reviewed by: Vladislav Vaintroub
2020-10-26 17:53:55 +02:00
Marko Mäkelä
ef3f71fa74 MDEV-23399 fixup: Interleaved doublewrite batches
Author: Vladislav Vaintroub
2020-10-26 17:53:54 +02:00
Marko Mäkelä
118e258aaa MDEV-23855: Shrink fil_space_t
Merge n_pending_ios, n_pending_ops to std::atomic<uint32_t> n_pending.
Change some more fil_space_t members to uint32_t to reduce
the memory footprint.

fil_space_t::add(), fil_ibd_create(): Attach the already opened
handle to the tablespace, and enforce the fil_system.n_open limit.

dict_boot(): Initialize fil_system.max_assigned_id.

srv_boot(): Call srv_thread_pool_init() before anything else,
so that files should be opened in the correct mode on Windows.

fil_ibd_create(): Create the file in OS_FILE_AIO mode, just like
fil_node_open_file_low() does it.

dict_table_t::is_accessible(): Replaces fil_table_accessible().

Reviewed by: Vladislav Vaintroub
2020-10-26 17:53:54 +02:00
Marko Mäkelä
45ed9dd957 MDEV-23855: Remove fil_system.LRU and reduce fil_system.mutex contention
Also fixes MDEV-23929: innodb_flush_neighbors is not being ignored
for system tablespace on SSD

When the maximum configured number of file is exceeded, InnoDB will
close data files. We used to maintain a fil_system.LRU list and
a counter fil_node_t::n_pending to achieve this, at the huge cost
of multiple fil_system.mutex operations per I/O operation.

fil_node_open_file_low(): Implement a FIFO replacement policy:
The last opened file will be moved to the end of fil_system.space_list,
and files will be closed from the start of the list. However, we will
not move tablespaces in fil_system.space_list while
i_s_tablespaces_encryption_fill_table() is executing
(producing output for INFORMATION_SCHEMA.INNODB_TABLESPACES_ENCRYPTION)
because it may cause information of some tablespaces to go missing.
We also avoid this in mariabackup --backup because datafiles_iter_next()
assumes that the ordering is not changed.

IORequest: Fold more parameters to IORequest::type.

fil_space_t::io(): Replaces fil_io().

fil_space_t::flush(): Replaces fil_flush().

OS_AIO_IBUF: Remove. We will always issue synchronous reads of the
change buffer pages in buf_read_page_low().

We will always ignore some errors for background reads.

This should reduce fil_system.mutex contention a little.

fil_node_t::complete_write(): Replaces fil_node_t::complete_io().
On both read and write completion, fil_space_t::release_for_io()
will have to be called.

fil_space_t::io(): Do not acquire fil_system.mutex in the normal
code path.

xb_delta_open_matching_space(): Do not try to open the system tablespace
which was already opened. This fixes a file sharing violation in
mariabackup --prepare --incremental.

Reviewed by: Vladislav Vaintroub
2020-10-26 17:09:01 +02:00
Marko Mäkelä
9028cc6b86 Cleanup: Make InnoDB page numbers uint32_t
InnoDB stores a 32-bit page number in page headers and in some
data structures, such as FIL_ADDR (consisting of a 32-bit page number
and a 16-bit byte offset within a page). For better compile-time
error detection and to reduce the memory footprint in some data
structures, let us use a uint32_t for the page number, instead
of ulint (size_t) which can be 64 bits.
2020-10-15 17:06:17 +03:00
Marko Mäkelä
7cffb5f6e8 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 d1ab89037a 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 17:04:56 +03:00
Marko Mäkelä
c7399db645 MDEV-21174 fixup: Remove buf_dblwr_being_created 2020-09-30 13:50:00 +03:00
Marko Mäkelä
3421223363 Merge 10.4 into 10.5 2020-09-09 16:57:30 +03:00
Marko Mäkelä
66ae50a564 Merge 10.3 into 10.4 2020-09-09 15:00:21 +03:00
Marko Mäkelä
7e07e38cf6 Merge 10.2 into 10.3 2020-09-09 13:06:46 +03:00
Thirunarayanan Balathandayuthapani
b1009ae5c1 MDEV-23456 fil_space_crypt_t::write_page0() is accessing an uninitialized page
buf_page_create() is invoked when page is initialized. So that
previous contents of the page ignored. In few cases, it calls
buf_page_get_gen() is called to fetch the page from buffer pool.
It should take x-latch on the page. If other thread uses the block
or block io state is different from BUF_IO_NONE then release the
mutex and check the state and buffer fix count again. For compressed
page, use the existing free block from LRU list to create new page.
Retry to fetch the compressed page if it is in flush list

fseg_create(), fseg_create_general(): Introduce block as a parameter
where segment header is placed. It is used to avoid repetitive
x-latch on the same page

Change the assert to check whether the page has SX latch and
X latch in all callee function of buf_page_create()

mtr_t::get_fix_count(): Get the buffer fix count of the given
block added by the mtr

FindBlock is added to find the buffer fix count of the given
block acquired by the mini-transaction
2020-09-09 11:58:15 +05:30
Marko Mäkelä
52edc374ac Merge 10.4 into 10.5 2020-08-01 14:43:37 +03:00
Marko Mäkelä
70d4500c00 Merge 10.3 into 10.4 2020-07-31 18:09:32 +03:00
Marko Mäkelä
8bb2170d74 Merge 10.2 into 10.3 2020-07-31 14:13:34 +03:00
Marko Mäkelä
879ba1979b MDEV-11799 Doublewrite recovery can corrupt data pages
The purpose of the InnoDB doublewrite buffer is to make InnoDB
tolerant against cases where the server was killed in the middle
of a page write. (In Linux, killing a process may interrupt a
write system call, typically on a 4096-byte boundary.)

There may exist multiple copies of a page number in the doublewrite
buffer. Recovery should choose the latest valid copy of the page.
By design, the FIL_PAGE_LSN must not precede the latest checkpoint LSN
nor be later than the end of the recovered log.

For page_compressed and encrypted pages, we were missing proper
consistency checks. In the 10.4 data set generated for in MDEV-23231,
the data file contained a valid page_compressed page, and an
identical copy of that page was also present in the doublewrite
buffer. But, recovery would incorrectly consider the page invalid
and restore an uncompressed copy of the same page that had been
written before the log checkpoint. (In fact, no redo log was to
be applied to that page.)

buf_dblwr_process(): Validate the FIL_PAGE_LSN in the doublewrite
buffer pages, and always skip page 0, because those pages should
have been recovered by Datafile::restore_from_doublewrite() if
necessary.

Datafile::restore_from_doublewrite(): Choose the latest applicable
page from the doublewrite buffer.

recv_dblwr_t::find_page(): Also validate encrypted or
page_compressed pages.

recv_dblwr_t::validate_page(): New function to validate a page,
either a copy in a data file or in the doublewrite buffer.
Also validate encrypted or page_compressed pages.

This is joint work with Thirunarayanan Balathandayuthapani.
2020-07-31 11:54:35 +03:00
Marko Mäkelä
4d4865de6f Merge 10.4 into 10.5 2020-07-20 15:55:59 +03:00
Marko Mäkelä
4b959bd8df Merge 10.3 into 10.4 2020-07-20 15:34:59 +03:00
Marko Mäkelä
acc58fd835 Merge 10.2 into 10.3 2020-07-20 15:11:59 +03:00
Marko Mäkelä
ca9276e37e Merge 10.1 into 10.2 2020-07-20 14:53:24 +03:00
Marko Mäkelä
98e2c17e9e Cleanup: Remove fil_check_adress_in_tablespace() 2020-07-20 14:48:56 +03:00
Marko Mäkelä
054f10365c Merge 10.4 into 10.5 2020-07-16 07:15:06 +03:00
Marko Mäkelä
3280edda89 Merge 10.3 into 10.4 2020-07-16 06:57:50 +03:00
Marko Mäkelä
73aa31fbfd Merge 10.2 into 10.3 2020-07-16 06:55:23 +03:00
Marko Mäkelä
147d4b1ec0 MDEV-21347 innodb_log_optimize_ddl=OFF is not crash safe
In commit 0f90728bc0 (MDEV-16809)
we introduced the configuration option innodb_log_optimize_ddl
for controlling whether native index creation or table-rebuild
in InnoDB should avoid writing full redo log.

Fungo Wang reported that this option is causing occasional failures.
The reason is that pages may be written to data files in an
inconsistent state. Applying log records to such inconsistent pages
may fail.

The solution is to always invoke PageBulk::finish() before page latches
may be released, to ensure that the page contents is in a consistent
state.

Something similar was implemented in MySQL 8.0.13:
mysql/mysql-server@d1254b9473

buf_block_t::skip_flush_check: Remove. Suppressing consistency checks
is a bad idea.

PageBulk::needs_finish(): New predicate: Determine whether
PageBulk::finish() must fix up the page.

PageBulk::init(): Clear PAGE_DIRECTION to ensure that needs_finish()
will hold. We change the field from PAGE_NO_DIRECTION to 0
and back without writing redo log. This trick avoids the need
to introduce any new data member to PageBulk.

PageBulk::insert(): Replace some high-level accessors to bypass
debug assertions related to PAGE_HEAP_TOP that we will be violating
until finish() has been executed.

PageBulk::finish(): Tolerate m_rec_no==0. We must invoke this also
on an empty page, to ensure that PAGE_HEAP_TOP is initialized.

PageBulk::commit(): Always invoke finish().

PageBulk::release(), BtrBulk::pageSplit(), BtrBulk::storeExt(),
BtrBulk::finish(): Invoke PageBulk::finish().
2020-07-16 06:35:15 +03:00
Marko Mäkelä
f347b3e0e6 Merge 10.3 into 10.4 2020-07-02 07:39:33 +03:00
Marko Mäkelä
1df1a63924 Merge 10.2 into 10.3 2020-07-02 06:17:51 +03:00
Marko Mäkelä
c36834c832 MDEV-20377: Make WITH_MSAN more usable
MemorySanitizer (clang -fsanitize=memory) requires that all code
be compiled with instrumentation enabled. The only exception is the
C runtime library. Failure to use instrumented libraries will cause
bogus messages about memory being uninitialized.

In WITH_MSAN builds, we must avoid calling getservbyname(),
because even though it is a standard library function, it is
not instrumented, not even in clang 10.

Note: Before MariaDB Server 10.5, ./mtr will typically fail
due to the old PCRE library, which was updated in MDEV-14024.

The following cmake options were tested on 10.5
in commit 94d0bb4dbe:

cmake \
-DCMAKE_C_FLAGS='-march=native -O2' \
-DCMAKE_CXX_FLAGS='-stdlib=libc++ -march=native -O2' \
-DWITH_EMBEDDED_SERVER=OFF -DWITH_UNIT_TESTS=OFF -DCMAKE_BUILD_TYPE=Debug \
-DWITH_INNODB_{BZIP2,LZ4,LZMA,LZO,SNAPPY}=OFF \
-DPLUGIN_{ARCHIVE,TOKUDB,MROONGA,OQGRAPH,ROCKSDB,CONNECT,SPIDER}=NO \
-DWITH_SAFEMALLOC=OFF \
-DWITH_{ZLIB,SSL,PCRE}=bundled \
-DHAVE_LIBAIO_H=0 \
-DWITH_MSAN=ON

MEM_MAKE_DEFINED(): An alias for VALGRIND_MAKE_MEM_DEFINED()
and __msan_unpoison().

MEM_GET_VBITS(), MEM_SET_VBITS(): Aliases for
VALGRIND_GET_VBITS(), VALGRIND_SET_VBITS(), __msan_copy_shadow().

InnoDB: Replace the UNIV_MEM_ macros with corresponding MEM_ macros.

ut_crc32_8_hw(), ut_crc32_64_low_hw(): Use the compiler built-in
functions instead of inline assembler when building WITH_MSAN.
This will require at least -msse4.2 when building for IA-32 or AMD64.
The inline assembler would not be instrumented, and would thus cause
bogus failures.
2020-07-01 17:23:00 +03:00
Marko Mäkelä
6877ef9a7c Merge 10.4 into 10.5 2020-06-05 20:36:43 +03:00
Marko Mäkelä
68d9d512e9 Merge 10.3 into 10.4 2020-06-05 18:05:22 +03:00
Marko Mäkelä
680463a8d9 Merge 10.2 into 10.3 2020-06-05 16:51:26 +03:00
Marko Mäkelä
b1ab211dee 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 4ed7082eef
which introduced buf_block_t::mutex, which we are now removing.

Later, multiple instances of buf_pool_t were introduced
in commit c18084f71b
and recently removed by us in
commit 1a6f708ec5 (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 c5883debd6 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
Marko Mäkelä
eba2d10ac5 MDEV-22721 Remove bloat caused by InnoDB logger class
Introduce a new ATTRIBUTE_NOINLINE to
ib::logger member functions, and add UNIV_UNLIKELY hints to callers.

Also, remove some crash reporting output. If needed, the
information will be available using debugging tools.

Furthermore, remove some fts_enable_diag_print output that included
indexed words in raw form. The code seemed to assume that words are
NUL-terminated byte strings. It is not clear whether a NUL terminator
is always guaranteed to be present. Also, UCS2 or UTF-16 strings would
typically contain many NUL bytes.
2020-06-04 10:24:10 +03:00
Marko Mäkelä
64488a6f2d Cleanup: Remove global functions or redundant parameters
fil_flush_file_spaces(): Remove the constant parameter.

buf_flush_start(), buf_flush_end(): Use static linkage.

fil_page_get_prev(), fil_page_get_next(), fil_addr_is_null(): Remove.
2020-05-05 07:06:37 +03:00
Marko Mäkelä
37c14690fc Merge 10.4 into 10.5 2020-03-30 19:07:25 +03:00
Marko Mäkelä
e2f1f88fa6 Merge 10.3 into 10.4 2020-03-30 14:50:23 +03:00
Marko Mäkelä
1a9b6c4c7f Merge 10.2 into 10.3 2020-03-30 11:12:56 +03:00
Eugene Kosov
884d22f288 remove fishy reinterpret_cast from buf_page_is_zeroes()
In my micro-benchmarks memcmp(4196) 3 times faster than old
implementation. Also, it's generally better to use as less
reinterpret_casts<> as possible.

buf_is_zeroes(): renamed from buf_page_is_zeroes() and
argument changed to span<> for convenience.

st_::span<T>::const_iterator: fixed

page_zip-verify_checksum(): make argument byte* instead of void*
2020-03-20 21:35:42 +03:00
Marko Mäkelä
f8a9f90667 MDEV-12353: Remove support for crash-upgrade
We tighten some assertions regarding dict_index_t::is_dummy
and crash recovery, now that redo log processing will
no longer create dummy objects.
2020-02-13 19:13:45 +02:00
Marko Mäkelä
ded128aa9b Merge 10.4 into 10.5 2020-01-20 16:48:56 +02:00
Marko Mäkelä
87a61355e8 Merge 10.3 into 10.4
The MDEV-17062 fix in commit c4195305b2
was omitted.
2020-01-20 15:49:48 +02:00
Sergei Petrunia
e709eb9bf7 Merge branch '10.2' into 10.3
# Conflicts:
#	mysql-test/suite/galera/r/MW-388.result
#	mysql-test/suite/galera/t/MW-388.test
#	mysql-test/suite/innodb/r/truncate_inject.result
#	mysql-test/suite/innodb/t/truncate_inject.test
#	mysql-test/suite/rpl/r/rpl_stop_slave.result
#	mysql-test/suite/rpl/t/rpl_stop_slave.test
#	sql/sp_head.cc
#	sql/sp_head.h
#	sql/sql_lex.cc
#	sql/sql_yacc.yy
#	storage/xtradb/buf/buf0dblwr.cc
2020-01-17 00:46:40 +03:00
Sergei Petrunia
b04429434a Merge branch '10.1' into 10.2
# Conflicts:
#	sql/sp_head.cc
#	sql/sql_select.cc
#	sql/sql_trigger.cc
2020-01-17 00:24:17 +03:00