Merge pull request #63 from robertbaldyga/dynamic-queues

Dynamic queues management
This commit is contained in:
Jan Musiał 2019-02-27 11:03:36 +01:00 committed by GitHub
commit e235500472
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
38 changed files with 380 additions and 464 deletions

View File

@ -134,42 +134,6 @@ static void ctx_data_secure_erase(ctx_data_t *ctx_data)
{
}
/*
* Initialize queue thread. To keep this example simple we handle queues
* synchronously, thus it's left non-implemented.
*/
static int ctx_queue_init(ocf_queue_t q)
{
return 0;
}
/*
* Trigger queue asynchronously. Made synchronous for simplicity.
*/
static inline void ctx_queue_kick_async(ocf_queue_t q)
{
ocf_queue_run(q);
}
/*
* Trigger queue synchronously. May be implemented as asynchronous as well,
* but in some environments kicking queue synchronously may reduce latency,
* so to take advantage of such situations OCF call synchronous variant of
* queue kick callback where possible.
*/
static void ctx_queue_kick_sync(ocf_queue_t q)
{
ocf_queue_run(q);
}
/*
* Stop queue thread. To keep this example simple we handle queues
* synchronously, thus it's left non-implemented.
*/
static void ctx_queue_stop(ocf_queue_t q)
{
}
/*
* Initialize cleaner thread. Cleaner thread is left non-implemented,
* to keep this example as simple as possible.
@ -257,8 +221,6 @@ static int ctx_logger_dump_stack(ocf_logger_t logger)
* This structure describes context config, containing simple context info
* and pointers to ops callbacks. Ops are splitted into few categories:
* - data ops, providing context specific data handing interface,
* - queue ops, providing interface for starting, stoping and kicking
* queue thread in both synchronous and asynchronous way,
* - cleaner ops, providing interface to start and stop clener thread,
* - metadata updater ops, providing interface for starting, stoping
* and kicking metadata updater thread.
@ -280,13 +242,6 @@ static const struct ocf_ctx_config ctx_cfg = {
.secure_erase = ctx_data_secure_erase,
},
.queue = {
.init = ctx_queue_init,
.kick_sync = ctx_queue_kick_sync,
.kick = ctx_queue_kick_async,
.stop = ctx_queue_stop,
},
.cleaner = {
.init = ctx_cleaner_init,
.stop = ctx_cleaner_stop,

View File

@ -18,6 +18,44 @@ void error(char *msg)
exit(1);
}
/*
* Trigger queue asynchronously. Made synchronous for simplicity.
*/
static inline void queue_kick_async(ocf_queue_t q)
{
ocf_queue_run(q);
}
/*
* Trigger queue synchronously. May be implemented as asynchronous as well,
* but in some environments kicking queue synchronously may reduce latency,
* so to take advantage of such situations OCF call synchronous variant of
* queue kick callback where possible.
*/
static void queue_kick_sync(ocf_queue_t q)
{
ocf_queue_run(q);
}
/*
* Stop queue thread. To keep this example simple we handle queues
* synchronously, thus it's left non-implemented.
*/
static void queue_stop(ocf_queue_t q)
{
}
/*
* Queue ops providing interface for running queue thread in both synchronous
* and asynchronous way. The stop() operation in called just before queue is
* being destroyed.
*/
const struct ocf_queue_ops queue_ops = {
.kick_sync = queue_kick_sync,
.kick = queue_kick_async,
.stop = queue_stop,
};
/*
* Function starting cache and attaching cache device.
*/
@ -25,6 +63,7 @@ int initialize_cache(ocf_ctx_t ctx, ocf_cache_t *cache)
{
struct ocf_mngt_cache_config cache_cfg = { };
struct ocf_mngt_cache_device_config device_cfg = { };
ocf_queue_t queue;
int ret;
/* Cache configuration */
@ -33,7 +72,6 @@ int initialize_cache(ocf_ctx_t ctx, ocf_cache_t *cache)
cache_cfg.cache_line_size = ocf_cache_line_size_4;
cache_cfg.cache_mode = ocf_cache_mode_wt;
cache_cfg.metadata_volatile = true;
cache_cfg.io_queues = 1;
cache_cfg.name = "cache1";
/* Cache deivce (volume) configuration */
@ -48,6 +86,14 @@ int initialize_cache(ocf_ctx_t ctx, ocf_cache_t *cache)
if (ret)
return ret;
ret = ocf_queue_create(*cache, &queue, &queue_ops);
if (!queue) {
ocf_mngt_cache_stop(*cache);
return -ENOMEM;
}
ocf_cache_set_priv(*cache, queue);
/* Attach volume to cache */
ret = ocf_mngt_cache_attach(*cache, &device_cfg);
if (ret) {
@ -113,6 +159,8 @@ int submit_io(ocf_core_t core, struct volume_data *data,
uint64_t addr, uint64_t len, int dir, ocf_end_io_t cmpl)
{
struct ocf_io *io;
ocf_cache_t cache = ocf_core_get_cache(core);
ocf_queue_t queue = (ocf_queue_t)ocf_cache_get_priv(cache);
/* Allocate new io */
io = ocf_core_new_io(core);
@ -123,6 +171,8 @@ int submit_io(ocf_core_t core, struct volume_data *data,
ocf_io_configure(io, addr, len, dir, 0, 0);
/* Assign data to io */
ocf_io_set_data(io, data, 0);
/* Setup io queue to */
ocf_io_set_queue(io, queue);
/* Setup completion function */
ocf_io_set_cmpl(io, NULL, NULL, cmpl);
/* Submit io */

View File

@ -109,18 +109,6 @@ ocf_volume_t ocf_cache_get_volume(ocf_cache_t cache);
*/
ocf_cache_id_t ocf_cache_get_id(ocf_cache_t cache);
/**
* @brief Get queue object associated with cache
*
* @param[in] cache Cache object
* @param[in] id Queue id
* @param[out] q Queue object
*
* @retval 0 Success
* @retval Non-zero Fail
*/
int ocf_cache_get_queue(ocf_cache_t cache, unsigned id, ocf_queue_t *q);
/**
* @brief Set name of given cache object
*

View File

@ -34,16 +34,17 @@ void ocf_cleaner_set_cmpl(ocf_cleaner_t cleaner, ocf_cleaner_end_t fn);
* @brief Set cleaner queue
*
* @param[in] cleaner Cleaner instance
* @param[in] io_queue Queue number
* @param[in] io_queue Queue handle
*/
void ocf_cleaner_set_io_queue(ocf_cleaner_t cleaner, uint32_t io_queue);
void ocf_cleaner_set_io_queue(ocf_cleaner_t cleaner, ocf_queue_t io_queue);
/**
* @brief Run cleaner
*
* @param[in] c Cleaner instance to run
* @param[in] queue IO queue handle
*/
void ocf_cleaner_run(ocf_cleaner_t c);
void ocf_cleaner_run(ocf_cleaner_t c, ocf_queue_t queue);
/**
* @brief Set cleaner private data

View File

@ -128,54 +128,6 @@ struct ocf_data_ops {
void (*secure_erase)(ctx_data_t *dst);
};
/**
* @brief I/O queue operations
*/
struct ocf_queue_ops {
/**
* @brief Initialize I/O queue.
*
* This function should create worker, thread or any other queue
* processing related stuff specific to given environment.
*
* @param[in] q I/O queue to be initialized
*
* @retval 0 I/O queue has been initializaed successfully
* @retval Non-zero I/O queue initialization failure
*/
int (*init)(ocf_queue_t q);
/**
* @brief Kick I/O queue processing
*
* This function should inform worker, thread or any other queue
* processing mechanism, that there are new requests in queue to
* be processed. Processing requests inside current call is not allowed.
*
* @param[in] q I/O queue to be kicked
*/
void (*kick)(ocf_queue_t q);
/**
* @brief Kick I/O queue processing
*
* This function should inform worker, thread or any other queue
* processing mechanism, that there are new requests in queue to
* be processed. Kick function is allowed to process requests in current
* call
*
* @param[in] q I/O queue to be kicked
*/
void (*kick_sync)(ocf_queue_t q);
/**
* @brief Stop I/O queue
*
* @param[in] q I/O queue beeing stopped
*/
void (*stop)(ocf_queue_t q);
};
/**
* @brief Cleaner operations
*/
@ -243,9 +195,6 @@ struct ocf_ctx_ops {
/* Context data operations */
struct ocf_data_ops data;
/* Queue operations */
struct ocf_queue_ops queue;
/* Cleaner operations */
struct ocf_cleaner_ops cleaner;

View File

@ -84,9 +84,9 @@ struct ocf_io {
uint32_t dir;
/**
* @brief Queue id
* @brief Queue handle
*/
uint32_t io_queue;
ocf_queue_t io_queue;
/**
* @brief OCF IO start function
@ -258,12 +258,12 @@ static inline ctx_data_t *ocf_io_get_data(struct ocf_io *io)
}
/**
* @brief Set queue id to which IO should be submitted
* @brief Set queue to which IO should be submitted
*
* @param[in] io OCF IO to set up
* @param[in] queue IO queue id
* @param[in] queue IO queue handle
*/
static inline void ocf_io_set_queue(struct ocf_io *io, uint32_t queue)
static inline void ocf_io_set_queue(struct ocf_io *io, ocf_queue_t queue)
{
io->io_queue = queue;
}

View File

@ -240,11 +240,6 @@ struct ocf_mngt_cache_config {
uint32_t queue_unblock_size;
} backfill;
/**
* @brief Number of I/O queues to be created
*/
uint32_t io_queues;
/**
* @brief Start cache and keep it locked
*
@ -360,14 +355,12 @@ int ocf_mngt_cache_detach(ocf_cache_t cache);
*
* @param[in] ctx OCF context
* @param[out] cache Cache handle
* @param[in] cfg Cache configuration
* @param[in] device_cfg Caching device configuration
*
* @retval 0 Cache successfully loaded
* @retval Non-zero Error occurred during loading cache
*/
int ocf_mngt_cache_load(ocf_ctx_t ctx, ocf_cache_t *cache,
struct ocf_mngt_cache_config *cfg,
struct ocf_mngt_cache_device_config *device_cfg);
/* Adding and removing cores */

View File

@ -11,6 +11,72 @@
* @brief OCF queues API
*/
/**
* @brief I/O queue operations
*/
struct ocf_queue_ops {
/**
* @brief Kick I/O queue processing
*
* This function should inform worker, thread or any other queue
* processing mechanism, that there are new requests in queue to
* be processed. Processing requests synchronously in this function
* is not allowed.
*
* @param[in] q I/O queue to be kicked
*/
void (*kick)(ocf_queue_t q);
/**
* @brief Kick I/O queue processing
*
* This function should inform worker, thread or any other queue
* processing mechanism, that there are new requests in queue to
* be processed. Function kick_sync is allowed to process requests
* synchronously without delegating them to the worker.
*
* @param[in] q I/O queue to be kicked
*/
void (*kick_sync)(ocf_queue_t q);
/**
* @brief Stop I/O queue
*
* @param[in] q I/O queue beeing stopped
*/
void (*stop)(ocf_queue_t q);
};
/**
* @brief Allocate IO queue and add it to list in cache
*
* @param[in] cache Handle to cache instance
* @param[out] queue Handle to created queue
* @param[in] ops Queue operations
*
* @return Zero on success, otherwise error code
*/
int ocf_queue_create(ocf_cache_t cache, ocf_queue_t *queue,
const struct ocf_queue_ops *ops);
/**
* @brief Increase reference counter in queue
*
* @param[in] queue Queue
*
*/
void ocf_queue_get(ocf_queue_t queue);
/**
* @brief Decrease reference counter in queue
*
* @note If queue don't have any reference - deallocate it
*
* @param[in] queue Queue
*
*/
void ocf_queue_put(ocf_queue_t queue);
/**
* @brief Process single request from queue
*
@ -60,13 +126,4 @@ uint32_t ocf_queue_pending_io(ocf_queue_t q);
*/
ocf_cache_t ocf_queue_get_cache(ocf_queue_t q);
/**
* @brief Get I/O queue id
*
* @param[in] q I/O queue
*
* @retval I/O queue id
*/
uint32_t ocf_queue_get_id(ocf_queue_t q);
#endif

View File

@ -74,9 +74,6 @@ struct ocf_event_cache_desc {
/** Number of cores */
uint32_t cores_no;
/** Number of IO queues */
uint32_t io_queues_no;
/** Trace version */
uint32_t version;
};
@ -152,7 +149,7 @@ struct ocf_event_io_cmpl {
*
* @param[in] cache OCF cache
* @param[in] trace_ctx Tracing context
* @param[in] qid Queue Id
* @param[in] queue Queue handle
* @param[out] trace Event log
* @param[out] size Size of event log
*
@ -160,7 +157,7 @@ struct ocf_event_io_cmpl {
* @return Non-zero error
*/
typedef void (*ocf_trace_callback_t)(ocf_cache_t cache, void *trace_ctx,
uint32_t qid, const void* trace, const uint32_t size);
ocf_queue_t queue, const void* trace, const uint32_t size);
/**
* @brief Start tracing

View File

@ -536,7 +536,7 @@ static bool _acp_prepare_flush_data(struct acp_context *acp,
/* Clean at most 'flush_max_buffers' cache lines from current or newly
* selected chunk */
void cleaning_policy_acp_perform_cleaning(struct ocf_cache *cache,
void cleaning_policy_acp_perform_cleaning(ocf_cache_t cache,
ocf_cleaner_end_t cmpl)
{
struct acp_cleaning_policy_config *config;

View File

@ -449,7 +449,7 @@ void cleaning_policy_alru_setup(struct ocf_cache *cache)
config->activity_threshold = OCF_ALRU_DEFAULT_ACTIVITY_THRESHOLD;
}
int cleaning_policy_alru_initialize(struct ocf_cache *cache, int init_metadata)
int cleaning_policy_alru_initialize(ocf_cache_t cache, int init_metadata)
{
struct ocf_user_part *part;
ocf_part_id_t part_id;

View File

@ -11,6 +11,7 @@
#include "../ocf_ctx_priv.h"
#include "../mngt/ocf_mngt_common.h"
#include "../metadata/metadata.h"
#include "../ocf_queue_priv.h"
#define SLEEP_TIME_MS (1000)
@ -63,11 +64,6 @@ void ocf_cleaner_set_cmpl(ocf_cleaner_t cleaner, ocf_cleaner_end_t fn)
cleaner->end = fn;
}
void ocf_cleaner_set_io_queue(ocf_cleaner_t cleaner, uint32_t io_queue)
{
cleaner->io_queue = io_queue;
}
void ocf_cleaner_set_priv(ocf_cleaner_t c, void *priv)
{
OCF_CHECK_NULL(c);
@ -112,13 +108,21 @@ static void ocf_cleaner_run_complete(ocf_cleaner_t cleaner, uint32_t interval)
env_rwsem_up_write(&cache->lock);
cleaner->end(cleaner, interval);
ocf_queue_put(cleaner->io_queue);
cleaner->io_queue = NULL;
}
void ocf_cleaner_run(ocf_cleaner_t cleaner)
void ocf_cleaner_run(ocf_cleaner_t cleaner, ocf_queue_t queue)
{
ocf_cache_t cache = ocf_cleaner_get_cache(cleaner);
ocf_cache_t cache;
ocf_cleaning_t clean_type;
OCF_CHECK_NULL(cleaner);
OCF_CHECK_NULL(queue);
cache = ocf_cleaner_get_cache(cleaner);
/* Do not involve cleaning when cache is not running
* (error, etc.).
*/
@ -143,6 +147,9 @@ void ocf_cleaner_run(ocf_cleaner_t cleaner)
ENV_BUG_ON(clean_type >= ocf_cleaning_max);
ocf_queue_get(queue);
cleaner->io_queue = queue;
/* Call cleaning. */
if (cleaning_policy_ops[clean_type].perform_cleaning) {
cleaning_policy_ops[clean_type].perform_cleaning(cache,

View File

@ -59,7 +59,7 @@ extern struct cleaning_policy_ops cleaning_policy_ops[ocf_cleaning_max];
struct ocf_cleaner {
void *cleaning_policy_context;
uint32_t io_queue;
ocf_queue_t io_queue;
ocf_cleaner_end_t end;
void *priv;
};

View File

@ -106,8 +106,7 @@ const struct ocf_io_if *ocf_get_io_if(ocf_req_cache_mode_t req_cache_mode)
return cache_mode_io_if_map[req_cache_mode];
}
struct ocf_request *ocf_engine_pop_req(struct ocf_cache *cache,
struct ocf_queue *q)
struct ocf_request *ocf_engine_pop_req(ocf_cache_t cache, ocf_queue_t q)
{
unsigned long lock_flags = 0;
struct ocf_request *req;
@ -263,6 +262,7 @@ int ocf_engine_hndl_fast_req(struct ocf_request *req,
ocf_req_cache_mode_t req_cache_mode)
{
const struct ocf_io_if *io_if;
int ret;
io_if = ocf_get_io_if(req_cache_mode);
if (!io_if)
@ -270,12 +270,16 @@ int ocf_engine_hndl_fast_req(struct ocf_request *req,
switch (req->rw) {
case OCF_READ:
return io_if->read(req);
ret = io_if->read(req);
break;
case OCF_WRITE:
return io_if->write(req);
ret = io_if->write(req);
break;
default:
return OCF_FAST_PATH_NO;
}
return ret;
}
static void ocf_engine_hndl_2dc_req(struct ocf_request *req)

View File

@ -117,7 +117,7 @@ void ocf_engine_update_req_info(struct ocf_cache *cache,
req->info.dirty_any++;
/* Check if cache line is fully dirty */
if (metadata_test_dirty_sec(cache, _entry->coll_idx,
if (metadata_test_dirty_sec(cache, _entry->coll_idx,
start_sector, end_sector))
req->info.dirty_all++;
}
@ -488,14 +488,14 @@ void ocf_engine_update_request_stats(struct ocf_request *req)
void ocf_engine_push_req_back(struct ocf_request *req, bool allow_sync)
{
struct ocf_cache *cache = req->cache;
struct ocf_queue *q = NULL;
ocf_cache_t cache = req->cache;
ocf_queue_t q = NULL;
unsigned long lock_flags = 0;
INIT_LIST_HEAD(&req->list);
ENV_BUG_ON(req->io_queue >= cache->io_queues_no);
q = &cache->io_queues[req->io_queue];
ENV_BUG_ON(!req->io_queue);
q = req->io_queue;
env_spinlock_lock_irqsave(&q->io_list_lock, lock_flags);
@ -508,19 +508,18 @@ void ocf_engine_push_req_back(struct ocf_request *req, bool allow_sync)
env_atomic_set(&cache->last_access_ms,
env_ticks_to_msecs(env_get_tick_count()));
ctx_queue_kick(cache->owner, q, allow_sync);
ocf_queue_kick(q, allow_sync);
}
void ocf_engine_push_req_front(struct ocf_request *req, bool allow_sync)
{
struct ocf_cache *cache = req->cache;
struct ocf_queue *q = NULL;
ocf_cache_t cache = req->cache;
ocf_queue_t q = NULL;
unsigned long lock_flags = 0;
INIT_LIST_HEAD(&req->list);
ENV_BUG_ON(req->io_queue >= cache->io_queues_no);
q = &cache->io_queues[req->io_queue];
q = req->io_queue;
env_spinlock_lock_irqsave(&q->io_list_lock, lock_flags);
@ -533,7 +532,7 @@ void ocf_engine_push_req_front(struct ocf_request *req, bool allow_sync)
env_atomic_set(&cache->last_access_ms,
env_ticks_to_msecs(env_get_tick_count()));
ctx_queue_kick(cache->owner, q, allow_sync);
ocf_queue_kick(q, allow_sync);
}
void ocf_engine_push_req_front_if(struct ocf_request *req,

View File

@ -40,8 +40,8 @@ static uint32_t ocf_evict_calculate(struct ocf_user_part *part,
return to_evict;
}
static inline uint32_t ocf_evict_do(struct ocf_cache *cache,
uint32_t io_queue, const uint32_t evict_cline_no,
static inline uint32_t ocf_evict_do(ocf_cache_t cache,
ocf_queue_t io_queue, const uint32_t evict_cline_no,
ocf_core_id_t core_id, ocf_part_id_t target_part_id)
{
uint32_t to_evict = 0, evicted = 0;

View File

@ -32,22 +32,22 @@ union eviction_policy_meta {
* set core_id to -2 to purge the whole cache partition
*/
struct eviction_policy_ops {
void (*init_cline)(struct ocf_cache *cache,
void (*init_cline)(ocf_cache_t cache,
ocf_cache_line_t cline);
void (*rm_cline)(struct ocf_cache *cache,
void (*rm_cline)(ocf_cache_t cache,
ocf_cache_line_t cline);
bool (*can_evict)(struct ocf_cache *cache);
uint32_t (*req_clines)(struct ocf_cache *cache,
uint32_t io_queue, ocf_part_id_t part_id,
bool (*can_evict)(ocf_cache_t cache);
uint32_t (*req_clines)(ocf_cache_t cache,
ocf_queue_t io_queue, ocf_part_id_t part_id,
uint32_t cline_no, ocf_core_id_t core_id);
void (*hot_cline)(struct ocf_cache *cache,
void (*hot_cline)(ocf_cache_t cache,
ocf_cache_line_t cline);
void (*init_evp)(struct ocf_cache *cache,
void (*init_evp)(ocf_cache_t cache,
ocf_part_id_t part_id);
void (*dirty_cline)(struct ocf_cache *cache,
void (*dirty_cline)(ocf_cache_t cache,
ocf_part_id_t part_id,
uint32_t cline_no);
void (*clean_cline)(struct ocf_cache *cache,
void (*clean_cline)(ocf_cache_t cache,
ocf_part_id_t part_id,
uint32_t cline_no);
const char *name;
@ -62,9 +62,9 @@ extern struct eviction_policy_ops evict_policy_ops[ocf_eviction_max];
* or the destination partition ID for the free buffers
* (it matches label and is part of the object (#core_id) IO group)
*/
int space_managment_evict_do(struct ocf_cache *cache,
int space_managment_evict_do(ocf_cache_t cache,
struct ocf_request *req, uint32_t evict_cline_no);
int space_management_free(struct ocf_cache *cache, uint32_t count);
int space_management_free(ocf_cache_t cache, uint32_t count);
#endif

View File

@ -29,7 +29,7 @@
#define is_lru_tail(x) (x == collision_table_entries)
/* Sets the given collision_index as the new _head_ of the LRU list. */
static inline void update_lru_head(struct ocf_cache *cache,
static inline void update_lru_head(ocf_cache_t cache,
int partition_id, unsigned int collision_index,
int cline_dirty)
{
@ -43,7 +43,7 @@ static inline void update_lru_head(struct ocf_cache *cache,
}
/* Sets the given collision_index as the new _tail_ of the LRU list. */
static inline void update_lru_tail(struct ocf_cache *cache,
static inline void update_lru_tail(ocf_cache_t cache,
int partition_id, unsigned int collision_index,
int cline_dirty)
{
@ -58,7 +58,7 @@ static inline void update_lru_tail(struct ocf_cache *cache,
/* Sets the given collision_index as the new _head_ and _tail_ of
* the LRU list.
*/
static inline void update_lru_head_tail(struct ocf_cache *cache,
static inline void update_lru_head_tail(ocf_cache_t cache,
int partition_id, unsigned int collision_index, int cline_dirty)
{
update_lru_head(cache, partition_id, collision_index, cline_dirty);
@ -66,7 +66,7 @@ static inline void update_lru_head_tail(struct ocf_cache *cache,
}
/* Adds the given collision_index to the _head_ of the LRU list */
static void add_lru_head(struct ocf_cache *cache, int partition_id,
static void add_lru_head(ocf_cache_t cache, int partition_id,
unsigned int collision_index, int cline_dirty)
{
unsigned int curr_head_index;
@ -121,7 +121,7 @@ static void add_lru_head(struct ocf_cache *cache, int partition_id,
}
/* Deletes the node with the given collision_index from the lru list */
static void remove_lru_list(struct ocf_cache *cache, int partition_id,
static void remove_lru_list(ocf_cache_t cache, int partition_id,
unsigned int collision_index, int cline_dirty)
{
int is_clean_head = 0, is_clean_tail = 0, is_dirty_head = 0, is_dirty_tail = 0;
@ -250,7 +250,7 @@ static void remove_lru_list(struct ocf_cache *cache, int partition_id,
/*-- End of LRU functions*/
void evp_lru_init_cline(struct ocf_cache *cache, ocf_cache_line_t cline)
void evp_lru_init_cline(ocf_cache_t cache, ocf_cache_line_t cline)
{
union eviction_policy_meta eviction;
@ -264,7 +264,7 @@ void evp_lru_init_cline(struct ocf_cache *cache, ocf_cache_line_t cline)
/* the caller must hold the metadata lock */
void evp_lru_rm_cline(struct ocf_cache *cache, ocf_cache_line_t cline)
void evp_lru_rm_cline(ocf_cache_t cache, ocf_cache_line_t cline)
{
ocf_part_id_t part_id = ocf_metadata_get_partition_id(cache, cline);
@ -278,7 +278,7 @@ static void evp_lru_clean_end(void *private_data, int error)
env_atomic_set(cleaning_in_progress, 0);
}
static int evp_lru_clean_getter(struct ocf_cache *cache,
static int evp_lru_clean_getter(ocf_cache_t cache,
void *getter_context, uint32_t item, ocf_cache_line_t *line)
{
union eviction_policy_meta eviction;
@ -306,7 +306,7 @@ static int evp_lru_clean_getter(struct ocf_cache *cache,
return -1;
}
static void evp_lru_clean(struct ocf_cache *cache, uint32_t io_queue,
static void evp_lru_clean(ocf_cache_t cache, ocf_queue_t io_queue,
ocf_part_id_t part_id, uint32_t count)
{
env_atomic *progress = &cache->cleaning[part_id];
@ -342,7 +342,7 @@ static void evp_lru_zero_line_complete(struct ocf_request *ocf_req, int error)
env_atomic_dec(&ocf_req->cache->pending_eviction_clines);
}
static void evp_lru_zero_line(struct ocf_cache *cache, uint32_t io_queue,
static void evp_lru_zero_line(ocf_cache_t cache, ocf_queue_t io_queue,
ocf_cache_line_t line)
{
struct ocf_request *req;
@ -352,11 +352,11 @@ static void evp_lru_zero_line(struct ocf_cache *cache, uint32_t io_queue,
ocf_metadata_get_core_info(cache, line, &id, &core_line);
addr = core_line * ocf_line_size(cache);
req = ocf_req_new(cache, id, addr, ocf_line_size(cache), OCF_WRITE);
req = ocf_req_new(io_queue, &cache->core[id], addr,
ocf_line_size(cache), OCF_WRITE);
if (req) {
req->info.internal = true;
req->complete = evp_lru_zero_line_complete;
req->io_queue = io_queue;
env_atomic_inc(&cache->pending_eviction_clines);
@ -364,7 +364,7 @@ static void evp_lru_zero_line(struct ocf_cache *cache, uint32_t io_queue,
}
}
bool evp_lru_can_evict(struct ocf_cache *cache)
bool evp_lru_can_evict(ocf_cache_t cache)
{
if (env_atomic_read(&cache->pending_eviction_clines) >=
OCF_PENDING_EVICTION_LIMIT) {
@ -375,7 +375,7 @@ bool evp_lru_can_evict(struct ocf_cache *cache)
}
/* the caller must hold the metadata lock */
uint32_t evp_lru_req_clines(struct ocf_cache *cache, uint32_t io_queue,
uint32_t evp_lru_req_clines(ocf_cache_t cache, ocf_queue_t io_queue,
ocf_part_id_t part_id, uint32_t cline_no, ocf_core_id_t core_id)
{
uint32_t i;
@ -438,7 +438,7 @@ uint32_t evp_lru_req_clines(struct ocf_cache *cache, uint32_t io_queue,
}
/* the caller must hold the metadata lock */
void evp_lru_hot_cline(struct ocf_cache *cache, ocf_cache_line_t cline)
void evp_lru_hot_cline(ocf_cache_t cache, ocf_cache_line_t cline)
{
ocf_part_id_t part_id = ocf_metadata_get_partition_id(cache, cline);
struct ocf_user_part *part = &cache->user_parts[part_id];
@ -469,7 +469,7 @@ void evp_lru_hot_cline(struct ocf_cache *cache, ocf_cache_line_t cline)
add_lru_head(cache, part_id, cline, cline_dirty);
}
void evp_lru_init_evp(struct ocf_cache *cache, ocf_part_id_t part_id)
void evp_lru_init_evp(ocf_cache_t cache, ocf_part_id_t part_id)
{
unsigned int collision_table_entries =
cache->device->collision_table_entries;
@ -483,7 +483,7 @@ void evp_lru_init_evp(struct ocf_cache *cache, ocf_part_id_t part_id)
part->runtime->eviction.policy.lru.dirty_tail = collision_table_entries;
}
void evp_lru_clean_cline(struct ocf_cache *cache, ocf_part_id_t part_id,
void evp_lru_clean_cline(ocf_cache_t cache, ocf_part_id_t part_id,
uint32_t cline)
{
OCF_METADATA_EVICTION_LOCK();
@ -492,7 +492,7 @@ void evp_lru_clean_cline(struct ocf_cache *cache, ocf_part_id_t part_id,
OCF_METADATA_EVICTION_UNLOCK();
}
void evp_lru_dirty_cline(struct ocf_cache *cache, ocf_part_id_t part_id,
void evp_lru_dirty_cline(ocf_cache_t cache, ocf_part_id_t part_id,
uint32_t cline)
{
OCF_METADATA_EVICTION_LOCK();

View File

@ -12,7 +12,7 @@ void evp_lru_init_cline(struct ocf_cache *cache,
ocf_cache_line_t cline);
void evp_lru_rm_cline(struct ocf_cache *cache, ocf_cache_line_t cline);
bool evp_lru_can_evict(struct ocf_cache *cache);
uint32_t evp_lru_req_clines(struct ocf_cache *cache, uint32_t io_queue,
uint32_t evp_lru_req_clines(struct ocf_cache *cache, ocf_queue_t io_queue,
ocf_part_id_t part_id, uint32_t cline_no,
ocf_core_id_t core_id);
void evp_lru_hot_cline(struct ocf_cache *cache, ocf_cache_line_t cline);

View File

@ -53,7 +53,7 @@ static inline bool ocf_eviction_can_evict(struct ocf_cache *cache)
}
static inline uint32_t ocf_eviction_need_space(struct ocf_cache *cache,
uint32_t io_queue, ocf_part_id_t part_id, uint32_t clines,
ocf_queue_t io_queue, ocf_part_id_t part_id, uint32_t clines,
ocf_core_id_t core_id)
{
uint8_t type;

View File

@ -44,7 +44,7 @@ static struct ocf_io_if meta_restart_if = {
/*
* Get max pages for IO
*/
static uint32_t metadata_io_max_page(struct ocf_cache *cache)
static uint32_t metadata_io_max_page(ocf_cache_t cache)
{
return ocf_volume_get_max_io_size(&cache->device->volume) / PAGE_SIZE;
}
@ -65,7 +65,7 @@ static void metadata_io_read_i_atomic_end(struct ocf_io *io, int error)
/*
* Iterative read request
*/
int metadata_io_read_i_atomic(struct ocf_cache *cache,
int metadata_io_read_i_atomic(ocf_cache_t cache,
ocf_metadata_atomic_io_event_t hndl)
{
uint64_t i;
@ -159,7 +159,7 @@ static int ocf_restart_meta_io(struct ocf_request *req)
{
struct ocf_io *io;
struct metadata_io_request *meta_io_req;
struct ocf_cache *cache;
ocf_cache_t cache;
int i;
int ret;
@ -205,7 +205,7 @@ static void metadata_io_write_i_asynch_end(struct metadata_io_request *request,
int error)
{
struct metadata_io_request_asynch *a_req;
struct ocf_cache *cache;
ocf_cache_t cache;
OCF_CHECK_NULL(request);
@ -247,7 +247,7 @@ static void metadata_io_write_i_asynch_end(struct metadata_io_request *request,
ocf_metadata_updater_kick(cache);
}
static void metadata_io_req_error(struct ocf_cache *cache,
static void metadata_io_req_error(ocf_cache_t cache,
struct metadata_io_request_asynch *a_req,
uint32_t i, int error)
{
@ -262,7 +262,7 @@ static void metadata_io_req_error(struct ocf_cache *cache,
/*
* Iterative write request asynchronously
*/
int metadata_io_write_i_asynch(struct ocf_cache *cache, uint32_t queue,
int metadata_io_write_i_asynch(ocf_cache_t cache, ocf_queue_t queue,
void *context, uint32_t page, uint32_t count,
ocf_metadata_io_event_t fill_hndl,
ocf_metadata_io_hndl_on_write_t compl_hndl)
@ -439,7 +439,7 @@ static void metadata_io_end(struct ocf_io *io, int error)
ctx_data_t *data = ocf_io_get_data(io);
uint32_t page = BYTES_TO_PAGES(io->addr);
uint32_t count = BYTES_TO_PAGES(io->bytes);
struct ocf_cache *cache = mio->cache;
ocf_cache_t cache = mio->cache;
uint32_t i = 0;
if (error) {
@ -463,7 +463,7 @@ out:
}
static int metadata_submit_io(
struct ocf_cache *cache,
ocf_cache_t cache,
struct metadata_io *mio,
uint32_t count,
uint32_t written)
@ -532,7 +532,7 @@ static int metadata_io(struct metadata_io *mio)
unsigned char step = 0;
int err;
struct ocf_cache *cache = mio->cache;
ocf_cache_t cache = mio->cache;
/* Check direction value correctness */
switch (mio->dir) {
@ -572,7 +572,7 @@ static int metadata_io(struct metadata_io *mio)
/*
*
*/
int metadata_io_write_i(struct ocf_cache *cache,
int metadata_io_write_i(ocf_cache_t cache,
uint32_t page, uint32_t count,
ocf_metadata_io_event_t hndl_fn, void *hndl_cntx)
{
@ -591,7 +591,7 @@ int metadata_io_write_i(struct ocf_cache *cache,
/*
*
*/
int metadata_io_read_i(struct ocf_cache *cache,
int metadata_io_read_i(ocf_cache_t cache,
uint32_t page, uint32_t count,
ocf_metadata_io_event_t hndl_fn, void *hndl_cntx)
{
@ -610,7 +610,7 @@ int metadata_io_read_i(struct ocf_cache *cache,
/*
*
*/
static int metadata_io_write_fill(struct ocf_cache *cache,
static int metadata_io_write_fill(ocf_cache_t cache,
ctx_data_t *data, uint32_t page, void *context)
{
ctx_data_wr_check(cache->owner, data, context, PAGE_SIZE);
@ -620,7 +620,7 @@ static int metadata_io_write_fill(struct ocf_cache *cache,
/*
* Write request
*/
int metadata_io_write(struct ocf_cache *cache,
int metadata_io_write(ocf_cache_t cache,
void *data, uint32_t page)
{
struct metadata_io mio = {

View File

@ -25,7 +25,7 @@
* @retval 0 Success
* @retval Non-zero Error which will bee finally returned to the caller
*/
typedef int (*ocf_metadata_io_event_t)(struct ocf_cache *cache,
typedef int (*ocf_metadata_io_event_t)(ocf_cache_t cache,
ctx_data_t *data, uint32_t page, void *context);
/**
@ -36,7 +36,7 @@ typedef int (*ocf_metadata_io_event_t)(struct ocf_cache *cache,
* @param error - error
* @param page - page that was written
*/
typedef void (*ocf_metadata_io_hndl_on_write_t)(struct ocf_cache *cache,
typedef void (*ocf_metadata_io_hndl_on_write_t)(ocf_cache_t cache,
void *context, int error);
struct metadata_io_request_asynch;
@ -45,7 +45,7 @@ struct metadata_io_request_asynch;
* IO request context
*/
struct metadata_io_request {
struct ocf_cache *cache;
ocf_cache_t cache;
void *context;
uint32_t page;
uint32_t count;
@ -75,7 +75,7 @@ struct metadata_io_request_atomic {
struct metadata_io {
int error;
int dir;
struct ocf_cache *cache;
ocf_cache_t cache;
uint32_t page;
uint32_t count;
env_completion completion;
@ -88,7 +88,7 @@ struct metadata_io {
* Asynchronous IO request context
*/
struct metadata_io_request_asynch {
struct ocf_cache *cache;
ocf_cache_t cache;
struct metadata_io_request *reqs;
void *context;
int error;
@ -111,7 +111,7 @@ struct metadata_io_request_asynch {
* @retval Non-zero Error which will bee finally returned to the caller
*/
typedef int (*ocf_metadata_atomic_io_event_t)(
struct ocf_cache *cache, uint64_t sector_addr,
ocf_cache_t cache, uint64_t sector_addr,
uint32_t sector_no, ctx_data_t *data);
/**
@ -122,10 +122,10 @@ typedef int (*ocf_metadata_atomic_io_event_t)(
* @param page - Page of SSD (cache device) where data has to be placed
* @return 0 - No errors, otherwise error occurred
*/
int metadata_io_write(struct ocf_cache *cache,
int metadata_io_write(ocf_cache_t cache,
void *data, uint32_t page);
int metadata_io_read_i_atomic(struct ocf_cache *cache,
int metadata_io_read_i_atomic(ocf_cache_t cache,
ocf_metadata_atomic_io_event_t hndl);
/**
@ -139,7 +139,7 @@ int metadata_io_read_i_atomic(struct ocf_cache *cache,
*
* @return 0 - No errors, otherwise error occurred
*/
int metadata_io_write_i(struct ocf_cache *cache,
int metadata_io_write_i(ocf_cache_t cache,
uint32_t page, uint32_t count,
ocf_metadata_io_event_t hndl_fn, void *hndl_cntx);
@ -154,7 +154,7 @@ int metadata_io_write_i(struct ocf_cache *cache,
*
* @return 0 - No errors, otherwise error occurred
*/
int metadata_io_read_i(struct ocf_cache *cache,
int metadata_io_read_i(ocf_cache_t cache,
uint32_t page, uint32_t count,
ocf_metadata_io_event_t hndl_fn, void *hndl_cntx);
@ -170,7 +170,7 @@ int metadata_io_read_i(struct ocf_cache *cache,
*
* @return 0 - No errors, otherwise error occurred
*/
int metadata_io_write_i_asynch(struct ocf_cache *cache, uint32_t queue,
int metadata_io_write_i_asynch(ocf_cache_t cache, ocf_queue_t queue,
void *context, uint32_t page, uint32_t count,
ocf_metadata_io_event_t fill_hndl,
ocf_metadata_io_hndl_on_write_t compl_hndl);

View File

@ -72,7 +72,7 @@ static bool _raw_ssd_page_is_valid(struct ocf_metadata_raw *raw, uint32_t page)
/*
* RAM Implementation - De-Initialize
*/
static int _raw_ram_deinit(struct ocf_cache *cache,
static int _raw_ram_deinit(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
OCF_DEBUG_TRACE(cache);
@ -88,7 +88,7 @@ static int _raw_ram_deinit(struct ocf_cache *cache,
/*
* RAM Implementation - Initialize
*/
static int _raw_ram_init(struct ocf_cache *cache,
static int _raw_ram_init(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
size_t mem_pool_size;
@ -109,7 +109,7 @@ static int _raw_ram_init(struct ocf_cache *cache,
/*
* RAM Implementation - Size of
*/
static size_t _raw_ram_size_of(struct ocf_cache *cache,
static size_t _raw_ram_size_of(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
size_t size;
@ -123,7 +123,7 @@ static size_t _raw_ram_size_of(struct ocf_cache *cache,
/*
* RAM Implementation - Size on SSD
*/
static uint32_t _raw_ram_size_on_ssd(struct ocf_cache *cache,
static uint32_t _raw_ram_size_on_ssd(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
const size_t alignment = 128 * KiB / PAGE_SIZE;
@ -134,7 +134,7 @@ static uint32_t _raw_ram_size_on_ssd(struct ocf_cache *cache,
/*
* RAM Implementation - Checksum
*/
static uint32_t _raw_ram_checksum(struct ocf_cache *cache,
static uint32_t _raw_ram_checksum(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
uint64_t i;
@ -152,7 +152,7 @@ static uint32_t _raw_ram_checksum(struct ocf_cache *cache,
/*
* RAM Implementation - Get entry
*/
static int _raw_ram_get(struct ocf_cache *cache,
static int _raw_ram_get(ocf_cache_t cache,
struct ocf_metadata_raw *raw, ocf_cache_line_t line,
void *data, uint32_t size)
{
@ -164,7 +164,7 @@ static int _raw_ram_get(struct ocf_cache *cache,
/*
* RAM Implementation - Read only entry access
*/
static const void *_raw_ram_rd_access(struct ocf_cache *cache,
static const void *_raw_ram_rd_access(ocf_cache_t cache,
struct ocf_metadata_raw *raw, ocf_cache_line_t line,
uint32_t size)
{
@ -176,7 +176,7 @@ static const void *_raw_ram_rd_access(struct ocf_cache *cache,
/*
* RAM Implementation - Read only entry access
*/
static void *_raw_ram_wr_access(struct ocf_cache *cache,
static void *_raw_ram_wr_access(ocf_cache_t cache,
struct ocf_metadata_raw *raw, ocf_cache_line_t line,
uint32_t size)
{
@ -188,7 +188,7 @@ static void *_raw_ram_wr_access(struct ocf_cache *cache,
/*
* RAM Implementation - Set Entry
*/
static int _raw_ram_set(struct ocf_cache *cache,
static int _raw_ram_set(ocf_cache_t cache,
struct ocf_metadata_raw *raw, ocf_cache_line_t line,
void *data, uint32_t size)
{
@ -200,7 +200,7 @@ static int _raw_ram_set(struct ocf_cache *cache,
/*
* RAM Implementation - Flush specified element from SSD
*/
static int _raw_ram_flush(struct ocf_cache *cache,
static int _raw_ram_flush(ocf_cache_t cache,
struct ocf_metadata_raw *raw, ocf_cache_line_t line)
{
OCF_DEBUG_PARAM(cache, "Line = %u", line);
@ -215,7 +215,7 @@ static int _raw_ram_flush(struct ocf_cache *cache,
/*
* RAM Implementation - Load all IO callback
*/
static int _raw_ram_load_all_io(struct ocf_cache *cache,
static int _raw_ram_load_all_io(ocf_cache_t cache,
ctx_data_t *data, uint32_t page, void *context)
{
ocf_cache_line_t line;
@ -241,7 +241,7 @@ static int _raw_ram_load_all_io(struct ocf_cache *cache,
/*
* RAM Implementation - Load all metadata elements from SSD
*/
static int _raw_ram_load_all(struct ocf_cache *cache,
static int _raw_ram_load_all(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
OCF_DEBUG_TRACE(cache);
@ -253,7 +253,7 @@ static int _raw_ram_load_all(struct ocf_cache *cache,
/*
* RAM Implementation - Flush IO callback - Fill page
*/
static int _raw_ram_flush_all_fill(struct ocf_cache *cache,
static int _raw_ram_flush_all_fill(ocf_cache_t cache,
ctx_data_t *data, uint32_t page, void *context)
{
ocf_cache_line_t line;
@ -278,7 +278,7 @@ static int _raw_ram_flush_all_fill(struct ocf_cache *cache,
/*
* RAM Implementation - Flush all elements
*/
static int _raw_ram_flush_all(struct ocf_cache *cache,
static int _raw_ram_flush_all(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
OCF_DEBUG_TRACE(cache);
@ -290,7 +290,7 @@ static int _raw_ram_flush_all(struct ocf_cache *cache,
/*
* RAM RAM Implementation - Mark to Flush
*/
static void _raw_ram_flush_mark(struct ocf_cache *cache,
static void _raw_ram_flush_mark(ocf_cache_t cache,
struct ocf_request *req, uint32_t map_idx, int to_state,
uint8_t start, uint8_t stop)
{
@ -311,7 +311,7 @@ struct _raw_ram_flush_ctx {
int error;
};
static void _raw_ram_flush_do_asynch_io_complete(struct ocf_cache *cache,
static void _raw_ram_flush_do_asynch_io_complete(ocf_cache_t cache,
void *context, int error)
{
struct _raw_ram_flush_ctx *ctx = context;
@ -336,7 +336,7 @@ static void _raw_ram_flush_do_asynch_io_complete(struct ocf_cache *cache,
/*
* RAM Implementation - Flush IO callback - Fill page
*/
static int _raw_ram_flush_do_asynch_fill(struct ocf_cache *cache,
static int _raw_ram_flush_do_asynch_fill(ocf_cache_t cache,
ctx_data_t *data, uint32_t page, void *context)
{
ocf_cache_line_t line;
@ -400,7 +400,7 @@ static void __raw_ram_flush_do_asynch_add_pages(struct ocf_request *req,
*pages_to_flush = j;
}
static int _raw_ram_flush_do_asynch(struct ocf_cache *cache,
static int _raw_ram_flush_do_asynch(ocf_cache_t cache,
struct ocf_request *req, struct ocf_metadata_raw *raw,
ocf_req_end_t complete)
{
@ -575,7 +575,7 @@ static const struct raw_iface IRAW[metadata_raw_type_max] = {
* RAW Top interface implementation
******************************************************************************/
int ocf_metadata_raw_init(struct ocf_cache *cache,
int ocf_metadata_raw_init(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
ENV_BUG_ON(raw->raw_type < metadata_raw_type_min);
@ -585,7 +585,7 @@ int ocf_metadata_raw_init(struct ocf_cache *cache,
return raw->iface->init(cache, raw);
}
int ocf_metadata_raw_deinit(struct ocf_cache *cache,
int ocf_metadata_raw_deinit(ocf_cache_t cache,
struct ocf_metadata_raw *raw)
{
int result;
@ -599,7 +599,7 @@ int ocf_metadata_raw_deinit(struct ocf_cache *cache,
return result;
}
size_t ocf_metadata_raw_size_on_ssd(struct ocf_cache* cache,
size_t ocf_metadata_raw_size_on_ssd(ocf_cache_t cache,
struct ocf_metadata_raw* raw)
{
ENV_BUG_ON(raw->raw_type < metadata_raw_type_min);

View File

@ -77,13 +77,8 @@ struct ocf_cachemng_init_params {
bool metadata_inited : 1;
/*!< Metadata is inited to valid state */
bool queues_inited : 1;
bool cache_locked : 1;
/*!< Cache has been locked */
bool io_queues_started : 1;
/*!< queues are started */
} flags;
struct ocf_metadata_init_params {
@ -743,7 +738,6 @@ static int _ocf_mngt_init_prepare_cache(struct ocf_cachemng_init_params *param,
env_rwsem_down_write(&cache->lock); /* Lock cache during setup */
param->flags.cache_locked = true;
cache->io_queues_no = cfg->io_queues;
cache->pt_unaligned_io = cfg->pt_unaligned_io;
cache->use_submit_io_fast = cfg->use_submit_io_fast;
@ -1018,8 +1012,8 @@ static int check_ram_availability(ocf_ctx_t ctx,
*/
static int _ocf_mngt_init_post_action(struct ocf_cachemng_attach_params *attach_params)
{
int result = 0;
struct ocf_cache *cache = attach_params->cache;
int result;
/* clear clean shutdown status */
if (ocf_metadata_set_shutdown_status(cache,
@ -1059,17 +1053,16 @@ static int _ocf_mngt_init_post_action(struct ocf_cachemng_attach_params *attach_
static void _ocf_mngt_init_handle_error(ocf_cache_t cache,
ocf_ctx_t ctx, struct ocf_cachemng_init_params *params)
{
if (!params || params->flags.io_queues_started)
ocf_stop_queues(cache);
if (!params || params->flags.queues_inited)
ocf_free_queues(cache);
ocf_queue_t queue, tmp_queue;
if (!params || params->flags.metadata_inited)
ocf_metadata_deinit(cache);
env_mutex_lock(&ctx->lock);
list_for_each_entry_safe(queue, tmp_queue, &cache->io_queues, list)
ocf_queue_put(queue);
if (!params || params->flags.cache_alloc) {
list_del(&cache->list);
env_vfree(cache);
@ -1150,7 +1143,6 @@ static int _ocf_mngt_cache_init(ocf_cache_t cache,
struct ocf_cachemng_init_params *params)
{
int i;
int result;
/*
* Super block elements initialization
@ -1163,10 +1155,7 @@ static int _ocf_mngt_cache_init(ocf_cache_t cache,
&cache->conf_meta->user_parts[i];
}
result = ocf_alloc_queues(cache);
if (result)
return result;
params->flags.queues_inited = 1;
INIT_LIST_HEAD(&cache->io_queues);
/* Init Partitions */
ocf_part_init(cache);
@ -1219,16 +1208,6 @@ static int _ocf_mngt_cache_start(ocf_ctx_t ctx, ocf_cache_t *cache,
ocf_log(ctx, log_debug, "Metadata initialized\n");
params.flags.metadata_inited = true;
if (!params.flags.io_queues_started) {
result = ocf_start_queues(*cache);
if (result) {
ocf_log(ctx, log_err,
"Error while creating I/O queues\n");
return result;
}
params.flags.io_queues_started = true;
}
if (params.locked) {
/* Increment reference counter to match cache_lock /
cache_unlock convention. User is expected to call
@ -1405,9 +1384,6 @@ static int _ocf_mngt_cache_validate_cfg(struct ocf_mngt_cache_config *cfg)
if (!ocf_cache_line_size_is_valid(cfg->cache_line_size))
return -OCF_ERR_INVALID_CACHE_LINE_SIZE;
if (!cfg->io_queues)
return -OCF_ERR_INVAL;
if (cfg->metadata_layout >= ocf_metadata_layout_max ||
cfg->metadata_layout < 0) {
return -OCF_ERR_INVAL;
@ -1573,6 +1549,7 @@ static int _ocf_mngt_cache_stop(ocf_cache_t cache)
{
int i, j, no, result = 0;
ocf_ctx_t owner = cache->owner;
ocf_queue_t queue, tmp_queue;
no = cache->conf_meta->core_count;
@ -1596,7 +1573,8 @@ static int _ocf_mngt_cache_stop(ocf_cache_t cache)
if (env_atomic_read(&cache->attached))
result = _ocf_mngt_cache_unplug(cache, true);
ocf_stop_queues(cache);
list_for_each_entry_safe(queue, tmp_queue, &cache->io_queues, list)
ocf_queue_put(queue);
env_mutex_lock(&owner->lock);
/* Mark device uninitialized */
@ -1635,35 +1613,17 @@ static void _ocf_mngt_cache_load_log(ocf_cache_t cache)
}
int ocf_mngt_cache_load(ocf_ctx_t ctx, ocf_cache_t *cache,
struct ocf_mngt_cache_config *cfg,
struct ocf_mngt_cache_device_config *device_cfg)
{
int result;
if (!ctx || !cache || !cfg || !device_cfg)
if (!ctx || !cache || !device_cfg)
return -OCF_ERR_INVAL;
result = _ocf_mngt_cache_validate_cfg(cfg);
if (result)
return result;
result = _ocf_mngt_cache_validate_device_cfg(device_cfg);
if (result)
return result;
result = _ocf_mngt_cache_start(ctx, cache, cfg);
if (!result) {
ocf_cache_log(*cache, log_info, "Successfully added\n");
} else {
if (cfg->name) {
ocf_log(ctx, log_err, "Inserting cache %s failed\n",
cfg->name);
} else {
ocf_log(ctx, log_err, "Inserting cache failed\n");
}
return result;
}
result = _ocf_mngt_cache_attach(*cache, device_cfg, true);
if (result) {
_ocf_mngt_init_handle_error(*cache, ctx, NULL);

View File

@ -169,7 +169,6 @@ void ocf_mngt_cache_put(ocf_cache_t cache)
OCF_CHECK_NULL(cache);
if (env_atomic_dec_return(&cache->ref_count) == 0) {
ocf_free_queues(cache);
ocf_metadata_deinit(cache);
env_vfree(cache);
}

View File

@ -185,8 +185,7 @@ struct ocf_cache {
env_atomic pending_eviction_clines;
struct ocf_queue *io_queues;
uint32_t io_queues_no;
struct list_head io_queues;
uint16_t ocf_core_inactive_count;
struct ocf_core core[OCF_CORE_MAX];

View File

@ -273,9 +273,6 @@ static inline void dec_counter_if_req_was_dirty(struct ocf_core_io *core_io,
static inline int ocf_core_validate_io(struct ocf_io *io)
{
ocf_core_t core = ocf_volume_to_core(io->volume);
ocf_cache_t cache = ocf_core_get_cache(core);
if (!io->volume)
return -EINVAL;
@ -294,7 +291,7 @@ static inline int ocf_core_validate_io(struct ocf_io *io)
if (io->dir != OCF_READ && io->dir != OCF_WRITE)
return -EINVAL;
if (io->io_queue >= cache->io_queues_no)
if (!io->io_queue)
return -EINVAL;
if (!io->end)
@ -364,8 +361,8 @@ void ocf_core_submit_io_mode(struct ocf_io *io, ocf_cache_mode_t cache_mode)
dec_counter_if_req_was_dirty(core_io, cache);
}
core_io->req = ocf_req_new(cache, ocf_core_get_id(core),
io->addr, io->bytes, io->dir);
core_io->req = ocf_req_new(io->io_queue, core, io->addr, io->bytes,
io->dir);
if (!core_io->req) {
dec_counter_if_req_was_dirty(core_io, cache);
io->end(io, -ENOMEM);
@ -375,7 +372,6 @@ void ocf_core_submit_io_mode(struct ocf_io *io, ocf_cache_mode_t cache_mode)
if (core_io->req->d2c)
req_cache_mode = ocf_req_cache_mode_d2c;
core_io->req->io_queue = io->io_queue;
core_io->req->part_id = ocf_part_class2id(cache, io->io_class);
core_io->req->data = core_io->data;
core_io->req->complete = ocf_req_complete;
@ -453,7 +449,7 @@ int ocf_core_submit_io_fast(struct ocf_io *io)
req_cache_mode = ocf_req_cache_mode_fast;
}
core_io->req = ocf_req_new_extended(cache, ocf_core_get_id(core),
core_io->req = ocf_req_new_extended(io->io_queue, core,
io->addr, io->bytes, io->dir);
// We need additional pointer to req in case completion arrives before
// we leave this function and core_io is freed
@ -470,7 +466,6 @@ int ocf_core_submit_io_fast(struct ocf_io *io)
return -EIO;
}
req->io_queue = io->io_queue;
req->part_id = ocf_part_class2id(cache, io->io_class);
req->data = core_io->data;
req->complete = ocf_req_complete;
@ -489,8 +484,7 @@ int ocf_core_submit_io_fast(struct ocf_io *io)
fast = ocf_engine_hndl_fast_req(req, req_cache_mode);
if (fast != OCF_FAST_PATH_NO) {
ocf_trace_push(cache, core_io->req->io_queue,
&trace_event, sizeof(trace_event));
ocf_trace_push(io->io_queue, &trace_event, sizeof(trace_event));
ocf_seq_cutoff_update(core, req);
return 0;
}
@ -533,14 +527,13 @@ static void ocf_core_volume_submit_flush(struct ocf_io *io)
return;
}
core_io->req = ocf_req_new(cache, ocf_core_get_id(core),
io->addr, io->bytes, io->dir);
core_io->req = ocf_req_new(io->io_queue, core, io->addr, io->bytes,
io->dir);
if (!core_io->req) {
ocf_io_end(io, -ENOMEM);
return;
}
core_io->req->io_queue = io->io_queue;
core_io->req->complete = ocf_req_complete;
core_io->req->io = io;
core_io->req->data = core_io->data;
@ -576,14 +569,13 @@ static void ocf_core_volume_submit_discard(struct ocf_io *io)
return;
}
core_io->req = ocf_req_new_discard(cache, ocf_core_get_id(core),
core_io->req = ocf_req_new_discard(io->io_queue, core,
io->addr, io->bytes, OCF_WRITE);
if (!core_io->req) {
ocf_io_end(io, -ENOMEM);
return;
}
core_io->req->io_queue = io->io_queue;
core_io->req->complete = ocf_req_complete;
core_io->req->io = io;
core_io->req->data = core_io->data;
@ -653,7 +645,7 @@ static ctx_data_t *ocf_core_io_get_data(struct ocf_io *io)
return core_io->data;
}
const struct ocf_volume_properties ocf_core_volume_properties = {
const struct ocf_volume_properties ocf_core_volume_properties = {
.name = "OCF Core",
.io_priv_size = sizeof(struct ocf_core_io),
.volume_priv_size = sizeof(struct ocf_core_volume),

View File

@ -136,25 +136,6 @@ static inline void ctx_data_secure_erase(ocf_ctx_t ctx, ctx_data_t *dst)
return ctx->ops->data.secure_erase(dst);
}
static inline int ctx_queue_init(ocf_ctx_t ctx, ocf_queue_t queue)
{
return ctx->ops->queue.init(queue);
}
static inline void ctx_queue_kick(ocf_ctx_t ctx, ocf_queue_t queue,
bool allow_sync)
{
if (allow_sync && ctx->ops->queue.kick_sync)
ctx->ops->queue.kick_sync(queue);
else
ctx->ops->queue.kick(queue);
}
static inline void ctx_queue_stop(ocf_ctx_t ctx, ocf_queue_t queue)
{
ctx->ops->queue.stop(queue);
}
static inline int ctx_cleaner_init(ocf_ctx_t ctx, ocf_cleaner_t cleaner)
{
return ctx->ops->cleaner.init(cleaner);

View File

@ -13,67 +13,52 @@
#include "engine/cache_engine.h"
#include "ocf_def_priv.h"
int ocf_alloc_queues(struct ocf_cache *cache)
{
ENV_BUG_ON(!cache->io_queues_no);
cache->io_queues = env_zalloc(
sizeof(*cache->io_queues) * cache->io_queues_no, ENV_MEM_NORMAL);
if (!cache->io_queues)
return -ENOMEM;
return 0;
}
void ocf_free_queues(struct ocf_cache *cache)
{
env_free(cache->io_queues);
cache->io_queues_no = 0;
cache->io_queues = NULL;
}
static void ocf_init_queue(struct ocf_queue *q)
static void ocf_init_queue(ocf_queue_t q)
{
env_atomic_set(&q->io_no, 0);
env_spinlock_init(&q->io_list_lock);
INIT_LIST_HEAD(&q->io_list);
env_atomic_set(&q->ref_count, 1);
}
int ocf_start_queues(ocf_cache_t cache)
int ocf_queue_create(ocf_cache_t cache, ocf_queue_t *queue,
const struct ocf_queue_ops *ops)
{
int id, result = 0;
struct ocf_queue *q;
ocf_queue_t tmp_queue;
for (id = 0; id < cache->io_queues_no; id++) {
q = &cache->io_queues[id];
q->cache = cache;
q->id = id;
ocf_init_queue(q);
result = ctx_queue_init(cache->owner, q);
if (result)
break;
}
if (result) {
while (id) {
ctx_queue_stop(cache->owner,
&cache->io_queues[--id]);
}
}
OCF_CHECK_NULL(cache);
return result;
tmp_queue = env_zalloc(sizeof(*tmp_queue), ENV_MEM_NORMAL);
if (!tmp_queue)
return -ENOMEM;
tmp_queue->cache = cache;
ocf_init_queue(tmp_queue);
tmp_queue->ops = ops;
list_add(&tmp_queue->list, &cache->io_queues);
*queue = tmp_queue;
return 0;
}
void ocf_stop_queues(ocf_cache_t cache)
void ocf_queue_get(ocf_queue_t queue)
{
int i;
struct ocf_queue *curr;
OCF_CHECK_NULL(queue);
ocf_cache_wait_for_io_finish(cache);
env_atomic_inc(&queue->ref_count);
}
/* Stop IO threads. */
for (i = 0 ; i < cache->io_queues_no; i++) {
curr = &cache->io_queues[i];
ctx_queue_stop(cache->owner, curr);
void ocf_queue_put(ocf_queue_t queue)
{
OCF_CHECK_NULL(queue);
if (env_atomic_dec_return(&queue->ref_count) == 0) {
list_del(&queue->list);
queue->ops->stop(queue);
env_free(queue);
}
}
@ -92,7 +77,7 @@ void ocf_io_handle(struct ocf_io *io, void *opaque)
void ocf_queue_run_single(ocf_queue_t q)
{
struct ocf_request *io_req = NULL;
struct ocf_cache *cache;
ocf_cache_t cache;
OCF_CHECK_NULL(q);
@ -145,20 +130,3 @@ ocf_cache_t ocf_queue_get_cache(ocf_queue_t q)
OCF_CHECK_NULL(q);
return q->cache;
}
uint32_t ocf_queue_get_id(ocf_queue_t q)
{
OCF_CHECK_NULL(q);
return q->id;
}
int ocf_cache_get_queue(ocf_cache_t cache, unsigned id, ocf_queue_t *q)
{
OCF_CHECK_NULL(cache);
if (!q || id >= cache->io_queues_no)
return -OCF_ERR_INVAL;
*q = &cache->io_queues[id];
return 0;
}

View File

@ -9,11 +9,12 @@
#include "ocf_env.h"
struct ocf_queue {
struct ocf_cache *cache;
uint32_t id;
ocf_cache_t cache;
env_atomic io_no;
env_atomic ref_count;
struct list_head io_list;
env_spinlock io_list_lock;
@ -23,15 +24,19 @@ struct ocf_queue {
/* Tracing stop request */
env_atomic trace_stop;
struct list_head list;
const struct ocf_queue_ops *ops;
void *priv;
};
int ocf_alloc_queues(struct ocf_cache *cache);
int ocf_start_queues(struct ocf_cache *cache);
void ocf_stop_queues(struct ocf_cache *cache);
void ocf_free_queues(struct ocf_cache *cache);
static inline void ocf_queue_kick(ocf_queue_t queue, bool allow_sync)
{
if (allow_sync && queue->ops->kick_sync)
queue->ops->kick_sync(queue);
else
queue->ops->kick(queue);
}
#endif

View File

@ -120,7 +120,7 @@ struct ocf_request {
env_atomic master_remaining;
/*!< Atomic counter for core device */
struct ocf_cache *cache;
ocf_cache_t cache;
/*!< Handle to cache instance */
const struct ocf_io_if *io_if;
@ -165,8 +165,8 @@ struct ocf_request {
uint32_t alloc_core_line_count;
/*! Core line count for which request was initially allocated */
uint32_t io_queue;
/*!< I/O queue id for which request should be submitted */
ocf_queue_t io_queue;
/*!< I/O queue handle for which request should be submitted */
int error;
/*!< This filed indicates an error for OCF request */

View File

@ -13,7 +13,7 @@
struct core_trace_visitor_ctx {
ocf_cache_t cache;
uint32_t io_queue;
ocf_queue_t io_queue;
};
static int _ocf_core_desc(ocf_core_t core, void *ctx)
@ -31,13 +31,13 @@ static int _ocf_core_desc(ocf_core_t core, void *ctx)
core_desc.core_size = ocf_volume_get_length(
ocf_core_get_volume(core));
ocf_trace_push(cache, visitor_ctx->io_queue,
ocf_trace_push(visitor_ctx->io_queue,
&core_desc, sizeof(core_desc));
return 0;
}
static int _ocf_trace_cache_info(ocf_cache_t cache, uint32_t io_queue)
static int _ocf_trace_cache_info(ocf_cache_t cache, ocf_queue_t io_queue)
{
struct ocf_event_cache_desc cache_desc;
int retval;
@ -62,9 +62,8 @@ static int _ocf_trace_cache_info(ocf_cache_t cache, uint32_t io_queue)
cache_desc.cores_no = ocf_cache_get_core_count(cache);
cache_desc.version = OCF_EVENT_VERSION;
cache_desc.io_queues_no = cache->io_queues_no;
ocf_trace_push(cache, io_queue, &cache_desc, sizeof(cache_desc));
ocf_trace_push(io_queue, &cache_desc, sizeof(cache_desc));
visitor_ctx.cache = cache;
visitor_ctx.io_queue = io_queue;
@ -77,8 +76,8 @@ static int _ocf_trace_cache_info(ocf_cache_t cache, uint32_t io_queue)
int ocf_mngt_start_trace(ocf_cache_t cache, void *trace_ctx,
ocf_trace_callback_t trace_callback)
{
int queue, result;
uint32_t i;
ocf_queue_t queue;
int result;
OCF_CHECK_NULL(cache);
@ -96,11 +95,12 @@ int ocf_mngt_start_trace(ocf_cache_t cache, void *trace_ctx,
cache->trace.trace_ctx = trace_ctx;
// Reset trace stop flag
for (queue = 0; queue < cache->io_queues_no; queue++)
env_atomic_set(&cache->io_queues[queue].trace_stop, 0);
list_for_each_entry(queue, &cache->io_queues, list) {
env_atomic_set(&queue->trace_stop, 0);
}
for (i = 0; i < cache->io_queues_no; i++) {
result = _ocf_trace_cache_info(cache, i);
list_for_each_entry(queue, &cache->io_queues, list) {
result = _ocf_trace_cache_info(cache, queue);
if (result) {
cache->trace.trace_callback = NULL;
return result;
@ -115,7 +115,7 @@ int ocf_mngt_start_trace(ocf_cache_t cache, void *trace_ctx,
int ocf_mngt_stop_trace(ocf_cache_t cache)
{
int queue;
ocf_queue_t queue;
OCF_CHECK_NULL(cache);
@ -127,9 +127,8 @@ int ocf_mngt_stop_trace(ocf_cache_t cache)
}
// Set trace stop flag
for (queue = 0; queue < cache->io_queues_no; queue++) {
env_atomic_set(&cache->io_queues[queue].trace_stop,
OCF_TRACING_STOP);
list_for_each_entry(queue, &cache->io_queues, list) {
env_atomic_set(&queue->trace_stop, OCF_TRACING_STOP);
}
cache->trace.trace_callback = NULL;

View File

@ -16,10 +16,10 @@
static inline bool ocf_is_trace_ongoing(ocf_cache_t cache)
{
int i;
ocf_queue_t q;
for (i = 0; i < cache->io_queues_no; i++) {
if (env_atomic64_read(&cache->io_queues[i].trace_ref_cntr))
list_for_each_entry(q, &cache->io_queues, list) {
if (env_atomic64_read(&q->trace_ref_cntr))
return true;
}
@ -66,20 +66,24 @@ static inline void ocf_trace_prep_io_event(struct ocf_event_io *ev,
ev->io_class = rq->io->io_class;
}
static inline void ocf_trace_push(ocf_cache_t cache, uint32_t io_queue,
void *trace, uint32_t size)
static inline void ocf_trace_push(ocf_queue_t queue, void *trace, uint32_t size)
{
ocf_cache_t cache;
ocf_trace_callback_t trace_callback;
void *trace_ctx;
OCF_CHECK_NULL(queue);
cache = ocf_queue_get_cache(queue);
if (cache->trace.trace_callback == NULL)
return;
env_atomic64_inc(&cache->io_queues[io_queue].trace_ref_cntr);
env_atomic64_inc(&queue->trace_ref_cntr);
if (env_atomic_read(&cache->io_queues[io_queue].trace_stop)) {
if (env_atomic_read(&queue->trace_stop)) {
// Tracing stop was requested
env_atomic64_dec(&cache->io_queues[io_queue].trace_ref_cntr);
env_atomic64_dec(&queue->trace_ref_cntr);
return;
}
@ -93,10 +97,10 @@ static inline void ocf_trace_push(ocf_cache_t cache, uint32_t io_queue,
trace_ctx = cache->trace.trace_ctx;
if (trace_callback && trace_ctx) {
trace_callback(cache, trace_ctx, io_queue, trace, size);
trace_callback(cache, trace_ctx, queue, trace, size);
}
env_atomic64_dec(&cache->io_queues[io_queue].trace_ref_cntr);
env_atomic64_dec(&queue->trace_ref_cntr);
}
static inline void ocf_trace_io(struct ocf_core_io *io, ocf_event_operation_t dir, ocf_cache_t cache)
@ -110,7 +114,7 @@ static inline void ocf_trace_io(struct ocf_core_io *io, ocf_event_operation_t di
rq = io->req;
ocf_trace_prep_io_event(&ev, io, dir);
ocf_trace_push(cache, rq->io_queue, &ev, sizeof(ev));
ocf_trace_push(rq->io_queue, &ev, sizeof(ev));
}
static inline void ocf_trace_io_cmpl(struct ocf_core_io *io, ocf_cache_t cache)
@ -129,7 +133,7 @@ static inline void ocf_trace_io_cmpl(struct ocf_core_io *io, ocf_cache_t cache)
ev.rsid = io->sid;
ev.is_hit = ocf_engine_is_hit(rq);
ocf_trace_push(cache, rq->io_queue, &ev, sizeof(ev));
ocf_trace_push(rq->io_queue, &ev, sizeof(ev));
}
#endif /* __OCF_TRACE_PRIV_H__ */

View File

@ -37,8 +37,8 @@
static struct ocf_request *_ocf_cleaner_alloc_req(struct ocf_cache *cache,
uint32_t count, const struct ocf_cleaner_attribs *attribs)
{
struct ocf_request *req = ocf_req_new_extended(cache, 0, 0,
count * ocf_line_size(cache), OCF_READ);
struct ocf_request *req = ocf_req_new_extended(attribs->io_queue, NULL,
0, count * ocf_line_size(cache), OCF_READ);
int ret;
if (!req)
@ -62,8 +62,6 @@ static struct ocf_request *_ocf_cleaner_alloc_req(struct ocf_cache *cache,
return NULL;
}
req->io_queue = attribs->io_queue;
return req;
}

View File

@ -43,7 +43,7 @@ struct ocf_cleaner_attribs {
* to iterate over items
*/
uint32_t io_queue;
ocf_queue_t io_queue;
};
/**

View File

@ -8,6 +8,7 @@
#include "utils_cache_line.h"
#include "../ocf_request.h"
#include "../ocf_cache_priv.h"
#include "../ocf_queue_priv.h"
#define OCF_UTILS_RQ_DEBUG 0
@ -165,10 +166,11 @@ static void start_cache_req(struct ocf_request *req)
}
}
struct ocf_request *ocf_req_new(struct ocf_cache *cache,
ocf_core_id_t core_id, uint64_t addr, uint32_t bytes, int rw)
struct ocf_request *ocf_req_new(ocf_queue_t queue, ocf_core_t core,
uint64_t addr, uint32_t bytes, int rw)
{
uint64_t core_line_first, core_line_last, core_line_count;
ocf_cache_t cache = queue->cache;
struct ocf_request *req;
env_allocator *allocator;
@ -197,14 +199,17 @@ struct ocf_request *ocf_req_new(struct ocf_cache *cache,
OCF_DEBUG_TRACE(cache);
ocf_queue_get(queue);
req->io_queue = queue;
/* TODO: Store core pointer instead of id */
req->core_id = core ? ocf_core_get_id(core) : 0;
req->cache = cache;
env_atomic_inc(&cache->pending_requests);
start_cache_req(req);
req->io_queue = 0;
env_atomic_set(&req->ref_count, 1);
req->core_id = core_id;
req->byte_position = addr;
req->byte_length = bytes;
@ -232,12 +237,12 @@ int ocf_req_alloc_map(struct ocf_request *req)
return 0;
}
struct ocf_request *ocf_req_new_extended(struct ocf_cache *cache,
ocf_core_id_t core_id, uint64_t addr, uint32_t bytes, int rw)
struct ocf_request *ocf_req_new_extended(ocf_queue_t queue, ocf_core_t core,
uint64_t addr, uint32_t bytes, int rw)
{
struct ocf_request *req;
req = ocf_req_new(cache, core_id, addr, bytes, rw);
req = ocf_req_new(queue, core, addr, bytes, rw);
if (likely(req) && ocf_req_alloc_map(req)) {
ocf_req_put(req);
@ -247,14 +252,13 @@ struct ocf_request *ocf_req_new_extended(struct ocf_cache *cache,
return req;
}
struct ocf_request *ocf_req_new_discard(struct ocf_cache *cache,
ocf_core_id_t core_id, uint64_t addr, uint32_t bytes, int rw)
struct ocf_request *ocf_req_new_discard(ocf_queue_t queue, ocf_core_t core,
uint64_t addr, uint32_t bytes, int rw)
{
struct ocf_request *req;
req = ocf_req_new_extended(cache, core_id, addr,
OCF_MIN(bytes, MAX_TRIM_RQ_SIZE),rw);
req = ocf_req_new_extended(queue, core, addr,
OCF_MIN(bytes, MAX_TRIM_RQ_SIZE), rw);
if (!req)
return NULL;
@ -279,6 +283,8 @@ void ocf_req_put(struct ocf_request *req)
if (env_atomic_dec_return(&req->ref_count))
return;
ocf_queue_put(req->io_queue);
OCF_DEBUG_TRACE(req->cache);
if (!req->d2c && !env_atomic_dec_return(

View File

@ -33,16 +33,16 @@ void ocf_req_allocator_deinit(struct ocf_ctx *ocf_ctx);
/**
* @brief Allocate new OCF request
*
* @param cache - OCF cache instance
* @param core_id - Core id
* @param queue - I/O queue handle
* @param core - OCF core instance
* @param addr - LBA of request
* @param bytes - number of bytes of request
* @param rw - Read or Write
*
* @return new OCF request
*/
struct ocf_request *ocf_req_new(struct ocf_cache *cache,
ocf_core_id_t core_id, uint64_t addr, uint32_t bytes, int rw);
struct ocf_request *ocf_req_new(ocf_queue_t queue, ocf_core_t core,
uint64_t addr, uint32_t bytes, int rw);
/**
* @brief Allocate OCF request map
@ -57,8 +57,8 @@ int ocf_req_alloc_map(struct ocf_request *req);
/**
* @brief Allocate new OCF request with NOIO map allocation for huge request
*
* @param cache - OCF cache instance
* @param core_id - Core id
* @param queue - I/O queue handle
* @param core - OCF core instance
* @param addr - LBA of request
* @param bytes - number of bytes of request
* @param rw - Read or Write
@ -66,22 +66,22 @@ int ocf_req_alloc_map(struct ocf_request *req);
* @return new OCF request
*/
struct ocf_request *ocf_req_new_extended(struct ocf_cache *cache,
ocf_core_id_t core_id, uint64_t addr, uint32_t bytes, int rw);
struct ocf_request *ocf_req_new_extended(ocf_queue_t queue, ocf_core_t core,
uint64_t addr, uint32_t bytes, int rw);
/**
* @brief Allocate new OCF request for DISCARD operation
*
* @param cache - OCF cache instance
* @param core_id - Core id
* @param queue - I/O queue handle
* @param core - OCF core instance
* @param addr - LBA of request
* @param bytes - number of bytes of request
* @param rw - Read or Write
*
* @return new OCF request
*/
struct ocf_request *ocf_req_new_discard(struct ocf_cache *cache,
ocf_core_id_t core_id, uint64_t addr, uint32_t bytes, int rw);
struct ocf_request *ocf_req_new_discard(ocf_queue_t queue, ocf_core_t core,
uint64_t addr, uint32_t bytes, int rw);
/**
* @brief Get number of allocated requests

View File

@ -160,6 +160,11 @@ void __wrap_cleaning_policy_alru_get_cleaning_parameters(ocf_cache_t cache,
}
void __wrap_ocf_queue_get(ocf_queue_t queue)
{
}
int __wrap_cleaning_alru_perform_cleaning(struct ocf_cache *cache, ocf_cleaner_end_t cmpl)
{
function_called();
@ -252,7 +257,7 @@ static void ocf_cleaner_run_test01(void **state)
ocf_cleaner_set_cmpl(&cache.cleaner, cleaner_complete);
ocf_cleaner_run(&cache.cleaner);
ocf_cleaner_run(&cache.cleaner, 0xdeadbeef);
/* Release allocated memory if allocated with test_* functions */