×
Community Blog MySQL Memory Allocation and Management (Part II)

MySQL Memory Allocation and Management (Part II)

This article introduces the structure and usage of InnoDB memory. The code is based on version 8.0.25.

By Huaxiong Song, from the ApsaraDB RDS for MySQL kernel team

The introduction to MySQL memory allocation and management is divided into three parts. Part I introduces the memory allocation manager at the InnoDB layer and the SQL layer, in which InnoDB memory allocation mainly depends on ut_allocator and mem_heap_allocator. Part II introduces the structure and usage of InnoDB memory. The code is based on version 8.0.25.

InnoDB is the default storage engine of MySQL. When discussing InnoDB memory, the Buffer Pool cannot be overlooked as it has a significant impact on performance. However, the memory of InnoDB is not solely occupied by the Buffer Pool; it also contains many other important structures such as AHI, Change Buffer, and Log buffer, which occupy a considerable amount of memory space. Understanding the structure and usage of InnoDB memory is crucial for selecting and utilizing memory during MySQL operation.

1
Source: https://dev.mysql.com/doc/refman/8.0/en/innodb-architecture.html

1. Buffer Pool

The buffer pool (BP) is a region in the main memory of InnoDB, primarily used to cache data pages, index pages, undo pages, adaptive hash indexes, and lock information. When retrieving data, if the data is present in the BP, it can be directly accessed, avoiding disk I/O and enhancing performance. For writing data, modifications are first made to the data pages in the BP, followed by employing a certain strategy to flush and update the disk data.

Typically, the BP size is configured to be 50% to 75% of the available physical memory of the machine. When the database starts, this portion of virtual memory is allocated in advance, and the actual physical memory mapping occurs during use. The BP memory will remain in use until it is uniformly released when the DB instance is shut down. The actual BP size may differ from the setpoint, but it will always be rounded to an integral multiple of chunk_size * instances.

1.1 Data Structure

The key data structures of the BP include buf_pool_t, buf_chunk_t, and buf_block_t. The relationships between each structure are illustrated in the following diagram. The number of buf_pool_t instances can be set using the innodb_buffer_pool_instances parameter. The design of multiple instances can reduce resource contention within the buffer pool, improving the overall performance of the engine. Each BP instance contains at least one chunk, and during initialization, each chunk will divide the data page control body buf_block_t and the actual data page frame. Page management is handled by linked lists such as LRU, free, and flush.

2

1.1.1 buf_pool_t

buf_pool_t represents a BP instance, which contains a lot of information, such as instance number, size, chunk list, each linked list (free, LRU, and flush) and its mutex, hash table and its mutex, and zip_free linked list array. In addition, buf_pool_t also contains the memory allocator ut_allocator which is used to allocate the memory of chunks, xxx_old which is used to record the old data before BP resizing, the risk pointer Hp which is used to mark the address of the linked list node.

struct buf_pool_t {
    ...
    ulint                           instance_no;       // The number of the buffer pool instance.
    ulint                           curr_pool_size;    // The size of the buffer pool instance.
    buf_chunk_t                     *chunks;           // The list of physical blocks of the buffer pool instance.
    hash_table_t                    *page_hash;        // The page hash table.
    hash_table_t                    *zip_hash;         // The block hash table that the buddy system allocates to the frame.
    UT_LIST_BASE_NODE_T(buf_page_t) free;              // The idle linked list.
    UT_LIST_BASE_NODE_T(buf_page_t) LRU;               // The LRU linked list.
    UT_LIST_BASE_NODE_T(buf_page_t) flush_list;        // The flush linked list.
    UT_LIST_BASE_NODE_T(buf_buddy_free_t) zip_free[BUF_BUDDY_SIZES_MAX]; // The idle linked list allocated from the buddy system.
    BufListMutex                    free_list_mutex;   // The mutex of the idle linked list.
    BufListMutex                    LRU_list_mutex;    // The mutex of the LRU linked list.
    BufListMutex                    flush_state_mutex; // The mutex of the flush linked list.
    BufListMutex          zip_free_mutex;              // The mutex allocated from the buddy system.
    BufListMutex          zip_hash_mutex;
    BufListMutex          chunks_mutex;                // The chunk mutex linked list.
    ...
}

1.1.2 buf_chunk_t

An instance consists of chunks, which are the basic unit of physical memory allocation. Each chunk is split into a block string and a frame string. The block string comes first and the frame string comes next. Unrounded memory fragments may exist between the two strings. The block is a control block that contains the control information of the page. The frame stores the actual data and consists of bytes.

struct buf_chunk_t {
  ulint               size;           /*!< Number of frames[]/blocks[] */
  unsigned char       *mem;           /*!< The pointer in the frame memory area */
  ut_new_pfx_t        mem_pfx;        /*!< The monitoring information */
  buf_block_t         *blocks;        /*!< The array of control blocks */
  uint32_t            chunk_no;       /*! chunk number*/
  UT_LIST_BASE_NODE_T(buf_page_t)   chunk_page_list; /*!< The root node of the chunk list */
  ...
};

1.1.3 buf_block_t

buf_block_t is the control block of the page. It contains the data of the page in buf_page_t format, which must be the first member for forced conversion of the pointer. It also contains a pointer pointing to the data frame, and the memory block size pointed to by the frame is UNIV_PAGE_SIZE K (generally 16K); In addition, the block contains the mutex of the frame and the block, which is also responsible for protecting partial data in buf_page_t.

struct buf_block_t {
  buf_page_t    page;  // It is placed in the first position for forced conversion of the block and the page.
  BPageLock     lock;  // The read/write lock of the frame.
  byte        *frame;  // The actual data.
  ...
  BPageMutex    mutex; // block locks:state、io_fix、buf_fix_count、accessed
};

1.1.4 buf_page_t

The page contains information such as the ID, size, and lsn. io_fix and buf_fix_count are used to control the concurrency and determine whether the page is being accessed. The page also contains many bool variables, which are used to determine whether the page is in the corresponding linked list or hash table.

struct buf_page_t {
    ...
    page_id_t      id;                  // page id
    page_size_t    size;                // page size
    ib_uint32_t    buf_fix_count;       // For concurrency control.
    buf_io_fix     io_fix;              // For concurrency control.
    buf_page_state state;               // page status
    lsn_t          newest_modification; // The latest lsn, or the most recently modified lsn.
    lsn_t          oldest_modification; // The earliest lsn, or the first modified lsn.
    ...
}

1.2 Initialization Process

The process of BP initialization is closely related to the data structure at each layer. The main line process of initialization is BP initialization > BP instance initialization > chunk initialization > block initialization. In addition, the initialization of structures such as AHI and page_hash are also completed in this process.

1.2.1 buf_pool_init():

This process mainly involves three operations. First, initialize the BP instance pointer array. No actual memory is allocated. Second, use buf_pool_create() concurrently with multiple threads to build actual memory space. Third, start the AHI initialization after the BP initialization is completed.

// 1. Construct an array of BP instance pointers.
buf_pool_ptr =
    (buf_pool_t *)ut_zalloc_nokey(n_instances * sizeof *buf_pool_ptr);
// 2. Initialize a BP instance concurrently with multiple threads.
for (ulint id = i; id < n; ++id) {
    threads.emplace_back(std::thread(buf_pool_create, &buf_pool_ptr[id], size,
                                     id, &m, std::ref(errs[id]))); }
// 3. Initialize AHI.
btr_search_sys_create(buf_pool_get_curr_size() / sizeof(void *) / 64);

The buf_pool_create() function is responsible for building each BP instance. It involves the following operations:

  1. Initialize various locks, including chunk mutex, LRU linked list lock, free linked list lock, zip_free linked list lock, and hash table lock.
  2. Calculate the number of chunks and apply an array of chunk pointers.
  3. Initialize these linked lists.
  4. Loop call buf_chunk_init() to initialize the chunk.
  5. Set instance parameters such as the size and instance number.
  6. Build a hash table and related locks.
  7. Initialize the flush data, such as Hp pointers and linked list containment relationships.
static void buf_pool_create(buf_pool_t *buf_pool, ulint buf_pool_size,
                            ulint instance_no, std::mutex *mutex,
                            dberr_t &err) {
  ...
  // 1. Build lock information.
  mutex_create(LATCH_ID_BUF_POOL_CHUNKS, &buf_pool->chunks_mutex);
  mutex_create(LATCH_ID_BUF_POOL_LRU_LIST, &buf_pool->LRU_list_mutex);
  ...
  // 2. Calculate the number of chunks.
  buf_pool->n_chunks = buf_pool_size / srv_buf_pool_chunk_unit;
  chunk_size = srv_buf_pool_chunk_unit;
  buf_pool->chunks = reinterpret_cast<buf_chunk_t *>(
        ut_zalloc_nokey(buf_pool->n_chunks * sizeof(*chunk)));
  ...
  // 3. Initialize each linked list.
  UT_LIST_INIT(buf_pool->LRU, &buf_page_t::LRU);
  UT_LIST_INIT(buf_pool->free, &buf_page_t::list)
  UT_LIST_INIT(buf_pool->flush_list, &buf_page_t::list);
  ...
  // 4. Initialize chunks.
  do {
    if (!buf_chunk_init(buf_pool, chunk, chunk_size, mutex)) {
      ...
  } while (++chunk < buf_pool->chunks + buf_pool->n_chunks);
  ...
  // 5. Set the instance parameter.
  buf_pool->instance_no = instance_no;
  buf_pool->curr_pool_size = buf_pool->curr_size * UNIV_PAGE_SIZE;
  ...
  // 6. Build page_hash and locks.
  srv_n_page_hash_locks =
      static_cast<ulong>(ut_2_power_up(srv_n_page_hash_locks));
  buf_pool->page_hash =
      ib_create(2 * buf_pool->curr_size, LATCH_ID_HASH_TABLE_RW_LOCK,
                srv_n_page_hash_locks, MEM_HEAP_FOR_PAGE_HASH);
  buf_pool->zip_hash = hash_create(2 * buf_pool->curr_size);
  ...
  // 7. Initialize flush information, such as the Hp pointer.
  for (i = BUF_FLUSH_LRU; i < BUF_FLUSH_N_TYPES; i++) {
    buf_pool->no_flush[i] = os_event_create();
  }
  ...
  new (&buf_pool->flush_hp) FlushHp(buf_pool, &buf_pool->flush_list_mutex);
  ...
}

1.2.2 buf_chunk_init():

This function involves the following operations:

  1. Use ut_allocator + large to allocate the memory required by chunks.
  2. Split chunks into blocks and frames.
  3. Call buf_block_init() to initialize the block and add it to the free linked list. Specifically, the function initializes variables in block/page, including the state, chunk_no, and whether they are in various flag bits of the linked list, building mutexes and rwlocks.
  4. Register chunks.
static buf_chunk_t *buf_chunk_init(...) 
{
  // 1. Apply for memory by using ut_allocator + large.
  ...
  if (!buf_pool->allocate_chunk(mem_size, chunk)) {
    return (nullptr);
  }
  ...
  // 2. Split chunks into blocks and frames.
  chunk->blocks = (buf_block_t *)chunk->mem;
  frame = (byte *)ut_align(chunk->mem, UNIV_PAGE_SIZE);
  chunk->size = chunk->mem_pfx.m_size / UNIV_PAGE_SIZE - (frame != chunk->mem);
  {
    ulint size = chunk->size;
    while (frame < (byte *)(chunk->blocks + size)) {
      frame += UNIV_PAGE_SIZE;
      size--;
    }
    chunk->size = size;
  }
  ...
  // 3. Initialize the page control body block.
  for () {
    buf_block_init(buf_pool, block, frame, chunk, sync_init_nolock);
    UT_LIST_ADD_LAST(buf_pool->free, &block->page);
  }
  // 4. Register chunks.
  buf_pool_register_chunk(chunk);
  ...
}

1.2.3 Initialization of page_hash

buf_pool->page_hash =
     ib_create(2 * buf_pool->curr_size, LATCH_ID_HASH_TABLE_RW_LOCK,
               srv_n_page_hash_locks, MEM_HEAP_FOR_PAGE_HASH);
buf_pool->page_hash_old = nullptr;
buf_pool->zip_hash = hash_create(2 * buf_pool->curr_size);

It can be seen that page_hash and zip_hash are built differently. The former calls ib_create(), while the latter calls hash_create() directly. ib_create() does more work on the mem_heap_t structure heaps and lock initialization than hash_create(). The ib_create() function is further described in the AHI structure in Part III.

1.3 Page Management Linked List

Each linked list in BP is a two-way linked list. The node type is buf_block_t, and the base node holds the first and last node information and the length of the linked list. The following figure shows the general structure.

3

The following figure shows the page usage in BP. Each small square can be regarded as a page. Free indicates a free page, that is, the data corresponding to the page is null, and new data can be filled in at any time. Clean indicates a clean page, that is, the page contains data and the data has not been updated. Dirty indicates a dirty page, that is, the page contains data that has been updated. Different types of pages co-exist in BP and are linked together by each linked list for collaborative management.

4

1.3.1 free list

After initialization, the block is directly added to the free linked list. If data pages need to be used in BP, they are directly obtained from the idle linked list. When idle nodes are insufficient, certain policies will be adopted to eliminate nodes from the LRU list and the flush list to supplement the pages in the free state.

1.3.2 LRU list

The LRU list is the most important data structure in BP. Basically, all the data pages read are buffered on it. The LRU linked list eliminates nodes according to the Least Recently Used algorithm. InnoDB optimizes the LRU algorithm as follows to solve the problems of "read-ahead failure" and "buffer pool pollution".

  • LRU Optimization

LRU is divided into two sections: Old Sublist and New Sublist. Data is first loaded to the old region, and is transferred to the new region only when certain conditions are met. When an operation similar to a full table scan occurs, the elimination of LRU does not affect the real hot data, thus ensuring the popularity of the cache.

5
Source: https://dev.mysql.com/doc/refman/8.0/en/innodb-buffer-pool.html

  • Response Time Optimization

Set an interval innodb_old_blocks_time, and then record in the corresponding control block the time when the old region of the data page is accessed for the first time:

  • If the interval between the subsequent access and the first access is less than the innodb_old_blocks_time, the cached page is not moved from the old region to the new region.
  • If the interval between the subsequent access and the first access is greater than innodb_old_blocks_time, the cached page is moved from the old region to the head of the new region.

1.3.3 flush list

All dirty pages in the buffer pool are mounted in the flush list to wait for data to be dropped to the disk. Data is likely to be modified multiple times before it is flushed to disk. The latest modified lsn (newset_modification) and the earliest modified lsn (oldest_modification) are recorded in the control body of the data page. The newly added data page is placed at the head of the linked list, and the data is refreshed from the end of the linked list. This means the page node added earliest is refreshed first.

1.3.4 zip_free

The structure is a two-dimensional array composed of five linked lists, which are 1 KB, 2 KB, 4 BK, 8 KB, and 16 KB in size. It stores compressed pages read from disk. The engine uses a partner system to manage the structure.

2. Change Buffer

A change buffer is a common B+ tree. When a secondary index page is not in BP, the changes generated by DML operations, such as insertions, updates, and deletions, are cached in the change buffer. The caches are merged at appropriate times to reduce disk I/O operations. The memory source of a change buffer is the buffer pool. You can set the maximum size by using the innodb_change_buffer_max_size parameter. The default value is 25%, and the maximum value is 50%.

6
Source: https://dev.mysql.com/doc/refman/8.0/en/innodb-change-buffer.html

In earlier versions, the change buffer only supports insertions, thus it is also known as the insert buffer. For this historical reason, the structure of the change buffer in the code is ibuf_t , which contains basic information such as the size, max_size, free_list_len, and the number of merge operations. There is only one ibuf_t structure globally, which is built when the database is started. The creation and initialization of the change buffer is completed in the ibuf_init_at_db_start() function. The process is as follows:

  1. Construction of related mutexes.
  2. Initialization of ibuf parameters, including max_size, and index.
  3. Acquisition of root.
void ibuf_init_at_db_start(void) {
  ...
  // 1. Build mutexes.
  mutex_create(LATCH_ID_IBUF, &ibuf_mutex);
  ...
    
  // 2. Build the root.
  {
    buf_block_t *block;
    // IBUF_SPACE_ID = 0, FSP_IBUF_TREE_ROOT_PAGE_NO = 4
    block = buf_page_get(page_id_t(IBUF_SPACE_ID, FSP_IBUF_TREE_ROOT_PAGE_NO),
                          univ_page_size, RW_X_LATCH, &mtr);
    buf_block_dbg_add_level(block, SYNC_IBUF_TREE_NODE);
    // The corresponding frame is used as the root of the change buffer B+ tree.
    root = buf_block_get_frame(block);
  }
  ...
  // 3. Set parameters.
  // The default value of CHANGE_BUFFER_DEFAULT_SIZE is 25.
  ibuf->max_size = ((buf_pool_get_curr_size() / UNIV_PAGE_SIZE) *
                    CHANGE_BUFFER_DEFAULT_SIZE) /
                    100;
  ibuf->index =
      dict_mem_index_create("innodb_change_buffer", "CLUST_IND", IBUF_SPACE_ID,
                            DICT_CLUSTERED | DICT_IBUF, 1);
  ibuf->index->id = DICT_IBUF_ID_MIN + IBUF_SPACE_ID;
  ibuf->index->table = dict_mem_table_create("innodb_change_buffer",
                                              IBUF_SPACE_ID, 1, 0, 0, 0, 0);
  ...
}

ibuf_insert() calls ibuf_insert_low() at the underlying layer. It involves the following operations:

  1. Add information such as page.no on the basis of data records according to the data construction.
  2. Select the appropriate block to insert. Data is inserted in rec, and the block contains the data of rec.
  3. Merge as appropriate.
static MY_ATTRIBUTE((warn_unused_result)) dberr_t
    ibuf_insert_low(ulint mode, ibuf_op_t op, ibool no_counter,
                    const dtuple_t *entry, ulint entry_size,
                    dict_index_t *index, const page_id_t &page_id,
                    const page_size_t &page_size, que_thr_t *thr) {
  ...
  // 1. Create an entry.
  ibuf_entry = ibuf_entry_build(op, index, entry, page_id.space(), page_id.page_no(),
                                no_counter ? ULINT_UNDEFINED : 0xFFFF, heap);
  ...
  // Initialize the cursor.
  btr_pcur_open(ibuf->index, ibuf_entry, PAGE_CUR_LE, mode, &pcur, &mtr);
  ...
        
  // 2. Insert.
  err=btr_cur_optimistic_insert (...);// It may also be btr_cur_pessimistic_insert.
  block = btr_cur_get_block(cursor);
  ...
  // Finish pcur, including clearing rec and block.
  btr_pcur_close(&pcur);
        
  // 3. Merge as appropriate.
}

The change buffer does not have many additional memory requests and relies on the blocks in BP to perform operations. Mostly, it applies for some temporary mem_heap_t, which is released immediately after use and will not reside in the memory for a long time.

3. AHI

The index structure of InnoDB is B-tree. When querying, the leaf nodes are indexed based on conditions. To reduce overhead, InnoDB introduces an Adaptive Hash Index (AHI) to create a hash table for the prefix of the index to speed up queries. AHI is established for index pages that are frequently accessed. You can construe it as an index on the B-tree that contains multiple hash tables. The initial size of the array is buf_pool_get_curr_size() / sizeof(void *) / 64. The memory is allocated by using malloc. The final size of the array corresponds to the total number of cells/buckets in the hash table. In fact, this number needs to be processed by a prime number.

3.1 Data Structure

7

struct hash_cell_t {
  void *node; /*!< The hash chain */
};
/* The hash table structure */
struct hash_table_t {
  enum hash_table_sync_t type; /*!< MUTEX/RW_LOCK/NONE. */
  ibool adaptive;     
  ulint n_cells;      /* The bucket array */
  hash_cell_t *cells; /*!< The number of hash buckets */
  ulint n_sync_obj; /* The number of mutexes and locks */
  union {
    ib_mutex_t *mutexes; 
    rw_lock_t *rw_locks; 
  } sync_obj;
  mem_heap_t **heaps; // The memory array that is used to allocate the hash chain when there are multiple parts. The number is related to n_sync_obj if it is used in page_hash.
  mem_heap_t *heap; // The memory heap that is used to allocate the hash chain.
};

3.2 Memory Initialization

When AHI is built, it is divided into 8 parts, each responsible for a different bucket and having its own lock. The construction and initialization are divided into the following steps:

  1. The initialization of the lock. The number of locks is related to the number of parts.
  2. The initialization of hash_table. Call ib_create() at the underlying layer. Note that the type specified here is MEM_HEAP_FOR_BTR_SEARCH, which directly determines the type of heap in the hash table, that is, the source of memory.
void btr_search_sys_create(ulint hash_size) {
  /* Step-1: Allocate latches (1 per part). */
  btr_search_latches = reinterpret_cast<rw_lock_t **>(
      ut_malloc(sizeof(rw_lock_t *) * btr_ahi_parts, mem_key_ahi));
  for (ulint i = 0; i < btr_ahi_parts; ++i) {
    btr_search_latches[i] = reinterpret_cast<rw_lock_t *>(
        ut_malloc(sizeof(rw_lock_t), mem_key_ahi));
    rw_lock_create(btr_search_latch_key, btr_search_latches[i],
                    SYNC_SEARCH_SYS);
  }
  /* Step-2: Allocate hash tablees. */
  btr_search_sys = reinterpret_cast<btr_search_sys_t *>(
      ut_malloc(sizeof(btr_search_sys_t), mem_key_ahi));
  btr_search_sys->hash_tables = reinterpret_cast<hash_table_t **>(
      ut_malloc(sizeof(hash_table_t *) * btr_ahi_parts, mem_key_ahi));
  for (ulint i = 0; i < btr_ahi_parts; ++i) { // Loop call ib_create().
    btr_search_sys->hash_tables[i] =
        ib_create((hash_size / btr_ahi_parts), LATCH_ID_HASH_TABLE_MUTEX, 0,
                  MEM_HEAP_FOR_BTR_SEARCH);
  ...
}

Further, ib_create involves two main operations:

  • Call hash_create() to create a hash table.

The hash_table() function initializes the parameters in most hash table structures to 0/nullptr. The most important thing is to construct hash_table->cells, that is, the hash bucket. Hash buckets are built through malloc and memset, which is the most time-consuming step in the AHI construction process.

  • Initialize table->heap

Select MEM_HEAP_FOR_BTR_SEARCH as the initialization type, and the heap is constructed to prepare for the subsequent memory allocation of the hash chain pointed to by the hash bucket. For the description and use of MEM_HEAP_FOR_BTR_SEARCH types, see Part I.

4. Others

4.1 Log Buffer

A log buffer is a cache of logs, the size of which is specified by the parameter innodb_log_buffer_size. Generally speaking, this memory size is relatively small. The default value is 16 MB, with max and min limits.

ut_allocate()/ut_free() is called at the underlying layer to apply for/release the memory of the log buffer, and the parameter srv_log_buffer_size is the required size.

// Apply for memory.
static void log_allocate_buffer(log_t &log) {
  ...
  log.buf.create(srv_log_buffer_size);
}
// Release the memory.
static void log_deallocate_buffer(log_t &log) { log.buf.destroy(); }

4.2 table cache

MySQL limits the number of open tables and the number of table structures in the memory. The process of open_table involves the server layer and the engine layer. This article only describes the actions involved at the InnoDB layer.

The function ha_innobase::open() starts to open tables at the InnoDB layer, mainly including the construction of dict_table_t and the row_prebuilt_t structure. The order of obtaining ib_table is session_cache, dict_sys->hash_table, and dd_open_table(). If ib_table fails to be obtained from the current cache, it will be obtained from the cache at the next level. If all caches are not hit, the table opening operation will be executed.

int ha_innobase::open(const char *name, int, uint open_flags,
                      const dd::Table *table_def) {
  ...
  // Table cache at the session level.
  ib_table = thd_to_innodb_session(thd)->lookup_table_handler(norm_name);
  
  ...
  // Global table cache at the dict_sys level.
  ib_table = dict_table_check_if_in_cache_low(norm_name);
  
  ...
  // The table does not exist in the cache. Open the table directly.
  ib_table = dd_open_table(client, table, norm_name, table_def, thd);
  
  ...
  // Build the m_prebuilt structure.
  m_prebuilt = row_create_prebuilt(ib_table, table->s->reclength);
  ...
}

The following describes the source memory of ib_table.

1.  session_table_cache

Each connection corresponds to a THD structure. The THD stores the thread_local data, which can be used to obtain the m_open_tables mapping table under the session. The table is inserted, deleted, and searched based on std::map.

class innodb_session_t {
  table_cache_t m_open_tables;
  ...
};

2.  dict_sys->table_hash

dict_sys can be considered as a global cache. The construction of the hash table in dicy_sys is completed when the dictionary_init () is executed to initialize the data dictionary, mainly including the construction of lock, table_LRU linked list, and table_hash. The hash table is also built through the hash_create() interface.

void dict_init(void) {
  ...
  dict_sys->table_hash = hash_create(
      buf_pool_get_curr_size() / (DICT_POOL_PER_TABLE_HASH * UNIV_WORD_SIZE));
  dict_sys->table_id_hash = hash_create(
      buf_pool_get_curr_size() / (DICT_POOL_PER_TABLE_HASH * UNIV_WORD_SIZE));
  ...
}

3.  dd_open_table()

After the attempt to obtain the cache table fails, dict_table_t is finally constructed through the dd_open_table() interface. The underlying layer calls dict_mem_table_create() and constructs all structures of dict_table_t through mem_heap_t. After the construction is completed, the latest table is cached in the hash_table of the dict.

dd_open_table
    |->dd_open_table_one
    |    |->dd_fill_dict_table //create dict_table_t
    |        |->dict_mem_table_create // create
    |           {
    |               // dict_table_t and internal memory such as col and locks are allocated from this heap. DICT_HEAP_SIZE=100,
    |               heap = mem_heap_create(DICT_HEAP_SIZE);
    |               ...
    |               table = static_cast<dict_table_t *>(mem_heap_zalloc(heap, sizeof(*table)));
    |               ...
    |               table->heap = heap;
    |               table->cols = static_cast<dict_col_t *>(
    |               mem_heap_alloc(heap, table->n_cols * sizeof(dict_col_t)));
    |               table->v_cols = static_cast<dict_v_col_t *>(
    |                  mem_heap_alloc(heap, n_v_cols * sizeof(*table->v_cols)));
    |               table->autoinc_lock =
    |                      static_cast<ib_lock_t *>(mem_heap_alloc(heap, lock_get_size()));
    |               ...
    |           }
    |->dict_table_add_to_cache(m_table, TRUE, heap); // Add to cache.

The total table cache at the server layer is related to the number of open tables and the field length. The memory occupied by each table cache varies from tens of KBs to hundreds of KBs.

4.3 lock_sys_t

The lock system is also built when InnoDB starts/creates. It mainly contains data such as row locks hash table, predicate locks hash table, and predicate page locks hash table. The main construction and destruction operations are as follows.

void lock_sys_create(ulint n_cells)
{
  ...
  lock_sys->rec_hash = hash_create(n_cells);
  lock_sys->prdt_hash = hash_create(n_cells);
  lock_sys->prdt_page_hash = hash_create(n_cells);
  ...
}
void lock_sys_close(void) {
  ...
  hash_table_free(lock_sys->rec_hash);
  hash_table_free(lock_sys->prdt_hash);
  hash_table_free(lock_sys->prdt_page_hash);
  ...
}

Memory is mainly consumed in the construction of the three hash tables. hash_create() is directly called for the "naked" construction, which does not involve the initialization of heaps. All memory is constructed by malloc. The required memory size of each hash_table is related to srv_lock_table_size, the value of which is specified when InnoDB starts: srv_lock_table_size = 5 * (srv_buf_pool_size / UNIV_PAGE_SIZE).

4.4 os_event_t

The construction and initialization of most locks and mutexes will eventually fall to the construction of os_event_t, but scattered temporary mutexes do not cause much memory pressure. As mentioned earlier, there are a large number of mutexes and rw_lock initializations in the initialization of buf_block_t, whose life cycle is equivalent to BP. The number of initializations is equal to buf_block_t, so it will occupy a large part of memory space.

  • buf_block_init:
/** Initializes a buffer control block when the buf_pool is created. */
static void buf_block_init(
    buf_pool_t *buf_pool, /*!< in: buffer pool instance */
    buf_block_t *block,   /*!< in: pointer to control block */
    byte *frame,          /*!< in: pointer to buffer frame */
    buf_chunk_t *chunk,   /*!< in: pointer to chunk */
    bool sync_init_nolock)
{
  ...
  mutex_create(LATCH_ID_BUF_BLOCK_MUTEX, &block->mutex); // Build a mutex.
  ...
  rw_lock_create(PFS_NOT_INSTRUMENTED, &block->lock, SYNC_LEVEL_VARYING); // Build rw_lock.
  ...
}
  • mutex_create
mutex_create()
    |->mutex_init()
        |->TTASEventMutex::init()
          |->os_event_create() // Build os_event_t.
  • rw_lock_create
rw_lock_create()
    |->pfs_rw_lock_create_func()
    |->rw_lock_create_func()
        |->os_event_create() // Build os_event_t.

The underlying layer is implemented by calling os_event_create(). The system ultimately allocates this part of memory.

os_event_t os_event_create() {
  os_event_t ret = (UT_NEW_NOKEY(os_event()));
  return ret;
}

5. Summary

Finally, the memory usage involved in InnoDB is briefly summarized in the following table. It can be seen that when specifying innodb_buffer_pool_size, there are many other parts of memory in addition to BP that are indirectly determined. If these parts of memory are not paid attention to, its specific memory usage is not easy to know.

table

BP refers to innodb_buffer_pool_size, round means that the allocated size needs to be rounded and aligned, and prime indicates that the number needs to be processed by a prime number.

  • The size specified by BP is finally reflected in the memory of the chunk. There may be differences between the actual memory and the specified size.
  • The AHI structure uses malloc to apply for cells. However, in some cases, data is stored in mem_heap_t. The mem_heap_t part of the memory in AHI is obtained from the BP due to the setting of the MEM_HEAP_FOR_BTR_SEARCH flag bit. There are multiple hash tables in AHI, which are currently built in loop mode. Parallel initialization can be considered to improve initialization efficiency. Theoretically, reducing hash conflicts can reduce the memory usage of mem_heap_t.
  • Many memory structures are related to the hash table, and the actual memory usage of the hash table needs to be processed by a prime number.
  • Most os_event_t are generated during the initialization of buf_block_t. This part occupies a large amount of memory and has a long lifecycle.
  • In the actual memory allocation, the system will also generate additional memory in addition to the specified BP size. This section only lists some of them.

References

  1. https://dev.mysql.com/doc/refman/8.0/en/innodb-architecture.html
  2. https://dev.mysql.com/doc/refman/8.0/en/innodb-buffer-pool.html
  3. http://mysql.taobao.org/monthly/2021/01/06/
  4. https://www.leviathan.vip/2018/12/18/InnoDB%E7%9A%84BufferPool%E5%88%86%E6%9E%90/
  5. https://juejin.cn/post/7109811386091307039
  6. https://juejin.cn/post/6882298660965580814
  7. https://juejin.cn/post/6974389300884570149
  8. https://dev.mysql.com/doc/refman/8.0/en/innodb-change-buffer.html
  9. Chapter 11, MySQL Kernel: The InnoDB Storage Engine
  10. https://juejin.cn/post/6892659139794157575
  11. https://dev.mysql.com/doc/refman/8.0/en/innodb-adaptive-hash.html
  12. https://dev.mysql.com/doc/refman/8.0/en/innodb-redo-log-buffer.html
  13. https://dev.mysql.com/doc/refman/8.0/en/memory-use.html

Next Part: https://www.alibabacloud.com/blog/600994

0 1 0
Share on

ApsaraDB

377 posts | 57 followers

You may also like

Comments