1. 01 Jul, 2021 5 commits
    • Marko Mäkelä's avatar
      Cleanup: Remove pointer indirection for trx_t::xid · 0a67b15a
      Marko Mäkelä authored
      The trx_t::xid is always allocated, so we might as well allocate it
      directly in the trx_t object to improve the locality of reference.
      0a67b15a
    • Marko Mäkelä's avatar
      MDEV-24671 fixup: Fix an off-by-one error · 83234719
      Marko Mäkelä authored
      In commit e71e6133 we
      accidentally made innodb_lock_wait_timeout=100000000
      a "literal" value, not the smallest special value that
      would mean "infinite" timeout.
      83234719
    • Marko Mäkelä's avatar
      MDEV-25902 Unexpected ER_LOCK_WAIT_TIMEOUT and result · 161e4bfa
      Marko Mäkelä authored
      trans_rollback_to_savepoint(): Only release metadata locks (MDL)
      if the storage engines agree, after the changes were already rolled back.
      
      Ever since commit 3792693f
      and mysql/mysql-server@55ceedbc3feb911505dcba6cee8080d55ce86dda
      we used to cheat here and always release MDL if the binlog is disabled.
      
      MDL are supposed to prevent race conditions between DML and DDL also
      when no replication is in use. MDL are supposed to be a superset of
      InnoDB table locks: InnoDB table lock may only exist if the thread
      also holds MDL on the table name.
      
      In the included test case, ROLLBACK TO SAVEPOINT would wrongly release
      the MDL on both tables and let ALTER TABLE proceed, even though the DML
      transaction is actually holding locks on the table.
      
      Until commit 1bd681c8 (MDEV-25506)
      InnoDB worked around the locking violation in a blatantly non-ACID way:
      If locks exist on a table that is being dropped (in this case, actually
      a partition of a table that is being rebuilt by ALTER TABLE), InnoDB
      would move the table (or partition) into a queue, to be dropped after
      the locks and references had been released.
      
      The scenario of commit 3792693f
      is unaffected by this fix, because mariadb-dump (a.k.a. mysqldump)
      would use non-locking reads, and the transaction would not be holding
      any InnoDB locks during the execution of ROLLBACK TO SAVEPOINT.
      MVCC reads inside InnoDB are only covered by MDL and page latches,
      not by any table or record locks.
      
      FIXME: It would be nice if storage engines were specifically asked
      which MDL can be released, instead of only offering a choice
      between all or nothing. InnoDB should be able to release any
      locks for tables that are no longer in trx_t::mod_tables, except
      if another transaction had converted some implicit record locks
      to explicit ones, before the ROLLBACK TO SAVEPOINT had been completed.
      
      Reviewed by: Sergei Golubchik
      161e4bfa
    • Marko Mäkelä's avatar
      MDEV-26067 innodb_lock_wait_timeout values above 100,000,000 are useless · 8c5c3a45
      Marko Mäkelä authored
      The practical maximum value of the parameter innodb_lock_wait_timeout
      is 100,000,000. Any value larger than that specifies an infinite timeout.
      
      Therefore, we should make 100,000,000 the maximum value of the parameter.
      8c5c3a45
    • Marko Mäkelä's avatar
      Speed up the test innodb.lock_insert_into_empty · ce1c957a
      Marko Mäkelä authored
      Let us use innodb_lock_wait_timeout=0 for an immediate timeout.
      Also, do not override the timeout in the default connection,
      so that further tests will use the default setting.
      ce1c957a
  2. 30 Jun, 2021 9 commits
  3. 29 Jun, 2021 9 commits
    • Alexey Botchkov's avatar
      MDEV-25837 Assertion `thd->locked_tables_mode == LTM_NONE' failed in... · c2ebe814
      Alexey Botchkov authored
      MDEV-25837 Assertion `thd->locked_tables_mode == LTM_NONE' failed in Locked_tables_list::init_locked_tables.
      
      don't do prelocking for the FLUSH command.
      c2ebe814
    • Marko Mäkelä's avatar
      MDEV-26041 Recovery failure due to delete-marked SYS_FIELDS record · 0237e9bb
      Marko Mäkelä authored
      trx_t::drop_table(): Delete-mark the SYS_TABLES and SYS_INDEXES
      record before delete-marking any SYS_COLUMNS or SYS_FIELDS records.
      Otherwise, dict_load_indexes() could fail on recovery. This fixes up
      commit 1bd681c8 (MDEV-25506).
      0237e9bb
    • Marko Mäkelä's avatar
      MDEV-25496 Assertion 'trx->bulk_insert' failed on INSERT · e04bbf73
      Marko Mäkelä authored
      row_get_prebuilt_insert_row(): Remove some fallback code that had been
      added in commit 8ea923f5 (MDEV-24818).
      It seems that after all, statement boundaries are being reliably
      indicated by ha_innobase::start_stmt() or
      (for partitioned tables) ha_innobase::external_lock().
      e04bbf73
    • Marko Mäkelä's avatar
      MDEV-26029: Implement my_test_if_thinly_provisioned() for ScaleFlux · 4b0070f6
      Marko Mäkelä authored
      This is based on code that was contributed by Ning Zheng and Ray Kuan
      from ScaleFlux.
      4b0070f6
    • Marko Mäkelä's avatar
      MDEV-26029: Sparse files are inefficient on thinly provisioned storage · 30edd554
      Marko Mäkelä authored
      The MariaDB implementation of page_compressed tables for InnoDB used
      sparse files. In the worst case, in the data file, every data page
      will consist of some data followed by a hole. This may be extremely
      inefficient in some file systems.
      
      If the underlying storage device is thinly provisioned (can compress
      data on the fly), it would be good to write regular files (with sequences
      of NUL bytes at the end of each page_compressed block) and let the
      storage device take care of compressing the data.
      
      For reads, sparse file regions and regions containing NUL bytes will be
      indistinguishable.
      
      my_test_if_disable_punch_hole(): A new predicate for detecting thinly
      provisioned storage. (Not implemented yet.)
      
      innodb_atomic_writes: Correct the comment.
      
      buf_flush_page(): Support all values of fil_node_t::punch_hole.
      On a thinly provisioned storage device, we will always write
      NUL-padded innodb_page_size bytes also for page_compressed tables.
      
      buf_flush_freed_pages(): Remove a redundant condition.
      
      fil_space_t::atomic_write_supported: Remove. (This was duplicating
      fil_node_t::atomic_write.)
      
      fil_space_t::punch_hole: Remove. (Duplicated fil_node_t::punch_hole.)
      
      fil_node_t: Remove magic_n, and consolidate flags into bitfields.
      For punch_hole we introduce a third value that indicates a
      thinly provisioned storage device.
      
      fil_node_t::find_metadata(): Detect all attributes of the file.
      30edd554
    • Marko Mäkelä's avatar
      Merge 10.5 into 10.6 · b11aa0df
      Marko Mäkelä authored
      b11aa0df
    • Marko Mäkelä's avatar
      MDEV-26042 Atomic write capability is not detected correctly · 617dee34
      Marko Mäkelä authored
      my_init_atomic_write(): Detect all forms of SSD, in case multiple
      types of devices are installed in the same machine.
      This was broken in commit ed008a74
      and further in commit 70684afe.
      
      SAME_DEV(): Match block devices, ignoring partition numbers.
      
      Let us use stat() instead of lstat(), in case someone has a symbolic
      link in /dev.
      
      Instead of reporting errors with perror(), let us use fprintf(stderr)
      with the file name, the impact of the error, and the strerror(errno).
      Because this code is specific to Linux, we may depend on the
      GNU libc/uClibc/musl extension %m for strerror(errno).
      617dee34
    • Marko Mäkelä's avatar
      3d15e3c0
    • Andrei Elkin's avatar
      MDEV-26031 unnessary xid logging in one phase commit case · 39001478
      Andrei Elkin authored
      The bug was originally observed as hanging binlog background thread at
      shutdown similar to one of MDEV-21120.
      It occurred through unnessary xid logging in 1pc execution.
      
      Two parts of the issue are fixed.  Per engine loop by involved engine
      with attempt to mark a group requiring xid unlogging gets corrected in
      two ways. Do not execute it when the termination event is irrelevant
      for recovery, does not have xid in particular.  Do not break the loop
      anymore unconditionally at the end of the 1st iteration.
      39001478
  4. 28 Jun, 2021 3 commits
  5. 26 Jun, 2021 4 commits
    • Marko Mäkelä's avatar
      Merge 10.5 into 10.6 · 891a927e
      Marko Mäkelä authored
      891a927e
    • Marko Mäkelä's avatar
      MDEV-26017: Assertion stat.flush_list_bytes <= curr_pool_size · fc2ff464
      Marko Mäkelä authored
      buf_flush_relocate_on_flush_list(): If we are removing the block from
      buf_pool.flush_list, subtract its size from buf_pool.stat.flush_list_bytes.
      This fixes a regression that was introduced in
      commit 22b62eda (MDEV-25113).
      fc2ff464
    • Marko Mäkelä's avatar
      Cleanup: Remove unused mtr_block_dirtied · aa95c423
      Marko Mäkelä authored
      aa95c423
    • Marko Mäkelä's avatar
      MDEV-26010 fixup: Use acquire/release memory order · 759deaa0
      Marko Mäkelä authored
      In commit 5f22511e we depend on
      Total Store Ordering. For correct operation on ISAs that implement
      weaker memory ordering, we must explicitly use release/acquire stores
      and loads on buf_page_t::oldest_modification_ to prevent a race condition
      when buf_page_t::list does not happen to be on the same cache line.
      
      buf_page_t::clear_oldest_modification(): Assert that the block is
      not in buf_pool.flush_list, and use std::memory_order_release.
      
      buf_page_t::oldest_modification_acquire(): Read oldest_modification_
      with std::memory_order_acquire. In this way, if the return value is 0,
      the caller may safely assume that it will not observe the buf_page_t
      as being in buf_pool.flush_list, even if it is not holding
      buf_pool.flush_list_mutex.
      
      buf_flush_relocate_on_flush_list(), buf_LRU_free_page():
      Invoke buf_page_t::oldest_modification_acquire().
      759deaa0
  6. 24 Jun, 2021 7 commits
    • Marko Mäkelä's avatar
      Merge 10.5 into 10.6 · a8350cfb
      Marko Mäkelä authored
      a8350cfb
    • Marko Mäkelä's avatar
      MDEV-26010: Assertion lsn > 2 failed in buf_pool_t::get_oldest_modification · 5f22511e
      Marko Mäkelä authored
      In commit 22b62eda (MDEV-25113)
      we introduced a race condition. buf_LRU_free_page() would read
      buf_page_t::oldest_modification() as 0 and assume that
      buf_page_t::list can be used (for attaching the block to the
      buf_pool.free list). In the observed race condition,
      buf_pool_t::delete_from_flush_list() had cleared the field,
      and buf_pool_t::delete_from_flush_list_low() was executing
      concurrently with buf_LRU_block_free_non_file_page(),
      which resulted in buf_pool.flush_list.end becoming corrupted.
      
      buf_pool_t::delete_from_flush_list(), buf_flush_relocate_on_flush_list():
      First remove the block from buf_pool.flush_list, and only then
      invoke buf_page_t::clear_oldest_modification(), to ensure that
      reading oldest_modification()==0 really implies that the block
      no longer is in buf_pool.flush_list.
      5f22511e
    • Marko Mäkelä's avatar
      MDEV-25948 fixup: Demote a warning to a note · e329dc8d
      Marko Mäkelä authored
      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 762bcb81 (MDEV-25948)
      but had been accidentally omitted.
      e329dc8d
    • Marko Mäkelä's avatar
      MDEV-26012 InnoDB purge and shutdown hangs after failed ALTER TABLE · 82fe83a3
      Marko Mäkelä authored
      ha_innobase::commit_inplace_alter_table(): Invoke
      purge_sys.resume_FTS() on all error handling paths
      if purge_sys.stop_FTS() had been called.
      
      This fixes a regression that had been introduced in
      commit 1bd681c8 (MDEV-25506).
      82fe83a3
    • Marko Mäkelä's avatar
      MDEV-26007 Rollback unnecessarily initiates redo log write · 033e29b6
      Marko Mäkelä authored
      trx_t::commit_in_memory(): Do not initiate a redo log write if
      the transaction has no visible effect. If anything for this
      transaction had been made durable, crash recovery will roll back
      the transaction just fine even if the end of ROLLBACK is not
      durably written.
      
      Rollbacks of transactions that are associated with XA identifiers
      (possibly internally via the binlog) will always be persisted.
      The test rpl.rpl_gtid_crash covers this.
      033e29b6
    • Marko Mäkelä's avatar
      Merge 10.5 into 10.6 · b4c9cd20
      Marko Mäkelä authored
      b4c9cd20
    • Marko Mäkelä's avatar
      MDEV-26004 Excessive wait times in buf_LRU_get_free_block() · 60ed4797
      Marko Mäkelä authored
      buf_LRU_get_free_block(): Initially wait for a single block to be
      freed, signaled by buf_pool.done_free. Only if that fails and no
      LRU eviction flushing batch is already running, we initiate a
      flushing batch that should serve all threads that are currently
      waiting in buf_LRU_get_free_block().
      
      Note: In an extreme case, this may introduce a performance regression
      at larger numbers of connections. We observed this in sysbench
      oltp_update_index with 512MiB buffer pool, 4GiB of data on fast NVMe,
      and 1000 concurrent connections, on a 20-thread CPU. The contention point
      appears to be buf_pool.mutex, and the improvement would turn into a
      regression somewhere beyond 32 concurrent connections.
      
      On slower storage, such regression was not observed; instead, the
      throughput was improving and maximum latency was reduced.
      
      The excessive waits were pointed out by Vladislav Vaintroub.
      60ed4797
  7. 23 Jun, 2021 3 commits
    • Marko Mäkelä's avatar
      Merge 10.5 into 10.6 · 101da872
      Marko Mäkelä authored
      101da872
    • Marko Mäkelä's avatar
      MDEV-25113: Introduce a page cleaner mode before 'furious flush' · 6441bc61
      Marko Mäkelä authored
      MDEV-23855 changed the way how the page cleaner is signaled by
      user threads. If a threshold is exceeded, a mini-transaction commit
      would invoke buf_flush_ahead() in order to initiate page flushing
      before all writers would eventually grind to halt in
      log_free_check(), waiting for the checkpoint age to reduce.
      
      However, buf_flush_ahead() would always initiate 'furious flushing',
      making the buf_flush_page_cleaner thread write innodb_io_capacity_max
      pages per batch, and sleeping no time between batches, until the
      limit LSN is reached. Because this could saturate the I/O subsystem,
      system throughput could significantly reduce during these
      'furious flushing' spikes.
      
      With this change, we introduce a gentler version of flush-ahead,
      which would write innodb_io_capacity_max pages per second until
      the 'soft limit' is reached.
      
      buf_flush_ahead(): Add a parameter to specify whether furious flushing
      is requested.
      
      buf_flush_async_lsn: Similar to buf_flush_sync_lsn, a limit for
      the less intrusive flushing.
      
      buf_flush_page_cleaner(): Keep working until buf_flush_async_lsn
      has been reached.
      
      log_close(): Suppress a warning message in the event that a new log
      is being created during startup, when old logs did not exist.
      Return what type of page cleaning will be needed.
      
      mtr_t::finish_write(): Also when m_log.is_small(), invoke log_close().
      Return what type of page cleaning will be needed.
      
      mtr_t::commit(): Invoke buf_flush_ahead() based on the return value of
      mtr_t::finish_write().
      6441bc61
    • Marko Mäkelä's avatar
      MDEV-25113: Make page flushing faster · 22b62eda
      Marko Mäkelä authored
      buf_page_write_complete(): Reduce the buf_pool.mutex hold time,
      and do not acquire buf_pool.flush_list_mutex at all.
      Instead, mark blocks clean by setting oldest_modification to 1.
      Dirty pages of temporary tables will be identified by the special
      value 2 instead of the previous special value 1.
      (By design of the ib_logfile0 format, actual LSN values smaller
      than 2048 are not possible.)
      
      buf_LRU_free_page(), buf_pool_t::get_oldest_modification()
      and many other functions will remove the garbage (clean blocks)
      from buf_pool.flush_list while holding buf_pool.flush_list_mutex.
      
      buf_pool_t::n_flush_LRU, buf_pool_t::n_flush_list:
      Replaced with non-atomic variables, protected by buf_pool.mutex,
      to avoid unnecessary synchronization when modifying the counts.
      
      export_vars: Remove unnecessary indirection for
      innodb_pages_created, innodb_pages_read, innodb_pages_written.
      22b62eda