An error occurred fetching the project authors.
  1. 19 Mar, 2021 1 commit
  2. 12 Mar, 2021 1 commit
    • Krunal Bauskar's avatar
      MDEV-21212: buf_page_get_gen -> buf_pool->stat.n_page_gets++ is a cpu waste · 8d4e3ec2
      Krunal Bauskar authored
      n_page_gets is a global counter that is updated on each page access.
      This also means it is updated pretty often and with a multi-core machine
      it easily boils up to be the hottest counter as also reported by perf.
      
      Using existing distributed counter framework help ease the contention
      and improve the performance.
      
      Patch also tend to increase the slot of the distributed counter from original
      64 to 128 given that is new normal for next-generation machines.
      
      The original idea and patch came from Daniel Black which is now ported
      to 10.6 with some improvement and adjustment.
      8d4e3ec2
  3. 03 Mar, 2021 1 commit
  4. 22 Feb, 2021 1 commit
  5. 15 Dec, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-21452: Replace ib_mutex_t with mysql_mutex_t · ff5d306e
      Marko Mäkelä authored
      SHOW ENGINE INNODB MUTEX functionality is completely removed,
      as are the InnoDB latching order checks.
      
      We will enforce innodb_fatal_semaphore_wait_threshold
      only for dict_sys.mutex and lock_sys.mutex.
      
      dict_sys_t::mutex_lock(): A single entry point for dict_sys.mutex.
      
      lock_sys_t::mutex_lock(): A single entry point for lock_sys.mutex.
      
      FIXME: srv_sys should be removed altogether; it is duplicating tpool
      functionality.
      
      fil_crypt_threads_init(): To prevent SAFE_MUTEX warnings, we must
      not hold fil_system.mutex.
      
      fil_close_all_files(): To prevent SAFE_MUTEX warnings for
      fil_space_destroy_crypt_data(), we must not hold fil_system.mutex
      while invoking fil_space_free_low() on a detached tablespace.
      ff5d306e
  6. 03 Dec, 2020 4 commits
    • Marko Mäkelä's avatar
    • Marko Mäkelä's avatar
      MDEV-24142: Remove the LatchDebug interface to rw-locks · ac028ec5
      Marko Mäkelä authored
      The latching order checks for rw-locks have not caught many bugs
      in the past few years and they are greatly complicating the code.
      
      Last time the debug checks were useful was in
      commit 59caf2c3 (MDEV-13485).
      
      The B-tree hang MDEV-14637 was not caught by LatchDebug,
      because the granularity of the checks is not sufficient
      to distinguish the levels of non-leaf B-tree pages.
      
      The interface was already made dead code by the grandparent
      commit 03ca6495.
      ac028ec5
    • Marko Mäkelä's avatar
      MDEV-24142: Replace InnoDB rw_lock_t with sux_lock · 03ca6495
      Marko Mäkelä authored
      InnoDB buffer pool block and index tree latches depend on a
      special kind of read-update-write lock that allows reentrant
      (recursive) acquisition of the 'update' and 'write' locks
      as well as an upgrade from 'update' lock to 'write' lock.
      The 'update' lock allows any number of reader locks from
      other threads, but no concurrent 'update' or 'write' lock.
      
      If there were no requirement to support an upgrade from 'update'
      to 'write', we could compose the lock out of two srw_lock
      (implemented as any type of native rw-lock, such as SRWLOCK on
      Microsoft Windows). Removing this requirement is very difficult,
      so in commit f7e7f487d4b06695f91f6fbeb0396b9d87fc7bbf we
      implemented an 'update' mode to our srw_lock.
      
      Re-entrant or recursive locking is mostly needed when writing or
      freeing BLOB pages, but also in crash recovery or when merging
      buffered changes to an index page. The re-entrancy allows us to
      attach a previously acquired page to a sub-mini-transaction that
      will be committed before whatever else is holding the page latch.
      
      The SUX lock supports Shared ('read'), Update, and eXclusive ('write')
      locking modes. The S latches are not re-entrant, but a single S latch
      may be acquired even if the thread already holds an U latch.
      
      The idea of the U latch is to allow a write of something that concurrent
      readers do not care about (such as the contents of BTR_SEG_LEAF,
      BTR_SEG_TOP and other page allocation metadata structures, or
      the MDEV-6076 PAGE_ROOT_AUTO_INC). (The PAGE_ROOT_AUTO_INC field
      is only updated when a dict_table_t for the table exists, and only
      read when a dict_table_t for the table is being added to dict_sys.)
      
      block_lock::u_lock_try(bool for_io=true) is used in buf_flush_page()
      to allow concurrent readers but no concurrent modifications while the
      page is being written to the data file. That latch will be released
      by buf_page_write_complete() in a different thread. Hence, we use
      the special lock owner value FOR_IO.
      
      The index_lock::u_lock() improves concurrency on operations that
      involve non-leaf index pages.
      
      The interface has been cleaned up a little. We will use
      x_lock_recursive() instead of x_lock() when we know that a
      lock is already held by the current thread. Similarly,
      a lock upgrade from U to X is only allowed via u_x_upgrade()
      or x_lock_upgraded() but not via x_lock().
      
      We will disable the LatchDebug and sync_array interfaces to
      InnoDB rw-locks.
      
      The SEMAPHORES section of SHOW ENGINE INNODB STATUS output
      will no longer include any information about InnoDB rw-locks,
      only TTASEventMutex (cmake -DMUTEXTYPE=event) waits.
      This will make a part of the 'innotop' script dead code.
      
      The block_lock buf_block_t::lock will not be covered by any
      PERFORMANCE_SCHEMA instrumentation.
      
      SHOW ENGINE INNODB MUTEX and INFORMATION_SCHEMA.INNODB_MUTEXES
      will no longer output source code file names or line numbers.
      The dict_index_t::lock will be identified by index and table names,
      which should be much more useful. PERFORMANCE_SCHEMA is lumping
      information about all dict_index_t::lock together as
      event_name='wait/synch/sxlock/innodb/index_tree_rw_lock'.
      
      buf_page_free(): Remove the file,line parameters. The sux_lock will
      not store such diagnostic information.
      
      buf_block_dbg_add_level(): Define as empty macro, to be removed
      in a subsequent commit.
      
      Unless the build was configured with cmake -DPLUGIN_PERFSCHEMA=NO
      the index_lock dict_index_t::lock will be instrumented via
      PERFORMANCE_SCHEMA. Similar to
      commit 1669c889
      we will distinguish lock waits by registering shared_lock,exclusive_lock
      events instead of try_shared_lock,try_exclusive_lock.
      Actual 'try' operations will not be instrumented at all.
      
      rw_lock_list: Remove. After MDEV-24167, this only covered
      buf_block_t::lock and dict_index_t::lock. We will output their
      information by traversing buf_pool or dict_sys.
      03ca6495
    • Marko Mäkelä's avatar
      MDEV-24167 fixup: Improve the PERFORMANCE_SCHEMA instrumentation · 1669c889
      Marko Mäkelä authored
      Let us try to avoid code bloat for the common case that
      performance_schema is disabled at runtime, and use
      ATTRIBUTE_NOINLINE member functions for instrumented latch acquisition.
      
      Also, let us distinguish lock waits from non-contended lock requests
      by using write_lock,read_lock for the requests that lead to waits,
      and try_write_lock,try_read_lock for the wait-free lock acquisitions.
      Actual 'try' operations are not being instrumented at all.
      1669c889
  7. 24 Nov, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-24167: Use lightweight srw_lock for btr_search_latch · c561f9e6
      Marko Mäkelä authored
      Many InnoDB rw-locks unnecessarily depend on the complex
      InnoDB rw_lock_t implementation that support the SX lock mode
      as well as recursive acquisition of X or SX locks.
      One of them is the bunch of adaptive hash index search latches,
      instrumented as btr_search_latch in PERFORMANCE_SCHEMA.
      Let us introduce a simpler lock for those in order to
      reduce overhead.
      
      srw_lock: A simple read-write lock that does not support recursion.
      On Microsoft Windows, this wraps SRWLOCK, only adding
      runtime overhead if PERFORMANCE_SCHEMA is enabled.
      On Linux (all architectures), this is implemented with
      std::atomic<uint32_t> and the futex system call.
      On other platforms, we will wrap mysql_rwlock_t with
      zero runtime overhead.
      
      The PERFORMANCE_SCHEMA instrumentation differs
      from InnoDB rw_lock_t in that we will only invoke
      PSI_RWLOCK_CALL(start_rwlock_wrwait) or
      PSI_RWLOCK_CALL(start_rwlock_rdwait)
      if there is an actual conflict.
      c561f9e6
  8. 19 Oct, 2020 1 commit
  9. 15 Oct, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-23399: Performance regression with write workloads · 7cffb5f6
      Marko Mäkelä authored
      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 d1ab8903 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
      7cffb5f6
  10. 23 Sep, 2020 1 commit
  11. 30 Jul, 2020 1 commit
    • Thirunarayanan Balathandayuthapani's avatar
      MDEV-23332 Index online status assert failure in btr_search_drop_page_hash_index · 8a612314
      Thirunarayanan Balathandayuthapani authored
      Problem:
      ========
      In row_merge_drop_indexes(), InnoDB drops only the index from
      dictionary and frees the index pages but it maintains the index
      object if the table is being used by other DML threads. It sets
      the online status of the index to ONLINE_INDEX_ABORTED_DROPPED.
      Removing the index from dictionary doesn't remove the
      corressponding ahi entries of the index. When block is being
      reused, InnoDB tries to remove ahi entries for the block and
      it fails if index online status is ONLINE_INDEX_ABORTED_DROPPED.
      
      Fix:
      ====
      MDEV-22456 allows the index ahi entries to be dropped lazily.
      so checking online status in btr_search_drop_page_hash_index()
      is meaningless and should be removed.
      8a612314
  12. 13 Jul, 2020 1 commit
  13. 19 Jun, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-22871 follow-up fix: AHI corruption & leak · e341fb0d
      Marko Mäkelä authored
      Commit bf3c862f
      accidentally introduced two bugs.
      
      btr_search_update_hash_ref(): Pass the correct parameter part->heap.
      
      btr_search_sys_t::free(): Free all memory.
      
      Thanks to Michael Widenius and Thirunarayanan Balathandayuthapani
      for pointing out these bugs.
      e341fb0d
  14. 18 Jun, 2020 4 commits
    • Marko Mäkelä's avatar
      MDEV-22871: Reduce InnoDB buf_pool.page_hash contention · 5155a300
      Marko Mäkelä authored
      The rw_lock_s_lock() calls for the buf_pool.page_hash became a
      clear bottleneck after MDEV-15053 reduced the contention on
      buf_pool.mutex. We will replace that use of rw_lock_t with a
      special implementation that is optimized for memory bus traffic.
      
      The hash_table_locks instrumentation will be removed.
      
      buf_pool_t::page_hash: Use a special implementation whose API is
      compatible with hash_table_t, and store the custom rw-locks
      directly in buf_pool.page_hash.array, intentionally sharing
      cache lines with the hash table pointers.
      
      rw_lock: A low-level rw-lock implementation based on std::atomic<uint32_t>
      where read_trylock() becomes a simple fetch_add(1).
      
      buf_pool_t::page_hash_latch: The special of rw_lock for the page_hash.
      
      buf_pool_t::page_hash_latch::read_lock(): Assert that buf_pool.mutex
      is not being held by the caller.
      
      buf_pool_t::page_hash_latch::write_lock() may be called while not holding
      buf_pool.mutex. buf_pool_t::watch_set() is such a caller.
      
      buf_pool_t::page_hash_latch::read_lock_wait(),
      page_hash_latch::write_lock_wait(): The spin loops.
      These will obey the global parameters innodb_sync_spin_loops and
      innodb_sync_spin_wait_delay.
      
      buf_pool_t::freed_page_hash: A singly linked list of copies of
      buf_pool.page_hash that ever existed. The fact that we never
      free any buf_pool.page_hash.array guarantees that all
      page_hash_latch that ever existed will remain valid until shutdown.
      
      buf_pool_t::resize_hash(): Replaces buf_pool_resize_hash().
      Prepend a shallow copy of the old page_hash to freed_page_hash.
      
      buf_pool_t::page_hash_table::n_cells: Declare as Atomic_relaxed.
      
      buf_pool_t::page_hash_table::lock(): Explain what prevents a
      race condition with buf_pool_t::resize_hash().
      5155a300
    • Marko Mäkelä's avatar
      MDEV-22871: Remove pointer indirection for InnoDB hash_table_t · cfd3d70c
      Marko Mäkelä authored
      hash_get_n_cells(): Remove. Access n_cells directly.
      
      hash_get_nth_cell(): Remove. Access array directly.
      
      hash_table_clear(): Replaced with hash_table_t::clear().
      
      hash_table_create(), hash_table_free(): Remove.
      
      hash0hash.cc: Remove.
      cfd3d70c
    • Marko Mäkelä's avatar
      MDEV-22871: Clean up btr_search_sys · bf3c862f
      Marko Mäkelä authored
      btr_search_sys::parts[]: A single structure for the partitions of
      the adaptive hash index. Replaces the 3 separate arrays:
      btr_search_latches[], btr_search_sys->hash_tables,
      btr_search_sys->hash_tables[i]->heap.
      
      hash_table_t::heap, hash_table_t::adaptive: Remove.
      
      ha0ha.cc: Remove. Move all code to btr0sea.cc.
      bf3c862f
    • Marko Mäkelä's avatar
      MDEV-22871: Clean up hash_table_t · 9159b897
      Marko Mäkelä authored
      HASH_TABLE_SYNC_MUTEX was kind-of used for the adaptive hash index,
      even though that hash table is already protected by btr_search_latches[].
      
      HASH_TABLE_SYNC_RWLOCK was only being used for buf_pool.page_hash.
      It is cleaner to decouple that synchronization from hash_table_t,
      and move it to the actual user.
      
      buf_pool_t::page_hash_latches[]: Synchronization for buf_pool.page_hash.
      
      LATCH_ID_HASH_TABLE_MUTEX: Remove.
      
      hash_table_t::sync_obj, hash_table_t::n_sync_obj: Remove.
      
      hash_table_t::type, hash_table_sync_t: Remove.
      
      HASH_ASSERT_OWN(), hash_get_mutex(), hash_get_nth_mutex(): Remove.
      
      ib_recreate(): Merge to the only caller, buf_pool_resize_hash().
      
      ib_create(): Merge to the callers.
      
      ha_clear(): Merge to the only caller buf_pool_t::close().
      
      buf_pool_t::create(): Merge the ib_create() and
      hash_create_sync_obj() invocations.
      
      ha_insert_for_fold_func(): Clarify an assertion.
      
      buf_pool_t::page_hash_lock(): Simplify the logic.
      
      hash_assert_can_search(), hash_assert_can_modify(): Remove.
      These predicates were only being invoked for the adaptive hash index,
      while they only are effective for buf_pool.page_hash.
      
      HASH_DELETE_AND_COMPACT(): Merge to ha_delete_hash_node().
      
      hash_get_sync_obj_index(): Remove.
      
      hash_table_t::heaps[], hash_get_nth_heap(): Remove. It was actually unused!
      
      hash_get_heap(): Remove. It was only used in ha_delete_hash_node(),
      where we always use hash_table_t::heap.
      
      hash_table_t::calc_hash(): Replaces hash_calc_hash().
      9159b897
  15. 16 Jun, 2020 2 commits
  16. 12 Jun, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-22877 Avoid unnecessary buf_pool.page_hash S-latch acquisition · d2c593c2
      Marko Mäkelä authored
      MDEV-15053 did not remove all unnecessary buf_pool.page_hash S-latch
      acquisition. There are code paths where we are holding buf_pool.mutex
      (which will sufficiently protect buf_pool.page_hash against changes)
      and unnecessarily acquire the latch. Many invocations of
      buf_page_hash_get_locked() can be replaced with the much simpler
      buf_pool.page_hash_get_low().
      
      In the worst case the thread that is holding buf_pool.mutex will become
      a victim of MDEV-22871, suffering from a spurious reader-reader conflict
      with another thread that genuinely needs to acquire a buf_pool.page_hash
      S-latch.
      
      In many places, we were also evaluating page_id_t::fold() while holding
      buf_pool.mutex. Low-level functions such as buf_pool.page_hash_get_low()
      must get the page_id_t::fold() as a parameter.
      
      buf_buddy_relocate(): Defer the hash_lock acquisition to the critical
      section that starts by calling buf_page_t::can_relocate().
      d2c593c2
  17. 05 Jun, 2020 3 commits
    • Marko Mäkelä's avatar
      MDEV-22790 Race between btr_page_mtr_lock() dropping AHI on the same block · 138c11cc
      Marko Mäkelä authored
      This race condition was introduced by
      commit ad6171b9 (MDEV-22456).
      
      In the observed case, two threads were executing
      btr_search_drop_page_hash_index() on the same block,
      to free a stale entry that was attached to a dropped index.
      Both threads were holding an S latch on the block.
      
      We must prevent the double-free of block->index by holding
      block->lock in exclusive mode.
      
      btr_search_guess_on_hash(): Do not invoke
      btr_search_drop_page_hash_index(block) to get rid of
      stale entries, because we are not necessarily holding
      an exclusive block->lock here.
      
      buf_defer_drop_ahi(): New function, to safely drop stale
      entries in buf_page_mtr_lock(). We will skip the call to
      btr_search_drop_page_hash_index(block) when only requesting
      bufferfixing (no page latch), because in that case, we should
      not be accessing the adaptive hash index, and we might get
      a deadlock if we acquired the page latch.
      138c11cc
    • Marko Mäkelä's avatar
      MDEV-22646: Fix a memory leak · 3677dd5c
      Marko Mäkelä authored
      btr_search_sys_free(): Free btr_search_sys->hash_tables.
      
      The leak was introduced in commit ad2bf112.
      3677dd5c
    • Marko Mäkelä's avatar
      MDEV-15053 Reduce buf_pool_t::mutex contention · b1ab211d
      Marko Mäkelä authored
      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 4ed7082e
      which introduced buf_block_t::mutex, which we are now removing.
      
      Later, multiple instances of buf_pool_t were introduced
      in commit c18084f7
      and recently removed by us in
      commit 1a6f708e (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 c5883deb 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.
      b1ab211d
  18. 03 Jun, 2020 1 commit
    • Thirunarayanan Balathandayuthapani's avatar
      MDEV-22646 Assertion `table2->cached' failed in dict_table_t::add_to_cache · ad2bf112
      Thirunarayanan Balathandayuthapani authored
      Problem:
      ========
        During buffer pool resizing, InnoDB recreates the dictionary hash
      tables. Dictionary hash table reuses the heap of AHI hash tables.
      It leads to memory corruption.
      
      Fix:
      ====
      - While disabling AHI, free the heap and AHI hash tables. Recreate the
      AHI hash tables and assign new heap when AHI is enabled.
      
      - btr_blob_free() access invalid page if page was reallocated during
      buffer poolresizing. So btr_blob_free() should get the page from
      buf_pool instead of using existing block.
      
      - btr_search_enabled and block->index should be checked after
      acquiring the btr_search_sys latch
      
      - Moved the buffer_pool_scan debug sync to earlier before accessing the
      btr_search_sys latches to avoid the hang of truncate_purge_debug
      test case
      
      - srv_printf_innodb_monitor() should acquire btr_search_sys latches
      before AHI hash tables.
      ad2bf112
  19. 18 May, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-22611 Assertion btr_search_enabled failed during buffer pool resizing · 5b6bcb59
      Marko Mäkelä authored
      In commit ad6171b9 (MDEV-22456)
      we removed the acquisition of the adaptive hash index latch
      from the caller of btr_search_update_hash_ref().
      The tests innodb.innodb_buffer_pool_resize_with_chunks
      and innodb.innodb_buffer_pool_resize
      would occasionally fail starting with 10.3,
      due to MDEV-12288 causing more purge activity during the test.
      
      btr_search_update_hash_ref(): After acquiring the adaptive hash index
      latch, check that the adaptive hash index is still enabled on the page.
      5b6bcb59
  20. 15 May, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-22456 Dropping the adaptive hash index may cause DDL to lock up InnoDB · ad6171b9
      Marko Mäkelä authored
      If the InnoDB buffer pool contains many pages for a table or index
      that is being dropped or rebuilt, and if many of such pages are
      pointed to by the adaptive hash index, dropping the adaptive hash index
      may consume a lot of time.
      
      The time-consuming operation of dropping the adaptive hash index entries
      is being executed while the InnoDB data dictionary cache dict_sys is
      exclusively locked.
      
      It is not actually necessary to drop all adaptive hash index entries
      at the time a table or index is being dropped or rebuilt. We can let
      the LRU replacement policy of the buffer pool take care of this gradually.
      For this to work, we must detach the dict_table_t and dict_index_t
      objects from the main dict_sys cache, and once the last
      adaptive hash index entry for the detached table is removed
      (when the garbage page is evicted from the buffer pool) we can free
      the dict_table_t and dict_index_t object.
      
      Related to this, in MDEV-16283, we made ALTER TABLE...DISCARD TABLESPACE
      skip both the buffer pool eviction and the drop of the adaptive hash index.
      We shifted the burden to ALTER TABLE...IMPORT TABLESPACE or DROP TABLE.
      We can remove the eviction from DROP TABLE. We must retain the eviction
      in the ALTER TABLE...IMPORT TABLESPACE code path, so that in case the
      discarded table is being re-imported with the same tablespace identifier,
      the fresh data from the imported tablespace will replace any stale pages
      in the buffer pool.
      
      rpl.rpl_failed_drop_tbl_binlog: Remove the test. DROP TABLE can
      no longer be interrupted inside InnoDB.
      
      fseg_free_page(), fseg_free_step(), fseg_free_step_not_header(),
      fseg_free_page_low(), fseg_free_extent(): Remove the parameter
      that specifies whether the adaptive hash index should be dropped.
      
      btr_search_lazy_free(): Lazily free an index when the last
      reference to it is dropped from the adaptive hash index.
      
      buf_pool_clear_hash_index(): Declare static, and move to the
      same compilation unit with the bulk of the adaptive hash index
      code.
      
      dict_index_t::clone(), dict_index_t::clone_if_needed():
      Clone an index that is being rebuilt while adaptive hash index
      entries exist. The original index will be inserted into
      dict_table_t::freed_indexes and dict_index_t::set_freed()
      will be called.
      
      dict_index_t::set_freed(), dict_index_t::freed(): Note that
      or check whether the index has been freed. We will use the
      impossible page number 1 to denote this condition.
      
      dict_index_t::n_ahi_pages(): Replaces btr_search_info_get_ref_count().
      
      dict_index_t::detach_columns(): Move the assignment n_fields=0
      to ha_innobase_inplace_ctx::clear_added_indexes().
      We must have access to the columns when freeing the
      adaptive hash index. Note: dict_table_t::v_cols[] will remain
      valid. If virtual columns are dropped or added, the table
      definition will be reloaded in ha_innobase::commit_inplace_alter_table().
      
      buf_page_mtr_lock(): Drop a stale adaptive hash index if needed.
      
      We will also reduce the number of btr_get_search_latch() calls
      and enclose some more code inside #ifdef BTR_CUR_HASH_ADAPT
      in order to benefit cmake -DWITH_INNODB_AHI=OFF.
      ad6171b9
  21. 29 Apr, 2020 1 commit
  22. 19 Mar, 2020 1 commit
  23. 18 Mar, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-21962 Allocate buf_pool statically · a786f50d
      Marko Mäkelä authored
      Thanks to MDEV-15058, there is only one InnoDB buffer pool.
      Allocating buf_pool statically removes one level of pointer indirection
      and makes code more readable, and removes the awkward initialization of
      some buf_pool members.
      
      While doing this, we will also declare some buf_pool_t data members
      private and replace some functions with member functions. This is
      mostly affecting buffer pool resizing.
      
      This is not aiming to be a complete rewrite of buf_pool_t to
      a proper class. Most of the buffer pool interface, such as
      buf_page_get_gen(), will remain in the C programming style
      for now.
      
      buf_pool_t::withdrawing: Replaces buf_pool_withdrawing.
      buf_pool_t::withdraw_clock_: Replaces buf_withdraw_clock.
      
      buf_pool_t::create(): Repalces buf_pool_init().
      buf_pool_t::close(): Replaces buf_pool_free().
      
      buf_bool_t::will_be_withdrawn(): Replaces buf_block_will_be_withdrawn(),
      buf_frame_will_be_withdrawn().
      
      buf_pool_t::clear_hash_index(): Replaces buf_pool_clear_hash_index().
      buf_pool_t::get_n_pages(): Replaces buf_pool_get_n_pages().
      buf_pool_t::validate(): Replaces buf_validate().
      buf_pool_t::print(): Replaces buf_print().
      buf_pool_t::block_from_ahi(): Replaces buf_block_from_ahi().
      buf_pool_t::is_block_field(): Replaces buf_pointer_is_block_field().
      buf_pool_t::is_block_mutex(): Replaces buf_pool_is_block_mutex().
      buf_pool_t::is_block_lock(): Replaces buf_pool_is_block_lock().
      buf_pool_t::is_obsolete(): Replaces buf_pool_is_obsolete().
      buf_pool_t::io_buf: Make default-constructible.
      buf_pool_t::io_buf::create(): Delayed 'constructor'
      buf_pool_t::io_buf::close(): Early 'destructor'
      
      HazardPointer: Make default-constructible. Define all member functions
      inline, also for derived classes.
      a786f50d
  24. 12 Mar, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-21907: InnoDB: Enable -Wconversion on clang and GCC · f2245252
      Marko Mäkelä authored
      The -Wconversion in GCC seems to be stricter than in clang.
      GCC at least since version 4.4.7 issues truncation warnings for
      assignments to bitfields, while clang 10 appears to only issue
      warnings when the sizes in bytes rounded to the nearest integer
      powers of 2 are different.
      
      Before GCC 10.0.0, -Wconversion required more casts and would not
      allow some operations, such as x<<=1 or x+=1 on a data type that
      is narrower than int.
      
      GCC 5 (but not GCC 4, GCC 6, or any later version) is complaining
      about x|=y even when x and y are compatible types that are narrower
      than int.  Hence, we must rewrite some x|=y as
      x=static_cast<byte>(x|y) or similar, or we must disable -Wconversion.
      
      In GCC 6 and later, the warning for assigning wider to bitfields
      that are narrower than 8, 16, or 32 bits can be suppressed by
      applying a bitwise & with the exact bitmask of the bitfield.
      For older GCC, we must disable -Wconversion for GCC 4 or 5 in such
      cases.
      
      The bitwise negation operator appears to promote short integers
      to a wider type, and hence we must add explicit truncation casts
      around them. Microsoft Visual C does not allow a static_cast to
      truncate a constant, such as static_cast<byte>(1) truncating int.
      Hence, we will use the constructor-style cast byte(~1) for such cases.
      
      This has been tested at least with GCC 4.8.5, 5.4.0, 7.4.0, 9.2.1, 10.0.0,
      clang 9.0.1, 10.0.0, and MSVC 14.22.27905 (Microsoft Visual Studio 2019)
      on 64-bit and 32-bit targets (IA-32, AMD64, POWER 8, POWER 9, ARMv8).
      f2245252
  25. 10 Mar, 2020 1 commit
    • Thirunarayanan Balathandayuthapani's avatar
      MDEV-15528 Punch holes when pages are freed · a35b4ae8
      Thirunarayanan Balathandayuthapani authored
      When a InnoDB data file page is freed, its contents becomes garbage,
      and any storage allocated in the data file is wasted. During flushing,
      InnoDB initializes the page with zeros if scrubbing is enabled. If the
      tablespace is compressed then InnoDB should punch a hole else ignore the
      flushing of the freed page.
      
      buf_page_t:
      - Replaced the variable file_page_was_freed, init_on_flush in buf_page_t
      with status enum variable.
      - Changed all debug assert of file_page_was_freed to DBUG_ASSERT
      of buf_page_t::status
      
      Removed buf_page_set_file_page_was_freed(),
      buf_page_reset_file_page_was_freed().
      
      buf_page_free(): Newly added function which takes X-lock on the page
      before marking the status as FREED. So that InnoDB flush handler can
      avoid concurrent flush of the freed page. Also while flushing the page,
      InnoDB make sure that redo log which does freeing of the page also written
      to the disk. Currently, this function only marks the page as FREED if
      it is in buffer pool
      
      buf_flush_freed_page(): Newly added function which initializes zeros
      asynchorously if innodb_immediate_scrub_data_uncompressed is enabled.
      Punch a hole to the file synchorously if page_compressed is enabled.
      Reset the io_fix to NORMAL. Release the block from flush list and
      associated mutex before writing zeros or punch a hole to the file.
      
      buf_flush_page(): Removed the unnecessary usage of temporary
      variable "flush"
      
      fil_io(): Introduce new parameter called punch_hole. It allows fil_io()
      to punch the hole to the file for the given offset.
      
      buf_page_create(): Let the callers assign buf_page_t::status.
      Every caller should eventually invoke mtr_t::init().
      
      fsp_page_create(): Remove the unused mtr_t parameter.
      
      In all other callers of buf_page_create() except fsp_page_create(),
      before invoking mtr_t::init(), invoke
      mtr_t::sx_latch_at_savepoint() or mtr_t::x_latch_at_savepoint().
      
      mtr_t::init(): Initialize buf_page_t::status also for the temporary
      tablespace (when redo logging is disabled), to avoid assertion failures.
      a35b4ae8
  26. 12 Feb, 2020 1 commit
    • Marko Mäkelä's avatar
      MDEV-15058: Deprecate and ignore innodb_buffer_pool_instances · 1a6f708e
      Marko Mäkelä authored
      Our benchmarking efforts indicate that the reasons for splitting the
      buf_pool in commit c18084f7
      have mostly gone away, possibly as a result of
      mysql/mysql-server@ce6109ebfdedfdf185e391a0c97dc6d33867ed78
      or similar work.
      
      Only in one write-heavy benchmark where the working set size is
      ten times the buffer pool size, the buf_pool->mutex would be
      less contended with 4 buffer pool instances than with 1 instance,
      in buf_page_io_complete(). That contention could be alleviated
      further by making more use of std::atomic and by splitting
      buf_pool_t::mutex further (MDEV-15053).
      
      We will deprecate and ignore the following parameters:
      
      	innodb_buffer_pool_instances
      	innodb_page_cleaners
      
      There will be only one buffer pool and one page cleaner task.
      
      In a number of INFORMATION_SCHEMA views, columns that indicated
      the buffer pool instance will be removed:
      
      	information_schema.innodb_buffer_page.pool_id
      	information_schema.innodb_buffer_page_lru.pool_id
      	information_schema.innodb_buffer_pool_stats.pool_id
      	information_schema.innodb_cmpmem.buffer_pool_instance
      	information_schema.innodb_cmpmem_reset.buffer_pool_instance
      1a6f708e
  27. 12 Dec, 2019 1 commit
    • Eugene Kosov's avatar
      MDEV-20950 Reduce size of record offsets · f0aa073f
      Eugene Kosov authored
      offset_t: this is a type which represents one record offset.
      It's unsigned short int.
      
      a lot of functions: replace ulint with offset_t
      
      btr_pcur_restore_position_func(),
      page_validate(),
      row_ins_scan_sec_index_for_duplicate(),
      row_upd_clust_rec_by_insert_inherit_func(),
      row_vers_impl_x_locked_low(),
      trx_undo_prev_version_build():
        allocate record offsets on the stack instead of waiting for rec_get_offsets()
        to allocate it from mem_heap_t. So, reducing  memory allocations.
      
      RECORD_OFFSET, INDEX_OFFSET:
        now it's less convenient to store pointers in offset_t*
        array. One pointer occupies now several offset_t. And those constant are start
        indexes into array to places where to store pointer values
      
      REC_OFFS_HEADER_SIZE: adjusted for the new reality
      
      REC_OFFS_NORMAL_SIZE:
        increase size from 100 to 300 which means less heap allocations.
        And sizeof(offset_t[REC_OFFS_NORMAL_SIZE]) now is 600 bytes which
        is smaller than previous 800 bytes.
      
      REC_OFFS_SEC_INDEX_SIZE: adjusted for the new reality
      
      rem0rec.h, rem0rec.ic, rem0rec.cc:
        various arguments, return values and local variables types were changed to
        fix numerous integer conversions issues.
      
      enum field_type_t:
        offset types concept was introduces which replaces old offset flags stuff.
        Like in earlier version, 2 upper bits are used to store offset type.
        And this enum represents those types.
      
      REC_OFFS_SQL_NULL, REC_OFFS_MASK: removed
      
      get_type(), set_type(), get_value(), combine():
        these are convenience functions to work with offsets and it's types
      
      rec_offs_base()[0]:
        still uses an old scheme with flags REC_OFFS_COMPACT and REC_OFFS_EXTERNAL
      
      rec_offs_base()[i]:
        these have type offset_t now. Two upper bits contains type.
      f0aa073f
  28. 03 Dec, 2019 1 commit
  29. 11 Nov, 2019 1 commit
  30. 11 Oct, 2019 1 commit
    • Marko Mäkelä's avatar
      MDEV-19514 Defer change buffer merge until pages are requested · b42294bc
      Marko Mäkelä authored
      We will remove the InnoDB background operation of merging buffered
      changes to secondary index leaf pages. Changes will only be merged as a
      result of an operation that accesses a secondary index leaf page,
      such as a SQL statement that performs a lookup via that index,
      or is modifying the index. Also ROLLBACK and some background operations,
      such as purging the history of committed transactions, or computing
      index cardinality statistics, can cause change buffer merge.
      Encryption key rotation will not perform change buffer merge.
      
      The motivation of this change is to simplify the I/O logic and to
      allow crash recovery to happen in the background (MDEV-14481).
      We also hope that this will reduce the number of "mystery" crashes
      due to corrupted data. Because change buffer merge will typically
      take place as a result of executing SQL statements, there should be
      a clearer connection between the crash and the SQL statements that
      were executed when the server crashed.
      
      In many cases, a slight performance improvement was observed.
      
      This is joint work with Thirunarayanan Balathandayuthapani
      and was tested by Axel Schwenke and Matthias Leich.
      
      The InnoDB monitor counter innodb_ibuf_merge_usec will be removed.
      
      On slow shutdown (innodb_fast_shutdown=0), we will continue to
      merge all buffered changes (and purge all undo log history).
      
      Two InnoDB configuration parameters will be changed as follows:
      
      innodb_disable_background_merge: Removed.
      This parameter existed only in debug builds.
      All change buffer merges will use synchronous reads.
      
      innodb_force_recovery will be changed as follows:
      * innodb_force_recovery=4 will be the same as innodb_force_recovery=3
      (the change buffer merge cannot be disabled; it can only happen as
      a result of an operation that accesses a secondary index leaf page).
      The option used to be capable of corrupting secondary index leaf pages.
      Now that capability is removed, and innodb_force_recovery=4 becomes 'safe'.
      * innodb_force_recovery=5 (which essentially hard-wires
      SET GLOBAL TRANSACTION ISOLATION LEVEL READ UNCOMMITTED)
      becomes safe to use. Bogus data can be returned to SQL, but
      persistent InnoDB data files will not be corrupted further.
      * innodb_force_recovery=6 (ignore the redo log files)
      will be the only option that can potentially cause
      persistent corruption of InnoDB data files.
      
      Code changes:
      
      buf_page_t::ibuf_exist: New flag, to indicate whether buffered
      changes exist for a buffer pool page. Pages with pending changes
      can be returned by buf_page_get_gen(). Previously, the changes
      were always merged inside buf_page_get_gen() if needed.
      
      ibuf_page_exists(const buf_page_t&): Check if a buffered changes
      exist for an X-latched or read-fixed page.
      
      buf_page_get_gen(): Add the parameter allow_ibuf_merge=false.
      All callers that know that they may be accessing a secondary index
      leaf page must pass this parameter as allow_ibuf_merge=true,
      unless it does not matter for that caller whether all buffered
      changes have been applied. Assert that whenever allow_ibuf_merge
      holds, the page actually is a leaf page. Attempt change buffer
      merge only to secondary B-tree index leaf pages.
      
      btr_block_get(): Add parameter 'bool merge'.
      All callers of btr_block_get() should know whether the page could be
      a secondary index leaf page. If it is not, we should avoid consulting
      the change buffer bitmap to even consider a merge. This is the main
      interface to requesting index pages from the buffer pool.
      
      ibuf_merge_or_delete_for_page(), recv_recover_page(): Replace
      buf_page_get_known_nowait() with much simpler logic, because
      it is now guaranteed that that the block is x-latched or read-fixed.
      
      mlog_init_t::mark_ibuf_exist(): Renamed from mlog_init_t::ibuf_merge().
      On crash recovery, we will no longer merge any buffered changes
      for the pages that we read into the buffer pool during the last batch
      of applying log records.
      
      buf_page_get_gen_known_nowait(), BUF_MAKE_YOUNG, BUF_KEEP_OLD: Remove.
      
      btr_search_guess_on_hash(): Merge buf_page_get_gen_known_nowait()
      to its only remaining caller.
      
      buf_page_make_young_if_needed(): Define as an inline function.
      Add the parameter buf_pool.
      
      buf_page_peek_if_young(), buf_page_peek_if_too_old(): Add the
      parameter buf_pool.
      
      fil_space_validate_for_mtr_commit(): Remove a bogus comment
      about background merge of the change buffer.
      
      btr_cur_open_at_rnd_pos_func(), btr_cur_search_to_nth_level_func(),
      btr_cur_open_at_index_side_func(): Use narrower data types and scopes.
      
      ibuf_read_merge_pages(): Replaces buf_read_ibuf_merge_pages().
      Merge the change buffer by invoking buf_page_get_gen().
      b42294bc
  31. 17 May, 2019 1 commit
    • Marko Mäkelä's avatar
      MDEV-19513: Allocate dict_sys statically · 5fd7502e
      Marko Mäkelä authored
      dict_sys_t::create(): Renamed from dict_init().
      
      dict_sys_t::close(): Renamed from dict_close().
      
      dict_sys_t::add(): Sliced from dict_table_t::add_to_cache().
      
      dict_sys_t::remove(): Renamed from dict_table_remove_from_cache().
      
      dict_sys_t::prevent_eviction(): Renamed from
      dict_table_move_from_lru_to_non_lru().
      
      dict_sys_t::acquire(): Replaces dict_move_to_mru() and some more logic.
      
      dict_sys_t::resize(): Renamed from dict_resize().
      
      dict_sys_t::find(): Replaces dict_lru_find_table() and
      dict_non_lru_find_table().
      5fd7502e