/* * Copyright(c) 2012-2022 Intel Corporation * Copyright(c) 2024 Huawei Technologies * SPDX-License-Identifier: BSD-3-Clause */ #include "ocf/ocf.h" #include "../ocf_cache_priv.h" #include "cleaning.h" #include "../metadata/metadata.h" #include "../utils/utils_cleaner.h" #include "../utils/utils_cache_line.h" #include "../utils/utils_parallelize.h" #include "../ocf_request.h" #include "../cleaning/acp.h" #include "../engine/engine_common.h" #include "../concurrency/ocf_cache_line_concurrency.h" #include "../concurrency/ocf_metadata_concurrency.h" #include "cleaning_priv.h" #define OCF_ACP_DEBUG 0 #if 1 == OCF_ACP_DEBUG #define OCF_DEBUG_PREFIX "[Clean] %s():%d " #define OCF_DEBUG_LOG(cache, format, ...) \ ocf_cache_log_prefix(cache, log_info, OCF_DEBUG_PREFIX, \ format"\n", __func__, __LINE__, ##__VA_ARGS__) #define OCF_DEBUG_TRACE(cache) OCF_DEBUG_LOG(cache, "") #define OCF_DEBUG_MSG(cache, msg) OCF_DEBUG_LOG(cache, "- %s", msg) #define OCF_DEBUG_PARAM(cache, format, ...) OCF_DEBUG_LOG(cache, "- "format, \ ##__VA_ARGS__) #define ACP_DEBUG_INIT(acp) acp->checksum = 0 #define ACP_DEBUG_BEGIN(acp, cache_line) acp->checksum ^= cache_line #define ACP_DEBUG_END(acp, cache_line) acp->checksum ^= cache_line #define ACP_DEBUG_CHECK(acp) ENV_BUG_ON(acp->checksum) #else #define OCF_DEBUG_PREFIX #define OCF_DEBUG_LOG(cache, format, ...) #define OCF_DEBUG_TRACE(cache) #define OCF_DEBUG_MSG(cache, msg) #define OCF_DEBUG_PARAM(cache, format, ...) #define ACP_DEBUG_INIT(acp) #define ACP_DEBUG_BEGIN(acp, cache_line) #define ACP_DEBUG_END(acp, cache_line) #define ACP_DEBUG_CHECK(acp) #endif #define ACP_CHUNK_SIZE (100 * MiB) /* minimal time to chunk cleaning after error */ #define ACP_CHUNK_CLEANING_BACKOFF_TIME 5 /* time to sleep when nothing to clean in ms */ #define ACP_BACKOFF_TIME_MS 1000 #define ACP_MAX_BUCKETS 11 /* Upper thresholds for buckets in percent dirty pages. First bucket should have * threshold=0 - it isn't cleaned and we don't want dirty chunks staying dirty * forever. Last bucket also should stay at 100 for obvious reasons */ static const uint16_t ACP_BUCKET_DEFAULTS[ACP_MAX_BUCKETS] = { 0, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100 }; struct acp_flush_context { /* number of cache lines in flush */ uint64_t size; /* chunk_for error handling */ struct acp_chunk_info *chunk; /* cache lines to flush */ struct flush_data data[OCF_ACP_MAX_FLUSH_MAX_BUFFERS]; /* flush error code */ int error; }; struct acp_state { /* currently cleaned chunk */ struct acp_chunk_info *chunk; /* cache line iterator within current chunk */ unsigned iter; /* true if there are cache lines to process * current chunk */ bool in_progress; }; struct acp_chunk_info { struct list_head list; uint64_t chunk_id; uint64_t next_cleaning_timestamp; ocf_core_id_t core_id; uint16_t num_dirty; uint8_t bucket_id; }; struct acp_bucket { struct list_head chunk_list; uint16_t threshold; /* threshold in clines */ }; struct acp_context { env_rwsem chunks_lock; /* number of chunks per core */ uint64_t num_chunks[OCF_CORE_MAX]; /* per core array of all chunks */ struct acp_chunk_info *chunk_info[OCF_CORE_MAX]; struct acp_bucket bucket_info[ACP_MAX_BUCKETS]; /* total number of chunks in cache */ uint64_t chunks_total; /* structure to keep track of I/O in progress */ struct acp_flush_context flush; /* cleaning state persistent over subsequent calls to perform_cleaning */ struct acp_state state; /* cache handle */ ocf_cache_t cache; /* cleaner completion callback */ ocf_cleaner_end_t cmpl; #if 1 == OCF_ACP_DEBUG /* debug only */ uint64_t checksum; #endif }; struct acp_core_line_info { ocf_cache_line_t cache_line; ocf_core_id_t core_id; uint64_t core_line; }; #define ACP_LOCK_CHUNKS_RD() env_rwsem_down_read(&acp->chunks_lock) #define ACP_UNLOCK_CHUNKS_RD() env_rwsem_up_read(&acp->chunks_lock) #define ACP_LOCK_CHUNKS_WR() env_rwsem_down_write(&acp->chunks_lock) #define ACP_UNLOCK_CHUNKS_WR() env_rwsem_up_write(&acp->chunks_lock) static struct acp_context *_acp_get_ctx_from_cache(struct ocf_cache *cache) { return cache->cleaner.cleaning_policy_context; } static struct acp_cleaning_policy_meta* _acp_meta_get( struct ocf_cache *cache, uint32_t cache_line) { return &ocf_metadata_get_cleaning_policy(cache, cache_line)->meta.acp; } static struct acp_core_line_info _acp_core_line_info(struct ocf_cache *cache, ocf_cache_line_t cache_line) { struct acp_core_line_info acp_core_line_info = {.cache_line = cache_line, }; ocf_metadata_get_core_info(cache, cache_line, &acp_core_line_info.core_id, &acp_core_line_info.core_line); return acp_core_line_info; } static struct acp_chunk_info *_acp_get_chunk(struct ocf_cache *cache, uint32_t cache_line) { struct acp_context *acp = _acp_get_ctx_from_cache(cache); struct acp_core_line_info core_line = _acp_core_line_info(cache, cache_line); uint64_t chunk_id; chunk_id = core_line.core_line * ocf_line_size(cache) / ACP_CHUNK_SIZE; return &acp->chunk_info[core_line.core_id][chunk_id]; } static void _acp_remove_cores(struct ocf_cache *cache) { ocf_core_t core; ocf_core_id_t core_id; for_each_core(cache, core, core_id) cleaning_policy_acp_remove_core(cache, core_id); } static int _acp_load_cores(struct ocf_cache *cache) { ocf_core_t core; ocf_core_id_t core_id; int err = 0; for_each_core(cache, core, core_id) { OCF_DEBUG_PARAM(cache, "loading core %i\n", core_id); err = cleaning_policy_acp_add_core(cache, core_id); if (err) break; } if (err) _acp_remove_cores(cache); return err; } void cleaning_policy_acp_init_cache_block(struct ocf_cache *cache, uint32_t cache_line) { struct acp_cleaning_policy_meta *acp_meta; acp_meta = _acp_meta_get(cache, cache_line); acp_meta->dirty = 0; } void cleaning_policy_acp_deinitialize(struct ocf_cache *cache) { struct acp_context *acp; _acp_remove_cores(cache); acp = cache->cleaner.cleaning_policy_context; env_rwsem_destroy(&acp->chunks_lock); env_vfree(cache->cleaner.cleaning_policy_context); cache->cleaner.cleaning_policy_context = NULL; } void cleaning_policy_acp_setup(struct ocf_cache *cache) { struct acp_cleaning_policy_config *config; config = (void *)&cache->conf_meta->cleaning[ocf_cleaning_acp].data; config->thread_wakeup_time = OCF_ACP_DEFAULT_WAKE_UP; config->flush_max_buffers = OCF_ACP_DEFAULT_FLUSH_MAX_BUFFERS; } int cleaning_policy_acp_initialize(ocf_cache_t cache, int kick_cleaner) { struct acp_context *acp; int err, i; /* bug if max chunk number would overflow dirty_no array type */ #if defined (BUILD_BUG_ON) BUILD_BUG_ON(ACP_CHUNK_SIZE / ocf_cache_line_size_min >= 1U << (sizeof(acp->chunk_info[0][0].num_dirty) * 8)); #else ENV_BUG_ON(ACP_CHUNK_SIZE / ocf_cache_line_size_min >= 1U << (sizeof(acp->chunk_info[0][0].num_dirty) * 8)); #endif ENV_BUG_ON(cache->cleaner.cleaning_policy_context); acp = env_vzalloc(sizeof(*acp)); if (!acp) { ocf_cache_log(cache, log_err, "acp context allocation error\n"); return -OCF_ERR_NO_MEM; } err = env_rwsem_init(&acp->chunks_lock); if (err) { env_vfree(acp); return err; } cache->cleaner.cleaning_policy_context = acp; acp->cache = cache; for (i = 0; i < ACP_MAX_BUCKETS; i++) { INIT_LIST_HEAD(&acp->bucket_info[i].chunk_list); acp->bucket_info[i].threshold = ((ACP_CHUNK_SIZE/ocf_line_size(cache)) * ACP_BUCKET_DEFAULTS[i]) / 100; } if (cache->conf_meta->core_count > 0) { err = _acp_load_cores(cache); if (err) { cleaning_policy_acp_deinitialize(cache); return err; } } if (kick_cleaner) ocf_kick_cleaner(cache); return 0; } #define OCF_ACP_POPULATE_SHARDS_CNT 32 struct ocf_acp_populate_context { ocf_cache_t cache; struct { uint16_t *chunk[OCF_CORE_MAX]; struct { struct list_head chunk_list; } bucket[ACP_MAX_BUCKETS]; } shard[OCF_ACP_POPULATE_SHARDS_CNT]; ocf_cleaning_populate_end_t cmpl; void *priv; }; static int ocf_acp_populate_handle(ocf_parallelize_t parallelize, void *priv, unsigned shard_id, unsigned shards_cnt) { struct ocf_acp_populate_context *context = priv; ocf_cache_t cache = context->cache; ocf_cache_line_t entries = cache->device->collision_table_entries; ocf_cache_line_t cline, portion; uint64_t begin, end; struct acp_cleaning_policy_meta *acp_meta; struct acp_chunk_info *chunk; ocf_core_id_t core_id; uint32_t step = 0; portion = OCF_DIV_ROUND_UP((uint64_t)entries, shards_cnt); begin = portion*shard_id; end = OCF_MIN(portion*(shard_id + 1), entries); for (cline = begin; cline < end; cline++) { ocf_metadata_get_core_and_part_id(cache, cline, &core_id, NULL); OCF_COND_RESCHED_DEFAULT(step); if (core_id == OCF_CORE_MAX) continue; if (!metadata_test_dirty(cache, cline)) { cleaning_policy_acp_init_cache_block(cache, cline); continue; } acp_meta = _acp_meta_get(cache, cline); acp_meta->dirty = 1; chunk = _acp_get_chunk(cache, cline); context->shard[shard_id].chunk[core_id][chunk->chunk_id]++; } return 0; } static void ocf_acp_populate_chunk(struct ocf_acp_populate_context *context, struct acp_chunk_info *chunk) { ocf_cache_t cache = context->cache; struct acp_context *acp = _acp_get_ctx_from_cache(cache); struct acp_bucket *bucket; unsigned shard_id; uint8_t bucket_id; chunk->num_dirty = 0; for (shard_id = 0; shard_id < OCF_ACP_POPULATE_SHARDS_CNT; shard_id++) { chunk->num_dirty += context->shard[shard_id] .chunk[chunk->core_id][chunk->chunk_id]; } for (bucket_id = 1; bucket_id < ACP_MAX_BUCKETS; bucket_id++) { bucket = &acp->bucket_info[bucket_id]; if (chunk->num_dirty < bucket->threshold) break; } bucket = &acp->bucket_info[--bucket_id]; chunk->bucket_id = bucket_id; list_move_tail(&chunk->list, &bucket->chunk_list); } static void ocf_acp_populate_finish(ocf_parallelize_t parallelize, void *priv, int error) { struct ocf_acp_populate_context *context = priv; ocf_cache_t cache = context->cache; struct acp_context *acp = _acp_get_ctx_from_cache(cache); ocf_core_id_t core_id; ocf_core_t core; uint64_t core_size; uint64_t num_chunks; uint64_t chunk_id; uint32_t step = 0; for_each_core(cache, core, core_id) { core_size = core->conf_meta->length; num_chunks = OCF_DIV_ROUND_UP(core_size, ACP_CHUNK_SIZE); for (chunk_id = 0; chunk_id < num_chunks; chunk_id++) { ocf_acp_populate_chunk(context, &acp->chunk_info[core_id][chunk_id]); OCF_COND_RESCHED_DEFAULT(step); } } ocf_cache_log(cache, log_info, "Finished rebuilding ACP metadata\n"); ocf_kick_cleaner(cache); context->cmpl(context->priv, error); for_each_core(cache, core, core_id) { if (context->shard[0].chunk[core_id]) env_vfree(context->shard[0].chunk[core_id]); } ocf_parallelize_destroy(parallelize); } void cleaning_policy_acp_populate(ocf_cache_t cache, ocf_cleaning_populate_end_t cmpl, void *priv) { struct ocf_acp_populate_context *context; ocf_parallelize_t parallelize; ocf_core_id_t core_id; ocf_core_t core; unsigned shards_cnt = OCF_ACP_POPULATE_SHARDS_CNT; unsigned shard_id; uint64_t core_size; uint64_t num_chunks; uint16_t *chunks; int result; result = ocf_parallelize_create(¶llelize, cache, OCF_ACP_POPULATE_SHARDS_CNT, sizeof(*context), ocf_acp_populate_handle, ocf_acp_populate_finish); if (result) { cmpl(priv, result); return; } context = ocf_parallelize_get_priv(parallelize); context->cache = cache; context->cmpl = cmpl; context->priv = priv; for_each_core(cache, core, core_id) { core_size = core->conf_meta->length; num_chunks = OCF_DIV_ROUND_UP(core_size, ACP_CHUNK_SIZE); chunks = env_vzalloc(sizeof(*chunks) * num_chunks * shards_cnt); if (!chunks) { result = -OCF_ERR_NO_MEM; goto err; } for (shard_id = 0; shard_id < shards_cnt; shard_id++) { context->shard[shard_id].chunk[core_id] = &chunks[num_chunks * shard_id]; } } ocf_parallelize_run(parallelize); return; err: for_each_core(cache, core, core_id) { if (context->shard[0].chunk[core_id]) env_vfree(context->shard[0].chunk[core_id]); } ocf_parallelize_destroy(parallelize); cmpl(priv, result); } int cleaning_policy_acp_set_cleaning_param(ocf_cache_t cache, uint32_t param_id, uint32_t param_value) { struct acp_cleaning_policy_config *config; config = (void *)&cache->conf_meta->cleaning[ocf_cleaning_acp].data; switch (param_id) { case ocf_acp_wake_up_time: OCF_CLEANING_CHECK_PARAM(cache, param_value, OCF_ACP_MIN_WAKE_UP, OCF_ACP_MAX_WAKE_UP, "thread_wakeup_time"); config->thread_wakeup_time = param_value; ocf_cache_log(cache, log_info, "Write-back flush thread " "wake-up time: %d\n", config->thread_wakeup_time); ocf_kick_cleaner(cache); break; case ocf_acp_flush_max_buffers: OCF_CLEANING_CHECK_PARAM(cache, param_value, OCF_ACP_MIN_FLUSH_MAX_BUFFERS, OCF_ACP_MAX_FLUSH_MAX_BUFFERS, "flush_max_buffers"); config->flush_max_buffers = param_value; ocf_cache_log(cache, log_info, "Write-back flush thread max " "buffers flushed per iteration: %d\n", config->flush_max_buffers); break; default: return -OCF_ERR_INVAL; } return 0; } int cleaning_policy_acp_get_cleaning_param(ocf_cache_t cache, uint32_t param_id, uint32_t *param_value) { struct acp_cleaning_policy_config *config; config = (void *)&cache->conf_meta->cleaning[ocf_cleaning_acp].data; switch (param_id) { case ocf_acp_flush_max_buffers: *param_value = config->flush_max_buffers; break; case ocf_acp_wake_up_time: *param_value = config->thread_wakeup_time; break; default: return -OCF_ERR_INVAL; } return 0; } /* attempt to lock cache line if it's dirty */ static ocf_cache_line_t _acp_trylock_dirty(struct ocf_cache *cache, uint32_t core_id, uint64_t core_line) { struct ocf_map_info info; bool locked = false; unsigned lock_idx = ocf_metadata_concurrency_next_idx( cache->cleaner.io_queue); ocf_hb_cline_prot_lock_rd(&cache->metadata.lock, lock_idx, core_id, core_line); ocf_engine_lookup_map_entry(cache, &info, core_id, core_line); if (info.status == LOOKUP_HIT && metadata_test_dirty(cache, info.coll_idx)) { locked = ocf_cache_line_try_lock_rd( ocf_cache_line_concurrency(cache), info.coll_idx); } ocf_hb_cline_prot_unlock_rd(&cache->metadata.lock, lock_idx, core_id, core_line); return locked ? info.coll_idx : cache->device->collision_table_entries; } static void _acp_handle_flush_error(struct ocf_cache *cache, struct acp_context *acp) { struct acp_flush_context *flush = &acp->flush; flush->chunk->next_cleaning_timestamp = env_get_tick_count() + env_secs_to_ticks(ACP_CHUNK_CLEANING_BACKOFF_TIME); if (ocf_cache_log_rl(cache)) { ocf_core_log(&cache->core[flush->chunk->core_id], log_err, "Cleaning error (%d) in range" " <%llu; %llu) backing off for %u seconds\n", flush->error, flush->chunk->chunk_id * ACP_CHUNK_SIZE, (flush->chunk->chunk_id * ACP_CHUNK_SIZE) + ACP_CHUNK_SIZE, ACP_CHUNK_CLEANING_BACKOFF_TIME); } } static inline bool _acp_can_clean_chunk(struct ocf_cache *cache, struct acp_chunk_info *chunk) { /* Check if core device is opened and if timeout after cleaning error * expired or wasn't set in the first place */ return (cache->core[chunk->core_id].opened && (chunk->next_cleaning_timestamp > env_get_tick_count() || !chunk->next_cleaning_timestamp)); } static struct acp_chunk_info *_acp_get_cleaning_candidate(ocf_cache_t cache) { int i; struct acp_chunk_info *cur; struct acp_context *acp = cache->cleaner.cleaning_policy_context; ACP_LOCK_CHUNKS_RD(); /* go through all buckets in descending order, excluding bucket 0 which * is supposed to contain all clean chunks */ for (i = ACP_MAX_BUCKETS - 1; i > 0; i--) { list_for_each_entry(cur, &acp->bucket_info[i].chunk_list, list) { if (_acp_can_clean_chunk(cache, cur)) { ACP_UNLOCK_CHUNKS_RD(); return cur; } } } ACP_UNLOCK_CHUNKS_RD(); return NULL; } /* called after flush request completed */ static void _acp_flush_end(void *priv, int error) { struct acp_cleaning_policy_config *config; struct acp_context *acp = priv; struct acp_flush_context *flush = &acp->flush; ocf_cache_t cache = acp->cache; int i; config = (void *)&cache->conf_meta->cleaning[ocf_cleaning_acp].data; for (i = 0; i < flush->size; i++) { ocf_cache_line_unlock_rd( ocf_cache_line_concurrency(cache), flush->data[i].cache_line); ACP_DEBUG_END(acp, flush->data[i].cache_line); } if (error) { flush->error = error; _acp_handle_flush_error(cache, acp); } ACP_DEBUG_CHECK(acp); acp->cmpl(&cache->cleaner, config->thread_wakeup_time); } /* flush data */ static void _acp_flush(struct acp_context *acp) { ocf_cache_t cache = acp->cache; struct ocf_cleaner_attribs attribs = { .cmpl_context = acp, .cmpl_fn = _acp_flush_end, .lock_cacheline = false, .lock_metadata = true, .cmpl_queue = true, .io_queue = cache->cleaner.io_queue, }; ocf_cleaner_do_flush_data_async(cache, acp->flush.data, acp->flush.size, &attribs); } static bool _acp_prepare_flush_data(struct acp_context *acp, uint32_t flush_max_buffers) { ocf_cache_t cache = acp->cache; struct acp_state *state = &acp->state; struct acp_chunk_info *chunk = state->chunk; size_t lines_per_chunk = ACP_CHUNK_SIZE / ocf_line_size(cache); uint64_t first_core_line = chunk->chunk_id * lines_per_chunk; OCF_DEBUG_PARAM(cache, "lines per chunk %llu chunk %llu " "first_core_line %llu\n", (uint64_t)lines_per_chunk, chunk->chunk_id, first_core_line); acp->flush.size = 0; acp->flush.chunk = chunk; for (; state->iter < lines_per_chunk && acp->flush.size < flush_max_buffers; state->iter++) { uint64_t core_line = first_core_line + state->iter; ocf_cache_line_t cache_line; cache_line = _acp_trylock_dirty(cache, chunk->core_id, core_line); if (cache_line == cache->device->collision_table_entries) continue; ACP_DEBUG_BEGIN(acp, cache_line); acp->flush.data[acp->flush.size].core_id = chunk->core_id; acp->flush.data[acp->flush.size].core_line = core_line; acp->flush.data[acp->flush.size].cache_line = cache_line; acp->flush.size++; } if (state->iter == lines_per_chunk) { /* reached end of chunk - reset state */ state->in_progress = false; } return (acp->flush.size > 0); } /* Clean at most 'flush_max_buffers' cache lines from current or newly * selected chunk */ void cleaning_policy_acp_perform_cleaning(ocf_cache_t cache, ocf_cleaner_end_t cmpl) { struct acp_cleaning_policy_config *config; struct acp_context *acp = _acp_get_ctx_from_cache(cache); struct acp_state *state = &acp->state; acp->cmpl = cmpl; if (!state->in_progress) { /* get next chunk to clean */ state->chunk = _acp_get_cleaning_candidate(cache); if (!state->chunk) { /* nothing co clean */ cmpl(&cache->cleaner, ACP_BACKOFF_TIME_MS); return; } /* new cleaning cycle - reset state */ state->iter = 0; state->in_progress = true; } ACP_DEBUG_INIT(acp); config = (void *)&cache->conf_meta->cleaning[ocf_cleaning_acp].data; if (_acp_prepare_flush_data(acp, config->flush_max_buffers)) _acp_flush(acp); else _acp_flush_end(acp, 0); } static void _acp_update_bucket(struct acp_context *acp, struct acp_chunk_info *chunk) { struct acp_bucket *bucket = &acp->bucket_info[chunk->bucket_id]; if (chunk->num_dirty > bucket->threshold) { ENV_BUG_ON(chunk->bucket_id == ACP_MAX_BUCKETS - 1); chunk->bucket_id++; /* buckets are stored in array, move up one bucket. * No overflow here. ENV_BUG_ON made sure of no incrementation on * last bucket */ bucket++; list_move_tail(&chunk->list, &bucket->chunk_list); } else if (chunk->bucket_id && chunk->num_dirty <= (bucket - 1)->threshold) { chunk->bucket_id--; /* move down one bucket, we made sure we won't underflow */ bucket--; list_move(&chunk->list, &bucket->chunk_list); } } void cleaning_policy_acp_set_hot_cache_line(struct ocf_cache *cache, uint32_t cache_line) { struct acp_context *acp = _acp_get_ctx_from_cache(cache); struct acp_cleaning_policy_meta *acp_meta; struct acp_chunk_info *chunk; ACP_LOCK_CHUNKS_WR(); acp_meta = _acp_meta_get(cache, cache_line); chunk = _acp_get_chunk(cache, cache_line); if (!acp_meta->dirty) { acp_meta->dirty = 1; chunk->num_dirty++; } _acp_update_bucket(acp, chunk); ACP_UNLOCK_CHUNKS_WR(); } void cleaning_policy_acp_purge_block(struct ocf_cache *cache, uint32_t cache_line) { struct acp_context *acp = _acp_get_ctx_from_cache(cache); struct acp_cleaning_policy_meta *acp_meta; struct acp_chunk_info *chunk; ACP_LOCK_CHUNKS_WR(); acp_meta = _acp_meta_get(cache, cache_line); chunk = _acp_get_chunk(cache, cache_line); if (acp_meta->dirty) { acp_meta->dirty = 0; chunk->num_dirty--; } _acp_update_bucket(acp, chunk); ACP_UNLOCK_CHUNKS_WR(); } int cleaning_policy_acp_purge_range(struct ocf_cache *cache, int core_id, uint64_t start_byte, uint64_t end_byte) { return ocf_metadata_actor(cache, PARTITION_UNSPECIFIED, core_id, start_byte, end_byte, cleaning_policy_acp_purge_block); } void cleaning_policy_acp_remove_core(ocf_cache_t cache, ocf_core_id_t core_id) { struct acp_context *acp = _acp_get_ctx_from_cache(cache); uint64_t i; ENV_BUG_ON(acp->chunks_total < acp->num_chunks[core_id]); ENV_BUG_ON(!acp->chunk_info[core_id]); if (acp->state.in_progress && acp->state.chunk->core_id == core_id) { acp->state.in_progress = false; acp->state.iter = 0; acp->state.chunk = NULL; } ACP_LOCK_CHUNKS_WR(); for (i = 0; i < acp->num_chunks[core_id]; i++) list_del(&acp->chunk_info[core_id][i].list); acp->chunks_total -= acp->num_chunks[core_id]; acp->num_chunks[core_id] = 0; env_vfree(acp->chunk_info[core_id]); acp->chunk_info[core_id] = NULL; ACP_UNLOCK_CHUNKS_WR(); } int cleaning_policy_acp_add_core(ocf_cache_t cache, ocf_core_id_t core_id) { ocf_core_t core = ocf_cache_get_core(cache, core_id); uint64_t core_size = core->conf_meta->length; uint64_t num_chunks = OCF_DIV_ROUND_UP(core_size, ACP_CHUNK_SIZE); struct acp_context *acp = _acp_get_ctx_from_cache(cache); int i; OCF_DEBUG_PARAM(cache, "%s core_id %llu num_chunks %llu\n", __func__, (uint64_t)core_id, (uint64_t) num_chunks); ACP_LOCK_CHUNKS_WR(); ENV_BUG_ON(acp->chunk_info[core_id]); acp->chunk_info[core_id] = env_vzalloc(num_chunks * sizeof(acp->chunk_info[0][0])); if (!acp->chunk_info[core_id]) { ACP_UNLOCK_CHUNKS_WR(); OCF_DEBUG_PARAM(cache, "failed to allocate acp tables\n"); return -OCF_ERR_NO_MEM; } OCF_DEBUG_PARAM(cache, "successfully allocated acp tables\n"); /* increment counters */ acp->num_chunks[core_id] = num_chunks; acp->chunks_total += num_chunks; for (i = 0; i < acp->num_chunks[core_id]; i++) { /* fill in chunk metadata and add to the clean bucket */ acp->chunk_info[core_id][i].core_id = core_id; acp->chunk_info[core_id][i].chunk_id = i; list_add(&acp->chunk_info[core_id][i].list, &acp->bucket_info[0].chunk_list); } ACP_UNLOCK_CHUNKS_WR(); return 0; }