From ddb12d6714a5c28c87826180a331f5c68a66c5a5 Mon Sep 17 00:00:00 2001 From: Matthew Sakai Date: Thu, 16 Nov 2023 21:05:22 -0500 Subject: dm vdo: add the block map The block map contains the logical to physical mapping. It can be thought of as an array with one entry per logical address. Each entry is 5 bytes: 36 bits contain the physical block number which holds the data for the given logical address, and the remaining 4 bits are used to indicate the nature of the mapping. Of the 16 possible states, one represents a logical address which is unmapped (i.e. it has never been written, or has been discarded), one represents an uncompressed block, and the other 14 states are used to indicate that the mapped data is compressed, and which of the compression slots in the compressed block this logical address maps to. Co-developed-by: J. corwin Coburn Signed-off-by: J. corwin Coburn Co-developed-by: Michael Sclafani Signed-off-by: Michael Sclafani Co-developed-by: Sweet Tea Dorminy Signed-off-by: Sweet Tea Dorminy Signed-off-by: Matthew Sakai Signed-off-by: Mike Snitzer --- drivers/md/dm-vdo/block-map.h | 225 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 225 insertions(+) create mode 100644 drivers/md/dm-vdo/block-map.h (limited to 'drivers/md/dm-vdo/block-map.h') diff --git a/drivers/md/dm-vdo/block-map.h b/drivers/md/dm-vdo/block-map.h new file mode 100644 index 000000000000..eaf27d41af8b --- /dev/null +++ b/drivers/md/dm-vdo/block-map.h @@ -0,0 +1,225 @@ +/* SPDX-License-Identifier: GPL-2.0-only */ +/* + * Copyright 2023 Red Hat + */ + +#ifndef VDO_BLOCK_MAP_H +#define VDO_BLOCK_MAP_H + +#include + +#include "numeric.h" + +#include "admin-state.h" +#include "completion.h" +#include "encodings.h" +#include "int-map.h" +#include "statistics.h" +#include "types.h" +#include "vio.h" +#include "wait-queue.h" + +enum { + BLOCK_MAP_VIO_POOL_SIZE = 64, +}; + +/* + * Generation counter for page references. + */ +typedef u32 vdo_page_generation; + +extern const struct block_map_entry UNMAPPED_BLOCK_MAP_ENTRY; + +struct forest; + +struct tree_page { + struct waiter waiter; + + /* Dirty list entry */ + struct list_head entry; + + /* If dirty, the tree zone flush generation in which it was last dirtied. */ + u8 generation; + + /* Whether this page is an interior tree page being written out. */ + bool writing; + + /* If writing, the tree zone flush generation of the copy being written. */ + u8 writing_generation; + + /* + * Sequence number of the earliest recovery journal block containing uncommitted updates to + * this page + */ + sequence_number_t recovery_lock; + + /* The value of recovery_lock when the this page last started writing */ + sequence_number_t writing_recovery_lock; + + char page_buffer[VDO_BLOCK_SIZE]; +}; + +enum block_map_page_type { + VDO_TREE_PAGE, + VDO_CACHE_PAGE, +}; + +typedef struct list_head dirty_era_t[2]; + +struct dirty_lists { + /** The number of periods after which an element will be expired */ + block_count_t maximum_age; + /** The oldest period which has unexpired elements */ + sequence_number_t oldest_period; + /** One more than the current period */ + sequence_number_t next_period; + /** The offset in the array of lists of the oldest period */ + block_count_t offset; + /** Expired pages */ + dirty_era_t expired; + /** The lists of dirty pages */ + dirty_era_t eras[]; +}; + +struct block_map_zone { + zone_count_t zone_number; + thread_id_t thread_id; + struct admin_state state; + struct block_map *block_map; + /* Dirty pages, by era*/ + struct dirty_lists *dirty_lists; + struct vdo_page_cache page_cache; + data_vio_count_t active_lookups; + struct int_map *loading_pages; + struct vio_pool *vio_pool; + /* The tree page which has issued or will be issuing a flush */ + struct tree_page *flusher; + struct wait_queue flush_waiters; + /* The generation after the most recent flush */ + u8 generation; + u8 oldest_generation; + /* The counts of dirty pages in each generation */ + u32 dirty_page_counts[256]; +}; + +struct block_map { + struct vdo *vdo; + struct action_manager *action_manager; + /* The absolute PBN of the first root of the tree part of the block map */ + physical_block_number_t root_origin; + block_count_t root_count; + + /* The era point we are currently distributing to the zones */ + sequence_number_t current_era_point; + /* The next era point */ + sequence_number_t pending_era_point; + + /* The number of entries in block map */ + block_count_t entry_count; + nonce_t nonce; + struct recovery_journal *journal; + + /* The trees for finding block map pages */ + struct forest *forest; + /* The expanded trees awaiting growth */ + struct forest *next_forest; + /* The number of entries after growth */ + block_count_t next_entry_count; + + zone_count_t zone_count; + struct block_map_zone zones[]; +}; + +/** + * typedef vdo_entry_callback_fn - A function to be called for each allocated PBN when traversing + * the forest. + * @pbn: A PBN of a tree node. + * @completion: The parent completion of the traversal. + * + * Return: VDO_SUCCESS or an error. + */ +typedef int (*vdo_entry_callback_fn)(physical_block_number_t pbn, + struct vdo_completion *completion); + +static inline struct block_map_page * __must_check +vdo_as_block_map_page(struct tree_page *tree_page) +{ + return (struct block_map_page *) tree_page->page_buffer; +} + +bool vdo_copy_valid_page(char *buffer, nonce_t nonce, + physical_block_number_t pbn, + struct block_map_page *page); + +void vdo_find_block_map_slot(struct data_vio *data_vio); + +physical_block_number_t vdo_find_block_map_page_pbn(struct block_map *map, + page_number_t page_number); + +void vdo_write_tree_page(struct tree_page *page, struct block_map_zone *zone); + +void vdo_traverse_forest(struct block_map *map, vdo_entry_callback_fn callback, + struct vdo_completion *parent); + +int __must_check vdo_decode_block_map(struct block_map_state_2_0 state, + block_count_t logical_blocks, struct vdo *vdo, + struct recovery_journal *journal, nonce_t nonce, + page_count_t cache_size, block_count_t maximum_age, + struct block_map **map_ptr); + +void vdo_drain_block_map(struct block_map *map, const struct admin_state_code *operation, + struct vdo_completion *parent); + +void vdo_resume_block_map(struct block_map *map, struct vdo_completion *parent); + +int __must_check vdo_prepare_to_grow_block_map(struct block_map *map, + block_count_t new_logical_blocks); + +void vdo_grow_block_map(struct block_map *map, struct vdo_completion *parent); + +void vdo_abandon_block_map_growth(struct block_map *map); + +void vdo_free_block_map(struct block_map *map); + +struct block_map_state_2_0 __must_check vdo_record_block_map(const struct block_map *map); + +void vdo_initialize_block_map_from_journal(struct block_map *map, + struct recovery_journal *journal); + +zone_count_t vdo_compute_logical_zone(struct data_vio *data_vio); + +void vdo_advance_block_map_era(struct block_map *map, + sequence_number_t recovery_block_number); + +void vdo_update_block_map_page(struct block_map_page *page, struct data_vio *data_vio, + physical_block_number_t pbn, + enum block_mapping_state mapping_state, + sequence_number_t *recovery_lock); + +void vdo_get_mapped_block(struct data_vio *data_vio); + +void vdo_put_mapped_block(struct data_vio *data_vio); + +struct block_map_statistics __must_check vdo_get_block_map_statistics(struct block_map *map); + +/** + * vdo_convert_maximum_age() - Convert the maximum age to reflect the new recovery journal format + * @age: The configured maximum age + * + * Return: The converted age + * + * In the old recovery journal format, each journal block held 311 entries, and every write bio + * made two entries. The old maximum age was half the usable journal length. In the new format, + * each block holds only 217 entries, but each bio only makes one entry. We convert the configured + * age so that the number of writes in a block map era is the same in the old and new formats. This + * keeps the bound on the amount of work required to recover the block map from the recovery + * journal the same across the format change. It also keeps the amortization of block map page + * writes to write bios the same. + */ +static inline block_count_t vdo_convert_maximum_age(block_count_t age) +{ + return DIV_ROUND_UP(age * RECOVERY_JOURNAL_1_ENTRIES_PER_BLOCK, + 2 * RECOVERY_JOURNAL_ENTRIES_PER_BLOCK); +} + +#endif /* VDO_BLOCK_MAP_H */ -- cgit v1.2.3 From 14d531d7b7a124151c54f05e929d6e6fa896b68d Mon Sep 17 00:00:00 2001 From: Matthew Sakai Date: Thu, 16 Nov 2023 21:06:33 -0500 Subject: dm vdo: implement the block map page cache The set of leaf pages of the block map tree is too large to fit in memory, so each block map zone maintains a cache of leaf pages. This patch adds the implementation of that cache. Co-developed-by: J. corwin Coburn Signed-off-by: J. corwin Coburn Co-developed-by: Michael Sclafani Signed-off-by: Michael Sclafani Co-developed-by: Sweet Tea Dorminy Signed-off-by: Sweet Tea Dorminy Signed-off-by: Matthew Sakai Signed-off-by: Mike Snitzer --- drivers/md/dm-vdo/block-map.c | 1235 +++++++++++++++++++++++++++++++++++++++++ drivers/md/dm-vdo/block-map.h | 154 +++++ 2 files changed, 1389 insertions(+) (limited to 'drivers/md/dm-vdo/block-map.h') diff --git a/drivers/md/dm-vdo/block-map.c b/drivers/md/dm-vdo/block-map.c index bc50564f7316..e703cb9eafc6 100644 --- a/drivers/md/dm-vdo/block-map.c +++ b/drivers/md/dm-vdo/block-map.c @@ -103,6 +103,8 @@ struct cursors { struct cursor cursors[]; }; +static const physical_block_number_t NO_PAGE = 0xFFFFFFFFFFFFFFFF; + /* Used to indicate that the page holding the location of a tree root has been "loaded". */ static const physical_block_number_t VDO_INVALID_PBN = 0xFFFFFFFFFFFFFFFF; @@ -112,6 +114,1239 @@ const struct block_map_entry UNMAPPED_BLOCK_MAP_ENTRY = { .pbn_low_word = __cpu_to_le32(VDO_ZERO_BLOCK & UINT_MAX), }; +enum { + LOG_INTERVAL = 4000, + DISPLAY_INTERVAL = 100000, +}; + +/* + * For adjusting VDO page cache statistic fields which are only mutated on the logical zone thread. + * Prevents any compiler shenanigans from affecting other threads reading those stats. + */ +#define ADD_ONCE(value, delta) WRITE_ONCE(value, (value) + (delta)) + +static inline bool is_dirty(const struct page_info *info) +{ + return info->state == PS_DIRTY; +} + +static inline bool is_present(const struct page_info *info) +{ + return (info->state == PS_RESIDENT) || (info->state == PS_DIRTY); +} + +static inline bool is_in_flight(const struct page_info *info) +{ + return (info->state == PS_INCOMING) || (info->state == PS_OUTGOING); +} + +static inline bool is_incoming(const struct page_info *info) +{ + return info->state == PS_INCOMING; +} + +static inline bool is_outgoing(const struct page_info *info) +{ + return info->state == PS_OUTGOING; +} + +static inline bool is_valid(const struct page_info *info) +{ + return is_present(info) || is_outgoing(info); +} + +static char *get_page_buffer(struct page_info *info) +{ + struct vdo_page_cache *cache = info->cache; + + return &cache->pages[(info - cache->infos) * VDO_BLOCK_SIZE]; +} + +static inline struct vdo_page_completion *page_completion_from_waiter(struct waiter *waiter) +{ + struct vdo_page_completion *completion; + + if (waiter == NULL) + return NULL; + + completion = container_of(waiter, struct vdo_page_completion, waiter); + vdo_assert_completion_type(&completion->completion, VDO_PAGE_COMPLETION); + return completion; +} + +/** + * initialize_info() - Initialize all page info structures and put them on the free list. + * + * Return: VDO_SUCCESS or an error. + */ +static int initialize_info(struct vdo_page_cache *cache) +{ + struct page_info *info; + + INIT_LIST_HEAD(&cache->free_list); + for (info = cache->infos; info < cache->infos + cache->page_count; info++) { + int result; + + info->cache = cache; + info->state = PS_FREE; + info->pbn = NO_PAGE; + + result = create_metadata_vio(cache->vdo, VIO_TYPE_BLOCK_MAP, + VIO_PRIORITY_METADATA, info, + get_page_buffer(info), &info->vio); + if (result != VDO_SUCCESS) + return result; + + /* The thread ID should never change. */ + info->vio->completion.callback_thread_id = cache->zone->thread_id; + + INIT_LIST_HEAD(&info->state_entry); + list_add_tail(&info->state_entry, &cache->free_list); + INIT_LIST_HEAD(&info->lru_entry); + } + + return VDO_SUCCESS; +} + +/** + * allocate_cache_components() - Allocate components of the cache which require their own + * allocation. + * @maximum_age: The number of journal blocks before a dirtied page is considered old and must be + * written out. + * + * The caller is responsible for all clean up on errors. + * + * Return: VDO_SUCCESS or an error code. + */ +static int __must_check allocate_cache_components(struct vdo_page_cache *cache) +{ + u64 size = cache->page_count * (u64) VDO_BLOCK_SIZE; + int result; + + result = uds_allocate(cache->page_count, struct page_info, "page infos", + &cache->infos); + if (result != UDS_SUCCESS) + return result; + + result = uds_allocate_memory(size, VDO_BLOCK_SIZE, "cache pages", &cache->pages); + if (result != UDS_SUCCESS) + return result; + + result = vdo_make_int_map(cache->page_count, 0, &cache->page_map); + if (result != UDS_SUCCESS) + return result; + + return initialize_info(cache); +} + +/** + * assert_on_cache_thread() - Assert that a function has been called on the VDO page cache's + * thread. + */ +static inline void assert_on_cache_thread(struct vdo_page_cache *cache, + const char *function_name) +{ + thread_id_t thread_id = vdo_get_callback_thread_id(); + + ASSERT_LOG_ONLY((thread_id == cache->zone->thread_id), + "%s() must only be called on cache thread %d, not thread %d", + function_name, cache->zone->thread_id, thread_id); +} + +/** assert_io_allowed() - Assert that a page cache may issue I/O. */ +static inline void assert_io_allowed(struct vdo_page_cache *cache) +{ + ASSERT_LOG_ONLY(!vdo_is_state_quiescent(&cache->zone->state), + "VDO page cache may issue I/O"); +} + +/** report_cache_pressure() - Log and, if enabled, report cache pressure. */ +static void report_cache_pressure(struct vdo_page_cache *cache) +{ + ADD_ONCE(cache->stats.cache_pressure, 1); + if (cache->waiter_count > cache->page_count) { + if ((cache->pressure_report % LOG_INTERVAL) == 0) + uds_log_info("page cache pressure %u", cache->stats.cache_pressure); + + if (++cache->pressure_report >= DISPLAY_INTERVAL) + cache->pressure_report = 0; + } +} + +/** + * get_page_state_name() - Return the name of a page state. + * + * If the page state is invalid a static string is returned and the invalid state is logged. + * + * Return: A pointer to a static page state name. + */ +static const char * __must_check get_page_state_name(enum vdo_page_buffer_state state) +{ + int result; + static const char * const state_names[] = { + "UDS_FREE", "INCOMING", "FAILED", "RESIDENT", "DIRTY", "OUTGOING" + }; + + BUILD_BUG_ON(ARRAY_SIZE(state_names) != PAGE_STATE_COUNT); + + result = ASSERT(state < ARRAY_SIZE(state_names), + "Unknown page_state value %d", state); + if (result != UDS_SUCCESS) + return "[UNKNOWN PAGE STATE]"; + + return state_names[state]; +} + +/** + * update_counter() - Update the counter associated with a given state. + * @info: The page info to count. + * @delta: The delta to apply to the counter. + */ +static void update_counter(struct page_info *info, s32 delta) +{ + struct block_map_statistics *stats = &info->cache->stats; + + switch (info->state) { + case PS_FREE: + ADD_ONCE(stats->free_pages, delta); + return; + + case PS_INCOMING: + ADD_ONCE(stats->incoming_pages, delta); + return; + + case PS_OUTGOING: + ADD_ONCE(stats->outgoing_pages, delta); + return; + + case PS_FAILED: + ADD_ONCE(stats->failed_pages, delta); + return; + + case PS_RESIDENT: + ADD_ONCE(stats->clean_pages, delta); + return; + + case PS_DIRTY: + ADD_ONCE(stats->dirty_pages, delta); + return; + + default: + return; + } +} + +/** update_lru() - Update the lru information for an active page. */ +static void update_lru(struct page_info *info) +{ + if (info->cache->lru_list.prev != &info->lru_entry) + list_move_tail(&info->lru_entry, &info->cache->lru_list); +} + +/** + * set_info_state() - Set the state of a page_info and put it on the right list, adjusting + * counters. + */ +static void set_info_state(struct page_info *info, enum vdo_page_buffer_state new_state) +{ + if (new_state == info->state) + return; + + update_counter(info, -1); + info->state = new_state; + update_counter(info, 1); + + switch (info->state) { + case PS_FREE: + case PS_FAILED: + list_move_tail(&info->state_entry, &info->cache->free_list); + return; + + case PS_OUTGOING: + list_move_tail(&info->state_entry, &info->cache->outgoing_list); + return; + + case PS_DIRTY: + return; + + default: + list_del_init(&info->state_entry); + } +} + +/** set_info_pbn() - Set the pbn for an info, updating the map as needed. */ +static int __must_check set_info_pbn(struct page_info *info, physical_block_number_t pbn) +{ + struct vdo_page_cache *cache = info->cache; + + /* Either the new or the old page number must be NO_PAGE. */ + int result = ASSERT((pbn == NO_PAGE) || (info->pbn == NO_PAGE), + "Must free a page before reusing it."); + if (result != VDO_SUCCESS) + return result; + + if (info->pbn != NO_PAGE) + vdo_int_map_remove(cache->page_map, info->pbn); + + info->pbn = pbn; + + if (pbn != NO_PAGE) { + result = vdo_int_map_put(cache->page_map, pbn, info, true, NULL); + if (result != UDS_SUCCESS) + return result; + } + return VDO_SUCCESS; +} + +/** reset_page_info() - Reset page info to represent an unallocated page. */ +static int reset_page_info(struct page_info *info) +{ + int result; + + result = ASSERT(info->busy == 0, "VDO Page must not be busy"); + if (result != UDS_SUCCESS) + return result; + + result = ASSERT(!vdo_has_waiters(&info->waiting), + "VDO Page must not have waiters"); + if (result != UDS_SUCCESS) + return result; + + result = set_info_pbn(info, NO_PAGE); + set_info_state(info, PS_FREE); + list_del_init(&info->lru_entry); + return result; +} + +/** + * find_free_page() - Find a free page. + * + * Return: A pointer to the page info structure (if found), NULL otherwise. + */ +static struct page_info * __must_check find_free_page(struct vdo_page_cache *cache) +{ + struct page_info *info; + + info = list_first_entry_or_null(&cache->free_list, struct page_info, + state_entry); + if (info != NULL) + list_del_init(&info->state_entry); + + return info; +} + +/** + * find_page() - Find the page info (if any) associated with a given pbn. + * @pbn: The absolute physical block number of the page. + * + * Return: The page info for the page if available, or NULL if not. + */ +static struct page_info * __must_check find_page(struct vdo_page_cache *cache, + physical_block_number_t pbn) +{ + if ((cache->last_found != NULL) && (cache->last_found->pbn == pbn)) + return cache->last_found; + + cache->last_found = vdo_int_map_get(cache->page_map, pbn); + return cache->last_found; +} + +/** + * select_lru_page() - Determine which page is least recently used. + * + * Picks the least recently used from among the non-busy entries at the front of each of the lru + * ring. Since whenever we mark a page busy we also put it to the end of the ring it is unlikely + * that the entries at the front are busy unless the queue is very short, but not impossible. + * + * Return: A pointer to the info structure for a relevant page, or NULL if no such page can be + * found. The page can be dirty or resident. + */ +static struct page_info * __must_check select_lru_page(struct vdo_page_cache *cache) +{ + struct page_info *info; + + list_for_each_entry(info, &cache->lru_list, lru_entry) + if ((info->busy == 0) && !is_in_flight(info)) + return info; + + return NULL; +} + +/* ASYNCHRONOUS INTERFACE BEYOND THIS POINT */ + +/** + * complete_with_page() - Helper to complete the VDO Page Completion request successfully. + * @info: The page info representing the result page. + * @vdo_page_comp: The VDO page completion to complete. + */ +static void complete_with_page(struct page_info *info, + struct vdo_page_completion *vdo_page_comp) +{ + bool available = vdo_page_comp->writable ? is_present(info) : is_valid(info); + + if (!available) { + uds_log_error_strerror(VDO_BAD_PAGE, + "Requested cache page %llu in state %s is not %s", + (unsigned long long) info->pbn, + get_page_state_name(info->state), + vdo_page_comp->writable ? "present" : "valid"); + vdo_fail_completion(&vdo_page_comp->completion, VDO_BAD_PAGE); + return; + } + + vdo_page_comp->info = info; + vdo_page_comp->ready = true; + vdo_finish_completion(&vdo_page_comp->completion); +} + +/** + * complete_waiter_with_error() - Complete a page completion with an error code. + * @waiter: The page completion, as a waiter. + * @result_ptr: A pointer to the error code. + * + * Implements waiter_callback_fn. + */ +static void complete_waiter_with_error(struct waiter *waiter, void *result_ptr) +{ + int *result = result_ptr; + + vdo_fail_completion(&page_completion_from_waiter(waiter)->completion, *result); +} + +/** + * complete_waiter_with_page() - Complete a page completion with a page. + * @waiter: The page completion, as a waiter. + * @page_info: The page info to complete with. + * + * Implements waiter_callback_fn. + */ +static void complete_waiter_with_page(struct waiter *waiter, void *page_info) +{ + complete_with_page(page_info, page_completion_from_waiter(waiter)); +} + +/** + * distribute_page_over_queue() - Complete a queue of VDO page completions with a page result. + * + * Upon completion the queue will be empty. + * + * Return: The number of pages distributed. + */ +static unsigned int distribute_page_over_queue(struct page_info *info, + struct wait_queue *queue) +{ + size_t pages; + + update_lru(info); + pages = vdo_count_waiters(queue); + + /* + * Increment the busy count once for each pending completion so that this page does not + * stop being busy until all completions have been processed (VDO-83). + */ + info->busy += pages; + + vdo_notify_all_waiters(queue, complete_waiter_with_page, info); + return pages; +} + +/** + * set_persistent_error() - Set a persistent error which all requests will receive in the future. + * @context: A string describing what triggered the error. + * + * Once triggered, all enqueued completions will get this error. Any future requests will result in + * this error as well. + */ +static void set_persistent_error(struct vdo_page_cache *cache, const char *context, + int result) +{ + struct page_info *info; + /* If we're already read-only, there's no need to log. */ + struct vdo *vdo = cache->zone->block_map->vdo; + + if ((result != VDO_READ_ONLY) && !vdo_is_read_only(vdo)) { + uds_log_error_strerror(result, "VDO Page Cache persistent error: %s", + context); + vdo_enter_read_only_mode(vdo, result); + } + + assert_on_cache_thread(cache, __func__); + + vdo_notify_all_waiters(&cache->free_waiters, complete_waiter_with_error, + &result); + cache->waiter_count = 0; + + for (info = cache->infos; info < cache->infos + cache->page_count; info++) + vdo_notify_all_waiters(&info->waiting, complete_waiter_with_error, + &result); +} + +/** + * validate_completed_page() - Check that a page completion which is being freed to the cache + * referred to a valid page and is in a valid state. + * @writable: Whether a writable page is required. + * + * Return: VDO_SUCCESS if the page was valid, otherwise as error + */ +static int __must_check validate_completed_page(struct vdo_page_completion *completion, + bool writable) +{ + int result; + + result = ASSERT(completion->ready, "VDO Page completion not ready"); + if (result != UDS_SUCCESS) + return result; + + result = ASSERT(completion->info != NULL, + "VDO Page Completion must be complete"); + if (result != UDS_SUCCESS) + return result; + + result = ASSERT(completion->info->pbn == completion->pbn, + "VDO Page Completion pbn must be consistent"); + if (result != UDS_SUCCESS) + return result; + + result = ASSERT(is_valid(completion->info), + "VDO Page Completion page must be valid"); + if (result != UDS_SUCCESS) + return result; + + if (writable) { + result = ASSERT(completion->writable, "VDO Page Completion is writable"); + if (result != UDS_SUCCESS) + return result; + } + + return VDO_SUCCESS; +} + +static void check_for_drain_complete(struct block_map_zone *zone) +{ + if (vdo_is_state_draining(&zone->state) && + (zone->active_lookups == 0) && + !vdo_has_waiters(&zone->flush_waiters) && + !is_vio_pool_busy(zone->vio_pool) && + (zone->page_cache.outstanding_reads == 0) && + (zone->page_cache.outstanding_writes == 0)) { + vdo_finish_draining_with_result(&zone->state, + (vdo_is_read_only(zone->block_map->vdo) ? + VDO_READ_ONLY : VDO_SUCCESS)); + } +} + +static void enter_zone_read_only_mode(struct block_map_zone *zone, int result) +{ + vdo_enter_read_only_mode(zone->block_map->vdo, result); + + /* + * We are in read-only mode, so we won't ever write any page out. Just take all waiters off + * the queue so the zone can drain. + */ + while (vdo_has_waiters(&zone->flush_waiters)) + vdo_dequeue_next_waiter(&zone->flush_waiters); + + check_for_drain_complete(zone); +} + +static bool __must_check +validate_completed_page_or_enter_read_only_mode(struct vdo_page_completion *completion, + bool writable) +{ + int result = validate_completed_page(completion, writable); + + if (result == VDO_SUCCESS) + return true; + + enter_zone_read_only_mode(completion->info->cache->zone, result); + return false; +} + +/** + * handle_load_error() - Handle page load errors. + * @completion: The page read vio. + */ +static void handle_load_error(struct vdo_completion *completion) +{ + int result = completion->result; + struct page_info *info = completion->parent; + struct vdo_page_cache *cache = info->cache; + + assert_on_cache_thread(cache, __func__); + vio_record_metadata_io_error(as_vio(completion)); + vdo_enter_read_only_mode(cache->zone->block_map->vdo, result); + ADD_ONCE(cache->stats.failed_reads, 1); + set_info_state(info, PS_FAILED); + vdo_notify_all_waiters(&info->waiting, complete_waiter_with_error, &result); + reset_page_info(info); + + /* + * Don't decrement until right before calling check_for_drain_complete() to + * ensure that the above work can't cause the page cache to be freed out from under us. + */ + cache->outstanding_reads--; + check_for_drain_complete(cache->zone); +} + +/** + * page_is_loaded() - Callback used when a page has been loaded. + * @completion: The vio which has loaded the page. Its parent is the page_info. + */ +static void page_is_loaded(struct vdo_completion *completion) +{ + struct page_info *info = completion->parent; + struct vdo_page_cache *cache = info->cache; + nonce_t nonce = info->cache->zone->block_map->nonce; + struct block_map_page *page; + enum block_map_page_validity validity; + + assert_on_cache_thread(cache, __func__); + + page = (struct block_map_page *) get_page_buffer(info); + validity = vdo_validate_block_map_page(page, nonce, info->pbn); + if (validity == VDO_BLOCK_MAP_PAGE_BAD) { + physical_block_number_t pbn = vdo_get_block_map_page_pbn(page); + int result = uds_log_error_strerror(VDO_BAD_PAGE, + "Expected page %llu but got page %llu instead", + (unsigned long long) info->pbn, + (unsigned long long) pbn); + + vdo_continue_completion(completion, result); + return; + } + + if (validity == VDO_BLOCK_MAP_PAGE_INVALID) + vdo_format_block_map_page(page, nonce, info->pbn, false); + + info->recovery_lock = 0; + set_info_state(info, PS_RESIDENT); + distribute_page_over_queue(info, &info->waiting); + + /* + * Don't decrement until right before calling check_for_drain_complete() to + * ensure that the above work can't cause the page cache to be freed out from under us. + */ + cache->outstanding_reads--; + check_for_drain_complete(cache->zone); +} + +/** + * handle_rebuild_read_error() - Handle a read error during a read-only rebuild. + * @completion: The page load completion. + */ +static void handle_rebuild_read_error(struct vdo_completion *completion) +{ + struct page_info *info = completion->parent; + struct vdo_page_cache *cache = info->cache; + + assert_on_cache_thread(cache, __func__); + + /* + * We are doing a read-only rebuild, so treat this as a successful read of an uninitialized + * page. + */ + vio_record_metadata_io_error(as_vio(completion)); + ADD_ONCE(cache->stats.failed_reads, 1); + memset(get_page_buffer(info), 0, VDO_BLOCK_SIZE); + vdo_reset_completion(completion); + page_is_loaded(completion); +} + +static void load_cache_page_endio(struct bio *bio) +{ + struct vio *vio = bio->bi_private; + struct page_info *info = vio->completion.parent; + + continue_vio_after_io(vio, page_is_loaded, info->cache->zone->thread_id); +} + +/** + * launch_page_load() - Begin the process of loading a page. + * + * Return: VDO_SUCCESS or an error code. + */ +static int __must_check launch_page_load(struct page_info *info, + physical_block_number_t pbn) +{ + int result; + vdo_action_fn callback; + struct vdo_page_cache *cache = info->cache; + + assert_io_allowed(cache); + + result = set_info_pbn(info, pbn); + if (result != VDO_SUCCESS) + return result; + + result = ASSERT((info->busy == 0), "Page is not busy before loading."); + if (result != VDO_SUCCESS) + return result; + + set_info_state(info, PS_INCOMING); + cache->outstanding_reads++; + ADD_ONCE(cache->stats.pages_loaded, 1); + callback = (cache->rebuilding ? handle_rebuild_read_error : handle_load_error); + submit_metadata_vio(info->vio, pbn, load_cache_page_endio, + callback, REQ_OP_READ | REQ_PRIO); + return VDO_SUCCESS; +} + +static void write_pages(struct vdo_completion *completion); + +/** handle_flush_error() - Handle errors flushing the layer. */ +static void handle_flush_error(struct vdo_completion *completion) +{ + struct page_info *info = completion->parent; + + vio_record_metadata_io_error(as_vio(completion)); + set_persistent_error(info->cache, "flush failed", completion->result); + write_pages(completion); +} + +static void flush_endio(struct bio *bio) +{ + struct vio *vio = bio->bi_private; + struct page_info *info = vio->completion.parent; + + continue_vio_after_io(vio, write_pages, info->cache->zone->thread_id); +} + +/** save_pages() - Attempt to save the outgoing pages by first flushing the layer. */ +static void save_pages(struct vdo_page_cache *cache) +{ + struct page_info *info; + struct vio *vio; + + if ((cache->pages_in_flush > 0) || (cache->pages_to_flush == 0)) + return; + + assert_io_allowed(cache); + + info = list_first_entry(&cache->outgoing_list, struct page_info, state_entry); + + cache->pages_in_flush = cache->pages_to_flush; + cache->pages_to_flush = 0; + ADD_ONCE(cache->stats.flush_count, 1); + + vio = info->vio; + + /* + * We must make sure that the recovery journal entries that changed these pages were + * successfully persisted, and thus must issue a flush before each batch of pages is + * written to ensure this. + */ + submit_flush_vio(vio, flush_endio, handle_flush_error); +} + +/** + * schedule_page_save() - Add a page to the outgoing list of pages waiting to be saved. + * + * Once in the list, a page may not be used until it has been written out. + */ +static void schedule_page_save(struct page_info *info) +{ + if (info->busy > 0) { + info->write_status = WRITE_STATUS_DEFERRED; + return; + } + + info->cache->pages_to_flush++; + info->cache->outstanding_writes++; + set_info_state(info, PS_OUTGOING); +} + +/** + * launch_page_save() - Add a page to outgoing pages waiting to be saved, and then start saving + * pages if another save is not in progress. + */ +static void launch_page_save(struct page_info *info) +{ + schedule_page_save(info); + save_pages(info->cache); +} + +/** + * completion_needs_page() - Determine whether a given vdo_page_completion (as a waiter) is + * requesting a given page number. + * @context: A pointer to the pbn of the desired page. + * + * Implements waiter_match_fn. + * + * Return: true if the page completion is for the desired page number. + */ +static bool completion_needs_page(struct waiter *waiter, void *context) +{ + physical_block_number_t *pbn = context; + + return (page_completion_from_waiter(waiter)->pbn == *pbn); +} + +/** + * allocate_free_page() - Allocate a free page to the first completion in the waiting queue, and + * any other completions that match it in page number. + */ +static void allocate_free_page(struct page_info *info) +{ + int result; + struct waiter *oldest_waiter; + physical_block_number_t pbn; + struct vdo_page_cache *cache = info->cache; + + assert_on_cache_thread(cache, __func__); + + if (!vdo_has_waiters(&cache->free_waiters)) { + if (cache->stats.cache_pressure > 0) { + uds_log_info("page cache pressure relieved"); + WRITE_ONCE(cache->stats.cache_pressure, 0); + } + + return; + } + + result = reset_page_info(info); + if (result != VDO_SUCCESS) { + set_persistent_error(cache, "cannot reset page info", result); + return; + } + + oldest_waiter = vdo_get_first_waiter(&cache->free_waiters); + pbn = page_completion_from_waiter(oldest_waiter)->pbn; + + /* + * Remove all entries which match the page number in question and push them onto the page + * info's wait queue. + */ + vdo_dequeue_matching_waiters(&cache->free_waiters, completion_needs_page, + &pbn, &info->waiting); + cache->waiter_count -= vdo_count_waiters(&info->waiting); + + result = launch_page_load(info, pbn); + if (result != VDO_SUCCESS) + vdo_notify_all_waiters(&info->waiting, complete_waiter_with_error, &result); +} + +/** + * discard_a_page() - Begin the process of discarding a page. + * + * If no page is discardable, increments a count of deferred frees so that the next release of a + * page which is no longer busy will kick off another discard cycle. This is an indication that the + * cache is not big enough. + * + * If the selected page is not dirty, immediately allocates the page to the oldest completion + * waiting for a free page. + */ +static void discard_a_page(struct vdo_page_cache *cache) +{ + struct page_info *info = select_lru_page(cache); + + if (info == NULL) { + report_cache_pressure(cache); + return; + } + + if (!is_dirty(info)) { + allocate_free_page(info); + return; + } + + ASSERT_LOG_ONLY(!is_in_flight(info), + "page selected for discard is not in flight"); + + cache->discard_count++; + info->write_status = WRITE_STATUS_DISCARD; + launch_page_save(info); +} + +/** + * discard_page_for_completion() - Helper used to trigger a discard so that the completion can get + * a different page. + */ +static void discard_page_for_completion(struct vdo_page_completion *vdo_page_comp) +{ + struct vdo_page_cache *cache = vdo_page_comp->cache; + + cache->waiter_count++; + vdo_enqueue_waiter(&cache->free_waiters, &vdo_page_comp->waiter); + discard_a_page(cache); +} + +/** + * discard_page_if_needed() - Helper used to trigger a discard if the cache needs another free + * page. + * @cache: The page cache. + */ +static void discard_page_if_needed(struct vdo_page_cache *cache) +{ + if (cache->waiter_count > cache->discard_count) + discard_a_page(cache); +} + +/** + * write_has_finished() - Inform the cache that a write has finished (possibly with an error). + * @info: The info structure for the page whose write just completed. + * + * Return: true if the page write was a discard. + */ +static bool write_has_finished(struct page_info *info) +{ + bool was_discard = (info->write_status == WRITE_STATUS_DISCARD); + + assert_on_cache_thread(info->cache, __func__); + info->cache->outstanding_writes--; + + info->write_status = WRITE_STATUS_NORMAL; + return was_discard; +} + +/** + * handle_page_write_error() - Handler for page write errors. + * @completion: The page write vio. + */ +static void handle_page_write_error(struct vdo_completion *completion) +{ + int result = completion->result; + struct page_info *info = completion->parent; + struct vdo_page_cache *cache = info->cache; + + vio_record_metadata_io_error(as_vio(completion)); + + /* If we're already read-only, write failures are to be expected. */ + if (result != VDO_READ_ONLY) { + static DEFINE_RATELIMIT_STATE(error_limiter, DEFAULT_RATELIMIT_INTERVAL, + DEFAULT_RATELIMIT_BURST); + + if (__ratelimit(&error_limiter)) { + uds_log_error("failed to write block map page %llu", + (unsigned long long) info->pbn); + } + } + + set_info_state(info, PS_DIRTY); + ADD_ONCE(cache->stats.failed_writes, 1); + set_persistent_error(cache, "cannot write page", result); + + if (!write_has_finished(info)) + discard_page_if_needed(cache); + + check_for_drain_complete(cache->zone); +} + +static void page_is_written_out(struct vdo_completion *completion); + +static void write_cache_page_endio(struct bio *bio) +{ + struct vio *vio = bio->bi_private; + struct page_info *info = vio->completion.parent; + + continue_vio_after_io(vio, page_is_written_out, info->cache->zone->thread_id); +} + +/** + * page_is_written_out() - Callback used when a page has been written out. + * @completion: The vio which wrote the page. Its parent is a page_info. + */ +static void page_is_written_out(struct vdo_completion *completion) +{ + bool was_discard, reclaimed; + u32 reclamations; + struct page_info *info = completion->parent; + struct vdo_page_cache *cache = info->cache; + struct block_map_page *page = (struct block_map_page *) get_page_buffer(info); + + if (!page->header.initialized) { + page->header.initialized = true; + submit_metadata_vio(info->vio, info->pbn, + write_cache_page_endio, + handle_page_write_error, + (REQ_OP_WRITE | REQ_PRIO | REQ_PREFLUSH)); + return; + } + + /* Handle journal updates and torn write protection. */ + vdo_release_recovery_journal_block_reference(cache->zone->block_map->journal, + info->recovery_lock, + VDO_ZONE_TYPE_LOGICAL, + cache->zone->zone_number); + info->recovery_lock = 0; + was_discard = write_has_finished(info); + reclaimed = (!was_discard || (info->busy > 0) || vdo_has_waiters(&info->waiting)); + + set_info_state(info, PS_RESIDENT); + + reclamations = distribute_page_over_queue(info, &info->waiting); + ADD_ONCE(cache->stats.reclaimed, reclamations); + + if (was_discard) + cache->discard_count--; + + if (reclaimed) + discard_page_if_needed(cache); + else + allocate_free_page(info); + + check_for_drain_complete(cache->zone); +} + +/** + * write_pages() - Write the batch of pages which were covered by the layer flush which just + * completed. + * @flush_completion: The flush vio. + * + * This callback is registered in save_pages(). + */ +static void write_pages(struct vdo_completion *flush_completion) +{ + struct vdo_page_cache *cache = ((struct page_info *) flush_completion->parent)->cache; + + /* + * We need to cache these two values on the stack since in the error case below, it is + * possible for the last page info to cause the page cache to get freed. Hence once we + * launch the last page, it may be unsafe to dereference the cache [VDO-4724]. + */ + bool has_unflushed_pages = (cache->pages_to_flush > 0); + page_count_t pages_in_flush = cache->pages_in_flush; + + cache->pages_in_flush = 0; + while (pages_in_flush-- > 0) { + struct page_info *info = + list_first_entry(&cache->outgoing_list, struct page_info, + state_entry); + + list_del_init(&info->state_entry); + if (vdo_is_read_only(info->cache->zone->block_map->vdo)) { + struct vdo_completion *completion = &info->vio->completion; + + vdo_reset_completion(completion); + completion->callback = page_is_written_out; + completion->error_handler = handle_page_write_error; + vdo_fail_completion(completion, VDO_READ_ONLY); + continue; + } + ADD_ONCE(info->cache->stats.pages_saved, 1); + submit_metadata_vio(info->vio, info->pbn, write_cache_page_endio, + handle_page_write_error, REQ_OP_WRITE | REQ_PRIO); + } + + if (has_unflushed_pages) { + /* + * If there are unflushed pages, the cache can't have been freed, so this call is + * safe. + */ + save_pages(cache); + } +} + +/** + * vdo_release_page_completion() - Release a VDO Page Completion. + * + * The page referenced by this completion (if any) will no longer be held busy by this completion. + * If a page becomes discardable and there are completions awaiting free pages then a new round of + * page discarding is started. + */ +void vdo_release_page_completion(struct vdo_completion *completion) +{ + struct page_info *discard_info = NULL; + struct vdo_page_completion *page_completion = as_vdo_page_completion(completion); + struct vdo_page_cache *cache; + + if (completion->result == VDO_SUCCESS) { + if (!validate_completed_page_or_enter_read_only_mode(page_completion, false)) + return; + + if (--page_completion->info->busy == 0) + discard_info = page_completion->info; + } + + ASSERT_LOG_ONLY((page_completion->waiter.next_waiter == NULL), + "Page being released after leaving all queues"); + + page_completion->info = NULL; + cache = page_completion->cache; + assert_on_cache_thread(cache, __func__); + + if (discard_info != NULL) { + if (discard_info->write_status == WRITE_STATUS_DEFERRED) { + discard_info->write_status = WRITE_STATUS_NORMAL; + launch_page_save(discard_info); + } + + /* + * if there are excess requests for pages (that have not already started discards) + * we need to discard some page (which may be this one) + */ + discard_page_if_needed(cache); + } +} + +/** + * load_page_for_completion() - Helper function to load a page as described by a VDO Page + * Completion. + */ +static void load_page_for_completion(struct page_info *info, + struct vdo_page_completion *vdo_page_comp) +{ + int result; + + vdo_enqueue_waiter(&info->waiting, &vdo_page_comp->waiter); + result = launch_page_load(info, vdo_page_comp->pbn); + if (result != VDO_SUCCESS) + vdo_notify_all_waiters(&info->waiting, complete_waiter_with_error, &result); +} + +/** + * vdo_get_page() - Initialize a page completion and get a block map page. + * @page_completion: The vdo_page_completion to initialize. + * @zone: The block map zone of the desired page. + * @pbn: The absolute physical block of the desired page. + * @writable: Whether the page can be modified. + * @parent: The object to notify when the fetch is complete. + * @callback: The notification callback. + * @error_handler: The handler for fetch errors. + * @requeue: Whether we must requeue when notifying the parent. + * + * May cause another page to be discarded (potentially writing a dirty page) and the one nominated + * by the completion to be loaded from disk. When the callback is invoked, the page will be + * resident in the cache and marked busy. All callers must call vdo_release_page_completion() + * when they are done with the page to clear the busy mark. + */ +void vdo_get_page(struct vdo_page_completion *page_completion, + struct block_map_zone *zone, physical_block_number_t pbn, + bool writable, void *parent, vdo_action_fn callback, + vdo_action_fn error_handler, bool requeue) +{ + struct vdo_page_cache *cache = &zone->page_cache; + struct vdo_completion *completion = &page_completion->completion; + struct page_info *info; + + assert_on_cache_thread(cache, __func__); + ASSERT_LOG_ONLY((page_completion->waiter.next_waiter == NULL), + "New page completion was not already on a wait queue"); + + *page_completion = (struct vdo_page_completion) { + .pbn = pbn, + .writable = writable, + .cache = cache, + }; + + vdo_initialize_completion(completion, cache->vdo, VDO_PAGE_COMPLETION); + vdo_prepare_completion(completion, callback, error_handler, + cache->zone->thread_id, parent); + completion->requeue = requeue; + + if (page_completion->writable && vdo_is_read_only(cache->zone->block_map->vdo)) { + vdo_fail_completion(completion, VDO_READ_ONLY); + return; + } + + if (page_completion->writable) + ADD_ONCE(cache->stats.write_count, 1); + else + ADD_ONCE(cache->stats.read_count, 1); + + info = find_page(cache, page_completion->pbn); + if (info != NULL) { + /* The page is in the cache already. */ + if ((info->write_status == WRITE_STATUS_DEFERRED) || + is_incoming(info) || + (is_outgoing(info) && page_completion->writable)) { + /* The page is unusable until it has finished I/O. */ + ADD_ONCE(cache->stats.wait_for_page, 1); + vdo_enqueue_waiter(&info->waiting, &page_completion->waiter); + return; + } + + if (is_valid(info)) { + /* The page is usable. */ + ADD_ONCE(cache->stats.found_in_cache, 1); + if (!is_present(info)) + ADD_ONCE(cache->stats.read_outgoing, 1); + update_lru(info); + info->busy++; + complete_with_page(info, page_completion); + return; + } + + /* Something horrible has gone wrong. */ + ASSERT_LOG_ONLY(false, "Info found in a usable state."); + } + + /* The page must be fetched. */ + info = find_free_page(cache); + if (info != NULL) { + ADD_ONCE(cache->stats.fetch_required, 1); + load_page_for_completion(info, page_completion); + return; + } + + /* The page must wait for a page to be discarded. */ + ADD_ONCE(cache->stats.discard_required, 1); + discard_page_for_completion(page_completion); +} + +/** + * vdo_request_page_write() - Request that a VDO page be written out as soon as it is not busy. + * @completion: The vdo_page_completion containing the page. + */ +void vdo_request_page_write(struct vdo_completion *completion) +{ + struct page_info *info; + struct vdo_page_completion *vdo_page_comp = as_vdo_page_completion(completion); + + if (!validate_completed_page_or_enter_read_only_mode(vdo_page_comp, true)) + return; + + info = vdo_page_comp->info; + set_info_state(info, PS_DIRTY); + launch_page_save(info); +} + +/** + * vdo_get_cached_page() - Get the block map page from a page completion. + * @completion: A vdo page completion whose callback has been called. + * @page_ptr: A pointer to hold the page + * + * Return: VDO_SUCCESS or an error + */ +int vdo_get_cached_page(struct vdo_completion *completion, + struct block_map_page **page_ptr) +{ + int result; + struct vdo_page_completion *vpc; + + vpc = as_vdo_page_completion(completion); + result = validate_completed_page(vpc, true); + if (result == VDO_SUCCESS) + *page_ptr = (struct block_map_page *) get_page_buffer(vpc->info); + + return result; +} + +/** + * vdo_invalidate_page_cache() - Invalidate all entries in the VDO page cache. + * + * There must not be any dirty pages in the cache. + * + * Return: A success or error code. + */ +int vdo_invalidate_page_cache(struct vdo_page_cache *cache) +{ + struct page_info *info; + + assert_on_cache_thread(cache, __func__); + + /* Make sure we don't throw away any dirty pages. */ + for (info = cache->infos; info < cache->infos + cache->page_count; info++) { + int result = ASSERT(!is_dirty(info), "cache must have no dirty pages"); + + if (result != VDO_SUCCESS) + return result; + } + + /* Reset the page map by re-allocating it. */ + vdo_free_int_map(uds_forget(cache->page_map)); + return vdo_make_int_map(cache->page_count, 0, &cache->page_map); +} + /** * get_tree_page_by_index() - Get the tree page for a given height and page index. * diff --git a/drivers/md/dm-vdo/block-map.h b/drivers/md/dm-vdo/block-map.h index eaf27d41af8b..dc807111b0e6 100644 --- a/drivers/md/dm-vdo/block-map.h +++ b/drivers/md/dm-vdo/block-map.h @@ -30,6 +30,140 @@ typedef u32 vdo_page_generation; extern const struct block_map_entry UNMAPPED_BLOCK_MAP_ENTRY; +/* The VDO Page Cache abstraction. */ +struct vdo_page_cache { + /* the VDO which owns this cache */ + struct vdo *vdo; + /* number of pages in cache */ + page_count_t page_count; + /* number of pages to write in the current batch */ + page_count_t pages_in_batch; + /* Whether the VDO is doing a read-only rebuild */ + bool rebuilding; + + /* array of page information entries */ + struct page_info *infos; + /* raw memory for pages */ + char *pages; + /* cache last found page info */ + struct page_info *last_found; + /* map of page number to info */ + struct int_map *page_map; + /* main LRU list (all infos) */ + struct list_head lru_list; + /* free page list (oldest first) */ + struct list_head free_list; + /* outgoing page list */ + struct list_head outgoing_list; + /* number of read I/O operations pending */ + page_count_t outstanding_reads; + /* number of write I/O operations pending */ + page_count_t outstanding_writes; + /* number of pages covered by the current flush */ + page_count_t pages_in_flush; + /* number of pages waiting to be included in the next flush */ + page_count_t pages_to_flush; + /* number of discards in progress */ + unsigned int discard_count; + /* how many VPCs waiting for free page */ + unsigned int waiter_count; + /* queue of waiters who want a free page */ + struct wait_queue free_waiters; + /* + * Statistics are only updated on the logical zone thread, but are accessed from other + * threads. + */ + struct block_map_statistics stats; + /* counter for pressure reports */ + u32 pressure_report; + /* the block map zone to which this cache belongs */ + struct block_map_zone *zone; +}; + +/* + * The state of a page buffer. If the page buffer is free no particular page is bound to it, + * otherwise the page buffer is bound to particular page whose absolute pbn is in the pbn field. If + * the page is resident or dirty the page data is stable and may be accessed. Otherwise the page is + * in flight (incoming or outgoing) and its data should not be accessed. + * + * @note Update the static data in get_page_state_name() if you change this enumeration. + */ +enum vdo_page_buffer_state { + /* this page buffer is not being used */ + PS_FREE, + /* this page is being read from store */ + PS_INCOMING, + /* attempt to load this page failed */ + PS_FAILED, + /* this page is valid and un-modified */ + PS_RESIDENT, + /* this page is valid and modified */ + PS_DIRTY, + /* this page is being written and should not be used */ + PS_OUTGOING, + /* not a state */ + PAGE_STATE_COUNT, +} __packed; + +/* + * The write status of page + */ +enum vdo_page_write_status { + WRITE_STATUS_NORMAL, + WRITE_STATUS_DISCARD, + WRITE_STATUS_DEFERRED, +} __packed; + +/* Per-page-slot information. */ +struct page_info { + /* Preallocated page struct vio */ + struct vio *vio; + /* back-link for references */ + struct vdo_page_cache *cache; + /* the pbn of the page */ + physical_block_number_t pbn; + /* page is busy (temporarily locked) */ + u16 busy; + /* the write status the page */ + enum vdo_page_write_status write_status; + /* page state */ + enum vdo_page_buffer_state state; + /* queue of completions awaiting this item */ + struct wait_queue waiting; + /* state linked list entry */ + struct list_head state_entry; + /* LRU entry */ + struct list_head lru_entry; + /* + * The earliest recovery journal block containing uncommitted updates to the block map page + * associated with this page_info. A reference (lock) is held on that block to prevent it + * from being reaped. When this value changes, the reference on the old value must be + * released and a reference on the new value must be acquired. + */ + sequence_number_t recovery_lock; +}; + +/* + * A completion awaiting a specific page. Also a live reference into the page once completed, until + * freed. + */ +struct vdo_page_completion { + /* The generic completion */ + struct vdo_completion completion; + /* The cache involved */ + struct vdo_page_cache *cache; + /* The waiter for the pending list */ + struct waiter waiter; + /* The absolute physical block number of the page on disk */ + physical_block_number_t pbn; + /* Whether the page may be modified */ + bool writable; + /* Whether the page is available */ + bool ready; + /* The info structure for the page, only valid when ready */ + struct page_info *info; +}; + struct forest; struct tree_page { @@ -141,6 +275,26 @@ struct block_map { typedef int (*vdo_entry_callback_fn)(physical_block_number_t pbn, struct vdo_completion *completion); +static inline struct vdo_page_completion *as_vdo_page_completion(struct vdo_completion *completion) +{ + vdo_assert_completion_type(completion, VDO_PAGE_COMPLETION); + return container_of(completion, struct vdo_page_completion, completion); +} + +void vdo_release_page_completion(struct vdo_completion *completion); + +void vdo_get_page(struct vdo_page_completion *page_completion, + struct block_map_zone *zone, physical_block_number_t pbn, + bool writable, void *parent, vdo_action_fn callback, + vdo_action_fn error_handler, bool requeue); + +void vdo_request_page_write(struct vdo_completion *completion); + +int __must_check vdo_get_cached_page(struct vdo_completion *completion, + struct block_map_page **page_ptr); + +int __must_check vdo_invalidate_page_cache(struct vdo_page_cache *cache); + static inline struct block_map_page * __must_check vdo_as_block_map_page(struct tree_page *tree_page) { -- cgit v1.2.3 From d6e260cc426164820a496528fda809add15be1ea Mon Sep 17 00:00:00 2001 From: Mike Snitzer Date: Mon, 20 Nov 2023 17:29:16 -0500 Subject: dm vdo wait-queue: add proper namespace to interface Rename various interfaces and structs associated with vdo's wait-queue, e.g.: s/wait_queue/vdo_wait_queue/, s/waiter/vdo_waiter/, etc. Now all function names start with "vdo_waitq_" or "vdo_waiter_". Reviewed-by: Ken Raeburn Signed-off-by: Mike Snitzer Signed-off-by: Matthew Sakai --- drivers/md/dm-vdo/block-map.c | 134 ++++++++++++------------ drivers/md/dm-vdo/block-map.h | 10 +- drivers/md/dm-vdo/data-vio.c | 14 +-- drivers/md/dm-vdo/data-vio.h | 12 +-- drivers/md/dm-vdo/dedupe.c | 48 ++++----- drivers/md/dm-vdo/dump.c | 12 +-- drivers/md/dm-vdo/flush.c | 32 +++--- drivers/md/dm-vdo/flush.h | 2 +- drivers/md/dm-vdo/physical-zone.c | 4 +- drivers/md/dm-vdo/recovery-journal.c | 69 +++++++------ drivers/md/dm-vdo/recovery-journal.h | 10 +- drivers/md/dm-vdo/slab-depot.c | 99 +++++++++--------- drivers/md/dm-vdo/slab-depot.h | 22 ++-- drivers/md/dm-vdo/vio.c | 12 +-- drivers/md/dm-vdo/vio.h | 2 +- drivers/md/dm-vdo/wait-queue.c | 190 ++++++++++++++++++----------------- drivers/md/dm-vdo/wait-queue.h | 130 +++++++++++++----------- 17 files changed, 413 insertions(+), 389 deletions(-) (limited to 'drivers/md/dm-vdo/block-map.h') diff --git a/drivers/md/dm-vdo/block-map.c b/drivers/md/dm-vdo/block-map.c index 1edb3b2a80eb..a1f2c9d38192 100644 --- a/drivers/md/dm-vdo/block-map.c +++ b/drivers/md/dm-vdo/block-map.c @@ -85,7 +85,7 @@ struct cursor_level { struct cursors; struct cursor { - struct waiter waiter; + struct vdo_waiter waiter; struct block_map_tree *tree; height_t height; struct cursors *parent; @@ -162,7 +162,7 @@ static char *get_page_buffer(struct page_info *info) return &cache->pages[(info - cache->infos) * VDO_BLOCK_SIZE]; } -static inline struct vdo_page_completion *page_completion_from_waiter(struct waiter *waiter) +static inline struct vdo_page_completion *page_completion_from_waiter(struct vdo_waiter *waiter) { struct vdo_page_completion *completion; @@ -407,7 +407,7 @@ static int reset_page_info(struct page_info *info) if (result != UDS_SUCCESS) return result; - result = ASSERT(!vdo_has_waiters(&info->waiting), + result = ASSERT(!vdo_waitq_has_waiters(&info->waiting), "VDO Page must not have waiters"); if (result != UDS_SUCCESS) return result; @@ -506,7 +506,7 @@ static void complete_with_page(struct page_info *info, * * Implements waiter_callback_fn. */ -static void complete_waiter_with_error(struct waiter *waiter, void *result_ptr) +static void complete_waiter_with_error(struct vdo_waiter *waiter, void *result_ptr) { int *result = result_ptr; @@ -520,25 +520,25 @@ static void complete_waiter_with_error(struct waiter *waiter, void *result_ptr) * * Implements waiter_callback_fn. */ -static void complete_waiter_with_page(struct waiter *waiter, void *page_info) +static void complete_waiter_with_page(struct vdo_waiter *waiter, void *page_info) { complete_with_page(page_info, page_completion_from_waiter(waiter)); } /** - * distribute_page_over_queue() - Complete a queue of VDO page completions with a page result. + * distribute_page_over_waitq() - Complete a waitq of VDO page completions with a page result. * - * Upon completion the queue will be empty. + * Upon completion the waitq will be empty. * * Return: The number of pages distributed. */ -static unsigned int distribute_page_over_queue(struct page_info *info, - struct wait_queue *queue) +static unsigned int distribute_page_over_waitq(struct page_info *info, + struct vdo_wait_queue *waitq) { size_t pages; update_lru(info); - pages = vdo_count_waiters(queue); + pages = vdo_waitq_num_waiters(waitq); /* * Increment the busy count once for each pending completion so that this page does not @@ -546,7 +546,7 @@ static unsigned int distribute_page_over_queue(struct page_info *info, */ info->busy += pages; - vdo_notify_all_waiters(queue, complete_waiter_with_page, info); + vdo_waitq_notify_all_waiters(waitq, complete_waiter_with_page, info); return pages; } @@ -572,13 +572,14 @@ static void set_persistent_error(struct vdo_page_cache *cache, const char *conte assert_on_cache_thread(cache, __func__); - vdo_notify_all_waiters(&cache->free_waiters, complete_waiter_with_error, - &result); + vdo_waitq_notify_all_waiters(&cache->free_waiters, + complete_waiter_with_error, &result); cache->waiter_count = 0; - for (info = cache->infos; info < cache->infos + cache->page_count; info++) - vdo_notify_all_waiters(&info->waiting, complete_waiter_with_error, - &result); + for (info = cache->infos; info < cache->infos + cache->page_count; info++) { + vdo_waitq_notify_all_waiters(&info->waiting, + complete_waiter_with_error, &result); + } } /** @@ -625,7 +626,7 @@ static void check_for_drain_complete(struct block_map_zone *zone) { if (vdo_is_state_draining(&zone->state) && (zone->active_lookups == 0) && - !vdo_has_waiters(&zone->flush_waiters) && + !vdo_waitq_has_waiters(&zone->flush_waiters) && !is_vio_pool_busy(zone->vio_pool) && (zone->page_cache.outstanding_reads == 0) && (zone->page_cache.outstanding_writes == 0)) { @@ -643,8 +644,8 @@ static void enter_zone_read_only_mode(struct block_map_zone *zone, int result) * We are in read-only mode, so we won't ever write any page out. Just take all waiters off * the queue so the zone can drain. */ - while (vdo_has_waiters(&zone->flush_waiters)) - vdo_dequeue_next_waiter(&zone->flush_waiters); + while (vdo_waitq_has_waiters(&zone->flush_waiters)) + vdo_waitq_dequeue_next_waiter(&zone->flush_waiters); check_for_drain_complete(zone); } @@ -677,7 +678,7 @@ static void handle_load_error(struct vdo_completion *completion) vdo_enter_read_only_mode(cache->zone->block_map->vdo, result); ADD_ONCE(cache->stats.failed_reads, 1); set_info_state(info, PS_FAILED); - vdo_notify_all_waiters(&info->waiting, complete_waiter_with_error, &result); + vdo_waitq_notify_all_waiters(&info->waiting, complete_waiter_with_error, &result); reset_page_info(info); /* @@ -720,7 +721,7 @@ static void page_is_loaded(struct vdo_completion *completion) info->recovery_lock = 0; set_info_state(info, PS_RESIDENT); - distribute_page_over_queue(info, &info->waiting); + distribute_page_over_waitq(info, &info->waiting); /* * Don't decrement until right before calling check_for_drain_complete() to @@ -874,7 +875,7 @@ static void launch_page_save(struct page_info *info) * * Return: true if the page completion is for the desired page number. */ -static bool completion_needs_page(struct waiter *waiter, void *context) +static bool completion_needs_page(struct vdo_waiter *waiter, void *context) { physical_block_number_t *pbn = context; @@ -888,13 +889,13 @@ static bool completion_needs_page(struct waiter *waiter, void *context) static void allocate_free_page(struct page_info *info) { int result; - struct waiter *oldest_waiter; + struct vdo_waiter *oldest_waiter; physical_block_number_t pbn; struct vdo_page_cache *cache = info->cache; assert_on_cache_thread(cache, __func__); - if (!vdo_has_waiters(&cache->free_waiters)) { + if (!vdo_waitq_has_waiters(&cache->free_waiters)) { if (cache->stats.cache_pressure > 0) { uds_log_info("page cache pressure relieved"); WRITE_ONCE(cache->stats.cache_pressure, 0); @@ -909,20 +910,22 @@ static void allocate_free_page(struct page_info *info) return; } - oldest_waiter = vdo_get_first_waiter(&cache->free_waiters); + oldest_waiter = vdo_waitq_get_first_waiter(&cache->free_waiters); pbn = page_completion_from_waiter(oldest_waiter)->pbn; /* * Remove all entries which match the page number in question and push them onto the page * info's wait queue. */ - vdo_dequeue_matching_waiters(&cache->free_waiters, completion_needs_page, - &pbn, &info->waiting); - cache->waiter_count -= vdo_count_waiters(&info->waiting); + vdo_waitq_dequeue_matching_waiters(&cache->free_waiters, completion_needs_page, + &pbn, &info->waiting); + cache->waiter_count -= vdo_waitq_num_waiters(&info->waiting); result = launch_page_load(info, pbn); - if (result != VDO_SUCCESS) - vdo_notify_all_waiters(&info->waiting, complete_waiter_with_error, &result); + if (result != VDO_SUCCESS) { + vdo_waitq_notify_all_waiters(&info->waiting, + complete_waiter_with_error, &result); + } } /** @@ -966,7 +969,7 @@ static void discard_page_for_completion(struct vdo_page_completion *vdo_page_com struct vdo_page_cache *cache = vdo_page_comp->cache; cache->waiter_count++; - vdo_enqueue_waiter(&cache->free_waiters, &vdo_page_comp->waiter); + vdo_waitq_enqueue_waiter(&cache->free_waiters, &vdo_page_comp->waiter); discard_a_page(cache); } @@ -1069,11 +1072,11 @@ static void page_is_written_out(struct vdo_completion *completion) cache->zone->zone_number); info->recovery_lock = 0; was_discard = write_has_finished(info); - reclaimed = (!was_discard || (info->busy > 0) || vdo_has_waiters(&info->waiting)); + reclaimed = (!was_discard || (info->busy > 0) || vdo_waitq_has_waiters(&info->waiting)); set_info_state(info, PS_RESIDENT); - reclamations = distribute_page_over_queue(info, &info->waiting); + reclamations = distribute_page_over_waitq(info, &info->waiting); ADD_ONCE(cache->stats.reclaimed, reclamations); if (was_discard) @@ -1187,10 +1190,12 @@ static void load_page_for_completion(struct page_info *info, { int result; - vdo_enqueue_waiter(&info->waiting, &vdo_page_comp->waiter); + vdo_waitq_enqueue_waiter(&info->waiting, &vdo_page_comp->waiter); result = launch_page_load(info, vdo_page_comp->pbn); - if (result != VDO_SUCCESS) - vdo_notify_all_waiters(&info->waiting, complete_waiter_with_error, &result); + if (result != VDO_SUCCESS) { + vdo_waitq_notify_all_waiters(&info->waiting, + complete_waiter_with_error, &result); + } } /** @@ -1251,7 +1256,7 @@ void vdo_get_page(struct vdo_page_completion *page_completion, (is_outgoing(info) && page_completion->writable)) { /* The page is unusable until it has finished I/O. */ ADD_ONCE(cache->stats.wait_for_page, 1); - vdo_enqueue_waiter(&info->waiting, &page_completion->waiter); + vdo_waitq_enqueue_waiter(&info->waiting, &page_completion->waiter); return; } @@ -1476,7 +1481,7 @@ static void set_generation(struct block_map_zone *zone, struct tree_page *page, { u32 new_count; int result; - bool decrement_old = vdo_is_waiting(&page->waiter); + bool decrement_old = vdo_waiter_is_waiting(&page->waiter); u8 old_generation = page->generation; if (decrement_old && (old_generation == new_generation)) @@ -1498,12 +1503,12 @@ static void set_generation(struct block_map_zone *zone, struct tree_page *page, static void write_page(struct tree_page *tree_page, struct pooled_vio *vio); /* Implements waiter_callback_fn */ -static void write_page_callback(struct waiter *waiter, void *context) +static void write_page_callback(struct vdo_waiter *waiter, void *context) { write_page(container_of(waiter, struct tree_page, waiter), context); } -static void acquire_vio(struct waiter *waiter, struct block_map_zone *zone) +static void acquire_vio(struct vdo_waiter *waiter, struct block_map_zone *zone) { waiter->callback = write_page_callback; acquire_vio_from_pool(zone->vio_pool, waiter); @@ -1530,10 +1535,10 @@ static void enqueue_page(struct tree_page *page, struct block_map_zone *zone) return; } - vdo_enqueue_waiter(&zone->flush_waiters, &page->waiter); + vdo_waitq_enqueue_waiter(&zone->flush_waiters, &page->waiter); } -static void write_page_if_not_dirtied(struct waiter *waiter, void *context) +static void write_page_if_not_dirtied(struct vdo_waiter *waiter, void *context) { struct tree_page *page = container_of(waiter, struct tree_page, waiter); struct write_if_not_dirtied_context *write_context = context; @@ -1576,8 +1581,8 @@ static void finish_page_write(struct vdo_completion *completion) .generation = page->writing_generation, }; - vdo_notify_all_waiters(&zone->flush_waiters, - write_page_if_not_dirtied, &context); + vdo_waitq_notify_all_waiters(&zone->flush_waiters, + write_page_if_not_dirtied, &context); if (dirty && attempt_increment(zone)) { write_page(page, pooled); return; @@ -1588,10 +1593,10 @@ static void finish_page_write(struct vdo_completion *completion) if (dirty) { enqueue_page(page, zone); - } else if ((zone->flusher == NULL) && vdo_has_waiters(&zone->flush_waiters) && + } else if ((zone->flusher == NULL) && vdo_waitq_has_waiters(&zone->flush_waiters) && attempt_increment(zone)) { zone->flusher = - container_of(vdo_dequeue_next_waiter(&zone->flush_waiters), + container_of(vdo_waitq_dequeue_next_waiter(&zone->flush_waiters), struct tree_page, waiter); write_page(zone->flusher, pooled); return; @@ -1724,9 +1729,9 @@ static void finish_lookup(struct data_vio *data_vio, int result) continue_data_vio_with_error(data_vio, result); } -static void abort_lookup_for_waiter(struct waiter *waiter, void *context) +static void abort_lookup_for_waiter(struct vdo_waiter *waiter, void *context) { - struct data_vio *data_vio = waiter_as_data_vio(waiter); + struct data_vio *data_vio = vdo_waiter_as_data_vio(waiter); int result = *((int *) context); if (!data_vio->write) { @@ -1746,8 +1751,9 @@ static void abort_lookup(struct data_vio *data_vio, int result, char *what) if (data_vio->tree_lock.locked) { release_page_lock(data_vio, what); - vdo_notify_all_waiters(&data_vio->tree_lock.waiters, - abort_lookup_for_waiter, &result); + vdo_waitq_notify_all_waiters(&data_vio->tree_lock.waiters, + abort_lookup_for_waiter, + &result); } finish_lookup(data_vio, result); @@ -1813,9 +1819,9 @@ static void continue_with_loaded_page(struct data_vio *data_vio, load_block_map_page(data_vio->logical.zone->block_map_zone, data_vio); } -static void continue_load_for_waiter(struct waiter *waiter, void *context) +static void continue_load_for_waiter(struct vdo_waiter *waiter, void *context) { - struct data_vio *data_vio = waiter_as_data_vio(waiter); + struct data_vio *data_vio = vdo_waiter_as_data_vio(waiter); data_vio->tree_lock.height--; continue_with_loaded_page(data_vio, context); @@ -1845,7 +1851,7 @@ static void finish_block_map_page_load(struct vdo_completion *completion) /* Release our claim to the load and wake any waiters */ release_page_lock(data_vio, "load"); - vdo_notify_all_waiters(&tree_lock->waiters, continue_load_for_waiter, page); + vdo_waitq_notify_all_waiters(&tree_lock->waiters, continue_load_for_waiter, page); continue_with_loaded_page(data_vio, page); } @@ -1871,10 +1877,10 @@ static void load_page_endio(struct bio *bio) data_vio->logical.zone->thread_id); } -static void load_page(struct waiter *waiter, void *context) +static void load_page(struct vdo_waiter *waiter, void *context) { struct pooled_vio *pooled = context; - struct data_vio *data_vio = waiter_as_data_vio(waiter); + struct data_vio *data_vio = vdo_waiter_as_data_vio(waiter); struct tree_lock *lock = &data_vio->tree_lock; physical_block_number_t pbn = lock->tree_slots[lock->height - 1].block_map_slot.pbn; @@ -1916,7 +1922,7 @@ static int attempt_page_lock(struct block_map_zone *zone, struct data_vio *data_ } /* Someone else is loading or allocating the page we need */ - vdo_enqueue_waiter(&lock_holder->waiters, &data_vio->waiter); + vdo_waitq_enqueue_waiter(&lock_holder->waiters, &data_vio->waiter); return VDO_SUCCESS; } @@ -1948,9 +1954,9 @@ static void allocation_failure(struct vdo_completion *completion) abort_lookup(data_vio, completion->result, "allocation"); } -static void continue_allocation_for_waiter(struct waiter *waiter, void *context) +static void continue_allocation_for_waiter(struct vdo_waiter *waiter, void *context) { - struct data_vio *data_vio = waiter_as_data_vio(waiter); + struct data_vio *data_vio = vdo_waiter_as_data_vio(waiter); struct tree_lock *tree_lock = &data_vio->tree_lock; physical_block_number_t pbn = *((physical_block_number_t *) context); @@ -2010,7 +2016,7 @@ static void write_expired_elements(struct block_map_zone *zone) list_del_init(&page->entry); - result = ASSERT(!vdo_is_waiting(&page->waiter), + result = ASSERT(!vdo_waiter_is_waiting(&page->waiter), "Newly expired page not already waiting to write"); if (result != VDO_SUCCESS) { enter_zone_read_only_mode(zone, result); @@ -2089,7 +2095,7 @@ static void finish_block_map_allocation(struct vdo_completion *completion) VDO_MAPPING_STATE_UNCOMPRESSED, &tree_page->recovery_lock); - if (vdo_is_waiting(&tree_page->waiter)) { + if (vdo_waiter_is_waiting(&tree_page->waiter)) { /* This page is waiting to be written out. */ if (zone->flusher != tree_page) { /* @@ -2117,8 +2123,8 @@ static void finish_block_map_allocation(struct vdo_completion *completion) /* Release our claim to the allocation and wake any waiters */ release_page_lock(data_vio, "allocation"); - vdo_notify_all_waiters(&tree_lock->waiters, continue_allocation_for_waiter, - &pbn); + vdo_waitq_notify_all_waiters(&tree_lock->waiters, + continue_allocation_for_waiter, &pbn); if (tree_lock->height == 0) { finish_lookup(data_vio, VDO_SUCCESS); return; @@ -2324,7 +2330,7 @@ physical_block_number_t vdo_find_block_map_page_pbn(struct block_map *map, */ void vdo_write_tree_page(struct tree_page *page, struct block_map_zone *zone) { - bool waiting = vdo_is_waiting(&page->waiter); + bool waiting = vdo_waiter_is_waiting(&page->waiter); if (waiting && (zone->flusher == page)) return; @@ -2630,7 +2636,7 @@ static void traverse(struct cursor *cursor) * * Implements waiter_callback_fn. */ -static void launch_cursor(struct waiter *waiter, void *context) +static void launch_cursor(struct vdo_waiter *waiter, void *context) { struct cursor *cursor = container_of(waiter, struct cursor, waiter); struct pooled_vio *pooled = context; diff --git a/drivers/md/dm-vdo/block-map.h b/drivers/md/dm-vdo/block-map.h index dc807111b0e6..cc98d19309ce 100644 --- a/drivers/md/dm-vdo/block-map.h +++ b/drivers/md/dm-vdo/block-map.h @@ -68,7 +68,7 @@ struct vdo_page_cache { /* how many VPCs waiting for free page */ unsigned int waiter_count; /* queue of waiters who want a free page */ - struct wait_queue free_waiters; + struct vdo_wait_queue free_waiters; /* * Statistics are only updated on the logical zone thread, but are accessed from other * threads. @@ -129,7 +129,7 @@ struct page_info { /* page state */ enum vdo_page_buffer_state state; /* queue of completions awaiting this item */ - struct wait_queue waiting; + struct vdo_wait_queue waiting; /* state linked list entry */ struct list_head state_entry; /* LRU entry */ @@ -153,7 +153,7 @@ struct vdo_page_completion { /* The cache involved */ struct vdo_page_cache *cache; /* The waiter for the pending list */ - struct waiter waiter; + struct vdo_waiter waiter; /* The absolute physical block number of the page on disk */ physical_block_number_t pbn; /* Whether the page may be modified */ @@ -167,7 +167,7 @@ struct vdo_page_completion { struct forest; struct tree_page { - struct waiter waiter; + struct vdo_waiter waiter; /* Dirty list entry */ struct list_head entry; @@ -228,7 +228,7 @@ struct block_map_zone { struct vio_pool *vio_pool; /* The tree page which has issued or will be issuing a flush */ struct tree_page *flusher; - struct wait_queue flush_waiters; + struct vdo_wait_queue flush_waiters; /* The generation after the most recent flush */ u8 generation; u8 oldest_generation; diff --git a/drivers/md/dm-vdo/data-vio.c b/drivers/md/dm-vdo/data-vio.c index 54c06e86d321..821155ca3761 100644 --- a/drivers/md/dm-vdo/data-vio.c +++ b/drivers/md/dm-vdo/data-vio.c @@ -249,7 +249,7 @@ static void initialize_lbn_lock(struct data_vio *data_vio, logical_block_number_ lock->lbn = lbn; lock->locked = false; - vdo_initialize_wait_queue(&lock->waiters); + vdo_waitq_init(&lock->waiters); zone_number = vdo_compute_logical_zone(data_vio); lock->zone = &vdo->logical_zones->zones[zone_number]; } @@ -466,7 +466,7 @@ static void attempt_logical_block_lock(struct vdo_completion *completion) } data_vio->last_async_operation = VIO_ASYNC_OP_ATTEMPT_LOGICAL_BLOCK_LOCK; - vdo_enqueue_waiter(&lock_holder->logical.waiters, &data_vio->waiter); + vdo_waitq_enqueue_waiter(&lock_holder->logical.waiters, &data_vio->waiter); /* * Prevent writes and read-modify-writes from blocking indefinitely on lock holders in the @@ -1191,11 +1191,11 @@ static void transfer_lock(struct data_vio *data_vio, struct lbn_lock *lock) /* Another data_vio is waiting for the lock, transfer it in a single lock map operation. */ next_lock_holder = - waiter_as_data_vio(vdo_dequeue_next_waiter(&lock->waiters)); + vdo_waiter_as_data_vio(vdo_waitq_dequeue_next_waiter(&lock->waiters)); /* Transfer the remaining lock waiters to the next lock holder. */ - vdo_transfer_all_waiters(&lock->waiters, - &next_lock_holder->logical.waiters); + vdo_waitq_transfer_all_waiters(&lock->waiters, + &next_lock_holder->logical.waiters); result = vdo_int_map_put(lock->zone->lbn_operations, lock->lbn, next_lock_holder, true, (void **) &lock_holder); @@ -1213,7 +1213,7 @@ static void transfer_lock(struct data_vio *data_vio, struct lbn_lock *lock) * If there are still waiters, other data_vios must be trying to get the lock we just * transferred. We must ensure that the new lock holder doesn't block in the packer. */ - if (vdo_has_waiters(&next_lock_holder->logical.waiters)) + if (vdo_waitq_has_waiters(&next_lock_holder->logical.waiters)) cancel_data_vio_compression(next_lock_holder); /* @@ -1235,7 +1235,7 @@ static void release_logical_lock(struct vdo_completion *completion) assert_data_vio_in_logical_zone(data_vio); - if (vdo_has_waiters(&lock->waiters)) + if (vdo_waitq_has_waiters(&lock->waiters)) transfer_lock(data_vio, lock); else release_lock(data_vio, lock); diff --git a/drivers/md/dm-vdo/data-vio.h b/drivers/md/dm-vdo/data-vio.h index aa415b8c7d91..f5a683968d1c 100644 --- a/drivers/md/dm-vdo/data-vio.h +++ b/drivers/md/dm-vdo/data-vio.h @@ -54,7 +54,7 @@ enum async_operation_number { struct lbn_lock { logical_block_number_t lbn; bool locked; - struct wait_queue waiters; + struct vdo_wait_queue waiters; struct logical_zone *zone; }; @@ -75,7 +75,7 @@ struct tree_lock { /* The key for the lock map */ u64 key; /* The queue of waiters for the page this vio is allocating or loading */ - struct wait_queue waiters; + struct vdo_wait_queue waiters; /* The block map tree slots for this LBN */ struct block_map_tree_slot tree_slots[VDO_BLOCK_MAP_TREE_HEIGHT + 1]; }; @@ -168,13 +168,13 @@ struct reference_updater { bool increment; struct zoned_pbn zpbn; struct pbn_lock *lock; - struct waiter waiter; + struct vdo_waiter waiter; }; /* A vio for processing user data requests. */ struct data_vio { - /* The wait_queue entry structure */ - struct waiter waiter; + /* The vdo_wait_queue entry structure */ + struct vdo_waiter waiter; /* The logical block of this request */ struct lbn_lock logical; @@ -288,7 +288,7 @@ static inline struct data_vio *as_data_vio(struct vdo_completion *completion) return vio_as_data_vio(as_vio(completion)); } -static inline struct data_vio *waiter_as_data_vio(struct waiter *waiter) +static inline struct data_vio *vdo_waiter_as_data_vio(struct vdo_waiter *waiter) { if (waiter == NULL) return NULL; diff --git a/drivers/md/dm-vdo/dedupe.c b/drivers/md/dm-vdo/dedupe.c index 8cc31110f5a8..02e36896ca3c 100644 --- a/drivers/md/dm-vdo/dedupe.c +++ b/drivers/md/dm-vdo/dedupe.c @@ -270,7 +270,7 @@ struct hash_lock { * to get the information they all need to deduplicate--either against each other, or * against an existing duplicate on disk. */ - struct wait_queue waiters; + struct vdo_wait_queue waiters; }; enum { @@ -351,7 +351,7 @@ static void return_hash_lock_to_pool(struct hash_zone *zone, struct hash_lock *l memset(lock, 0, sizeof(*lock)); INIT_LIST_HEAD(&lock->pool_node); INIT_LIST_HEAD(&lock->duplicate_ring); - vdo_initialize_wait_queue(&lock->waiters); + vdo_waitq_init(&lock->waiters); list_add_tail(&lock->pool_node, &zone->lock_pool); } @@ -420,7 +420,7 @@ static void set_duplicate_lock(struct hash_lock *hash_lock, struct pbn_lock *pbn */ static inline struct data_vio *dequeue_lock_waiter(struct hash_lock *lock) { - return waiter_as_data_vio(vdo_dequeue_next_waiter(&lock->waiters)); + return vdo_waiter_as_data_vio(vdo_waitq_dequeue_next_waiter(&lock->waiters)); } /** @@ -536,7 +536,7 @@ static struct data_vio *retire_lock_agent(struct hash_lock *lock) */ static void wait_on_hash_lock(struct hash_lock *lock, struct data_vio *data_vio) { - vdo_enqueue_waiter(&lock->waiters, &data_vio->waiter); + vdo_waitq_enqueue_waiter(&lock->waiters, &data_vio->waiter); /* * Make sure the agent doesn't block indefinitely in the packer since it now has at least @@ -562,9 +562,9 @@ static void wait_on_hash_lock(struct hash_lock *lock, struct data_vio *data_vio) * @waiter: The data_vio's waiter link. * @context: Not used. */ -static void abort_waiter(struct waiter *waiter, void *context __always_unused) +static void abort_waiter(struct vdo_waiter *waiter, void *context __always_unused) { - write_data_vio(waiter_as_data_vio(waiter)); + write_data_vio(vdo_waiter_as_data_vio(waiter)); } /** @@ -602,7 +602,7 @@ void vdo_clean_failed_hash_lock(struct data_vio *data_vio) /* Ensure we don't attempt to update advice when cleaning up. */ lock->update_advice = false; - vdo_notify_all_waiters(&lock->waiters, abort_waiter, NULL); + vdo_waitq_notify_all_waiters(&lock->waiters, abort_waiter, NULL); if (lock->duplicate_lock != NULL) { /* The agent must reference the duplicate zone to launch it. */ @@ -650,7 +650,7 @@ static void finish_unlocking(struct vdo_completion *completion) */ lock->verified = false; - if (vdo_has_waiters(&lock->waiters)) { + if (vdo_waitq_has_waiters(&lock->waiters)) { /* * UNLOCKING -> LOCKING transition: A new data_vio entered the hash lock while the * agent was releasing the PBN lock. The current agent exits and the waiter has to @@ -750,7 +750,7 @@ static void finish_updating(struct vdo_completion *completion) */ lock->update_advice = false; - if (vdo_has_waiters(&lock->waiters)) { + if (vdo_waitq_has_waiters(&lock->waiters)) { /* * UPDATING -> DEDUPING transition: A new data_vio arrived during the UDS update. * Send it on the verified dedupe path. The agent is done with the lock, but the @@ -812,7 +812,7 @@ static void finish_deduping(struct hash_lock *lock, struct data_vio *data_vio) struct data_vio *agent = data_vio; ASSERT_LOG_ONLY(lock->agent == NULL, "shouldn't have an agent in DEDUPING"); - ASSERT_LOG_ONLY(!vdo_has_waiters(&lock->waiters), + ASSERT_LOG_ONLY(!vdo_waitq_has_waiters(&lock->waiters), "shouldn't have any lock waiters in DEDUPING"); /* Just release the lock reference if other data_vios are still deduping. */ @@ -917,9 +917,9 @@ static int __must_check acquire_lock(struct hash_zone *zone, * Implements waiter_callback_fn. Binds the data_vio that was waiting to a new hash lock and waits * on that lock. */ -static void enter_forked_lock(struct waiter *waiter, void *context) +static void enter_forked_lock(struct vdo_waiter *waiter, void *context) { - struct data_vio *data_vio = waiter_as_data_vio(waiter); + struct data_vio *data_vio = vdo_waiter_as_data_vio(waiter); struct hash_lock *new_lock = context; set_hash_lock(data_vio, new_lock); @@ -956,7 +956,7 @@ static void fork_hash_lock(struct hash_lock *old_lock, struct data_vio *new_agen set_hash_lock(new_agent, new_lock); new_lock->agent = new_agent; - vdo_notify_all_waiters(&old_lock->waiters, enter_forked_lock, new_lock); + vdo_waitq_notify_all_waiters(&old_lock->waiters, enter_forked_lock, new_lock); new_agent->is_duplicate = false; start_writing(new_lock, new_agent); @@ -1033,7 +1033,7 @@ static void start_deduping(struct hash_lock *lock, struct data_vio *agent, launch_dedupe(lock, agent, true); agent = NULL; } - while (vdo_has_waiters(&lock->waiters)) + while (vdo_waitq_has_waiters(&lock->waiters)) launch_dedupe(lock, dequeue_lock_waiter(lock), false); if (agent_is_done) { @@ -1454,7 +1454,7 @@ static void finish_writing(struct hash_lock *lock, struct data_vio *agent) lock->update_advice = true; /* If there are any waiters, we need to start deduping them. */ - if (vdo_has_waiters(&lock->waiters)) { + if (vdo_waitq_has_waiters(&lock->waiters)) { /* * WRITING -> DEDUPING transition: an asynchronously-written block failed to * compress, so the PBN lock on the written copy was already transferred. The agent @@ -1502,10 +1502,10 @@ static void finish_writing(struct hash_lock *lock, struct data_vio *agent) */ static struct data_vio *select_writing_agent(struct hash_lock *lock) { - struct wait_queue temp_queue; + struct vdo_wait_queue temp_queue; struct data_vio *data_vio; - vdo_initialize_wait_queue(&temp_queue); + vdo_waitq_init(&temp_queue); /* * Move waiters to the temp queue one-by-one until we find an allocation. Not ideal to @@ -1514,7 +1514,7 @@ static struct data_vio *select_writing_agent(struct hash_lock *lock) while (((data_vio = dequeue_lock_waiter(lock)) != NULL) && !data_vio_has_allocation(data_vio)) { /* Use the lower-level enqueue since we're just moving waiters around. */ - vdo_enqueue_waiter(&temp_queue, &data_vio->waiter); + vdo_waitq_enqueue_waiter(&temp_queue, &data_vio->waiter); } if (data_vio != NULL) { @@ -1522,13 +1522,13 @@ static struct data_vio *select_writing_agent(struct hash_lock *lock) * Move the rest of the waiters over to the temp queue, preserving the order they * arrived at the lock. */ - vdo_transfer_all_waiters(&lock->waiters, &temp_queue); + vdo_waitq_transfer_all_waiters(&lock->waiters, &temp_queue); /* * The current agent is being replaced and will have to wait to dedupe; make it the * first waiter since it was the first to reach the lock. */ - vdo_enqueue_waiter(&lock->waiters, &lock->agent->waiter); + vdo_waitq_enqueue_waiter(&lock->waiters, &lock->agent->waiter); lock->agent = data_vio; } else { /* No one has an allocation, so keep the current agent. */ @@ -1536,7 +1536,7 @@ static struct data_vio *select_writing_agent(struct hash_lock *lock) } /* Swap all the waiters back onto the lock's queue. */ - vdo_transfer_all_waiters(&temp_queue, &lock->waiters); + vdo_waitq_transfer_all_waiters(&temp_queue, &lock->waiters); return data_vio; } @@ -1577,7 +1577,7 @@ static void start_writing(struct hash_lock *lock, struct data_vio *agent) * If the agent compresses, it might wait indefinitely in the packer, which would be bad if * there are any other data_vios waiting. */ - if (vdo_has_waiters(&lock->waiters)) + if (vdo_waitq_has_waiters(&lock->waiters)) cancel_data_vio_compression(agent); /* @@ -1928,7 +1928,7 @@ void vdo_release_hash_lock(struct data_vio *data_vio) "unregistered hash lock must not be in the lock map"); } - ASSERT_LOG_ONLY(!vdo_has_waiters(&lock->waiters), + ASSERT_LOG_ONLY(!vdo_waitq_has_waiters(&lock->waiters), "hash lock returned to zone must have no waiters"); ASSERT_LOG_ONLY((lock->duplicate_lock == NULL), "hash lock returned to zone must not reference a PBN lock"); @@ -2812,7 +2812,7 @@ static void dump_hash_lock(const struct hash_lock *lock) lock, state, (lock->registered ? 'D' : 'U'), (unsigned long long) lock->duplicate.pbn, lock->duplicate.state, lock->reference_count, - vdo_count_waiters(&lock->waiters), lock->agent); + vdo_waitq_num_waiters(&lock->waiters), lock->agent); } static const char *index_state_to_string(struct hash_zones *zones, diff --git a/drivers/md/dm-vdo/dump.c b/drivers/md/dm-vdo/dump.c index 99266a946ed7..91bc8ed36aa7 100644 --- a/drivers/md/dm-vdo/dump.c +++ b/drivers/md/dm-vdo/dump.c @@ -146,25 +146,25 @@ void vdo_dump_all(struct vdo *vdo, const char *why) } /* - * Dump out the data_vio waiters on a wait queue. + * Dump out the data_vio waiters on a waitq. * wait_on should be the label to print for queue (e.g. logical or physical) */ -static void dump_vio_waiters(struct wait_queue *queue, char *wait_on) +static void dump_vio_waiters(struct vdo_wait_queue *waitq, char *wait_on) { - struct waiter *waiter, *first = vdo_get_first_waiter(queue); + struct vdo_waiter *waiter, *first = vdo_waitq_get_first_waiter(waitq); struct data_vio *data_vio; if (first == NULL) return; - data_vio = waiter_as_data_vio(first); + data_vio = vdo_waiter_as_data_vio(first); uds_log_info(" %s is locked. Waited on by: vio %px pbn %llu lbn %llu d-pbn %llu lastOp %s", wait_on, data_vio, data_vio->allocation.pbn, data_vio->logical.lbn, data_vio->duplicate.pbn, get_data_vio_operation_name(data_vio)); for (waiter = first->next_waiter; waiter != first; waiter = waiter->next_waiter) { - data_vio = waiter_as_data_vio(waiter); + data_vio = vdo_waiter_as_data_vio(waiter); uds_log_info(" ... and : vio %px pbn %llu lbn %llu d-pbn %llu lastOp %s", data_vio, data_vio->allocation.pbn, data_vio->logical.lbn, data_vio->duplicate.pbn, @@ -177,7 +177,7 @@ static void dump_vio_waiters(struct wait_queue *queue, char *wait_on) * logging brevity: * * R => vio completion result not VDO_SUCCESS - * W => vio is on a wait queue + * W => vio is on a waitq * D => vio is a duplicate * p => vio is a partial block operation * z => vio is a zero block diff --git a/drivers/md/dm-vdo/flush.c b/drivers/md/dm-vdo/flush.c index a99607e23fb0..e7195c677773 100644 --- a/drivers/md/dm-vdo/flush.c +++ b/drivers/md/dm-vdo/flush.c @@ -31,9 +31,9 @@ struct flusher { /** The first unacknowledged flush generation */ sequence_number_t first_unacknowledged_generation; /** The queue of flush requests waiting to notify other threads */ - struct wait_queue notifiers; + struct vdo_wait_queue notifiers; /** The queue of flush requests waiting for VIOs to complete */ - struct wait_queue pending_flushes; + struct vdo_wait_queue pending_flushes; /** The flush generation for which notifications are being sent */ sequence_number_t notify_generation; /** The logical zone to notify next */ @@ -93,7 +93,7 @@ static inline struct vdo_flush *completion_as_vdo_flush(struct vdo_completion *c * * Return: The wait queue entry as a vdo_flush. */ -static struct vdo_flush *waiter_as_flush(struct waiter *waiter) +static struct vdo_flush *vdo_waiter_as_flush(struct vdo_waiter *waiter) { return container_of(waiter, struct vdo_flush, waiter); } @@ -195,10 +195,10 @@ static void finish_notification(struct vdo_completion *completion) assert_on_flusher_thread(flusher, __func__); - vdo_enqueue_waiter(&flusher->pending_flushes, - vdo_dequeue_next_waiter(&flusher->notifiers)); + vdo_waitq_enqueue_waiter(&flusher->pending_flushes, + vdo_waitq_dequeue_next_waiter(&flusher->notifiers)); vdo_complete_flushes(flusher); - if (vdo_has_waiters(&flusher->notifiers)) + if (vdo_waitq_has_waiters(&flusher->notifiers)) notify_flush(flusher); } @@ -248,7 +248,8 @@ static void increment_generation(struct vdo_completion *completion) */ static void notify_flush(struct flusher *flusher) { - struct vdo_flush *flush = waiter_as_flush(vdo_get_first_waiter(&flusher->notifiers)); + struct vdo_flush *flush = + vdo_waiter_as_flush(vdo_waitq_get_first_waiter(&flusher->notifiers)); flusher->notify_generation = flush->flush_generation; flusher->logical_zone_to_notify = &flusher->vdo->logical_zones->zones[0]; @@ -280,8 +281,8 @@ static void flush_vdo(struct vdo_completion *completion) } flush->flush_generation = flusher->flush_generation++; - may_notify = !vdo_has_waiters(&flusher->notifiers); - vdo_enqueue_waiter(&flusher->notifiers, &flush->waiter); + may_notify = !vdo_waitq_has_waiters(&flusher->notifiers); + vdo_waitq_enqueue_waiter(&flusher->notifiers, &flush->waiter); if (may_notify) notify_flush(flusher); } @@ -294,7 +295,8 @@ static void check_for_drain_complete(struct flusher *flusher) { bool drained; - if (!vdo_is_state_draining(&flusher->state) || vdo_has_waiters(&flusher->pending_flushes)) + if (!vdo_is_state_draining(&flusher->state) || + vdo_waitq_has_waiters(&flusher->pending_flushes)) return; spin_lock(&flusher->lock); @@ -321,9 +323,9 @@ void vdo_complete_flushes(struct flusher *flusher) min(oldest_active_generation, READ_ONCE(zone->oldest_active_generation)); - while (vdo_has_waiters(&flusher->pending_flushes)) { + while (vdo_waitq_has_waiters(&flusher->pending_flushes)) { struct vdo_flush *flush = - waiter_as_flush(vdo_get_first_waiter(&flusher->pending_flushes)); + vdo_waiter_as_flush(vdo_waitq_get_first_waiter(&flusher->pending_flushes)); if (flush->flush_generation >= oldest_active_generation) return; @@ -333,7 +335,7 @@ void vdo_complete_flushes(struct flusher *flusher) "acknowledged next expected flush, %llu, was: %llu", (unsigned long long) flusher->first_unacknowledged_generation, (unsigned long long) flush->flush_generation); - vdo_dequeue_next_waiter(&flusher->pending_flushes); + vdo_waitq_dequeue_next_waiter(&flusher->pending_flushes); vdo_complete_flush(flush); flusher->first_unacknowledged_generation++; } @@ -352,8 +354,8 @@ void vdo_dump_flusher(const struct flusher *flusher) (unsigned long long) flusher->flush_generation, (unsigned long long) flusher->first_unacknowledged_generation); uds_log_info(" notifiers queue is %s; pending_flushes queue is %s", - (vdo_has_waiters(&flusher->notifiers) ? "not empty" : "empty"), - (vdo_has_waiters(&flusher->pending_flushes) ? "not empty" : "empty")); + (vdo_waitq_has_waiters(&flusher->notifiers) ? "not empty" : "empty"), + (vdo_waitq_has_waiters(&flusher->pending_flushes) ? "not empty" : "empty")); } /** diff --git a/drivers/md/dm-vdo/flush.h b/drivers/md/dm-vdo/flush.h index 4d40908462bb..97252d6656e0 100644 --- a/drivers/md/dm-vdo/flush.h +++ b/drivers/md/dm-vdo/flush.h @@ -18,7 +18,7 @@ struct vdo_flush { /* The flush bios covered by this request */ struct bio_list bios; /* The wait queue entry for this flush */ - struct waiter waiter; + struct vdo_waiter waiter; /* Which flush this struct represents */ sequence_number_t flush_generation; }; diff --git a/drivers/md/dm-vdo/physical-zone.c b/drivers/md/dm-vdo/physical-zone.c index d3fc4666c3c2..9b99c9a820a3 100644 --- a/drivers/md/dm-vdo/physical-zone.c +++ b/drivers/md/dm-vdo/physical-zone.c @@ -519,9 +519,9 @@ static int allocate_and_lock_block(struct allocation *allocation) * @waiter: The allocating_vio that was waiting to allocate. * @context: The context (unused). */ -static void retry_allocation(struct waiter *waiter, void *context __always_unused) +static void retry_allocation(struct vdo_waiter *waiter, void *context __always_unused) { - struct data_vio *data_vio = waiter_as_data_vio(waiter); + struct data_vio *data_vio = vdo_waiter_as_data_vio(waiter); /* Now that some slab has scrubbed, restart the allocation process. */ data_vio->allocation.wait_for_clean_slab = false; diff --git a/drivers/md/dm-vdo/recovery-journal.c b/drivers/md/dm-vdo/recovery-journal.c index 2dfc39deef94..5126e670e97e 100644 --- a/drivers/md/dm-vdo/recovery-journal.c +++ b/drivers/md/dm-vdo/recovery-journal.c @@ -267,9 +267,9 @@ static void assert_on_journal_thread(struct recovery_journal *journal, * Invoked whenever a data_vio is to be released from the journal, either because its entry was * committed to disk, or because there was an error. Implements waiter_callback_fn. */ -static void continue_waiter(struct waiter *waiter, void *context) +static void continue_waiter(struct vdo_waiter *waiter, void *context) { - continue_data_vio_with_error(waiter_as_data_vio(waiter), *((int *) context)); + continue_data_vio_with_error(vdo_waiter_as_data_vio(waiter), *((int *) context)); } /** @@ -287,8 +287,8 @@ static inline bool has_block_waiters(struct recovery_journal *journal) * has waiters. */ return ((block != NULL) && - (vdo_has_waiters(&block->entry_waiters) || - vdo_has_waiters(&block->commit_waiters))); + (vdo_waitq_has_waiters(&block->entry_waiters) || + vdo_waitq_has_waiters(&block->commit_waiters))); } static void recycle_journal_blocks(struct recovery_journal *journal); @@ -343,14 +343,14 @@ static void check_for_drain_complete(struct recovery_journal *journal) recycle_journal_blocks(journal); /* Release any data_vios waiting to be assigned entries. */ - vdo_notify_all_waiters(&journal->entry_waiters, continue_waiter, - &result); + vdo_waitq_notify_all_waiters(&journal->entry_waiters, + continue_waiter, &result); } if (!vdo_is_state_draining(&journal->state) || journal->reaping || has_block_waiters(journal) || - vdo_has_waiters(&journal->entry_waiters) || + vdo_waitq_has_waiters(&journal->entry_waiters) || !suspend_lock_counter(&journal->lock_counter)) return; @@ -721,7 +721,7 @@ int vdo_decode_recovery_journal(struct recovery_journal_state_7_0 state, nonce_t INIT_LIST_HEAD(&journal->free_tail_blocks); INIT_LIST_HEAD(&journal->active_tail_blocks); - vdo_initialize_wait_queue(&journal->pending_writes); + vdo_waitq_init(&journal->pending_writes); journal->thread_id = vdo->thread_config.journal_thread; journal->origin = partition->offset; @@ -1047,7 +1047,7 @@ static void schedule_block_write(struct recovery_journal *journal, struct recovery_journal_block *block) { if (!block->committing) - vdo_enqueue_waiter(&journal->pending_writes, &block->write_waiter); + vdo_waitq_enqueue_waiter(&journal->pending_writes, &block->write_waiter); /* * At the end of adding entries, or discovering this partial block is now full and ready to * rewrite, we will call write_blocks() and write a whole batch. @@ -1084,9 +1084,9 @@ static void update_usages(struct recovery_journal *journal, struct data_vio *dat * * Implements waiter_callback_fn. */ -static void assign_entry(struct waiter *waiter, void *context) +static void assign_entry(struct vdo_waiter *waiter, void *context) { - struct data_vio *data_vio = waiter_as_data_vio(waiter); + struct data_vio *data_vio = vdo_waiter_as_data_vio(waiter); struct recovery_journal_block *block = context; struct recovery_journal *journal = block->journal; @@ -1099,10 +1099,10 @@ static void assign_entry(struct waiter *waiter, void *context) update_usages(journal, data_vio); journal->available_space--; - if (!vdo_has_waiters(&block->entry_waiters)) + if (!vdo_waitq_has_waiters(&block->entry_waiters)) journal->events.blocks.started++; - vdo_enqueue_waiter(&block->entry_waiters, &data_vio->waiter); + vdo_waitq_enqueue_waiter(&block->entry_waiters, &data_vio->waiter); block->entry_count++; block->uncommitted_entry_count++; journal->events.entries.started++; @@ -1127,9 +1127,10 @@ static void assign_entries(struct recovery_journal *journal) } journal->adding_entries = true; - while (vdo_has_waiters(&journal->entry_waiters) && prepare_to_assign_entry(journal)) { - vdo_notify_next_waiter(&journal->entry_waiters, assign_entry, - journal->active_block); + while (vdo_waitq_has_waiters(&journal->entry_waiters) && + prepare_to_assign_entry(journal)) { + vdo_waitq_notify_next_waiter(&journal->entry_waiters, + assign_entry, journal->active_block); } /* Now that we've finished with entries, see if we have a batch of blocks to write. */ @@ -1170,9 +1171,9 @@ static void recycle_journal_block(struct recovery_journal_block *block) * * Implements waiter_callback_fn. */ -static void continue_committed_waiter(struct waiter *waiter, void *context) +static void continue_committed_waiter(struct vdo_waiter *waiter, void *context) { - struct data_vio *data_vio = waiter_as_data_vio(waiter); + struct data_vio *data_vio = vdo_waiter_as_data_vio(waiter); struct recovery_journal *journal = context; int result = (is_read_only(journal) ? VDO_READ_ONLY : VDO_SUCCESS); bool has_decrement; @@ -1216,11 +1217,12 @@ static void notify_commit_waiters(struct recovery_journal *journal) if (block->committing) return; - vdo_notify_all_waiters(&block->commit_waiters, continue_committed_waiter, - journal); + vdo_waitq_notify_all_waiters(&block->commit_waiters, + continue_committed_waiter, journal); if (is_read_only(journal)) { - vdo_notify_all_waiters(&block->entry_waiters, - continue_committed_waiter, journal); + vdo_waitq_notify_all_waiters(&block->entry_waiters, + continue_committed_waiter, + journal); } else if (is_block_dirty(block) || !is_block_full(block)) { /* Stop at partially-committed or partially-filled blocks. */ return; @@ -1328,9 +1330,9 @@ static void complete_write_endio(struct bio *bio) */ static void add_queued_recovery_entries(struct recovery_journal_block *block) { - while (vdo_has_waiters(&block->entry_waiters)) { + while (vdo_waitq_has_waiters(&block->entry_waiters)) { struct data_vio *data_vio = - waiter_as_data_vio(vdo_dequeue_next_waiter(&block->entry_waiters)); + vdo_waiter_as_data_vio(vdo_waitq_dequeue_next_waiter(&block->entry_waiters)); struct tree_lock *lock = &data_vio->tree_lock; struct packed_recovery_journal_entry *packed_entry; struct recovery_journal_entry new_entry; @@ -1357,7 +1359,7 @@ static void add_queued_recovery_entries(struct recovery_journal_block *block) data_vio->recovery_sequence_number = block->sequence_number; /* Enqueue the data_vio to wait for its entry to commit. */ - vdo_enqueue_waiter(&block->commit_waiters, &data_vio->waiter); + vdo_waitq_enqueue_waiter(&block->commit_waiters, &data_vio->waiter); } } @@ -1366,17 +1368,18 @@ static void add_queued_recovery_entries(struct recovery_journal_block *block) * * Implements waiter_callback_fn. */ -static void write_block(struct waiter *waiter, void *context __always_unused) +static void write_block(struct vdo_waiter *waiter, void *context __always_unused) { struct recovery_journal_block *block = container_of(waiter, struct recovery_journal_block, write_waiter); struct recovery_journal *journal = block->journal; struct packed_journal_header *header = get_block_header(block); - if (block->committing || !vdo_has_waiters(&block->entry_waiters) || is_read_only(journal)) + if (block->committing || !vdo_waitq_has_waiters(&block->entry_waiters) || + is_read_only(journal)) return; - block->entries_in_commit = vdo_count_waiters(&block->entry_waiters); + block->entries_in_commit = vdo_waitq_num_waiters(&block->entry_waiters); add_queued_recovery_entries(block); journal->pending_write_count += 1; @@ -1419,7 +1422,7 @@ static void write_blocks(struct recovery_journal *journal) return; /* Write all the full blocks. */ - vdo_notify_all_waiters(&journal->pending_writes, write_block, NULL); + vdo_waitq_notify_all_waiters(&journal->pending_writes, write_block, NULL); /* * Do we need to write the active block? Only if we have no outstanding writes, even after @@ -1459,7 +1462,7 @@ void vdo_add_recovery_journal_entry(struct recovery_journal *journal, "journal lock not held for new entry"); vdo_advance_journal_point(&journal->append_point, journal->entries_per_block); - vdo_enqueue_waiter(&journal->entry_waiters, &data_vio->waiter); + vdo_waitq_enqueue_waiter(&journal->entry_waiters, &data_vio->waiter); assign_entries(journal); } @@ -1721,8 +1724,8 @@ static void dump_recovery_block(const struct recovery_journal_block *block) uds_log_info(" sequence number %llu; entries %u; %s; %zu entry waiters; %zu commit waiters", (unsigned long long) block->sequence_number, block->entry_count, (block->committing ? "committing" : "waiting"), - vdo_count_waiters(&block->entry_waiters), - vdo_count_waiters(&block->commit_waiters)); + vdo_waitq_num_waiters(&block->entry_waiters), + vdo_waitq_num_waiters(&block->commit_waiters)); } /** @@ -1745,7 +1748,7 @@ void vdo_dump_recovery_journal_statistics(const struct recovery_journal *journal (unsigned long long) journal->slab_journal_reap_head, (unsigned long long) stats.disk_full, (unsigned long long) stats.slab_journal_commits_requested, - vdo_count_waiters(&journal->entry_waiters)); + vdo_waitq_num_waiters(&journal->entry_waiters)); uds_log_info(" entries: started=%llu written=%llu committed=%llu", (unsigned long long) stats.entries.started, (unsigned long long) stats.entries.written, diff --git a/drivers/md/dm-vdo/recovery-journal.h b/drivers/md/dm-vdo/recovery-journal.h index c6d83019f918..19fa7ed9648a 100644 --- a/drivers/md/dm-vdo/recovery-journal.h +++ b/drivers/md/dm-vdo/recovery-journal.h @@ -113,7 +113,7 @@ struct recovery_journal_block { /* The doubly linked pointers for the free or active lists */ struct list_head list_node; /* The waiter for the pending full block list */ - struct waiter write_waiter; + struct vdo_waiter write_waiter; /* The journal to which this block belongs */ struct recovery_journal *journal; /* A pointer to the current sector in the packed block buffer */ @@ -133,9 +133,9 @@ struct recovery_journal_block { /* The number of new entries in the current commit */ journal_entry_count_t entries_in_commit; /* The queue of vios which will make entries for the next commit */ - struct wait_queue entry_waiters; + struct vdo_wait_queue entry_waiters; /* The queue of vios waiting for the current commit */ - struct wait_queue commit_waiters; + struct vdo_wait_queue commit_waiters; }; struct recovery_journal { @@ -146,7 +146,7 @@ struct recovery_journal { /* The block map which can hold locks on this journal */ struct block_map *block_map; /* The queue of vios waiting to make entries */ - struct wait_queue entry_waiters; + struct vdo_wait_queue entry_waiters; /* The number of free entries in the journal */ u64 available_space; /* The number of decrement entries which need to be made */ @@ -184,7 +184,7 @@ struct recovery_journal { /* A pointer to the active block (the one we are adding entries to now) */ struct recovery_journal_block *active_block; /* Journal blocks that need writing */ - struct wait_queue pending_writes; + struct vdo_wait_queue pending_writes; /* The new block map reap head after reaping */ sequence_number_t block_map_reap_head; /* The head block number for the block map rebuild range */ diff --git a/drivers/md/dm-vdo/slab-depot.c b/drivers/md/dm-vdo/slab-depot.c index 670a464ddbb0..2125e256aa86 100644 --- a/drivers/md/dm-vdo/slab-depot.c +++ b/drivers/md/dm-vdo/slab-depot.c @@ -65,7 +65,7 @@ static bool is_slab_open(struct vdo_slab *slab) static inline bool __must_check must_make_entries_to_flush(struct slab_journal *journal) { return ((journal->slab->status != VDO_SLAB_REBUILDING) && - vdo_has_waiters(&journal->entry_waiters)); + vdo_waitq_has_waiters(&journal->entry_waiters)); } /** @@ -122,7 +122,7 @@ static bool __must_check block_is_full(struct slab_journal *journal) static void add_entries(struct slab_journal *journal); static void update_tail_block_location(struct slab_journal *journal); -static void release_journal_locks(struct waiter *waiter, void *context); +static void release_journal_locks(struct vdo_waiter *waiter, void *context); /** * is_slab_journal_blank() - Check whether a slab's journal is blank. @@ -184,7 +184,7 @@ static void check_if_slab_drained(struct vdo_slab *slab) code = vdo_get_admin_state_code(&slab->state); read_only = vdo_is_read_only(slab->allocator->depot->vdo); if (!read_only && - vdo_has_waiters(&slab->dirty_blocks) && + vdo_waitq_has_waiters(&slab->dirty_blocks) && (code != VDO_ADMIN_STATE_SUSPENDING) && (code != VDO_ADMIN_STATE_RECOVERING)) return; @@ -229,14 +229,13 @@ static u8 __must_check compute_fullness_hint(struct slab_depot *depot, */ static void check_summary_drain_complete(struct block_allocator *allocator) { - struct vdo *vdo = allocator->depot->vdo; - if (!vdo_is_state_draining(&allocator->summary_state) || (allocator->summary_write_count > 0)) return; vdo_finish_operation(&allocator->summary_state, - (vdo_is_read_only(vdo) ? VDO_READ_ONLY : VDO_SUCCESS)); + (vdo_is_read_only(allocator->depot->vdo) ? + VDO_READ_ONLY : VDO_SUCCESS)); } /** @@ -245,11 +244,12 @@ static void check_summary_drain_complete(struct block_allocator *allocator) * @queue: The queue to notify. */ static void notify_summary_waiters(struct block_allocator *allocator, - struct wait_queue *queue) + struct vdo_wait_queue *queue) { - int result = (vdo_is_read_only(allocator->depot->vdo) ? VDO_READ_ONLY : VDO_SUCCESS); + int result = (vdo_is_read_only(allocator->depot->vdo) ? + VDO_READ_ONLY : VDO_SUCCESS); - vdo_notify_all_waiters(queue, NULL, &result); + vdo_waitq_notify_all_waiters(queue, NULL, &result); } static void launch_write(struct slab_summary_block *summary_block); @@ -264,7 +264,7 @@ static void finish_updating_slab_summary_block(struct slab_summary_block *block) notify_summary_waiters(block->allocator, &block->current_update_waiters); block->writing = false; block->allocator->summary_write_count--; - if (vdo_has_waiters(&block->next_update_waiters)) + if (vdo_waitq_has_waiters(&block->next_update_waiters)) launch_write(block); else check_summary_drain_complete(block->allocator); @@ -320,8 +320,8 @@ static void launch_write(struct slab_summary_block *block) return; allocator->summary_write_count++; - vdo_transfer_all_waiters(&block->next_update_waiters, - &block->current_update_waiters); + vdo_waitq_transfer_all_waiters(&block->next_update_waiters, + &block->current_update_waiters); block->writing = true; if (vdo_is_read_only(depot->vdo)) { @@ -351,7 +351,7 @@ static void launch_write(struct slab_summary_block *block) * @is_clean: Whether the slab is clean. * @free_blocks: The number of free blocks. */ -static void update_slab_summary_entry(struct vdo_slab *slab, struct waiter *waiter, +static void update_slab_summary_entry(struct vdo_slab *slab, struct vdo_waiter *waiter, tail_block_offset_t tail_block_offset, bool load_ref_counts, bool is_clean, block_count_t free_blocks) @@ -382,7 +382,7 @@ static void update_slab_summary_entry(struct vdo_slab *slab, struct waiter *wait .is_dirty = !is_clean, .fullness_hint = compute_fullness_hint(allocator->depot, free_blocks), }; - vdo_enqueue_waiter(&block->next_update_waiters, waiter); + vdo_waitq_enqueue_waiter(&block->next_update_waiters, waiter); launch_write(block); } @@ -441,7 +441,7 @@ static void flush_endio(struct bio *bio) * @waiter: The journal as a flush waiter. * @context: The newly acquired flush vio. */ -static void flush_for_reaping(struct waiter *waiter, void *context) +static void flush_for_reaping(struct vdo_waiter *waiter, void *context) { struct slab_journal *journal = container_of(waiter, struct slab_journal, flush_waiter); @@ -550,7 +550,7 @@ static void adjust_slab_journal_block_reference(struct slab_journal *journal, * * Implements waiter_callback_fn. */ -static void release_journal_locks(struct waiter *waiter, void *context) +static void release_journal_locks(struct vdo_waiter *waiter, void *context) { sequence_number_t first, i; struct slab_journal *journal = @@ -734,7 +734,7 @@ static void write_slab_journal_endio(struct bio *bio) * * Callback from acquire_vio_from_pool() registered in commit_tail(). */ -static void write_slab_journal_block(struct waiter *waiter, void *context) +static void write_slab_journal_block(struct vdo_waiter *waiter, void *context) { struct pooled_vio *pooled = context; struct vio *vio = &pooled->vio; @@ -1006,7 +1006,7 @@ static bool requires_reaping(const struct slab_journal *journal) } /** finish_summary_update() - A waiter callback that resets the writing state of a slab. */ -static void finish_summary_update(struct waiter *waiter, void *context) +static void finish_summary_update(struct vdo_waiter *waiter, void *context) { struct vdo_slab *slab = container_of(waiter, struct vdo_slab, summary_waiter); int result = *((int *) context); @@ -1021,7 +1021,7 @@ static void finish_summary_update(struct waiter *waiter, void *context) check_if_slab_drained(slab); } -static void write_reference_block(struct waiter *waiter, void *context); +static void write_reference_block(struct vdo_waiter *waiter, void *context); /** * launch_reference_block_write() - Launch the write of a dirty reference block by first acquiring @@ -1032,7 +1032,7 @@ static void write_reference_block(struct waiter *waiter, void *context); * This can be asynchronous since the writer will have to wait if all VIOs in the pool are * currently in use. */ -static void launch_reference_block_write(struct waiter *waiter, void *context) +static void launch_reference_block_write(struct vdo_waiter *waiter, void *context) { struct vdo_slab *slab = context; @@ -1047,7 +1047,8 @@ static void launch_reference_block_write(struct waiter *waiter, void *context) static void save_dirty_reference_blocks(struct vdo_slab *slab) { - vdo_notify_all_waiters(&slab->dirty_blocks, launch_reference_block_write, slab); + vdo_waitq_notify_all_waiters(&slab->dirty_blocks, + launch_reference_block_write, slab); check_if_slab_drained(slab); } @@ -1084,7 +1085,7 @@ static void finish_reference_block_write(struct vdo_completion *completion) /* Re-queue the block if it was re-dirtied while it was writing. */ if (block->is_dirty) { - vdo_enqueue_waiter(&block->slab->dirty_blocks, &block->waiter); + vdo_waitq_enqueue_waiter(&block->slab->dirty_blocks, &block->waiter); if (vdo_is_state_draining(&slab->state)) { /* We must be saving, and this block will otherwise not be relaunched. */ save_dirty_reference_blocks(slab); @@ -1097,7 +1098,7 @@ static void finish_reference_block_write(struct vdo_completion *completion) * Mark the slab as clean in the slab summary if there are no dirty or writing blocks * and no summary update in progress. */ - if ((slab->active_count > 0) || vdo_has_waiters(&slab->dirty_blocks)) { + if ((slab->active_count > 0) || vdo_waitq_has_waiters(&slab->dirty_blocks)) { check_if_slab_drained(slab); return; } @@ -1175,7 +1176,7 @@ static void handle_io_error(struct vdo_completion *completion) * @waiter: The waiter of the dirty block. * @context: The VIO returned by the pool. */ -static void write_reference_block(struct waiter *waiter, void *context) +static void write_reference_block(struct vdo_waiter *waiter, void *context) { size_t block_offset; physical_block_number_t pbn; @@ -1213,7 +1214,7 @@ static void reclaim_journal_space(struct slab_journal *journal) { block_count_t length = journal_length(journal); struct vdo_slab *slab = journal->slab; - block_count_t write_count = vdo_count_waiters(&slab->dirty_blocks); + block_count_t write_count = vdo_waitq_num_waiters(&slab->dirty_blocks); block_count_t written; if ((length < journal->flushing_threshold) || (write_count == 0)) @@ -1228,8 +1229,8 @@ static void reclaim_journal_space(struct slab_journal *journal) } for (written = 0; written < write_count; written++) { - vdo_notify_next_waiter(&slab->dirty_blocks, - launch_reference_block_write, slab); + vdo_waitq_notify_next_waiter(&slab->dirty_blocks, + launch_reference_block_write, slab); } } @@ -1263,7 +1264,7 @@ static void dirty_block(struct reference_block *block) block->is_dirty = true; if (!block->is_writing) - vdo_enqueue_waiter(&block->slab->dirty_blocks, &block->waiter); + vdo_waitq_enqueue_waiter(&block->slab->dirty_blocks, &block->waiter); } /** @@ -1678,7 +1679,7 @@ static int __must_check adjust_reference_count(struct vdo_slab *slab, * This callback is invoked by add_entries() once it has determined that we are ready to make * another entry in the slab journal. Implements waiter_callback_fn. */ -static void add_entry_from_waiter(struct waiter *waiter, void *context) +static void add_entry_from_waiter(struct vdo_waiter *waiter, void *context) { int result; struct reference_updater *updater = @@ -1744,7 +1745,7 @@ static void add_entry_from_waiter(struct waiter *waiter, void *context) */ static inline bool is_next_entry_a_block_map_increment(struct slab_journal *journal) { - struct waiter *waiter = vdo_get_first_waiter(&journal->entry_waiters); + struct vdo_waiter *waiter = vdo_waitq_get_first_waiter(&journal->entry_waiters); struct reference_updater *updater = container_of(waiter, struct reference_updater, waiter); @@ -1767,7 +1768,7 @@ static void add_entries(struct slab_journal *journal) } journal->adding_entries = true; - while (vdo_has_waiters(&journal->entry_waiters)) { + while (vdo_waitq_has_waiters(&journal->entry_waiters)) { struct slab_journal_block_header *header = &journal->tail_header; if (journal->partial_write_in_progress || @@ -1864,8 +1865,8 @@ static void add_entries(struct slab_journal *journal) } } - vdo_notify_next_waiter(&journal->entry_waiters, - add_entry_from_waiter, journal); + vdo_waitq_notify_next_waiter(&journal->entry_waiters, + add_entry_from_waiter, journal); } journal->adding_entries = false; @@ -1873,7 +1874,7 @@ static void add_entries(struct slab_journal *journal) /* If there are no waiters, and we are flushing or saving, commit the tail block. */ if (vdo_is_state_draining(&journal->slab->state) && !vdo_is_state_suspending(&journal->slab->state) && - !vdo_has_waiters(&journal->entry_waiters)) + !vdo_waitq_has_waiters(&journal->entry_waiters)) commit_tail(journal); } @@ -2259,7 +2260,7 @@ static void load_reference_block_endio(struct bio *bio) * @waiter: The waiter of the block to load. * @context: The VIO returned by the pool. */ -static void load_reference_block(struct waiter *waiter, void *context) +static void load_reference_block(struct vdo_waiter *waiter, void *context) { struct pooled_vio *pooled = context; struct vio *vio = &pooled->vio; @@ -2284,7 +2285,7 @@ static void load_reference_blocks(struct vdo_slab *slab) slab->free_blocks = slab->block_count; slab->active_count = slab->reference_block_count; for (i = 0; i < slab->reference_block_count; i++) { - struct waiter *waiter = &slab->reference_blocks[i].waiter; + struct vdo_waiter *waiter = &slab->reference_blocks[i].waiter; waiter->callback = load_reference_block; acquire_vio_from_pool(slab->allocator->vio_pool, waiter); @@ -2455,7 +2456,7 @@ static void handle_load_error(struct vdo_completion *completion) * * This is the success callback from acquire_vio_from_pool() when loading a slab journal. */ -static void read_slab_journal_tail(struct waiter *waiter, void *context) +static void read_slab_journal_tail(struct vdo_waiter *waiter, void *context) { struct slab_journal *journal = container_of(waiter, struct slab_journal, resource_waiter); @@ -2662,7 +2663,7 @@ static void uninitialize_scrubber_vio(struct slab_scrubber *scrubber) */ static void finish_scrubbing(struct slab_scrubber *scrubber, int result) { - bool notify = vdo_has_waiters(&scrubber->waiters); + bool notify = vdo_waitq_has_waiters(&scrubber->waiters); bool done = !has_slabs_to_scrub(scrubber); struct block_allocator *allocator = container_of(scrubber, struct block_allocator, scrubber); @@ -2709,7 +2710,7 @@ static void finish_scrubbing(struct slab_scrubber *scrubber, int result) * Fortunately if there were waiters, we can't have been freed yet. */ if (notify) - vdo_notify_all_waiters(&scrubber->waiters, NULL, NULL); + vdo_waitq_notify_all_waiters(&scrubber->waiters, NULL, NULL); } static void scrub_next_slab(struct slab_scrubber *scrubber); @@ -2933,7 +2934,7 @@ static void scrub_next_slab(struct slab_scrubber *scrubber) * Note: this notify call is always safe only because scrubbing can only be started when * the VDO is quiescent. */ - vdo_notify_all_waiters(&scrubber->waiters, NULL, NULL); + vdo_waitq_notify_all_waiters(&scrubber->waiters, NULL, NULL); if (vdo_is_read_only(completion->vdo)) { finish_scrubbing(scrubber, VDO_READ_ONLY); @@ -3053,7 +3054,7 @@ static struct vdo_slab *next_slab(struct slab_iterator *iterator) * This callback is invoked on all vios waiting to make slab journal entries after the VDO has gone * into read-only mode. Implements waiter_callback_fn. */ -static void abort_waiter(struct waiter *waiter, void *context __always_unused) +static void abort_waiter(struct vdo_waiter *waiter, void *context __always_unused) { struct reference_updater *updater = container_of(waiter, struct reference_updater, waiter); @@ -3079,8 +3080,8 @@ static void notify_block_allocator_of_read_only_mode(void *listener, while (iterator.next != NULL) { struct vdo_slab *slab = next_slab(&iterator); - vdo_notify_all_waiters(&slab->journal.entry_waiters, - abort_waiter, &slab->journal); + vdo_waitq_notify_all_waiters(&slab->journal.entry_waiters, + abort_waiter, &slab->journal); check_if_slab_drained(slab); } @@ -3210,7 +3211,7 @@ int vdo_allocate_block(struct block_allocator *allocator, * some other error otherwise. */ int vdo_enqueue_clean_slab_waiter(struct block_allocator *allocator, - struct waiter *waiter) + struct vdo_waiter *waiter) { if (vdo_is_read_only(allocator->depot->vdo)) return VDO_READ_ONLY; @@ -3218,7 +3219,7 @@ int vdo_enqueue_clean_slab_waiter(struct block_allocator *allocator, if (vdo_is_state_quiescent(&allocator->scrubber.admin_state)) return VDO_NO_SPACE; - vdo_enqueue_waiter(&allocator->scrubber.waiters, waiter); + vdo_waitq_enqueue_waiter(&allocator->scrubber.waiters, waiter); return VDO_SUCCESS; } @@ -3244,7 +3245,7 @@ void vdo_modify_reference_count(struct vdo_completion *completion, return; } - vdo_enqueue_waiter(&slab->journal.entry_waiters, &updater->waiter); + vdo_waitq_enqueue_waiter(&slab->journal.entry_waiters, &updater->waiter); if ((slab->status != VDO_SLAB_REBUILT) && requires_reaping(&slab->journal)) register_slab_for_scrubbing(slab, true); @@ -3587,7 +3588,7 @@ void vdo_dump_block_allocator(const struct block_allocator *allocator) } uds_log_info(" slab journal: entry_waiters=%zu waiting_to_commit=%s updating_slab_summary=%s head=%llu unreapable=%llu tail=%llu next_commit=%llu summarized=%llu last_summarized=%llu recovery_lock=%llu dirty=%s", - vdo_count_waiters(&journal->entry_waiters), + vdo_waitq_num_waiters(&journal->entry_waiters), uds_bool_to_string(journal->waiting_to_commit), uds_bool_to_string(journal->updating_slab_summary), (unsigned long long) journal->head, @@ -3608,7 +3609,7 @@ void vdo_dump_block_allocator(const struct block_allocator *allocator) uds_log_info(" slab: free=%u/%u blocks=%u dirty=%zu active=%zu journal@(%llu,%u)", slab->free_blocks, slab->block_count, slab->reference_block_count, - vdo_count_waiters(&slab->dirty_blocks), + vdo_waitq_num_waiters(&slab->dirty_blocks), slab->active_count, (unsigned long long) slab->slab_journal_point.sequence_number, slab->slab_journal_point.entry_count); @@ -3628,7 +3629,7 @@ void vdo_dump_block_allocator(const struct block_allocator *allocator) uds_log_info("slab_scrubber slab_count %u waiters %zu %s%s", READ_ONCE(scrubber->slab_count), - vdo_count_waiters(&scrubber->waiters), + vdo_waitq_num_waiters(&scrubber->waiters), vdo_get_admin_state_code(&scrubber->admin_state)->name, scrubber->high_priority_only ? ", high_priority_only " : ""); } diff --git a/drivers/md/dm-vdo/slab-depot.h b/drivers/md/dm-vdo/slab-depot.h index 169021b0811a..efdef566709a 100644 --- a/drivers/md/dm-vdo/slab-depot.h +++ b/drivers/md/dm-vdo/slab-depot.h @@ -60,13 +60,13 @@ struct journal_lock { struct slab_journal { /* A waiter object for getting a VIO pool entry */ - struct waiter resource_waiter; + struct vdo_waiter resource_waiter; /* A waiter object for updating the slab summary */ - struct waiter slab_summary_waiter; + struct vdo_waiter slab_summary_waiter; /* A waiter object for getting a vio with which to flush */ - struct waiter flush_waiter; + struct vdo_waiter flush_waiter; /* The queue of VIOs waiting to make an entry */ - struct wait_queue entry_waiters; + struct vdo_wait_queue entry_waiters; /* The parent slab reference of this journal */ struct vdo_slab *slab; @@ -149,7 +149,7 @@ struct slab_journal { */ struct reference_block { /* This block waits on the ref_counts to tell it to write */ - struct waiter waiter; + struct vdo_waiter waiter; /* The slab to which this reference_block belongs */ struct vdo_slab *slab; /* The number of references in this block that represent allocations */ @@ -241,12 +241,12 @@ struct vdo_slab { struct search_cursor search_cursor; /* A list of the dirty blocks waiting to be written out */ - struct wait_queue dirty_blocks; + struct vdo_wait_queue dirty_blocks; /* The number of blocks which are currently writing */ size_t active_count; /* A waiter object for updating the slab summary */ - struct waiter summary_waiter; + struct vdo_waiter summary_waiter; /* The latest slab journal for which there has been a reference count update */ struct journal_point slab_journal_point; @@ -271,7 +271,7 @@ struct slab_scrubber { /* The queue of slabs to scrub once there are no high_priority_slabs */ struct list_head slabs; /* The queue of VIOs waiting for a slab to be scrubbed */ - struct wait_queue waiters; + struct vdo_wait_queue waiters; /* * The number of slabs that are unrecovered or being scrubbed. This field is modified by @@ -341,9 +341,9 @@ struct slab_summary_block { /* Whether this block has a write outstanding */ bool writing; /* Ring of updates waiting on the outstanding write */ - struct wait_queue current_update_waiters; + struct vdo_wait_queue current_update_waiters; /* Ring of updates waiting on the next write */ - struct wait_queue next_update_waiters; + struct vdo_wait_queue next_update_waiters; /* The active slab_summary_entry array for this block */ struct slab_summary_entry *entries; /* The vio used to write this block */ @@ -522,7 +522,7 @@ int __must_check vdo_allocate_block(struct block_allocator *allocator, physical_block_number_t *block_number_ptr); int vdo_enqueue_clean_slab_waiter(struct block_allocator *allocator, - struct waiter *waiter); + struct vdo_waiter *waiter); void vdo_modify_reference_count(struct vdo_completion *completion, struct reference_updater *updater); diff --git a/drivers/md/dm-vdo/vio.c b/drivers/md/dm-vdo/vio.c index f83b56acc8e4..6acaba149c75 100644 --- a/drivers/md/dm-vdo/vio.c +++ b/drivers/md/dm-vdo/vio.c @@ -25,7 +25,7 @@ struct vio_pool { /** The list of objects which are available */ struct list_head available; /** The queue of requestors waiting for objects from the pool */ - struct wait_queue waiting; + struct vdo_wait_queue waiting; /** The number of objects currently in use */ size_t busy_count; /** The list of objects which are in use */ @@ -364,7 +364,7 @@ void free_vio_pool(struct vio_pool *pool) return; /* Remove all available vios from the object pool. */ - ASSERT_LOG_ONLY(!vdo_has_waiters(&pool->waiting), + ASSERT_LOG_ONLY(!vdo_waitq_has_waiters(&pool->waiting), "VIO pool must not have any waiters when being freed"); ASSERT_LOG_ONLY((pool->busy_count == 0), "VIO pool must not have %zu busy entries when being freed", @@ -400,7 +400,7 @@ bool is_vio_pool_busy(struct vio_pool *pool) * @pool: The vio pool. * @waiter: Object that is requesting a vio. */ -void acquire_vio_from_pool(struct vio_pool *pool, struct waiter *waiter) +void acquire_vio_from_pool(struct vio_pool *pool, struct vdo_waiter *waiter) { struct pooled_vio *pooled; @@ -408,7 +408,7 @@ void acquire_vio_from_pool(struct vio_pool *pool, struct waiter *waiter) "acquire from active vio_pool called from correct thread"); if (list_empty(&pool->available)) { - vdo_enqueue_waiter(&pool->waiting, waiter); + vdo_waitq_enqueue_waiter(&pool->waiting, waiter); return; } @@ -430,8 +430,8 @@ void return_vio_to_pool(struct vio_pool *pool, struct pooled_vio *vio) vio->vio.completion.error_handler = NULL; vio->vio.completion.parent = NULL; - if (vdo_has_waiters(&pool->waiting)) { - vdo_notify_next_waiter(&pool->waiting, NULL, vio); + if (vdo_waitq_has_waiters(&pool->waiting)) { + vdo_waitq_notify_next_waiter(&pool->waiting, NULL, vio); return; } diff --git a/drivers/md/dm-vdo/vio.h b/drivers/md/dm-vdo/vio.h index 3c72fded69b0..71585424f85b 100644 --- a/drivers/md/dm-vdo/vio.h +++ b/drivers/md/dm-vdo/vio.h @@ -193,7 +193,7 @@ int __must_check make_vio_pool(struct vdo *vdo, size_t pool_size, thread_id_t th void *context, struct vio_pool **pool_ptr); void free_vio_pool(struct vio_pool *pool); bool __must_check is_vio_pool_busy(struct vio_pool *pool); -void acquire_vio_from_pool(struct vio_pool *pool, struct waiter *waiter); +void acquire_vio_from_pool(struct vio_pool *pool, struct vdo_waiter *waiter); void return_vio_to_pool(struct vio_pool *pool, struct pooled_vio *vio); #endif /* VIO_H */ diff --git a/drivers/md/dm-vdo/wait-queue.c b/drivers/md/dm-vdo/wait-queue.c index 8acc24e79d2b..9c12a9893823 100644 --- a/drivers/md/dm-vdo/wait-queue.c +++ b/drivers/md/dm-vdo/wait-queue.c @@ -12,211 +12,213 @@ #include "status-codes.h" /** - * vdo_enqueue_waiter() - Add a waiter to the tail end of a wait queue. - * @queue: The queue to which to add the waiter. - * @waiter: The waiter to add to the queue. + * vdo_waitq_enqueue_waiter() - Add a waiter to the tail end of a waitq. + * @waitq: The vdo_wait_queue to which to add the waiter. + * @waiter: The waiter to add to the waitq. * - * The waiter must not already be waiting in a queue. - * - * Return: VDO_SUCCESS or an error code. + * The waiter must not already be waiting in a waitq. */ -void vdo_enqueue_waiter(struct wait_queue *queue, struct waiter *waiter) +void vdo_waitq_enqueue_waiter(struct vdo_wait_queue *waitq, struct vdo_waiter *waiter) { BUG_ON(waiter->next_waiter != NULL); - if (queue->last_waiter == NULL) { + if (waitq->last_waiter == NULL) { /* - * The queue is empty, so form the initial circular list by self-linking the + * The waitq is empty, so form the initial circular list by self-linking the * initial waiter. */ waiter->next_waiter = waiter; } else { - /* Splice the new waiter in at the end of the queue. */ - waiter->next_waiter = queue->last_waiter->next_waiter; - queue->last_waiter->next_waiter = waiter; + /* Splice the new waiter in at the end of the waitq. */ + waiter->next_waiter = waitq->last_waiter->next_waiter; + waitq->last_waiter->next_waiter = waiter; } /* In both cases, the waiter we added to the ring becomes the last waiter. */ - queue->last_waiter = waiter; - queue->queue_length += 1; + waitq->last_waiter = waiter; + waitq->length += 1; } /** - * vdo_transfer_all_waiters() - Transfer all waiters from one wait queue to a second queue, - * emptying the first queue. - * @from_queue: The queue containing the waiters to move. - * @to_queue: The queue that will receive the waiters from the first queue. + * vdo_waitq_transfer_all_waiters() - Transfer all waiters from one waitq to + * a second waitq, emptying the first waitq. + * @from_waitq: The waitq containing the waiters to move. + * @to_waitq: The waitq that will receive the waiters from the first waitq. */ -void vdo_transfer_all_waiters(struct wait_queue *from_queue, struct wait_queue *to_queue) +void vdo_waitq_transfer_all_waiters(struct vdo_wait_queue *from_waitq, + struct vdo_wait_queue *to_waitq) { - /* If the source queue is empty, there's nothing to do. */ - if (!vdo_has_waiters(from_queue)) + /* If the source waitq is empty, there's nothing to do. */ + if (!vdo_waitq_has_waiters(from_waitq)) return; - if (vdo_has_waiters(to_queue)) { + if (vdo_waitq_has_waiters(to_waitq)) { /* - * Both queues are non-empty. Splice the two circular lists together by swapping - * the next (head) pointers in the list tails. + * Both are non-empty. Splice the two circular lists together + * by swapping the next (head) pointers in the list tails. */ - struct waiter *from_head = from_queue->last_waiter->next_waiter; - struct waiter *to_head = to_queue->last_waiter->next_waiter; + struct vdo_waiter *from_head = from_waitq->last_waiter->next_waiter; + struct vdo_waiter *to_head = to_waitq->last_waiter->next_waiter; - to_queue->last_waiter->next_waiter = from_head; - from_queue->last_waiter->next_waiter = to_head; + to_waitq->last_waiter->next_waiter = from_head; + from_waitq->last_waiter->next_waiter = to_head; } - to_queue->last_waiter = from_queue->last_waiter; - to_queue->queue_length += from_queue->queue_length; - vdo_initialize_wait_queue(from_queue); + to_waitq->last_waiter = from_waitq->last_waiter; + to_waitq->length += from_waitq->length; + vdo_waitq_init(from_waitq); } /** - * vdo_notify_all_waiters() - Notify all the entries waiting in a queue. - * @queue: The wait queue containing the waiters to notify. + * vdo_waitq_notify_all_waiters() - Notify all the entries waiting in a waitq. + * @waitq: The vdo_wait_queue containing the waiters to notify. * @callback: The function to call to notify each waiter, or NULL to invoke the callback field * registered in each waiter. * @context: The context to pass to the callback function. * - * Notifies all the entries waiting in a queue to continue execution by invoking a callback - * function on each of them in turn. The queue is copied and emptied before invoking any callbacks, - * and only the waiters that were in the queue at the start of the call will be notified. + * Notifies all the entries waiting in a waitq to continue execution by invoking a callback + * function on each of them in turn. The waitq is copied and emptied before invoking any callbacks, + * and only the waiters that were in the waitq at the start of the call will be notified. */ -void vdo_notify_all_waiters(struct wait_queue *queue, waiter_callback_fn callback, - void *context) +void vdo_waitq_notify_all_waiters(struct vdo_wait_queue *waitq, + vdo_waiter_callback_fn callback, void *context) { /* - * Copy and empty the queue first, avoiding the possibility of an infinite loop if entries - * are returned to the queue by the callback function. + * Copy and empty the waitq first, avoiding the possibility of an infinite + * loop if entries are returned to the waitq by the callback function. */ - struct wait_queue waiters; + struct vdo_wait_queue waiters; - vdo_initialize_wait_queue(&waiters); - vdo_transfer_all_waiters(queue, &waiters); + vdo_waitq_init(&waiters); + vdo_waitq_transfer_all_waiters(waitq, &waiters); - /* Drain the copied queue, invoking the callback on every entry. */ - while (vdo_has_waiters(&waiters)) - vdo_notify_next_waiter(&waiters, callback, context); + /* Drain the copied waitq, invoking the callback on every entry. */ + while (vdo_waitq_has_waiters(&waiters)) + vdo_waitq_notify_next_waiter(&waiters, callback, context); } /** - * vdo_get_first_waiter() - Return the waiter that is at the head end of a wait queue. - * @queue: The queue from which to get the first waiter. + * vdo_waitq_get_first_waiter() - Return the waiter that is at the head end of a waitq. + * @waitq: The vdo_wait_queue from which to get the first waiter. * - * Return: The first (oldest) waiter in the queue, or NULL if the queue is empty. + * Return: The first (oldest) waiter in the waitq, or NULL if the waitq is empty. */ -struct waiter *vdo_get_first_waiter(const struct wait_queue *queue) +struct vdo_waiter *vdo_waitq_get_first_waiter(const struct vdo_wait_queue *waitq) { - struct waiter *last_waiter = queue->last_waiter; + struct vdo_waiter *last_waiter = waitq->last_waiter; if (last_waiter == NULL) { /* There are no waiters, so we're done. */ return NULL; } - /* The queue is circular, so the last entry links to the head of the queue. */ + /* The waitq is circular, so the last entry links to the head of the waitq. */ return last_waiter->next_waiter; } /** - * vdo_dequeue_matching_waiters() - Remove all waiters that match based on the specified matching - * method and append them to a wait_queue. - * @queue: The wait queue to process. - * @match_method: The method to determine matching. + * vdo_waitq_dequeue_matching_waiters() - Remove all waiters that match based on the specified + * matching method and append them to a vdo_wait_queue. + * @waitq: The vdo_wait_queue to process. + * @waiter_match: The method to determine matching. * @match_context: Contextual info for the match method. - * @matched_queue: A wait_queue to store matches. + * @matched_waitq: A wait_waitq to store matches. */ -void vdo_dequeue_matching_waiters(struct wait_queue *queue, waiter_match_fn match_method, - void *match_context, struct wait_queue *matched_queue) +void vdo_waitq_dequeue_matching_waiters(struct vdo_wait_queue *waitq, + vdo_waiter_match_fn waiter_match, + void *match_context, + struct vdo_wait_queue *matched_waitq) { - struct wait_queue matched_waiters, iteration_queue; + // FIXME: copying a waitq just to iterate it, with matching, is unfortunate + struct vdo_wait_queue matched_waiters, iteration_waitq; - vdo_initialize_wait_queue(&matched_waiters); + vdo_waitq_init(&matched_waiters); + vdo_waitq_init(&iteration_waitq); + vdo_waitq_transfer_all_waiters(waitq, &iteration_waitq); - vdo_initialize_wait_queue(&iteration_queue); - vdo_transfer_all_waiters(queue, &iteration_queue); - while (vdo_has_waiters(&iteration_queue)) { - struct waiter *waiter = vdo_dequeue_next_waiter(&iteration_queue); + while (vdo_waitq_has_waiters(&iteration_waitq)) { + struct vdo_waiter *waiter = vdo_waitq_dequeue_next_waiter(&iteration_waitq); - vdo_enqueue_waiter((match_method(waiter, match_context) ? - &matched_waiters : queue), waiter); + vdo_waitq_enqueue_waiter((waiter_match(waiter, match_context) ? + &matched_waiters : waitq), waiter); } - vdo_transfer_all_waiters(&matched_waiters, matched_queue); + vdo_waitq_transfer_all_waiters(&matched_waiters, matched_waitq); } /** - * vdo_dequeue_next_waiter() - Remove the first waiter from the head end of a wait queue. - * @queue: The wait queue from which to remove the first entry. + * vdo_waitq_dequeue_next_waiter() - Remove the first waiter from the head end of a waitq. + * @waitq: The vdo_wait_queue from which to remove the first entry. * * The caller will be responsible for waking the waiter by invoking the correct callback function * to resume its execution. * - * Return: The first (oldest) waiter in the queue, or NULL if the queue is empty. + * Return: The first (oldest) waiter in the waitq, or NULL if the waitq is empty. */ -struct waiter *vdo_dequeue_next_waiter(struct wait_queue *queue) +struct vdo_waiter *vdo_waitq_dequeue_next_waiter(struct vdo_wait_queue *waitq) { - struct waiter *first_waiter = vdo_get_first_waiter(queue); - struct waiter *last_waiter = queue->last_waiter; + struct vdo_waiter *first_waiter = vdo_waitq_get_first_waiter(waitq); + struct vdo_waiter *last_waiter = waitq->last_waiter; if (first_waiter == NULL) return NULL; if (first_waiter == last_waiter) { - /* The queue has a single entry, so just empty it out by nulling the tail. */ - queue->last_waiter = NULL; + /* The waitq has a single entry, so just empty it out by nulling the tail. */ + waitq->last_waiter = NULL; } else { /* - * The queue has more than one entry, so splice the first waiter out of the - * circular queue. + * The waitq has more than one entry, so splice the first waiter out of the + * circular waitq. */ last_waiter->next_waiter = first_waiter->next_waiter; } - /* The waiter is no longer in a wait queue. */ + /* The waiter is no longer in a waitq. */ first_waiter->next_waiter = NULL; - queue->queue_length -= 1; + waitq->length -= 1; return first_waiter; } /** - * vdo_notify_next_waiter() - Notify the next entry waiting in a queue. - * @queue: The wait queue containing the waiter to notify. + * vdo_waitq_notify_next_waiter() - Notify the next entry waiting in a waitq. + * @waitq: The vdo_wait_queue containing the waiter to notify. * @callback: The function to call to notify the waiter, or NULL to invoke the callback field * registered in the waiter. * @context: The context to pass to the callback function. * - * Notifies the next entry waiting in a queue to continue execution by invoking a callback function - * on it after removing it from the queue. + * Notifies the next entry waiting in a waitq to continue execution by invoking a callback function + * on it after removing it from the waitq. * - * Return: true if there was a waiter in the queue. + * Return: true if there was a waiter in the waitq. */ -bool vdo_notify_next_waiter(struct wait_queue *queue, waiter_callback_fn callback, - void *context) +bool vdo_waitq_notify_next_waiter(struct vdo_wait_queue *waitq, + vdo_waiter_callback_fn callback, void *context) { - struct waiter *waiter = vdo_dequeue_next_waiter(queue); + struct vdo_waiter *waiter = vdo_waitq_dequeue_next_waiter(waitq); if (waiter == NULL) return false; if (callback == NULL) callback = waiter->callback; - (*callback)(waiter, context); + callback(waiter, context); return true; } /** - * vdo_get_next_waiter() - Get the waiter after this one, for debug iteration. - * @queue: The wait queue. + * vdo_waitq_get_next_waiter() - Get the waiter after this one, for debug iteration. + * @waitq: The vdo_wait_queue. * @waiter: A waiter. * * Return: The next waiter, or NULL. */ -const struct waiter *vdo_get_next_waiter(const struct wait_queue *queue, - const struct waiter *waiter) +const struct vdo_waiter *vdo_waitq_get_next_waiter(const struct vdo_wait_queue *waitq, + const struct vdo_waiter *waiter) { - struct waiter *first_waiter = vdo_get_first_waiter(queue); + struct vdo_waiter *first_waiter = vdo_waitq_get_first_waiter(waitq); if (waiter == NULL) return first_waiter; diff --git a/drivers/md/dm-vdo/wait-queue.h b/drivers/md/dm-vdo/wait-queue.h index 50f1e2a1ea67..b92f12dd5b4b 100644 --- a/drivers/md/dm-vdo/wait-queue.h +++ b/drivers/md/dm-vdo/wait-queue.h @@ -10,122 +10,132 @@ #include /** - * DOC: Wait queues. + * A vdo_wait_queue is a circular singly linked list of entries waiting to be notified + * of a change in a condition. Keeping a circular list allows the vdo_wait_queue + * structure to simply be a pointer to the tail (newest) entry, supporting + * constant-time enqueue and dequeue operations. A null pointer is an empty waitq. * - * A wait queue is a circular list of entries waiting to be notified of a change in a condition. - * Keeping a circular list allows the queue structure to simply be a pointer to the tail (newest) - * entry in the queue, supporting constant-time enqueue and dequeue operations. A null pointer is - * an empty queue. + * An empty waitq: + * waitq0.last_waiter -> NULL * - * An empty queue: - * queue0.last_waiter -> NULL + * A singleton waitq: + * waitq1.last_waiter -> entry1 -> entry1 -> [...] * - * A singleton queue: - * queue1.last_waiter -> entry1 -> entry1 -> [...] + * A three-element waitq: + * waitq2.last_waiter -> entry3 -> entry1 -> entry2 -> entry3 -> [...] * - * A three-element queue: - * queue2.last_waiter -> entry3 -> entry1 -> entry2 -> entry3 -> [...] + * linux/wait.h's wait_queue_head is _not_ used because vdo_wait_queue's + * interface is much less complex (doesn't need locking, priorities or timers). + * Made possible by vdo's thread-based resource allocation and locking; and + * the polling nature of vdo_wait_queue consumers. + * + * FIXME: could be made to use a linux/list.h's list_head but its extra barriers + * really aren't needed. Nor is a doubly linked list, but vdo_wait_queue could + * make use of __list_del_clearprev() -- but that would compromise the ability + * to make full use of linux's list interface. */ -struct waiter; +struct vdo_waiter; -struct wait_queue { +struct vdo_wait_queue { /* The tail of the queue, the last (most recently added) entry */ - struct waiter *last_waiter; + struct vdo_waiter *last_waiter; /* The number of waiters currently in the queue */ - size_t queue_length; + size_t length; }; /** - * typedef waiter_callback_fn - Callback type for functions which will be called to resume - * processing of a waiter after it has been removed from its wait - * queue. + * vdo_waiter_callback_fn - Callback type that will be called to resume processing + * of a waiter after it has been removed from its wait queue. */ -typedef void (*waiter_callback_fn)(struct waiter *waiter, void *context); +typedef void (*vdo_waiter_callback_fn)(struct vdo_waiter *waiter, void *context); /** - * typedef waiter_match_fn - Method type for waiter matching methods. + * vdo_waiter_match_fn - Method type for waiter matching methods. * - * A waiter_match_fn method returns false if the waiter does not match. + * Returns false if the waiter does not match. */ -typedef bool (*waiter_match_fn)(struct waiter *waiter, void *context); +typedef bool (*vdo_waiter_match_fn)(struct vdo_waiter *waiter, void *context); -/* The queue entry structure for entries in a wait_queue. */ -struct waiter { +/* The structure for entries in a vdo_wait_queue. */ +struct vdo_waiter { /* - * The next waiter in the queue. If this entry is the last waiter, then this is actually a - * pointer back to the head of the queue. + * The next waiter in the waitq. If this entry is the last waiter, then this + * is actually a pointer back to the head of the waitq. */ - struct waiter *next_waiter; + struct vdo_waiter *next_waiter; - /* Optional waiter-specific callback to invoke when waking this waiter. */ - waiter_callback_fn callback; + /* Optional waiter-specific callback to invoke when dequeuing this waiter. */ + vdo_waiter_callback_fn callback; }; /** - * is_waiting() - Check whether a waiter is waiting. + * vdo_waiter_is_waiting() - Check whether a waiter is waiting. * @waiter: The waiter to check. * - * Return: true if the waiter is on some wait_queue. + * Return: true if the waiter is on some vdo_wait_queue. */ -static inline bool vdo_is_waiting(struct waiter *waiter) +static inline bool vdo_waiter_is_waiting(struct vdo_waiter *waiter) { return (waiter->next_waiter != NULL); } /** - * initialize_wait_queue() - Initialize a wait queue. - * @queue: The queue to initialize. + * vdo_waitq_init() - Initialize a vdo_wait_queue. + * @waitq: The vdo_wait_queue to initialize. */ -static inline void vdo_initialize_wait_queue(struct wait_queue *queue) +static inline void vdo_waitq_init(struct vdo_wait_queue *waitq) { - *queue = (struct wait_queue) { + *waitq = (struct vdo_wait_queue) { .last_waiter = NULL, - .queue_length = 0, + .length = 0, }; } /** - * has_waiters() - Check whether a wait queue has any entries waiting in it. - * @queue: The queue to query. + * vdo_waitq_has_waiters() - Check whether a vdo_wait_queue has any entries waiting. + * @waitq: The vdo_wait_queue to query. * - * Return: true if there are any waiters in the queue. + * Return: true if there are any waiters in the waitq. */ -static inline bool __must_check vdo_has_waiters(const struct wait_queue *queue) +static inline bool __must_check vdo_waitq_has_waiters(const struct vdo_wait_queue *waitq) { - return (queue->last_waiter != NULL); + return (waitq->last_waiter != NULL); } -void vdo_enqueue_waiter(struct wait_queue *queue, struct waiter *waiter); +void vdo_waitq_enqueue_waiter(struct vdo_wait_queue *waitq, + struct vdo_waiter *waiter); -void vdo_notify_all_waiters(struct wait_queue *queue, waiter_callback_fn callback, - void *context); +void vdo_waitq_notify_all_waiters(struct vdo_wait_queue *waitq, + vdo_waiter_callback_fn callback, void *context); -bool vdo_notify_next_waiter(struct wait_queue *queue, waiter_callback_fn callback, - void *context); +bool vdo_waitq_notify_next_waiter(struct vdo_wait_queue *waitq, + vdo_waiter_callback_fn callback, void *context); -void vdo_transfer_all_waiters(struct wait_queue *from_queue, - struct wait_queue *to_queue); +void vdo_waitq_transfer_all_waiters(struct vdo_wait_queue *from_waitq, + struct vdo_wait_queue *to_waitq); -struct waiter *vdo_get_first_waiter(const struct wait_queue *queue); +struct vdo_waiter *vdo_waitq_get_first_waiter(const struct vdo_wait_queue *waitq); -void vdo_dequeue_matching_waiters(struct wait_queue *queue, waiter_match_fn match_method, - void *match_context, struct wait_queue *matched_queue); +void vdo_waitq_dequeue_matching_waiters(struct vdo_wait_queue *waitq, + vdo_waiter_match_fn waiter_match, + void *match_context, + struct vdo_wait_queue *matched_waitq); -struct waiter *vdo_dequeue_next_waiter(struct wait_queue *queue); +struct vdo_waiter *vdo_waitq_dequeue_next_waiter(struct vdo_wait_queue *waitq); /** - * count_waiters() - Count the number of waiters in a wait queue. - * @queue: The wait queue to query. + * vdo_waitq_num_waiters() - Return the number of waiters in a vdo_wait_queue. + * @waitq: The vdo_wait_queue to query. * - * Return: The number of waiters in the queue. + * Return: The number of waiters in the waitq. */ -static inline size_t __must_check vdo_count_waiters(const struct wait_queue *queue) +static inline size_t __must_check vdo_waitq_num_waiters(const struct vdo_wait_queue *waitq) { - return queue->queue_length; + return waitq->length; } -const struct waiter * __must_check vdo_get_next_waiter(const struct wait_queue *queue, - const struct waiter *waiter); +const struct vdo_waiter * __must_check +vdo_waitq_get_next_waiter(const struct vdo_wait_queue *waitq, const struct vdo_waiter *waiter); #endif /* VDO_WAIT_QUEUE_H */ -- cgit v1.2.3 From b06d5c37b88b97359e98820eecae99e243512c1b Mon Sep 17 00:00:00 2001 From: Mike Snitzer Date: Fri, 26 Jan 2024 21:35:49 -0500 Subject: dm vdo block-map: rename struct cursors member to 'completion' 'completion' is more informative name for a 'struct vdo_completion' than 'parent'. Signed-off-by: Mike Snitzer Signed-off-by: Matthew Sakai --- drivers/md/dm-vdo/block-map.c | 20 +++++++++----------- drivers/md/dm-vdo/block-map.h | 2 +- 2 files changed, 10 insertions(+), 12 deletions(-) (limited to 'drivers/md/dm-vdo/block-map.h') diff --git a/drivers/md/dm-vdo/block-map.c b/drivers/md/dm-vdo/block-map.c index 0ce136612869..50dda7ae7074 100644 --- a/drivers/md/dm-vdo/block-map.c +++ b/drivers/md/dm-vdo/block-map.c @@ -98,7 +98,7 @@ struct cursors { struct block_map_zone *zone; struct vio_pool *pool; vdo_entry_callback_fn entry_callback; - struct vdo_completion *parent; + struct vdo_completion *completion; root_count_t active_roots; struct cursor cursors[]; }; @@ -2501,7 +2501,7 @@ static void replace_forest(struct block_map *map) static void finish_cursor(struct cursor *cursor) { struct cursors *cursors = cursor->parent; - struct vdo_completion *parent = cursors->parent; + struct vdo_completion *completion = cursors->completion; return_vio_to_pool(cursors->pool, uds_forget(cursor->vio)); if (--cursors->active_roots > 0) @@ -2509,7 +2509,7 @@ static void finish_cursor(struct cursor *cursor) uds_free(cursors); - vdo_finish_completion(parent); + vdo_finish_completion(completion); } static void traverse(struct cursor *cursor); @@ -2595,12 +2595,10 @@ static void traverse(struct cursor *cursor) if (cursor->height < VDO_BLOCK_MAP_TREE_HEIGHT - 1) { int result = cursor->parent->entry_callback(location.pbn, - cursor->parent->parent); - + cursor->parent->completion); if (result != VDO_SUCCESS) { page->entries[level->slot] = UNMAPPED_BLOCK_MAP_ENTRY; - vdo_write_tree_page(tree_page, - cursor->parent->zone); + vdo_write_tree_page(tree_page, cursor->parent->zone); continue; } } @@ -2676,10 +2674,10 @@ static struct boundary compute_boundary(struct block_map *map, root_count_t root /** * vdo_traverse_forest() - Walk the entire forest of a block map. * @callback: A function to call with the pbn of each allocated node in the forest. - * @parent: The completion to notify on each traversed PBN, and when the traversal is complete. + * @completion: The completion to notify on each traversed PBN, and when traversal completes. */ void vdo_traverse_forest(struct block_map *map, vdo_entry_callback_fn callback, - struct vdo_completion *parent) + struct vdo_completion *completion) { root_count_t root; struct cursors *cursors; @@ -2688,14 +2686,14 @@ void vdo_traverse_forest(struct block_map *map, vdo_entry_callback_fn callback, result = uds_allocate_extended(struct cursors, map->root_count, struct cursor, __func__, &cursors); if (result != VDO_SUCCESS) { - vdo_fail_completion(parent, result); + vdo_fail_completion(completion, result); return; } cursors->zone = &map->zones[0]; cursors->pool = cursors->zone->vio_pool; cursors->entry_callback = callback; - cursors->parent = parent; + cursors->completion = completion; cursors->active_roots = map->root_count; for (root = 0; root < map->root_count; root++) { struct cursor *cursor = &cursors->cursors[root]; diff --git a/drivers/md/dm-vdo/block-map.h b/drivers/md/dm-vdo/block-map.h index cc98d19309ce..c574bd524bc2 100644 --- a/drivers/md/dm-vdo/block-map.h +++ b/drivers/md/dm-vdo/block-map.h @@ -313,7 +313,7 @@ physical_block_number_t vdo_find_block_map_page_pbn(struct block_map *map, void vdo_write_tree_page(struct tree_page *page, struct block_map_zone *zone); void vdo_traverse_forest(struct block_map *map, vdo_entry_callback_fn callback, - struct vdo_completion *parent); + struct vdo_completion *completion); int __must_check vdo_decode_block_map(struct block_map_state_2_0 state, block_count_t logical_blocks, struct vdo *vdo, -- cgit v1.2.3 From ea9ca07affd80668b207703919eaba849654e11f Mon Sep 17 00:00:00 2001 From: Matthew Sakai Date: Tue, 6 Feb 2024 22:00:42 -0500 Subject: dm vdo: add documentation details on zones and locking Add details describing the vdo zone and thread model to the documentation comments for major vdo components. Also added some high-level description of the block map structure. Signed-off-by: Matthew Sakai Signed-off-by: Mike Snitzer --- drivers/md/dm-vdo/block-map.h | 15 +++++++++++++++ drivers/md/dm-vdo/dedupe.c | 5 +++++ drivers/md/dm-vdo/recovery-journal.h | 4 ++++ drivers/md/dm-vdo/slab-depot.h | 16 +++++++++++----- 4 files changed, 35 insertions(+), 5 deletions(-) (limited to 'drivers/md/dm-vdo/block-map.h') diff --git a/drivers/md/dm-vdo/block-map.h b/drivers/md/dm-vdo/block-map.h index c574bd524bc2..b662c318c2ea 100644 --- a/drivers/md/dm-vdo/block-map.h +++ b/drivers/md/dm-vdo/block-map.h @@ -19,6 +19,21 @@ #include "vio.h" #include "wait-queue.h" +/* + * The block map is responsible for tracking all the logical to physical mappings of a VDO. It + * consists of a collection of 60 radix trees gradually allocated as logical addresses are used. + * Each tree is assigned to a logical zone such that it is easy to compute which zone must handle + * each logical address. Each logical zone also has a dedicated portion of the leaf page cache. + * + * Each logical zone has a single dedicated queue and thread for performing all updates to the + * radix trees assigned to that zone. The concurrency guarantees of this single-threaded model + * allow the code to omit more fine-grained locking for the block map structures. + * + * Load operations must be performed on the admin thread. Normal operations, such as reading and + * updating mappings, must be performed on the appropriate logical zone thread. Save operations + * must be launched from the same admin thread as the original load operation. + */ + enum { BLOCK_MAP_VIO_POOL_SIZE = 64, }; diff --git a/drivers/md/dm-vdo/dedupe.c b/drivers/md/dm-vdo/dedupe.c index 4b00135511dd..d81065a0951c 100644 --- a/drivers/md/dm-vdo/dedupe.c +++ b/drivers/md/dm-vdo/dedupe.c @@ -14,6 +14,11 @@ * deduplicate against a single block instead of being serialized through a PBN read lock. Only one * index query is needed for each hash_lock, instead of one for every data_vio. * + * Hash_locks are assigned to hash_zones by computing a modulus on the hash itself. Each hash_zone + * has a single dedicated queue and thread for performing all operations on the hash_locks assigned + * to that zone. The concurrency guarantees of this single-threaded model allow the code to omit + * more fine-grained locking for the hash_lock structures. + * * A hash_lock acts like a state machine perhaps more than as a lock. Other than the starting and * ending states INITIALIZING and BYPASSING, every state represents and is held for the duration of * an asynchronous operation. All state transitions are performed on the thread of the hash_zone diff --git a/drivers/md/dm-vdo/recovery-journal.h b/drivers/md/dm-vdo/recovery-journal.h index 19fa7ed9648a..d78c6c7da4ea 100644 --- a/drivers/md/dm-vdo/recovery-journal.h +++ b/drivers/md/dm-vdo/recovery-journal.h @@ -26,6 +26,10 @@ * write amplification of writes by providing amortization of slab journal and block map page * updates. * + * The recovery journal has a single dedicated queue and thread for performing all journal updates. + * The concurrency guarantees of this single-threaded model allow the code to omit more + * fine-grained locking for recovery journal structures. + * * The journal consists of a set of on-disk blocks arranged as a circular log with monotonically * increasing sequence numbers. Three sequence numbers serve to define the active extent of the * journal. The 'head' is the oldest active block in the journal. The 'tail' is the end of the diff --git a/drivers/md/dm-vdo/slab-depot.h b/drivers/md/dm-vdo/slab-depot.h index efdef566709a..fba293f9713e 100644 --- a/drivers/md/dm-vdo/slab-depot.h +++ b/drivers/md/dm-vdo/slab-depot.h @@ -29,11 +29,17 @@ * a single array of slabs in order to eliminate the need for additional math in order to compute * which physical zone a PBN is in. It also has a block_allocator per zone. * - * Load operations are required to be performed on a single thread. Normal operations are assumed - * to be performed in the appropriate zone. Allocations and reference count updates must be done - * from the thread of their physical zone. Requests to commit slab journal tail blocks from the - * recovery journal must be done on the journal zone thread. Save operations are required to be - * launched from the same thread as the original load operation. + * Each physical zone has a single dedicated queue and thread for performing all updates to the + * slabs assigned to that zone. The concurrency guarantees of this single-threaded model allow the + * code to omit more fine-grained locking for the various slab structures. Each physical zone + * maintains a separate copy of the slab summary to remove the need for explicit locking on that + * structure as well. + * + * Load operations must be performed on the admin thread. Normal operations, such as allocations + * and reference count updates, must be performed on the appropriate physical zone thread. Requests + * from the recovery journal to commit slab journal tail blocks must be scheduled from the recovery + * journal thread to run on the appropriate physical zone thread. Save operations must be launched + * from the same admin thread as the original load operation. */ enum { -- cgit v1.2.3 From 571eff3969a58cce63b02acda4462aa0ca12e116 Mon Sep 17 00:00:00 2001 From: Mike Snitzer Date: Tue, 13 Feb 2024 23:57:10 -0500 Subject: dm vdo: cleanup style for comments in structs Use /* ... */ rather than /** ... */ if for no other reason than syntax highlighting is improved (at least for me, in emacs: comments are now red, code is yellow. Previously comments were also yellow). Signed-off-by: Mike Snitzer Signed-off-by: Susan LeGendre-McGhee Signed-off-by: Matthew Sakai --- drivers/md/dm-vdo/block-map.h | 12 +-- drivers/md/dm-vdo/constants.h | 38 +++---- drivers/md/dm-vdo/encodings.h | 8 +- drivers/md/dm-vdo/flush.c | 28 ++--- drivers/md/dm-vdo/recovery-journal.h | 22 ++-- drivers/md/dm-vdo/statistics.h | 194 +++++++++++++++++------------------ drivers/md/dm-vdo/vdo.h | 8 +- drivers/md/dm-vdo/vio.c | 16 +-- 8 files changed, 163 insertions(+), 163 deletions(-) (limited to 'drivers/md/dm-vdo/block-map.h') diff --git a/drivers/md/dm-vdo/block-map.h b/drivers/md/dm-vdo/block-map.h index b662c318c2ea..39a13039e4a3 100644 --- a/drivers/md/dm-vdo/block-map.h +++ b/drivers/md/dm-vdo/block-map.h @@ -216,17 +216,17 @@ enum block_map_page_type { typedef struct list_head dirty_era_t[2]; struct dirty_lists { - /** The number of periods after which an element will be expired */ + /* The number of periods after which an element will be expired */ block_count_t maximum_age; - /** The oldest period which has unexpired elements */ + /* The oldest period which has unexpired elements */ sequence_number_t oldest_period; - /** One more than the current period */ + /* One more than the current period */ sequence_number_t next_period; - /** The offset in the array of lists of the oldest period */ + /* The offset in the array of lists of the oldest period */ block_count_t offset; - /** Expired pages */ + /* Expired pages */ dirty_era_t expired; - /** The lists of dirty pages */ + /* The lists of dirty pages */ dirty_era_t eras[]; }; diff --git a/drivers/md/dm-vdo/constants.h b/drivers/md/dm-vdo/constants.h index 24237bc29f87..a8c4d6e24b38 100644 --- a/drivers/md/dm-vdo/constants.h +++ b/drivers/md/dm-vdo/constants.h @@ -17,10 +17,10 @@ enum { */ VDO_BIO_ROTATION_INTERVAL_LIMIT = 1024, - /** The number of entries on a block map page */ + /* The number of entries on a block map page */ VDO_BLOCK_MAP_ENTRIES_PER_PAGE = 812, - /** The origin of the flat portion of the block map */ + /* The origin of the flat portion of the block map */ VDO_BLOCK_MAP_FLAT_PAGE_ORIGIN = 1, /* @@ -29,22 +29,22 @@ enum { */ VDO_BLOCK_MAP_TREE_HEIGHT = 5, - /** The default number of bio submission queues. */ + /* The default number of bio submission queues. */ DEFAULT_VDO_BIO_SUBMIT_QUEUE_COUNT = 4, - /** The number of contiguous PBNs to be submitted to a single bio queue. */ + /* The number of contiguous PBNs to be submitted to a single bio queue. */ DEFAULT_VDO_BIO_SUBMIT_QUEUE_ROTATE_INTERVAL = 64, - /** The number of trees in the arboreal block map */ + /* The number of trees in the arboreal block map */ DEFAULT_VDO_BLOCK_MAP_TREE_ROOT_COUNT = 60, - /** The default size of the recovery journal, in blocks */ + /* The default size of the recovery journal, in blocks */ DEFAULT_VDO_RECOVERY_JOURNAL_SIZE = 32 * 1024, - /** The default size of each slab journal, in blocks */ + /* The default size of each slab journal, in blocks */ DEFAULT_VDO_SLAB_JOURNAL_SIZE = 224, - /** Unit test minimum */ + /* Unit test minimum */ MINIMUM_VDO_SLAB_JOURNAL_BLOCKS = 2, /* @@ -54,16 +54,16 @@ enum { */ VDO_LOCK_MAP_CAPACITY = 10000, - /** The maximum number of logical zones */ + /* The maximum number of logical zones */ MAX_VDO_LOGICAL_ZONES = 60, - /** The maximum number of physical zones */ + /* The maximum number of physical zones */ MAX_VDO_PHYSICAL_ZONES = 16, - /** The base-2 logarithm of the maximum blocks in one slab */ + /* The base-2 logarithm of the maximum blocks in one slab */ MAX_VDO_SLAB_BITS = 23, - /** The maximum number of slabs the slab depot supports */ + /* The maximum number of slabs the slab depot supports */ MAX_VDO_SLABS = 8192, /* @@ -71,25 +71,25 @@ enum { */ MAXIMUM_SIMULTANEOUS_VDO_BLOCK_MAP_RESTORATION_READS = 1024, - /** The maximum number of entries in the slab summary */ + /* The maximum number of entries in the slab summary */ MAXIMUM_VDO_SLAB_SUMMARY_ENTRIES = MAX_VDO_SLABS * MAX_VDO_PHYSICAL_ZONES, - /** The maximum number of total threads in a VDO thread configuration. */ + /* The maximum number of total threads in a VDO thread configuration. */ MAXIMUM_VDO_THREADS = 100, - /** The maximum number of VIOs in the system at once */ + /* The maximum number of VIOs in the system at once */ MAXIMUM_VDO_USER_VIOS = 2048, - /** The only physical block size supported by VDO */ + /* The only physical block size supported by VDO */ VDO_BLOCK_SIZE = 4096, - /** The number of sectors per block */ + /* The number of sectors per block */ VDO_SECTORS_PER_BLOCK = (VDO_BLOCK_SIZE >> SECTOR_SHIFT), - /** The size of a sector that will not be torn */ + /* The size of a sector that will not be torn */ VDO_SECTOR_SIZE = 512, - /** The physical block number reserved for storing the zero block */ + /* The physical block number reserved for storing the zero block */ VDO_ZERO_BLOCK = 0, }; diff --git a/drivers/md/dm-vdo/encodings.h b/drivers/md/dm-vdo/encodings.h index 260822a3958a..ba3db9867f4a 100644 --- a/drivers/md/dm-vdo/encodings.h +++ b/drivers/md/dm-vdo/encodings.h @@ -155,7 +155,7 @@ struct block_map_page_header { __le64 nonce; __le64 pbn; - /** May be non-zero on disk */ + /* May be non-zero on disk */ u8 unused_long_word[8]; /* Whether this page has been written twice to disk */ @@ -197,11 +197,11 @@ extern const struct header VDO_BLOCK_MAP_HEADER_2_0; /* The state of the recovery journal as encoded in the VDO super block. */ struct recovery_journal_state_7_0 { - /** Sequence number to start the journal */ + /* Sequence number to start the journal */ sequence_number_t journal_start; - /** Number of logical blocks used by VDO */ + /* Number of logical blocks used by VDO */ block_count_t logical_blocks_used; - /** Number of block map pages allocated */ + /* Number of block map pages allocated */ block_count_t block_map_data_blocks; } __packed; diff --git a/drivers/md/dm-vdo/flush.c b/drivers/md/dm-vdo/flush.c index a6eeb425d721..330b18715027 100644 --- a/drivers/md/dm-vdo/flush.c +++ b/drivers/md/dm-vdo/flush.c @@ -22,33 +22,33 @@ struct flusher { struct vdo_completion completion; - /** The vdo to which this flusher belongs */ + /* The vdo to which this flusher belongs */ struct vdo *vdo; - /** The administrative state of the flusher */ + /* The administrative state of the flusher */ struct admin_state state; - /** The current flush generation of the vdo */ + /* The current flush generation of the vdo */ sequence_number_t flush_generation; - /** The first unacknowledged flush generation */ + /* The first unacknowledged flush generation */ sequence_number_t first_unacknowledged_generation; - /** The queue of flush requests waiting to notify other threads */ + /* The queue of flush requests waiting to notify other threads */ struct vdo_wait_queue notifiers; - /** The queue of flush requests waiting for VIOs to complete */ + /* The queue of flush requests waiting for VIOs to complete */ struct vdo_wait_queue pending_flushes; - /** The flush generation for which notifications are being sent */ + /* The flush generation for which notifications are being sent */ sequence_number_t notify_generation; - /** The logical zone to notify next */ + /* The logical zone to notify next */ struct logical_zone *logical_zone_to_notify; - /** The ID of the thread on which flush requests should be made */ + /* The ID of the thread on which flush requests should be made */ thread_id_t thread_id; - /** The pool of flush requests */ + /* The pool of flush requests */ mempool_t *flush_pool; - /** Bios waiting for a flush request to become available */ + /* Bios waiting for a flush request to become available */ struct bio_list waiting_flush_bios; - /** The lock to protect the previous fields */ + /* The lock to protect the previous fields */ spinlock_t lock; - /** The rotor for selecting the bio queue for submitting flush bios */ + /* The rotor for selecting the bio queue for submitting flush bios */ zone_count_t bio_queue_rotor; - /** The number of flushes submitted to the current bio queue */ + /* The number of flushes submitted to the current bio queue */ int flush_count; }; diff --git a/drivers/md/dm-vdo/recovery-journal.h b/drivers/md/dm-vdo/recovery-journal.h index d78c6c7da4ea..899071173015 100644 --- a/drivers/md/dm-vdo/recovery-journal.h +++ b/drivers/md/dm-vdo/recovery-journal.h @@ -89,27 +89,27 @@ enum vdo_zone_type { }; struct lock_counter { - /** The completion for notifying the owner of a lock release */ + /* The completion for notifying the owner of a lock release */ struct vdo_completion completion; - /** The number of logical zones which may hold locks */ + /* The number of logical zones which may hold locks */ zone_count_t logical_zones; - /** The number of physical zones which may hold locks */ + /* The number of physical zones which may hold locks */ zone_count_t physical_zones; - /** The number of locks */ + /* The number of locks */ block_count_t locks; - /** Whether the lock release notification is in flight */ + /* Whether the lock release notification is in flight */ atomic_t state; - /** The number of logical zones which hold each lock */ + /* The number of logical zones which hold each lock */ atomic_t *logical_zone_counts; - /** The number of physical zones which hold each lock */ + /* The number of physical zones which hold each lock */ atomic_t *physical_zone_counts; - /** The per-lock counts for the journal zone */ + /* The per-lock counts for the journal zone */ u16 *journal_counters; - /** The per-lock decrement counts for the journal zone */ + /* The per-lock decrement counts for the journal zone */ atomic_t *journal_decrement_counts; - /** The per-zone, per-lock reference counts for logical zones */ + /* The per-zone, per-lock reference counts for logical zones */ u16 *logical_counters; - /** The per-zone, per-lock reference counts for physical zones */ + /* The per-zone, per-lock reference counts for physical zones */ u16 *physical_counters; }; diff --git a/drivers/md/dm-vdo/statistics.h b/drivers/md/dm-vdo/statistics.h index da8b6a7fe24f..c88a75dffba3 100644 --- a/drivers/md/dm-vdo/statistics.h +++ b/drivers/md/dm-vdo/statistics.h @@ -13,11 +13,11 @@ enum { }; struct block_allocator_statistics { - /** The total number of slabs from which blocks may be allocated */ + /* The total number of slabs from which blocks may be allocated */ u64 slab_count; - /** The total number of slabs from which blocks have ever been allocated */ + /* The total number of slabs from which blocks have ever been allocated */ u64 slabs_opened; - /** The number of times since loading that a slab has been re-opened */ + /* The number of times since loading that a slab has been re-opened */ u64 slabs_reopened; }; @@ -29,235 +29,235 @@ struct block_allocator_statistics { * reporting purposes */ struct commit_statistics { - /** The total number of items on which processing has started */ + /* The total number of items on which processing has started */ u64 started; - /** The total number of items for which a write operation has been issued */ + /* The total number of items for which a write operation has been issued */ u64 written; - /** The total number of items for which a write operation has completed */ + /* The total number of items for which a write operation has completed */ u64 committed; }; /** Counters for events in the recovery journal */ struct recovery_journal_statistics { - /** Number of times the on-disk journal was full */ + /* Number of times the on-disk journal was full */ u64 disk_full; - /** Number of times the recovery journal requested slab journal commits. */ + /* Number of times the recovery journal requested slab journal commits. */ u64 slab_journal_commits_requested; - /** Write/Commit totals for individual journal entries */ + /* Write/Commit totals for individual journal entries */ struct commit_statistics entries; - /** Write/Commit totals for journal blocks */ + /* Write/Commit totals for journal blocks */ struct commit_statistics blocks; }; /** The statistics for the compressed block packer. */ struct packer_statistics { - /** Number of compressed data items written since startup */ + /* Number of compressed data items written since startup */ u64 compressed_fragments_written; - /** Number of blocks containing compressed items written since startup */ + /* Number of blocks containing compressed items written since startup */ u64 compressed_blocks_written; - /** Number of VIOs that are pending in the packer */ + /* Number of VIOs that are pending in the packer */ u64 compressed_fragments_in_packer; }; /** The statistics for the slab journals. */ struct slab_journal_statistics { - /** Number of times the on-disk journal was full */ + /* Number of times the on-disk journal was full */ u64 disk_full_count; - /** Number of times an entry was added over the flush threshold */ + /* Number of times an entry was added over the flush threshold */ u64 flush_count; - /** Number of times an entry was added over the block threshold */ + /* Number of times an entry was added over the block threshold */ u64 blocked_count; - /** Number of times a tail block was written */ + /* Number of times a tail block was written */ u64 blocks_written; - /** Number of times we had to wait for the tail to write */ + /* Number of times we had to wait for the tail to write */ u64 tail_busy_count; }; /** The statistics for the slab summary. */ struct slab_summary_statistics { - /** Number of blocks written */ + /* Number of blocks written */ u64 blocks_written; }; /** The statistics for the reference counts. */ struct ref_counts_statistics { - /** Number of reference blocks written */ + /* Number of reference blocks written */ u64 blocks_written; }; /** The statistics for the block map. */ struct block_map_statistics { - /** number of dirty (resident) pages */ + /* number of dirty (resident) pages */ u32 dirty_pages; - /** number of clean (resident) pages */ + /* number of clean (resident) pages */ u32 clean_pages; - /** number of free pages */ + /* number of free pages */ u32 free_pages; - /** number of pages in failed state */ + /* number of pages in failed state */ u32 failed_pages; - /** number of pages incoming */ + /* number of pages incoming */ u32 incoming_pages; - /** number of pages outgoing */ + /* number of pages outgoing */ u32 outgoing_pages; - /** how many times free page not avail */ + /* how many times free page not avail */ u32 cache_pressure; - /** number of get_vdo_page() calls for read */ + /* number of get_vdo_page() calls for read */ u64 read_count; - /** number of get_vdo_page() calls for write */ + /* number of get_vdo_page() calls for write */ u64 write_count; - /** number of times pages failed to read */ + /* number of times pages failed to read */ u64 failed_reads; - /** number of times pages failed to write */ + /* number of times pages failed to write */ u64 failed_writes; - /** number of gets that are reclaimed */ + /* number of gets that are reclaimed */ u64 reclaimed; - /** number of gets for outgoing pages */ + /* number of gets for outgoing pages */ u64 read_outgoing; - /** number of gets that were already there */ + /* number of gets that were already there */ u64 found_in_cache; - /** number of gets requiring discard */ + /* number of gets requiring discard */ u64 discard_required; - /** number of gets enqueued for their page */ + /* number of gets enqueued for their page */ u64 wait_for_page; - /** number of gets that have to fetch */ + /* number of gets that have to fetch */ u64 fetch_required; - /** number of page fetches */ + /* number of page fetches */ u64 pages_loaded; - /** number of page saves */ + /* number of page saves */ u64 pages_saved; - /** the number of flushes issued */ + /* the number of flushes issued */ u64 flush_count; }; /** The dedupe statistics from hash locks */ struct hash_lock_statistics { - /** Number of times the UDS advice proved correct */ + /* Number of times the UDS advice proved correct */ u64 dedupe_advice_valid; - /** Number of times the UDS advice proved incorrect */ + /* Number of times the UDS advice proved incorrect */ u64 dedupe_advice_stale; - /** Number of writes with the same data as another in-flight write */ + /* Number of writes with the same data as another in-flight write */ u64 concurrent_data_matches; - /** Number of writes whose hash collided with an in-flight write */ + /* Number of writes whose hash collided with an in-flight write */ u64 concurrent_hash_collisions; - /** Current number of dedupe queries that are in flight */ + /* Current number of dedupe queries that are in flight */ u32 curr_dedupe_queries; }; /** Counts of error conditions in VDO. */ struct error_statistics { - /** number of times VDO got an invalid dedupe advice PBN from UDS */ + /* number of times VDO got an invalid dedupe advice PBN from UDS */ u64 invalid_advice_pbn_count; - /** number of times a VIO completed with a VDO_NO_SPACE error */ + /* number of times a VIO completed with a VDO_NO_SPACE error */ u64 no_space_error_count; - /** number of times a VIO completed with a VDO_READ_ONLY error */ + /* number of times a VIO completed with a VDO_READ_ONLY error */ u64 read_only_error_count; }; struct bio_stats { - /** Number of REQ_OP_READ bios */ + /* Number of REQ_OP_READ bios */ u64 read; - /** Number of REQ_OP_WRITE bios with data */ + /* Number of REQ_OP_WRITE bios with data */ u64 write; - /** Number of bios tagged with REQ_PREFLUSH and containing no data */ + /* Number of bios tagged with REQ_PREFLUSH and containing no data */ u64 empty_flush; - /** Number of REQ_OP_DISCARD bios */ + /* Number of REQ_OP_DISCARD bios */ u64 discard; - /** Number of bios tagged with REQ_PREFLUSH */ + /* Number of bios tagged with REQ_PREFLUSH */ u64 flush; - /** Number of bios tagged with REQ_FUA */ + /* Number of bios tagged with REQ_FUA */ u64 fua; }; struct memory_usage { - /** Tracked bytes currently allocated. */ + /* Tracked bytes currently allocated. */ u64 bytes_used; - /** Maximum tracked bytes allocated. */ + /* Maximum tracked bytes allocated. */ u64 peak_bytes_used; }; /** UDS index statistics */ struct index_statistics { - /** Number of records stored in the index */ + /* Number of records stored in the index */ u64 entries_indexed; - /** Number of post calls that found an existing entry */ + /* Number of post calls that found an existing entry */ u64 posts_found; - /** Number of post calls that added a new entry */ + /* Number of post calls that added a new entry */ u64 posts_not_found; - /** Number of query calls that found an existing entry */ + /* Number of query calls that found an existing entry */ u64 queries_found; - /** Number of query calls that added a new entry */ + /* Number of query calls that added a new entry */ u64 queries_not_found; - /** Number of update calls that found an existing entry */ + /* Number of update calls that found an existing entry */ u64 updates_found; - /** Number of update calls that added a new entry */ + /* Number of update calls that added a new entry */ u64 updates_not_found; - /** Number of entries discarded */ + /* Number of entries discarded */ u64 entries_discarded; }; /** The statistics of the vdo service. */ struct vdo_statistics { u32 version; - /** Number of blocks used for data */ + /* Number of blocks used for data */ u64 data_blocks_used; - /** Number of blocks used for VDO metadata */ + /* Number of blocks used for VDO metadata */ u64 overhead_blocks_used; - /** Number of logical blocks that are currently mapped to physical blocks */ + /* Number of logical blocks that are currently mapped to physical blocks */ u64 logical_blocks_used; - /** number of physical blocks */ + /* number of physical blocks */ block_count_t physical_blocks; - /** number of logical blocks */ + /* number of logical blocks */ block_count_t logical_blocks; - /** Size of the block map page cache, in bytes */ + /* Size of the block map page cache, in bytes */ u64 block_map_cache_size; - /** The physical block size */ + /* The physical block size */ u64 block_size; - /** Number of times the VDO has successfully recovered */ + /* Number of times the VDO has successfully recovered */ u64 complete_recoveries; - /** Number of times the VDO has recovered from read-only mode */ + /* Number of times the VDO has recovered from read-only mode */ u64 read_only_recoveries; - /** String describing the operating mode of the VDO */ + /* String describing the operating mode of the VDO */ char mode[15]; - /** Whether the VDO is in recovery mode */ + /* Whether the VDO is in recovery mode */ bool in_recovery_mode; - /** What percentage of recovery mode work has been completed */ + /* What percentage of recovery mode work has been completed */ u8 recovery_percentage; - /** The statistics for the compressed block packer */ + /* The statistics for the compressed block packer */ struct packer_statistics packer; - /** Counters for events in the block allocator */ + /* Counters for events in the block allocator */ struct block_allocator_statistics allocator; - /** Counters for events in the recovery journal */ + /* Counters for events in the recovery journal */ struct recovery_journal_statistics journal; - /** The statistics for the slab journals */ + /* The statistics for the slab journals */ struct slab_journal_statistics slab_journal; - /** The statistics for the slab summary */ + /* The statistics for the slab summary */ struct slab_summary_statistics slab_summary; - /** The statistics for the reference counts */ + /* The statistics for the reference counts */ struct ref_counts_statistics ref_counts; - /** The statistics for the block map */ + /* The statistics for the block map */ struct block_map_statistics block_map; - /** The dedupe statistics from hash locks */ + /* The dedupe statistics from hash locks */ struct hash_lock_statistics hash_lock; - /** Counts of error conditions */ + /* Counts of error conditions */ struct error_statistics errors; - /** The VDO instance */ + /* The VDO instance */ u32 instance; - /** Current number of active VIOs */ + /* Current number of active VIOs */ u32 current_vios_in_progress; - /** Maximum number of active VIOs */ + /* Maximum number of active VIOs */ u32 max_vios; - /** Number of times the UDS index was too slow in responding */ + /* Number of times the UDS index was too slow in responding */ u64 dedupe_advice_timeouts; - /** Number of flush requests submitted to the storage device */ + /* Number of flush requests submitted to the storage device */ u64 flush_out; - /** Logical block size */ + /* Logical block size */ u64 logical_block_size; - /** Bios submitted into VDO from above */ + /* Bios submitted into VDO from above */ struct bio_stats bios_in; struct bio_stats bios_in_partial; - /** Bios submitted onward for user data */ + /* Bios submitted onward for user data */ struct bio_stats bios_out; - /** Bios submitted onward for metadata */ + /* Bios submitted onward for metadata */ struct bio_stats bios_meta; struct bio_stats bios_journal; struct bio_stats bios_page_cache; @@ -267,11 +267,11 @@ struct vdo_statistics { struct bio_stats bios_page_cache_completed; struct bio_stats bios_acknowledged; struct bio_stats bios_acknowledged_partial; - /** Current number of bios in progress */ + /* Current number of bios in progress */ struct bio_stats bios_in_progress; - /** Memory usage stats. */ + /* Memory usage stats. */ struct memory_usage memory_usage; - /** The statistics for the UDS index */ + /* The statistics for the UDS index */ struct index_statistics index; }; diff --git a/drivers/md/dm-vdo/vdo.h b/drivers/md/dm-vdo/vdo.h index 35d603066529..772317e6db52 100644 --- a/drivers/md/dm-vdo/vdo.h +++ b/drivers/md/dm-vdo/vdo.h @@ -25,13 +25,13 @@ #include "uds.h" enum notifier_state { - /** Notifications are allowed but not in progress */ + /* Notifications are allowed but not in progress */ MAY_NOTIFY, - /** A notification is in progress */ + /* A notification is in progress */ NOTIFYING, - /** Notifications are not allowed */ + /* Notifications are not allowed */ MAY_NOT_NOTIFY, - /** A notification has completed */ + /* A notification has completed */ NOTIFIED, }; diff --git a/drivers/md/dm-vdo/vio.c b/drivers/md/dm-vdo/vio.c index f9c54ec56176..eb6838ddabbb 100644 --- a/drivers/md/dm-vdo/vio.c +++ b/drivers/md/dm-vdo/vio.c @@ -20,21 +20,21 @@ /* A vio_pool is a collection of preallocated vios. */ struct vio_pool { - /** The number of objects managed by the pool */ + /* The number of objects managed by the pool */ size_t size; - /** The list of objects which are available */ + /* The list of objects which are available */ struct list_head available; - /** The queue of requestors waiting for objects from the pool */ + /* The queue of requestors waiting for objects from the pool */ struct vdo_wait_queue waiting; - /** The number of objects currently in use */ + /* The number of objects currently in use */ size_t busy_count; - /** The list of objects which are in use */ + /* The list of objects which are in use */ struct list_head busy; - /** The ID of the thread on which this pool may be used */ + /* The ID of the thread on which this pool may be used */ thread_id_t thread_id; - /** The buffer backing the pool's vios */ + /* The buffer backing the pool's vios */ char *buffer; - /** The pool entries */ + /* The pool entries */ struct pooled_vio vios[]; }; -- cgit v1.2.3