Commit graph

44 commits

Author SHA1 Message Date
Vladislav Vaintroub
a2f510fccf MDEV-33978 P_S.THREADS is not showing all server threads
This patch only makes sure Linux getevents thread is shown in PS
2025-01-07 15:49:20 +01:00
Vladislav Vaintroub
d64034770e MDEV-35273 tpool::worker_data - replace MY_ALIGNED with pad member 2024-10-28 16:24:53 +01:00
Dave Gosselin
db0c28eff8 MDEV-33746 Supply missing override markings
Find and fix missing virtual override markings.  Updates cmake
maintainer flags to include -Wsuggest-override and
-Winconsistent-missing-override.
2024-06-20 11:32:13 -04:00
Vladislav Vaintroub
f6e9600f42 MDEV-33840 tpool- switch to longer maintainence timer interval, if pool is idle
Previous solution, that would entirely switch timer off, turned out
to be deadlock prone.

This patch fixed previous attempt to switch between long/short interval
periods in MDEV-24295. Now, initial state of the timer is fixed (it is ON).
Also, avoid switching timer to longer periods if there is any activity in
the pool.
2024-04-17 10:49:59 +02:00
Vladislav Vaintroub
2ba79aba2b Revert "MDEV-33840 tpool : switch off maintenance timer when not needed."
This reverts commit 09bae92c16.
2024-04-17 09:58:34 +02:00
Vladislav Vaintroub
09bae92c16 MDEV-33840 tpool : switch off maintenance timer when not needed.
Before patch, maintenance timer will tick every 0.4 seconds.
After this patch, timer will tick every 0.4 seconds when necessary(
there are delayed thread creation), switching off completely after 20
seconds of being idle.
2024-04-09 08:31:32 +02:00
Marko Mäkelä
78c9a12c8f MDEV-32861 InnoDB hangs when running out of I/O slots
When the constant OS_AIO_N_PENDING_IOS_PER_THREAD is changed from 256 to 1
and the server is run with the minimum parameters
innodb_read_io_threads=1 and innodb_write_io_threads=2, two hangs
were observed.

tpool::cache<T>::put(T*): Ensure that get() in io_slots::acquire()
will be woken up when the cache previously was empty.

buf_pool_t::io_buf_t::reserve(): Schedule a possibly partial doublewrite
batch so that os_aio_wait_until_no_pending_writes() has a chance of
returning. Add a Boolean parameter and pass wait_for_reads=false inside
buf_page_decrypt_after_read(), because those calls will be executed
inside a read completion callback, and therefore
os_aio_wait_until_no_pending_reads() would block indefinitely.
2023-11-22 16:54:41 +02:00
Khem Raj
75bbf645a6 Add missing include <cstdio>
This is needed with GCC 13 and newer [1]

[1] https://www.gnu.org/software/gcc/gcc-13/porting_to.html

Signed-off-by: Khem Raj <raj.khem@gmail.com>
2023-01-27 12:43:38 +11:00
Vladislav Vaintroub
a3fd9e6b06 MDEV-29367 Refactor tpool::cache
Removed use std::vector's ba push_back(), pop_back() to  make it more
obvious that memory in the vectors won't be reallocated.

Also, "borrowed" elements can be debugged a little better now,
they are put into the start of the m_cache vector.
2022-08-24 13:36:49 +02:00
Vladislav Vaintroub
35f2cdcb99 MDEV-28920 Rescheduling of innodb_stats_func() missing
Fixed tpool timer implementation on POSIX.
Prior to this patch, under some specific rare circumstances (concurrency
related), timer callback execution might be skipped.
2022-06-23 05:53:55 +02:00
Daniel Black
d78173828e MDEV-27900: aio handle partial reads/writes
As btrfs showed, a partial read of data in AIO /O_DIRECT circumstances can
really confuse MariaDB.

Filipe Manana (SuSE)[1] showed how database programmers can assume
O_DIRECT is all or nothing.

While a fix was done in the kernel side, we can do better in our code by
requesting that the rest of the block be read/written synchronously if
we do only get a partial read/write.

Per the APIs, a partial read/write can occur before an error, so
reattempting the request will leave the caller with a concrete error to
handle.

[1] https://lore.kernel.org/linux-btrfs/CABVffENfbsC6HjGbskRZGR2NvxbnQi17gAuW65eOM+QRzsr8Bg@mail.gmail.com/T/#mb2738e675e48e0e0778a2e8d1537dec5ec0d3d3a

Also spell synchronously correctly in other files.
2022-03-12 09:47:53 +11:00
Vladislav Vaintroub
012e724deb MDEV-27796 Windows - starting server with huge innodb-log-buffer-size may fail
Fixed tpool::pread() and tpool::pwrite() to return SSIZE_T on Windows,
so that huge numbers are not converted to negatives.

Also, make sure to never attempt reading/writing more bytes than
DWORD can accomodate (4G)
2022-02-10 17:25:12 +01:00
Vladislav Vaintroub
78bd7d86a4 MDEV-25953 Tpool - prevent potential deadlock in simulated AIO
Do not execute user callback just after pwrite. Instead, submit user
function as task into thread pool. This way, the IO thread would not hog
aiocb, which is a limited (in Innodb) resource
2021-06-17 20:39:47 +02:00
Marko Mäkelä
f24b738318 MDEV-24313 (2 of 2): Silently ignored innodb_use_native_aio=1
In commit 5e62b6a5e0 (MDEV-16264)
the logic of os_aio_init() was changed so that it will never fail,
but instead automatically disable innodb_use_native_aio (which is
enabled by default) if the io_setup() system call would fail due
to resource limits being exceeded. This is questionable, especially
because falling back to simulated AIO may lead to significantly
reduced performance.

srv_n_file_io_threads, srv_n_read_io_threads, srv_n_write_io_threads:
Change the data type from ulong to uint.

os_aio_init(): Remove the parameters, and actually return an error code.

thread_pool::configure_aio(): Do not silently fall back to simulated AIO.

Reviewed by: Vladislav Vaintroub
2020-12-14 15:27:03 +02:00
Vladislav Vaintroub
3ee24b2306 Simplify clang workarounds. 2020-12-07 10:35:57 +01:00
Marko Mäkelä
f3a58ed801 MDEV-24295: Fix the non-clang build
Sorry, only tested commit 4174fc1a1b
on clang. Other compilers do not define __has_feature().
2020-12-02 22:04:57 +02:00
Marko Mäkelä
4174fc1a1b MDEV-24295: Fix the WITH_MSAN build
For some reason, commit 5bb5d4ad3a
made clang++-11 unhappy about a constexpr declaration.
2020-12-02 21:46:01 +02:00
Vladislav Vaintroub
1435f35bda Clarify some comments.
- the intention for my_getevents syscall is now better explained,
why are we using it (to be able to interrupt io_getevents syscall via
io_destroy()).

- Fix comment for MAX_EVENTS in getevent_thread_routine.
MAX_EVENTS is more of less arbitrary constant, chosen such that events array
is big enough to get multiple simultaneous io completions, but small
enough so it does not blow the thread's stack.
2020-11-30 16:46:06 +01:00
Vladislav Vaintroub
5bb5d4ad3a MDEV-24295 Reduce wakeups by tpool maintenance timer, when server is idle
If maintenance timer does not do much for prolonged time, it will
wake up less frequently, once every 4 seconds instead of once every 0.4
second.

It will wakeup more often if thread creation is throttled, to avoid stalls.
2020-11-30 16:46:05 +01:00
Monty
279b5f87de Avoid some DBUG prints from idle server in thread pool 2020-11-26 19:13:37 +02:00
Marko Mäkelä
f693b72547 MDEV-24270: Clarify some comments 2020-11-25 16:08:26 +02:00
Vladislav Vaintroub
2de95f7a1e Fix misspelling.
Kudos to Marko for finding.
2020-11-25 13:07:49 +01:00
Vladislav Vaintroub
c130c60b2b Cleanup. Provide accurate comment on my_getevents(). 2020-11-25 13:07:08 +01:00
Vladislav Vaintroub
78df9e37a6 Partially Revert "MDEV-24270: Collect multiple completed events at a time"
This partially reverts commit 6479006e14.

Remove the constant tpool::aio::N_PENDING, which has no
intrinsic meaning for the tpool.
2020-11-25 13:07:08 +01:00
Marko Mäkelä
6479006e14 MDEV-24270: Collect multiple completed events at a time
tpool::aio::N_PENDING: Replaces OS_AIO_N_PENDING_IOS_PER_THREAD.
This limits two similar things: the number of outstanding requests
that a thread may io_submit(), and the number of completed requests
collected at a time by io_getevents().
2020-11-25 09:42:38 +02:00
Marko Mäkelä
7a9405e3dc MDEV-24270 Misuse of io_getevents() causes wake-ups at least twice per second
In the asynchronous I/O interface, InnoDB is invoking io_getevents()
with a timeout value of half a second, and requesting exactly 1 event
at a time.

The reason to have such a short timeout is to facilitate shutdown.

We can do better: Use an infinite timeout, wait for a larger maximum
number of events. On shutdown, we will invoke io_destroy(), which
should lead to the io_getevents system call reporting EINVAL.

my_getevents(): Reimplement the libaio io_getevents() by only invoking
the system call. The library implementation would try to elide the
system call and return 0 immediately if aio_ring_is_empty() holds.
Here, we do want a blocking system call, not 100% CPU usage. Neither
do we want the aio_ring_is_empty() trigger SIGSEGV because it is
dereferencing some memory that was freed by io_destroy().
2020-11-25 09:40:12 +02:00
Marko Mäkelä
8cb01c51fb MDEV-16264 fixup: Clean up asynchronous I/O
os_aio_userdata_t: Remove. It was basically duplicating IORequest.

buf_page_write_complete(): Take only IORequest as a parameter.

os_aio_func(), pfs_os_aio_func(): Replaced with os_aio() that has
no redundant parameters. There is only one caller, so there is no
point to pass __FILE__, __LINE__ as a parameter.
2020-10-26 17:53:54 +02: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ä
c7920fa8ff MDEV-16264: Eliminate unsafe os_aio_userdata_t type cast 2020-03-12 19:43:45 +02:00
Vicențiu Ciorbaru
efa9079fbd Fix compilation error due to type mismatch in tpool_generic.cc
size_t compared to int
2020-02-13 13:42:01 +02:00
Marko Mäkelä
8b97eba31b MDEV-21674 purge_sys.stop() fails to wait for purge workers to complete
Since commit 5e62b6a5e0 (MDEV-16264),
purge_sys_t::stop() no longer waited for all purge activity to stop.

This caused problems on FLUSH TABLES...FOR EXPORT because of
purge running concurrently with the buffer pool flush.
The assertion at the end of buf_flush_dirty_pages() could fail.

The, implemented by Vladislav Vaintroub, aims to eliminate race
conditions when stopping or resuming purge:

waitable_task::disable(): Wait for the task to complete, then replace
the task callback function with noop.

waitable_task::enable(): Restore the original task callback function
after disable().

purge_sys_t::stop(): Invoke purge_coordinator_task.disable().

purge_sys_t::resume(): Invoke purge_coordinator_task.enable().

purge_sys_t::running(): Add const qualifier, and clarify the comment.
The purge coordinator task will remain active as long as any purge
worker task is active.

purge_worker_callback(): Assert purge_sys.running().

srv_purge_wakeup(): Merge with the only caller purge_sys_t::resume().

purge_coordinator_task: Use static linkage.
2020-02-07 08:12:58 +02:00
Vladislav Vaintroub
b19760b843 MDEV-21551 : Assertion `m_active_threads.size() >= m_long_tasks_count + m_waiting_task_count' failed"
Happened when running innodb_fts.sync_ddl

m_long_task_count could be wrongly reset to 0, if m_task_queue is
empty.
2020-01-23 15:23:46 +01:00
Vladislav Vaintroub
fde1589f9b MDEV-21551 Fix race condition in thread_pool_generic::wait_begin()
While waiting for mutex, thread_pool_generic::wait_begin(),
current task can be marked long-running. This is done by periodic
mantainence task, that runs in parallel.

Fix to recheck is_long_task() after the mutex acquisition.
2020-01-22 19:36:08 +01:00
Marko Mäkelä
588eac58fd MDEV-21551: Fix -Wsign-compare
An assertion added in commit c20bf8fd49
includes a sign mismatch. Make the affected data members unsigned.
2020-01-22 10:06:07 +02:00
Vladislav Vaintroub
c20bf8fd49 MDEV-21551 Fix calculation of current concurrency level in
maybe_wake_or_create_thread()

A task that is executed,could be counted as waiting (after wait_begin()
before wait_end()) or as long-running (callback runs for a long time).

If task is both marked waiting and long running, then calculation of
current concurrency (# of executing tasks - # of long tasks - #of waiting tasks)
is wrong, as task is counted twice.

Thus current concurrency could go negative, but with unsigned arithmetic
it will become a huge number.

As a result, maybe_wake_or_create_thread() would neither wake or create
a thread, when it should. Which may result in a deadlock.
2020-01-22 00:01:25 +01:00
Vladislav Vaintroub
508bc20a85 tpool - misc fixes 2020-01-12 21:34:59 +01:00
Vladislav Vaintroub
c27577a1ad MDEV-21326 : Address TSAN warnings in tpool.
1. Fix places where data race warnings were relevant.

tls_worker_data::m_state should be modified under mutex protection,
since both maintainence timer and current worker set this flag.

2. Suppress warnings that are legitimate, yet harmless.
Apparently, the dirty reads in waitable_task::get_ref_count() or
write_slots->pending_io_count()

Avoiding race entirely without side-effects here is tricky,
and the effects of race is harmless.

The worst thing that can happen due to race is an extra wait notification,
under rare circumstances.
2020-01-12 20:30:26 +01:00
Vladislav Vaintroub
bada05a883 tpool - implement post-task callback (for Innodb debugging) 2020-01-12 19:08:02 +01:00
Vladislav Vaintroub
66de4fef76 MDEV-16264 - some improvements
- wait notification, tpool_wait_begin/tpool_wait_end - to notify the
threadpool that current thread is going to wait

Use it to wait for IOs to complete and also when purge waits for workers.
2019-12-09 21:12:13 +01:00
Marko Mäkelä
57444a3b30 MDEV-16264: Minor cleanup
aio_linux::m_max_io_count: Unused data member; remove.

aiocb::m_ret_len: Declare as the more compatible type size_t.
Unfortunately, ssize_t is not available on Microsoft Visual Studio.
2019-12-03 11:05:18 +02:00
Vladislav Vaintroub
86407a59b3 MDEV-16264 - Fix assertion `m_queue.empty() && !m_tasks_running' in tpool::task_group destructor
This particular assertion happened when shutting down Innodb IO.IO shutdown properly waits for all IOs to finish


However there is a race condition -
right after releasing last IO slot and before decrementing task count
in group, pending_io_count will be 0, but tasks_running will be 1,
leading to assertion.

The fix is to make task_group destructor to wait for last running task
to finish.
2019-11-25 17:10:01 +01:00
Vladislav Vaintroub
c233d406cb Fix compile error on centos6. it does not like std::this_thread::sleep()
Simplify task_group destructor.
No tasks must be running or queued into task group is being destroyed.
2019-11-15 23:51:49 +01:00
Marko Mäkelä
8040998624 MDEV-16264: Fix some white space 2019-11-15 19:55:13 +02:00
Vladislav Vaintroub
00ee8d85c9 MDEV-16264: Add threadpool library
The library is capable of
- asynchronous execution of tasks (and optionally waiting for them)
- asynchronous file IO
  This is implemented using libaio on Linux and completion ports on
  Windows. Elsewhere, async io is "simulated", which means worker threads
  are performing synchronous IO.
- timers, scheduling work asynchronously in some point of the future.
  Also periodic timers are implemented.
2019-11-15 16:50:22 +01:00