LATCH_ID_OS_AIO_READ_MUTEX,
LATCH_ID_OS_AIO_WRITE_MUTEX,
LATCH_ID_OS_AIO_LOG_MUTEX,
LATCH_ID_OS_AIO_IBUF_MUTEX,
LATCH_ID_OS_AIO_SYNC_MUTEX: Remove. The tpool is not instrumented.
lock_set_timeout_event(): Remove.
srv_sys_mutex_key, srv_sys_t::mutex, SYNC_THREADS: Remove.
srv_slot_t::suspended: Remove. We only ever assigned this data member
true, so it is redundant.
ib_wqueue_wait(), ib_wqueue_timedwait(): Remove.
os_thread_join(): Remove.
os_thread_create(), os_thread_exit(): Remove redundant parameters.
These were missed in commit 5e62b6a5e0.
The parameters innodb_thread_concurrency and innodb_commit_concurrency
were useful years ago when both computing resources and the implementation
of some shared data structures were limited. MySQL 5.0 or 5.1 had trouble
scaling beyond 8 concurrent connections. Most of the scalability bottlenecks
have been removed since then, and the transactions per second delivered
by MariaDB Server 10.5 should not dramatically drop upon exceeding the
'optimal' number of connections.
Hence, enabling any concurrency throttling for InnoDB actually makes
things worse. We have seen many customers mistakenly setting this to a
small value like 16 or 64 and then complaining the server was slow.
Ignoring the parameters allows us to remove some normally unused code
and data structures, which could slightly improve performance.
innodb_thread_concurrency, innodb_commit_concurrency,
innodb_replication_delay, innodb_concurrency_tickets,
innodb_thread_sleep_delay, innodb_adaptive_max_sleep_delay:
Deprecate and ignore; hard-wire to 0.
The column INFORMATION_SCHEMA.INNODB_TRX.trx_concurrency_tickets
will always report 0.
The function wsrep_on() was being called rather frequently
in InnoDB and XtraDB. Let us cache it in trx_t and invoke
trx_t::is_wsrep() instead.
innobase_trx_init(): Cache trx->wsrep = wsrep_on(thd).
ha_innobase::write_row(): Replace many repeated calls to current_thd,
and test the cheapest condition first.
Almost all threads have gone
- the "ticking" threads, that sleep a while then do some work)
(srv_monitor_thread, srv_error_monitor_thread, srv_master_thread)
were replaced with timers. Some timers are periodic,
e.g the "master" timer.
- The btr_defragment_thread is also replaced by a timer , which
reschedules it self when current defragment "item" needs throttling
- the buf_resize_thread and buf_dump_threads are substitutes with tasks
Ditto with page cleaner workers.
- purge workers threads are not tasks as well, and purge cleaner
coordinator is a combination of a task and timer.
- All AIO is outsourced to tpool, Innodb just calls thread_pool::submit_io()
and provides the callback.
- The srv_slot_t was removed, and innodb_debug_sync used in purge
is currently not working, and needs reimplementation.
lock_t::requested_time: Document what the field is used for.
lock_t::wait_time: Document that the field is only used for
diagnostics and may be garbage if the system time is being adjusted.
srv_slot_t::suspend_time: Document that this is duplicating
trx_lock_t::wait_started.
lock_table_print(), lock_rec_print(): Declare in static scope.
Add a parameter for the current time.
lock_deadlock_check_and_resolve(), lock_deadlock_lock_print(),
lock_deadlock_joining_trx_print():
Add a parameter for the current time.
Replace ut_usectime() with my_interval_timer(),
which is equivalent, but monotonically counting nanoseconds
instead of counting the microseconds of real time.
os_event_wait_time_low(): Use my_hrtime() instead of ut_usectime().
FIXME: Set a clock attribute on the condition variable that allows
a monotonic clock to be chosen as the time base, so that the wait
is immune to adjustments of the system clock.
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().
Revert the dead code for MySQL 5.7 multi-master replication (GCS),
also known as
WL#6835: InnoDB: GCS Replication: Deterministic Deadlock Handling
(High Prio Transactions in InnoDB).
Also, make innodb_lock_schedule_algorithm=vats skip SPATIAL INDEX,
because the code does not seem to be compatible with them.
Add FIXME comments to some SPATIAL INDEX locking code. It looks
like Galera write-set replication might not work with SPATIAL INDEX.
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.
Let lock_print_info_all_transactions() iterate rw_trx_hash instead of
rw_trx_list.
When printing info of locks for transactions, InnoDB monitor doesn't
attempt to read relevant page from disk anymore. The code was prone
to race conditions.
Note that TrxListIterator didn't work as advertised: it iterated
rw_trx_list only.
This is 10.1 version where no merge error exists.
wsrep_on_check
New check function. Galera can't be enabled
if innodb-lock-schedule-algorithm=VATS.
innobase_kill_query
In Galera async kill we could own lock mutex.
innobase_init
If Variance-Aware-Transaction-Sheduling Algorithm (VATS) is
used on Galera we refuse to start InnoDB.
Changed innodb-lock-schedule-algorithm as read-only parameter
as it was designed to be.
lock_rec_other_has_expl_req,
lock_rec_other_has_conflicting,
lock_rec_lock_slow
lock_table_other_has_incompatible
lock_rec_insert_check_and_lock
Change pointer to conflicting lock to normal pointer as this
pointer contents could be changed later.
Problem was a merge error from MySQL wsrep i.e. Galera.
wsrep_on_check
New check function. Galera can't be enabled
if innodb-lock-schedule-algorithm=VATS.
innobase_kill_query
In Galera async kill we could own lock mutex.
innobase_init
If Variance-Aware-Transaction-Sheduling Algorithm (VATS) is
used on Galera we fall back to First-Come-First-Served (FCFS)
with notice to user.
Changed innodb-lock-schedule-algorithm as read-only parameter
as it was designed to be.
lock_reset_lock_and_trx_wait
Use ib::hex() to print out transaction ID.
lock_rec_other_has_expl_req,
lock_rec_other_has_conflicting,
RecLock::add_to_waitq
lock_rec_lock_slow
lock_table_other_has_incompatible
lock_rec_insert_check_and_lock
lock_prdt_other_has_conflicting
Change pointer to conflicting lock to normal pointer as this
pointer contents could be changed later.
RecLock::create
Conclicting lock pointer is moved to last parameter with
default value NULL. This conflicting transaction could
be selected as victim in Galera if requesting transaction
is BF (brute force) transaction. In this case contents
of conflicting lock pointer will be changed. Use ib::hex() to print
transaction ids.
Problem:- This crash happens because of thd = NULL , and while checking
for wsrep_on , we no longer check for thd != NULL (MDEV-7955). So this
problem is regression of MDEV-7955. However this patch not only solves
this regression , It solves all regression caused by MDEV-7955 patch.
To get all possible cases when thd can be null , assert(thd)/
assert(trx->mysql_thd) is place just before all wsrep_on and innodb test
suite is run. And the assert which caused failure are removed with a physical
check for thd != NULL. Rest assert are removed. Hopefully this method will
remove all current/potential regression of MDEV-7955.
There is a race condition related to the variable
srv_stats.n_lock_wait_current_count, which is only
incremented and decremented by the function lock_wait_suspend_thread(),
The incrementing is protected by lock_sys->wait_mutex, but the
decrementing does not appear to be protected by anything.
This mismatch could allow the counter to be corrupted when a
transactional InnoDB table or record lock wait is terminating
roughly at the same time with the start of a wait on a
(possibly different) lock.
ib_counter_t: Remove some unused methods. Prevent instantiation for N=1.
Add an inc() method that takes a slot index as a parameter.
single_indexer_t: Remove.
simple_counter<typename Type, bool atomic=false>: A new counter wrapper.
Optionally use atomic memory operations for modifying the counter.
Aligned to the cache line size.
lsn_ctr_1_t, ulint_ctr_1_t, int64_ctr_1_t: Define as simple_counter<Type>.
These counters are either only incremented (and we do not care about
losing some increment operations), or the increment/decrement operations
are protected by some mutex.
srv_stats_t::os_log_pending_writes: Document that the number is protected
by log_sys->mutex.
srv_stats_t::n_lock_wait_current_count: Use simple_counter<ulint, true>,
that is, atomic inc() and dec() operations.
lock_wait_suspend_thread(): Release the mutexes before incrementing
the counters. Avoid acquiring the lock mutex if the lock wait has
already been resolved. Atomically increment and decrement
srv_stats.n_lock_wait_current_count.
row_insert_for_mysql(), row_update_for_mysql(),
row_update_cascade_for_mysql(): Use the inc() method with the trx->id
as the slot index. This is a non-functional change, just using
inc() instead of add(1).
buf_LRU_get_free_block(): Replace the method add(index, n) with inc().
There is no slot index in the simple_counter.
InnoDB shutdown failed to properly take fil_crypt_thread() into account.
The encryption threads were signalled to shut down together with other
non-critical tasks. This could be much too early in case of slow shutdown,
which could need minutes to complete the purge. Furthermore, InnoDB
failed to wait for the fil_crypt_thread() to actually exit before
proceeding to the final steps of shutdown, causing the race conditions.
Furthermore, the log_scrub_thread() was shut down way too early.
Also it should remain until the SRV_SHUTDOWN_FLUSH_PHASE.
fil_crypt_threads_end(): Remove. This would cause the threads to
be terminated way too early.
srv_buf_dump_thread_active, srv_dict_stats_thread_active,
lock_sys->timeout_thread_active, log_scrub_thread_active,
srv_monitor_active, srv_error_monitor_active: Remove a race condition
between startup and shutdown, by setting these in the startup thread
that creates threads, not in each created thread. In this way, once the
flag is cleared, it will remain cleared during shutdown.
srv_n_fil_crypt_threads_started, fil_crypt_threads_event: Declare in
global rather than static scope.
log_scrub_event, srv_log_scrub_thread_active, log_scrub_thread():
Declare in static rather than global scope. Let these be created by
log_init() and freed by log_shutdown().
rotate_thread_t::should_shutdown(): Do not shut down before the
SRV_SHUTDOWN_FLUSH_PHASE.
srv_any_background_threads_are_active(): Remove. These checks now
exist in logs_empty_and_mark_files_at_shutdown().
logs_empty_and_mark_files_at_shutdown(): Shut down the threads in
the proper order. Keep fil_crypt_thread() and log_scrub_thread() alive
until SRV_SHUTDOWN_FLUSH_PHASE, and check that they actually terminate.
* remove new InnoDB-specific ER_ and HA_ERR_ codes
* renamed few old ER_ and HA_ERR_ error messages to be less MyISAM-specific
* remove duplicate enum definitions (durability_properties, icp_result)
* move new mysql-test include files to their owner suite
* rename xtradb.rdiff files to *-disabled
* remove mistakenly committed helper perl module
* remove long obsolete handler::ha_statistic_increment() method
* restore the standard C xid_t structure to not have setters and getters
* remove xid_t::reset that was cleaning too much
* move MySQL-5.7 ER_ codes where they belong
* fir innodb to include service_wsrep.h not internal wsrep headers
* update tests and results
Contains also:
MDEV-10549 mysqld: sql/handler.cc:2692: int handler::ha_index_first(uchar*): Assertion `table_share->tmp_table != NO_TMP_TABLE || m_lock_type != 2' failed. (branch bb-10.2-jan)
Unlike MySQL, InnoDB still uses THR_LOCK in MariaDB
MDEV-10548 Some of the debug sync waits do not work with InnoDB 5.7 (branch bb-10.2-jan)
enable tests that were fixed in MDEV-10549
MDEV-10548 Some of the debug sync waits do not work with InnoDB 5.7 (branch bb-10.2-jan)
fix main.innodb_mysql_sync - re-enable online alter for partitioned innodb tables
Contains also
MDEV-10547: Test multi_update_innodb fails with InnoDB 5.7
The failure happened because 5.7 has changed the signature of
the bool handler::primary_key_is_clustered() const
virtual function ("const" was added). InnoDB was using the old
signature which caused the function not to be used.
MDEV-10550: Parallel replication lock waits/deadlock handling does not work with InnoDB 5.7
Fixed mutexing problem on lock_trx_handle_wait. Note that
rpl_parallel and rpl_optimistic_parallel tests still
fail.
MDEV-10156 : Group commit tests fail on 10.2 InnoDB (branch bb-10.2-jan)
Reason: incorrect merge
MDEV-10550: Parallel replication can't sync with master in InnoDB 5.7 (branch bb-10.2-jan)
Reason: incorrect merge
Fixes a deadlock between applier and its victim transaction.
The deadlock would manifest when a BF victim was waiting for some lock
and was signaled to rollback, and the same time its wait
timeout expired. In such cases the victim would return from
lock_wait_suspend_thread() with error DB_LOCK_WAIT_TIMEOUT, as opposed to
DB_DEADLOCK. As a result only the last statement of the victim would rollback,
and eventually it would deadlock with the applier.