Merge pull request #258 from arutk/metadata_lock_per_bucket
Improve metadata lock granularity
This commit is contained in:
commit
ed1ae71992
@ -392,7 +392,7 @@ static ocf_cache_line_t _acp_trylock_dirty(struct ocf_cache *cache,
|
||||
struct ocf_map_info info;
|
||||
bool locked = false;
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_metadata_hash_lock_rd(&cache->metadata.lock, core_id, core_line);
|
||||
|
||||
ocf_engine_lookup_map_entry(cache, &info, core_id,
|
||||
core_line);
|
||||
@ -403,7 +403,7 @@ static ocf_cache_line_t _acp_trylock_dirty(struct ocf_cache *cache,
|
||||
locked = true;
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_metadata_hash_unlock_rd(&cache->metadata.lock, core_id, core_line);
|
||||
|
||||
return locked ? info.coll_idx : cache->device->collision_table_entries;
|
||||
}
|
||||
|
@ -779,7 +779,7 @@ static void alru_clean(struct alru_flush_ctx *fctx)
|
||||
return;
|
||||
}
|
||||
|
||||
if (OCF_METADATA_LOCK_WR_TRY()) {
|
||||
if (ocf_metadata_try_start_exclusive_access(&cache->metadata.lock)) {
|
||||
alru_clean_complete(fctx, 0);
|
||||
return;
|
||||
}
|
||||
@ -797,7 +797,7 @@ static void alru_clean(struct alru_flush_ctx *fctx)
|
||||
fctx->flush_perfomed = true;
|
||||
ocf_cleaner_do_flush_data_async(cache, fctx->flush_data, to_clean,
|
||||
&fctx->attribs);
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -806,7 +806,7 @@ static void alru_clean(struct alru_flush_ctx *fctx)
|
||||
env_ticks_to_secs(env_get_tick_count());
|
||||
|
||||
end:
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
alru_clean_complete(fctx, 0);
|
||||
}
|
||||
|
||||
|
@ -35,14 +35,11 @@
|
||||
|
||||
#define _WAITERS_LIST_ITEM(cache_line) ((cache_line) % _WAITERS_LIST_ENTRIES)
|
||||
|
||||
typedef void (*__on_lock)(void *ctx, uint32_t ctx_id, ocf_cache_line_t line,
|
||||
int rw);
|
||||
|
||||
struct __waiter {
|
||||
ocf_cache_line_t line;
|
||||
void *ctx;
|
||||
uint32_t ctx_id;
|
||||
__on_lock on_lock;
|
||||
ocf_req_async_lock_cb cb;
|
||||
struct list_head item;
|
||||
int rw;
|
||||
};
|
||||
@ -53,7 +50,6 @@ struct __waiters_list {
|
||||
};
|
||||
|
||||
struct ocf_cache_line_concurrency {
|
||||
env_rwlock lock;
|
||||
env_atomic *access;
|
||||
env_atomic waiting;
|
||||
size_t access_limit;
|
||||
@ -115,8 +111,6 @@ int ocf_cache_line_concurrency_init(struct ocf_cache *cache)
|
||||
env_spinlock_init(&c->waiters_lsts[i].lock);
|
||||
}
|
||||
|
||||
env_rwlock_init(&c->lock);
|
||||
|
||||
return 0;
|
||||
|
||||
ocf_cache_line_concurrency_init:
|
||||
@ -144,8 +138,6 @@ void ocf_cache_line_concurrency_deinit(struct ocf_cache *cache)
|
||||
|
||||
concurrency = cache->device->concurrency.cache_line;
|
||||
|
||||
env_rwlock_destroy(&concurrency->lock);
|
||||
|
||||
for (i = 0; i < _WAITERS_LIST_ENTRIES; i++)
|
||||
env_spinlock_destroy(&concurrency->waiters_lsts[i].lock);
|
||||
|
||||
@ -353,11 +345,37 @@ static inline bool __try_lock_rd2rd(struct ocf_cache_line_concurrency *c,
|
||||
return true;
|
||||
}
|
||||
|
||||
/*
|
||||
*
|
||||
*/
|
||||
static void _req_on_lock(void *ctx, ocf_req_async_lock_cb cb,
|
||||
uint32_t ctx_id, ocf_cache_line_t line, int rw)
|
||||
{
|
||||
struct ocf_request *req = ctx;
|
||||
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency.
|
||||
cache_line;
|
||||
|
||||
if (rw == OCF_READ)
|
||||
req->map[ctx_id].rd_locked = true;
|
||||
else if (rw == OCF_WRITE)
|
||||
req->map[ctx_id].wr_locked = true;
|
||||
else
|
||||
ENV_BUG();
|
||||
|
||||
if (env_atomic_dec_return(&req->lock_remaining) == 0) {
|
||||
/* All cache line locked, resume request */
|
||||
OCF_DEBUG_RQ(req, "Resume");
|
||||
ENV_BUG_ON(!cb);
|
||||
env_atomic_dec(&c->waiting);
|
||||
cb(req);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
*
|
||||
*/
|
||||
static inline bool __lock_cache_line_wr(struct ocf_cache_line_concurrency *c,
|
||||
const ocf_cache_line_t line, __on_lock on_lock,
|
||||
const ocf_cache_line_t line, ocf_req_async_lock_cb cb,
|
||||
void *ctx, uint32_t ctx_id)
|
||||
{
|
||||
struct __waiter *waiter;
|
||||
@ -367,8 +385,8 @@ static inline bool __lock_cache_line_wr(struct ocf_cache_line_concurrency *c,
|
||||
|
||||
if (__try_lock_wr(c, line)) {
|
||||
/* No activity before look get */
|
||||
if (on_lock)
|
||||
on_lock(ctx, ctx_id, line, OCF_WRITE);
|
||||
if (cb)
|
||||
_req_on_lock(ctx, cb, ctx_id, line, OCF_WRITE);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -382,7 +400,7 @@ static inline bool __lock_cache_line_wr(struct ocf_cache_line_concurrency *c,
|
||||
locked = true;
|
||||
} else {
|
||||
waiter = NULL;
|
||||
if (on_lock != NULL) {
|
||||
if (cb != NULL) {
|
||||
/* Need to create waiters and add it into list */
|
||||
waiter = env_allocator_new(c->allocator);
|
||||
}
|
||||
@ -391,7 +409,7 @@ static inline bool __lock_cache_line_wr(struct ocf_cache_line_concurrency *c,
|
||||
waiter->line = line;
|
||||
waiter->ctx = ctx;
|
||||
waiter->ctx_id = ctx_id;
|
||||
waiter->on_lock = on_lock;
|
||||
waiter->cb = cb;
|
||||
waiter->rw = OCF_WRITE;
|
||||
INIT_LIST_HEAD(&waiter->item);
|
||||
|
||||
@ -403,8 +421,8 @@ static inline bool __lock_cache_line_wr(struct ocf_cache_line_concurrency *c,
|
||||
|
||||
__unlock_waiters_list(c, line, flags);
|
||||
|
||||
if (locked && on_lock)
|
||||
on_lock(ctx, ctx_id, line, OCF_WRITE);
|
||||
if (locked && cb)
|
||||
_req_on_lock(ctx, cb, ctx_id, line, OCF_WRITE);
|
||||
|
||||
return locked || waiting;
|
||||
}
|
||||
@ -414,7 +432,7 @@ static inline bool __lock_cache_line_wr(struct ocf_cache_line_concurrency *c,
|
||||
* In case cache line is locked, attempt to add caller on wait list.
|
||||
*/
|
||||
static inline bool __lock_cache_line_rd(struct ocf_cache_line_concurrency *c,
|
||||
const ocf_cache_line_t line, __on_lock on_lock,
|
||||
const ocf_cache_line_t line, ocf_req_async_lock_cb cb,
|
||||
void *ctx, uint32_t ctx_id)
|
||||
{
|
||||
struct __waiter *waiter;
|
||||
@ -424,8 +442,8 @@ static inline bool __lock_cache_line_rd(struct ocf_cache_line_concurrency *c,
|
||||
|
||||
if (__try_lock_rd_idle(c, line)) {
|
||||
/* No activity before look get, it is first reader */
|
||||
if (on_lock)
|
||||
on_lock(ctx, ctx_id, line, OCF_READ);
|
||||
if (cb)
|
||||
_req_on_lock(ctx, cb, ctx_id, line, OCF_READ);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -444,7 +462,7 @@ static inline bool __lock_cache_line_rd(struct ocf_cache_line_concurrency *c,
|
||||
|
||||
if (!locked) {
|
||||
waiter = NULL;
|
||||
if (on_lock) {
|
||||
if (cb) {
|
||||
/* Need to create waiters and add it into list */
|
||||
waiter = env_allocator_new(c->allocator);
|
||||
}
|
||||
@ -453,7 +471,7 @@ static inline bool __lock_cache_line_rd(struct ocf_cache_line_concurrency *c,
|
||||
waiter->line = line;
|
||||
waiter->ctx = ctx;
|
||||
waiter->ctx_id = ctx_id;
|
||||
waiter->on_lock = on_lock;
|
||||
waiter->cb = cb;
|
||||
waiter->rw = OCF_READ;
|
||||
INIT_LIST_HEAD(&waiter->item);
|
||||
|
||||
@ -465,8 +483,8 @@ static inline bool __lock_cache_line_rd(struct ocf_cache_line_concurrency *c,
|
||||
|
||||
__unlock_waiters_list(c, line, flags);
|
||||
|
||||
if (locked && on_lock)
|
||||
on_lock(ctx, ctx_id, line, OCF_READ);
|
||||
if (locked && cb)
|
||||
_req_on_lock(ctx, cb, ctx_id, line, OCF_READ);
|
||||
|
||||
return locked || waiting;
|
||||
}
|
||||
@ -520,8 +538,8 @@ static inline void __unlock_cache_line_rd_common(struct ocf_cache_line_concurren
|
||||
exchanged = false;
|
||||
list_del(iter);
|
||||
|
||||
waiter->on_lock(waiter->ctx, waiter->ctx_id, line,
|
||||
waiter->rw);
|
||||
_req_on_lock(waiter->ctx, waiter->cb, waiter->ctx_id,
|
||||
line, waiter->rw);
|
||||
|
||||
env_allocator_del(c->allocator, waiter);
|
||||
} else {
|
||||
@ -601,7 +619,7 @@ static inline void __unlock_cache_line_wr_common(struct ocf_cache_line_concurren
|
||||
exchanged = false;
|
||||
list_del(iter);
|
||||
|
||||
waiter->on_lock(waiter->ctx, waiter->ctx_id, line,
|
||||
_req_on_lock(waiter->ctx, waiter->cb, waiter->ctx_id, line,
|
||||
waiter->rw);
|
||||
|
||||
env_allocator_del(c->allocator, waiter);
|
||||
@ -668,35 +686,10 @@ static inline void __remove_line_from_waiters_list(struct ocf_cache_line_concurr
|
||||
__unlock_waiters_list(c, line, flags);
|
||||
}
|
||||
|
||||
/*
|
||||
*
|
||||
*/
|
||||
static void _req_on_lock(void *ctx, uint32_t ctx_id,
|
||||
ocf_cache_line_t line, int rw)
|
||||
{
|
||||
struct ocf_request *req = ctx;
|
||||
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency.cache_line;
|
||||
|
||||
if (rw == OCF_READ)
|
||||
req->map[ctx_id].rd_locked = true;
|
||||
else if (rw == OCF_WRITE)
|
||||
req->map[ctx_id].wr_locked = true;
|
||||
else
|
||||
ENV_BUG();
|
||||
|
||||
if (env_atomic_dec_return(&req->lock_remaining) == 0) {
|
||||
/* All cache line locked, resume request */
|
||||
OCF_DEBUG_RQ(req, "Resume");
|
||||
ENV_BUG_ON(!req->io_if->resume);
|
||||
env_atomic_dec(&c->waiting);
|
||||
req->io_if->resume(req);
|
||||
}
|
||||
}
|
||||
|
||||
/* Try to read-lock request without adding waiters. Function should be called
|
||||
* under read lock, multiple threads may attempt to acquire the lock
|
||||
* concurrently. */
|
||||
static int _ocf_req_trylock_rd(struct ocf_request *req)
|
||||
int ocf_req_trylock_rd(struct ocf_request *req)
|
||||
{
|
||||
int32_t i;
|
||||
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency.
|
||||
@ -747,16 +740,15 @@ static int _ocf_req_trylock_rd(struct ocf_request *req)
|
||||
}
|
||||
|
||||
/*
|
||||
* Read-lock request cache lines. Must be called under cacheline concurrency
|
||||
* write lock.
|
||||
* Asynchronously read-lock request cache lines. Must be called under cacheline
|
||||
* concurrency write lock.
|
||||
*/
|
||||
static int _ocf_req_lock_rd(struct ocf_request *req)
|
||||
int ocf_req_async_lock_rd(struct ocf_request *req, ocf_req_async_lock_cb cb)
|
||||
{
|
||||
int32_t i;
|
||||
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency.
|
||||
cache_line;
|
||||
ocf_cache_line_t line;
|
||||
__on_lock on_lock = _req_on_lock;
|
||||
int ret = OCF_LOCK_NOT_ACQUIRED;
|
||||
|
||||
ENV_BUG_ON(env_atomic_read(&req->lock_remaining));
|
||||
@ -778,7 +770,7 @@ static int _ocf_req_lock_rd(struct ocf_request *req)
|
||||
ENV_BUG_ON(req->map[i].rd_locked);
|
||||
ENV_BUG_ON(req->map[i].wr_locked);
|
||||
|
||||
if (!__lock_cache_line_rd(c, line, on_lock, req, i)) {
|
||||
if (!__lock_cache_line_rd(c, line, cb, req, i)) {
|
||||
/* lock not acquired and not added to wait list */
|
||||
ret = -OCF_ERR_NO_MEM;
|
||||
goto err;
|
||||
@ -804,29 +796,10 @@ err:
|
||||
|
||||
}
|
||||
|
||||
int ocf_req_trylock_rd(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache_line_concurrency *c =
|
||||
req->cache->device->concurrency.cache_line;
|
||||
int lock;
|
||||
|
||||
env_rwlock_read_lock(&c->lock);
|
||||
lock = _ocf_req_trylock_rd(req);
|
||||
env_rwlock_read_unlock(&c->lock);
|
||||
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
env_rwlock_write_lock(&c->lock);
|
||||
lock = _ocf_req_lock_rd(req);
|
||||
env_rwlock_write_unlock(&c->lock);
|
||||
}
|
||||
|
||||
return lock;
|
||||
}
|
||||
|
||||
/* Try to write-lock request without adding waiters. Function should be called
|
||||
* under read lock, multiple threads may attempt to acquire the lock
|
||||
* concurrently. */
|
||||
static int _ocf_req_trylock_wr(struct ocf_request *req)
|
||||
int ocf_req_trylock_wr(struct ocf_request *req)
|
||||
{
|
||||
int32_t i;
|
||||
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency.
|
||||
@ -875,20 +848,19 @@ static int _ocf_req_trylock_wr(struct ocf_request *req)
|
||||
}
|
||||
|
||||
/*
|
||||
* Write-lock request cache lines. Must be called under cacheline concurrency
|
||||
* write lock.
|
||||
* Asynchronously write-lock request cache lines. Must be called under cacheline
|
||||
* concurrency write lock.
|
||||
*/
|
||||
static int _ocf_req_lock_wr(struct ocf_request *req)
|
||||
int ocf_req_async_lock_wr(struct ocf_request *req, ocf_req_async_lock_cb cb)
|
||||
{
|
||||
int32_t i;
|
||||
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency.
|
||||
cache_line;
|
||||
ocf_cache_line_t line;
|
||||
__on_lock on_lock = _req_on_lock;
|
||||
int ret = OCF_LOCK_NOT_ACQUIRED;
|
||||
|
||||
ENV_BUG_ON(env_atomic_read(&req->lock_remaining));
|
||||
ENV_BUG_ON(!req->io_if->resume);
|
||||
ENV_BUG_ON(!cb);
|
||||
|
||||
env_atomic_inc(&c->waiting);
|
||||
env_atomic_set(&req->lock_remaining, req->core_line_count);
|
||||
@ -907,7 +879,7 @@ static int _ocf_req_lock_wr(struct ocf_request *req)
|
||||
ENV_BUG_ON(req->map[i].rd_locked);
|
||||
ENV_BUG_ON(req->map[i].wr_locked);
|
||||
|
||||
if (!__lock_cache_line_wr(c, line, on_lock, req, i)) {
|
||||
if (!__lock_cache_line_wr(c, line, cb, req, i)) {
|
||||
/* lock not acquired and not added to wait list */
|
||||
ret = -OCF_ERR_NO_MEM;
|
||||
goto err;
|
||||
@ -932,26 +904,6 @@ err:
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ocf_req_trylock_wr(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache_line_concurrency *c =
|
||||
req->cache->device->concurrency.cache_line;
|
||||
int lock;
|
||||
|
||||
env_rwlock_read_lock(&c->lock);
|
||||
lock = _ocf_req_trylock_wr(req);
|
||||
env_rwlock_read_unlock(&c->lock);
|
||||
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
env_rwlock_write_lock(&c->lock);
|
||||
lock = _ocf_req_lock_wr(req);
|
||||
env_rwlock_write_unlock(&c->lock);
|
||||
}
|
||||
|
||||
return lock;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
*
|
||||
*/
|
||||
|
@ -50,31 +50,64 @@ uint32_t ocf_cache_line_concurrency_suspended_no(struct ocf_cache *cache);
|
||||
*/
|
||||
size_t ocf_cache_line_concurrency_size_of(struct ocf_cache *cache);
|
||||
|
||||
/* async request cacheline lock acquisition callback */
|
||||
typedef void (*ocf_req_async_lock_cb)(struct ocf_request *req);
|
||||
|
||||
/**
|
||||
* @brief Lock OCF request for WRITE access (Lock all cache lines in map info)
|
||||
* @brief Lock OCF request for write access asynchronously. Attempts to lock all
|
||||
* cache lines in map info.
|
||||
*
|
||||
* @note io_if->resume callback has to be set
|
||||
* @param req - OCF request
|
||||
* @param cb - async lock acquisition callback
|
||||
*
|
||||
* @returns lock acquisition status or negative error code in case of internal
|
||||
* error
|
||||
* @retval OCF_LOCK_ACQUIRED - OCF request has been locked and can be processed
|
||||
* @retval OCF_LOCK_NOT_ACQUIRED - OCF request lock not acquired, request was
|
||||
* added into waiting list. When lock is acquired, @cb will be
|
||||
* called.
|
||||
*/
|
||||
int ocf_req_async_lock_wr(struct ocf_request *req, ocf_req_async_lock_cb cb);
|
||||
|
||||
/**
|
||||
* @brief Try to lock OCF request for write access. Serves the same purpose as
|
||||
* ocf_req_async_lock_wr, except that this function fails if lock is already
|
||||
* held by someone else.
|
||||
*
|
||||
* @param req - OCF request
|
||||
*
|
||||
* @returns lock acquisition status
|
||||
* @retval OCF_LOCK_ACQUIRED - OCF request has been locked and can be processed
|
||||
*
|
||||
* @retval OCF_LOCK_NOT_ACQUIRED - OCF request lock not acquired, request was
|
||||
* added into waiting list. When lock will be acquired io_if->resume be called
|
||||
* @retval OCF_LOCK_NOT_ACQUIRED - OCF request lock not acquired
|
||||
*/
|
||||
int ocf_req_trylock_wr(struct ocf_request *req);
|
||||
|
||||
/**
|
||||
* @brief Lock OCF request for READ access (Lock all cache lines in map info)
|
||||
* @brief Lock OCF request for read access asynchronously. Attempts to lock all
|
||||
* cache lines in map info.
|
||||
*
|
||||
* @note io_if->resume callback has to be set
|
||||
* @param req - OCF request
|
||||
* @param cb - async lock acquisition callback
|
||||
*
|
||||
* @returns lock acquisition status or negative error code in case of internal
|
||||
* error
|
||||
* @retval OCF_LOCK_ACQUIRED - OCF request has been locked and can be processed
|
||||
* @retval OCF_LOCK_NOT_ACQUIRED - OCF request lock not acquired, request was
|
||||
* added into waiting list. When lock is acquired, @cb will be
|
||||
* called.
|
||||
*/
|
||||
int ocf_req_async_lock_rd(struct ocf_request *req, ocf_req_async_lock_cb cb);
|
||||
|
||||
/**
|
||||
* @brief Try to lock OCF request forread access. Serves the same purpose as
|
||||
* ocf_req_async_lock_rd, except that this function fails if lock is already
|
||||
* held by someone else.
|
||||
*
|
||||
* @param req - OCF request
|
||||
*
|
||||
* @returns lock acquisition status
|
||||
* @retval OCF_LOCK_ACQUIRED - OCF request has been locked and can be processed
|
||||
*
|
||||
* @retval OCF_LOCK_NOT_ACQUIRED - OCF request lock not acquired, request was
|
||||
* added into waiting list. When lock will be acquired io_if->resume be called
|
||||
* @retval OCF_LOCK_NOT_ACQUIRED - OCF request lock not acquired
|
||||
*/
|
||||
int ocf_req_trylock_rd(struct ocf_request *req);
|
||||
|
||||
|
@ -4,22 +4,265 @@
|
||||
*/
|
||||
|
||||
#include "ocf_metadata_concurrency.h"
|
||||
#include "../metadata/metadata_misc.h"
|
||||
|
||||
void ocf_metadata_concurrency_init(struct ocf_cache *cache)
|
||||
void ocf_metadata_concurrency_init(struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
env_spinlock_init(&cache->metadata.lock.eviction);
|
||||
env_rwlock_init(&cache->metadata.lock.status);
|
||||
env_rwsem_init(&cache->metadata.lock.collision);
|
||||
env_spinlock_init(&metadata_lock->eviction);
|
||||
env_rwlock_init(&metadata_lock->status);
|
||||
env_rwsem_init(&metadata_lock->global);
|
||||
}
|
||||
|
||||
void ocf_metadata_concurrency_deinit(struct ocf_cache *cache)
|
||||
void ocf_metadata_concurrency_deinit(struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
env_spinlock_destroy(&cache->metadata.lock.eviction);
|
||||
env_rwlock_destroy(&cache->metadata.lock.status);
|
||||
env_rwsem_destroy(&cache->metadata.lock.collision);
|
||||
env_spinlock_destroy(&metadata_lock->eviction);
|
||||
env_rwlock_destroy(&metadata_lock->status);
|
||||
env_rwsem_destroy(&metadata_lock->global);
|
||||
}
|
||||
|
||||
int ocf_metadata_concurrency_attached_init(struct ocf_cache *cache)
|
||||
int ocf_metadata_concurrency_attached_init(
|
||||
struct ocf_metadata_lock *metadata_lock, ocf_cache_t cache,
|
||||
uint64_t hash_table_entries)
|
||||
{
|
||||
uint64_t i;
|
||||
|
||||
metadata_lock->cache = cache;
|
||||
metadata_lock->num_hash_entries = hash_table_entries;
|
||||
metadata_lock->hash = env_vzalloc(sizeof(env_rwsem) *
|
||||
hash_table_entries);
|
||||
if (!metadata_lock->hash)
|
||||
return -OCF_ERR_NO_MEM;
|
||||
|
||||
for (i = 0; i < hash_table_entries; i++)
|
||||
env_rwsem_init(&metadata_lock->hash[i]);
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
void ocf_metadata_concurrency_attached_deinit(
|
||||
struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
uint64_t i;
|
||||
|
||||
for (i = 0; i < metadata_lock->num_hash_entries; i++)
|
||||
env_rwsem_destroy(&metadata_lock->hash[i]);
|
||||
|
||||
env_vfree(metadata_lock->hash);
|
||||
}
|
||||
|
||||
void ocf_metadata_start_exclusive_access(
|
||||
struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
env_rwsem_down_write(&metadata_lock->global);
|
||||
}
|
||||
|
||||
int ocf_metadata_try_start_exclusive_access(
|
||||
struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
return env_rwsem_down_write_trylock(&metadata_lock->global);
|
||||
}
|
||||
|
||||
void ocf_metadata_end_exclusive_access(
|
||||
struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
env_rwsem_up_write(&metadata_lock->global);
|
||||
}
|
||||
|
||||
void ocf_metadata_start_shared_access(
|
||||
struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
env_rwsem_down_read(&metadata_lock->global);
|
||||
}
|
||||
|
||||
int ocf_metadata_try_start_shared_access(
|
||||
struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
return env_rwsem_down_read_trylock(&metadata_lock->global);
|
||||
}
|
||||
|
||||
void ocf_metadata_end_shared_access(struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
env_rwsem_up_read(&metadata_lock->global);
|
||||
}
|
||||
|
||||
void ocf_metadata_hash_lock(struct ocf_metadata_lock *metadata_lock,
|
||||
ocf_cache_line_t hash, int rw)
|
||||
{
|
||||
ENV_BUG_ON(hash >= metadata_lock->num_hash_entries);
|
||||
|
||||
if (rw == OCF_METADATA_WR)
|
||||
env_rwsem_down_write(&metadata_lock->hash[hash]);
|
||||
else if (rw == OCF_METADATA_RD)
|
||||
env_rwsem_down_read(&metadata_lock->hash[hash]);
|
||||
else
|
||||
ENV_BUG();
|
||||
}
|
||||
|
||||
void ocf_metadata_hash_unlock(struct ocf_metadata_lock *metadata_lock,
|
||||
ocf_cache_line_t hash, int rw)
|
||||
{
|
||||
ENV_BUG_ON(hash >= metadata_lock->num_hash_entries);
|
||||
|
||||
if (rw == OCF_METADATA_WR)
|
||||
env_rwsem_up_write(&metadata_lock->hash[hash]);
|
||||
else if (rw == OCF_METADATA_RD)
|
||||
env_rwsem_up_read(&metadata_lock->hash[hash]);
|
||||
else
|
||||
ENV_BUG();
|
||||
}
|
||||
|
||||
int ocf_metadata_hash_try_lock(struct ocf_metadata_lock *metadata_lock,
|
||||
ocf_cache_line_t hash, int rw)
|
||||
{
|
||||
int result = -1;
|
||||
|
||||
ENV_BUG_ON(hash >= metadata_lock->num_hash_entries);
|
||||
|
||||
if (rw == OCF_METADATA_WR) {
|
||||
result = env_rwsem_down_write_trylock(
|
||||
&metadata_lock->hash[hash]);
|
||||
} else if (rw == OCF_METADATA_RD) {
|
||||
result = env_rwsem_down_read_trylock(
|
||||
&metadata_lock->hash[hash]);
|
||||
} else {
|
||||
ENV_BUG();
|
||||
}
|
||||
|
||||
if (!result)
|
||||
return -1;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
/* NOTE: attempt to acquire hash lock for multiple core lines may end up
|
||||
* in deadlock. In order to hash lock multiple core lines safely, use
|
||||
* ocf_req_hash_lock_* functions */
|
||||
void ocf_metadata_hash_lock_rd(struct ocf_metadata_lock *metadata_lock,
|
||||
uint32_t core_id, uint64_t core_line)
|
||||
{
|
||||
ocf_cache_line_t hash = ocf_metadata_hash_func(metadata_lock->cache,
|
||||
core_line, core_id);
|
||||
|
||||
ocf_metadata_start_shared_access(metadata_lock);
|
||||
ocf_metadata_hash_lock(metadata_lock, hash, OCF_METADATA_RD);
|
||||
}
|
||||
|
||||
void ocf_metadata_hash_unlock_rd(struct ocf_metadata_lock *metadata_lock,
|
||||
uint32_t core_id, uint64_t core_line)
|
||||
{
|
||||
ocf_cache_line_t hash = ocf_metadata_hash_func(metadata_lock->cache,
|
||||
core_line, core_id);
|
||||
|
||||
ocf_metadata_hash_unlock(metadata_lock, hash, OCF_METADATA_RD);
|
||||
ocf_metadata_end_shared_access(metadata_lock);
|
||||
}
|
||||
|
||||
void ocf_metadata_hash_lock_wr(struct ocf_metadata_lock *metadata_lock,
|
||||
uint32_t core_id, uint64_t core_line)
|
||||
{
|
||||
ocf_cache_line_t hash = ocf_metadata_hash_func(metadata_lock->cache,
|
||||
core_line, core_id);
|
||||
|
||||
ocf_metadata_start_shared_access(metadata_lock);
|
||||
ocf_metadata_hash_lock(metadata_lock, hash, OCF_METADATA_WR);
|
||||
}
|
||||
|
||||
void ocf_metadata_hash_unlock_wr(struct ocf_metadata_lock *metadata_lock,
|
||||
uint32_t core_id, uint64_t core_line)
|
||||
{
|
||||
ocf_cache_line_t hash = ocf_metadata_hash_func(metadata_lock->cache,
|
||||
core_line, core_id);
|
||||
|
||||
ocf_metadata_hash_unlock(metadata_lock, hash, OCF_METADATA_WR);
|
||||
ocf_metadata_end_shared_access(metadata_lock);
|
||||
}
|
||||
|
||||
#define _NUM_HASH_ENTRIES req->cache->metadata.lock.num_hash_entries
|
||||
|
||||
/*
|
||||
* Iterate over hash buckets for all core lines in the request in ascending hash
|
||||
* bucket value order. Each hash bucket is visited only once.
|
||||
*
|
||||
* @i is used as iteration counter, starting from 0
|
||||
* @hash stores hash values for each iteration
|
||||
* @start is internal helper variable. It set to the index of first occurence
|
||||
* of hash with minimal value within the request.
|
||||
*
|
||||
* Example hash iteration order for _NUM_HASH_ENTRIES == 5:
|
||||
* Request hashes Iteration order start
|
||||
* [2, 3, 4] [2, 3, 4] 0
|
||||
* [2, 3, 4, 0] [0, 2, 3, 4] 3
|
||||
* [2, 3, 4, 0, 1, 2, 3, 4, 0, 1] [0, 1, 2, 3, 4] 3
|
||||
* [4, 0] [0, 4] 1
|
||||
* [0, 1, 2, 3, 4, 0, 1] [0, 1, 2, 3, 4] 0
|
||||
*
|
||||
*/
|
||||
#define for_each_req_hash_asc(req, i, hash, start) \
|
||||
for (i = 0, start = (req->map[0].hash + req->core_line_count <= \
|
||||
_NUM_HASH_ENTRIES) ? 0 : (_NUM_HASH_ENTRIES - req->map[0].hash)\
|
||||
% _NUM_HASH_ENTRIES, hash = req->map[start].hash; \
|
||||
i < OCF_MIN(req->core_line_count, _NUM_HASH_ENTRIES); \
|
||||
i++, hash = req->map[(start + i) % req->core_line_count].hash)
|
||||
|
||||
void ocf_req_hash_lock_rd(struct ocf_request *req)
|
||||
{
|
||||
unsigned i, start;
|
||||
ocf_cache_line_t hash;
|
||||
|
||||
ocf_metadata_start_shared_access(&req->cache->metadata.lock);
|
||||
for_each_req_hash_asc(req, i, hash, start) {
|
||||
ocf_metadata_hash_lock(&req->cache->metadata.lock, hash,
|
||||
OCF_METADATA_RD);
|
||||
}
|
||||
}
|
||||
|
||||
void ocf_req_hash_unlock_rd(struct ocf_request *req)
|
||||
{
|
||||
unsigned i, start;
|
||||
ocf_cache_line_t hash;
|
||||
|
||||
for_each_req_hash_asc(req, i, hash, start) {
|
||||
ocf_metadata_hash_unlock(&req->cache->metadata.lock, hash,
|
||||
OCF_METADATA_RD);
|
||||
}
|
||||
ocf_metadata_end_shared_access(&req->cache->metadata.lock);
|
||||
}
|
||||
|
||||
void ocf_req_hash_lock_wr(struct ocf_request *req)
|
||||
{
|
||||
unsigned i, start;
|
||||
ocf_cache_line_t hash;
|
||||
|
||||
ocf_metadata_start_shared_access(&req->cache->metadata.lock);
|
||||
for_each_req_hash_asc(req, i, hash, start) {
|
||||
ocf_metadata_hash_lock(&req->cache->metadata.lock, hash,
|
||||
OCF_METADATA_WR);
|
||||
}
|
||||
}
|
||||
|
||||
void ocf_req_hash_lock_upgrade(struct ocf_request *req)
|
||||
{
|
||||
unsigned i, start;
|
||||
ocf_cache_line_t hash;
|
||||
|
||||
for_each_req_hash_asc(req, i, hash, start) {
|
||||
ocf_metadata_hash_unlock(&req->cache->metadata.lock, hash,
|
||||
OCF_METADATA_RD);
|
||||
}
|
||||
for_each_req_hash_asc(req, i, hash, start) {
|
||||
ocf_metadata_hash_lock(&req->cache->metadata.lock, hash,
|
||||
OCF_METADATA_WR);
|
||||
}
|
||||
}
|
||||
|
||||
void ocf_req_hash_unlock_wr(struct ocf_request *req)
|
||||
{
|
||||
unsigned i, start;
|
||||
ocf_cache_line_t hash;
|
||||
|
||||
for_each_req_hash_asc(req, i, hash, start) {
|
||||
ocf_metadata_hash_unlock(&req->cache->metadata.lock, hash,
|
||||
OCF_METADATA_WR);
|
||||
}
|
||||
ocf_metadata_end_shared_access(&req->cache->metadata.lock);
|
||||
}
|
||||
|
@ -10,125 +10,105 @@
|
||||
#define OCF_METADATA_RD 0
|
||||
#define OCF_METADATA_WR 1
|
||||
|
||||
void ocf_metadata_concurrency_init(struct ocf_cache *cache);
|
||||
void ocf_metadata_concurrency_init(struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
void ocf_metadata_concurrency_deinit(struct ocf_cache *cache);
|
||||
void ocf_metadata_concurrency_deinit(struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
int ocf_metadata_concurrency_attached_init(struct ocf_cache *cache);
|
||||
int ocf_metadata_concurrency_attached_init(
|
||||
struct ocf_metadata_lock *metadata_lock, ocf_cache_t cache,
|
||||
uint64_t hash_table_entries);
|
||||
|
||||
static inline void ocf_metadata_eviction_lock(struct ocf_cache *cache)
|
||||
void ocf_metadata_concurrency_attached_deinit(
|
||||
struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
static inline void ocf_metadata_eviction_lock(
|
||||
struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
env_spinlock_lock(&cache->metadata.lock.eviction);
|
||||
env_spinlock_lock(&metadata_lock->eviction);
|
||||
}
|
||||
|
||||
static inline void ocf_metadata_eviction_unlock(struct ocf_cache *cache)
|
||||
static inline void ocf_metadata_eviction_unlock(
|
||||
struct ocf_metadata_lock *metadata_lock)
|
||||
{
|
||||
env_spinlock_unlock(&cache->metadata.lock.eviction);
|
||||
env_spinlock_unlock(&metadata_lock->eviction);
|
||||
}
|
||||
|
||||
#define OCF_METADATA_EVICTION_LOCK() \
|
||||
ocf_metadata_eviction_lock(cache)
|
||||
ocf_metadata_eviction_lock(&cache->metadata.lock)
|
||||
|
||||
#define OCF_METADATA_EVICTION_UNLOCK() \
|
||||
ocf_metadata_eviction_unlock(cache)
|
||||
ocf_metadata_eviction_unlock(&cache->metadata.lock)
|
||||
|
||||
static inline void ocf_metadata_lock(struct ocf_cache *cache, int rw)
|
||||
{
|
||||
if (rw == OCF_METADATA_WR)
|
||||
env_rwsem_down_write(&cache->metadata.lock.collision);
|
||||
else if (rw == OCF_METADATA_RD)
|
||||
env_rwsem_down_read(&cache->metadata.lock.collision);
|
||||
else
|
||||
ENV_BUG();
|
||||
}
|
||||
void ocf_metadata_start_exclusive_access(
|
||||
struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
int ocf_metadata_try_start_exclusive_access(
|
||||
struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
static inline void ocf_metadata_unlock(struct ocf_cache *cache, int rw)
|
||||
{
|
||||
if (rw == OCF_METADATA_WR)
|
||||
env_rwsem_up_write(&cache->metadata.lock.collision);
|
||||
else if (rw == OCF_METADATA_RD)
|
||||
env_rwsem_up_read(&cache->metadata.lock.collision);
|
||||
else
|
||||
ENV_BUG();
|
||||
}
|
||||
void ocf_metadata_end_exclusive_access(
|
||||
struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
static inline int ocf_metadata_try_lock(struct ocf_cache *cache, int rw)
|
||||
{
|
||||
int result = 0;
|
||||
int ocf_metadata_try_start_shared_access(
|
||||
struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
if (rw == OCF_METADATA_WR) {
|
||||
result = env_rwsem_down_write_trylock(
|
||||
&cache->metadata.lock.collision);
|
||||
} else if (rw == OCF_METADATA_RD) {
|
||||
result = env_rwsem_down_read_trylock(
|
||||
&cache->metadata.lock.collision);
|
||||
} else {
|
||||
ENV_BUG();
|
||||
}
|
||||
void ocf_metadata_start_shared_access(
|
||||
struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
if (result)
|
||||
return -1;
|
||||
|
||||
return 0;
|
||||
}
|
||||
void ocf_metadata_end_shared_access(
|
||||
struct ocf_metadata_lock *metadata_lock);
|
||||
|
||||
static inline void ocf_metadata_status_bits_lock(
|
||||
struct ocf_cache *cache, int rw)
|
||||
struct ocf_metadata_lock *metadata_lock, int rw)
|
||||
{
|
||||
if (rw == OCF_METADATA_WR)
|
||||
env_rwlock_write_lock(&cache->metadata.lock.status);
|
||||
env_rwlock_write_lock(&metadata_lock->status);
|
||||
else if (rw == OCF_METADATA_RD)
|
||||
env_rwlock_read_lock(&cache->metadata.lock.status);
|
||||
env_rwlock_read_lock(&metadata_lock->status);
|
||||
else
|
||||
ENV_BUG();
|
||||
}
|
||||
|
||||
static inline void ocf_metadata_status_bits_unlock(
|
||||
struct ocf_cache *cache, int rw)
|
||||
struct ocf_metadata_lock *metadata_lock, int rw)
|
||||
{
|
||||
if (rw == OCF_METADATA_WR)
|
||||
env_rwlock_write_unlock(&cache->metadata.lock.status);
|
||||
env_rwlock_write_unlock(&metadata_lock->status);
|
||||
else if (rw == OCF_METADATA_RD)
|
||||
env_rwlock_read_unlock(&cache->metadata.lock.status);
|
||||
env_rwlock_read_unlock(&metadata_lock->status);
|
||||
else
|
||||
ENV_BUG();
|
||||
}
|
||||
|
||||
#define OCF_METADATA_LOCK_RD() \
|
||||
ocf_metadata_lock(cache, OCF_METADATA_RD)
|
||||
|
||||
#define OCF_METADATA_UNLOCK_RD() \
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_RD)
|
||||
|
||||
#define OCF_METADATA_LOCK_RD_TRY() \
|
||||
ocf_metadata_try_lock(cache, OCF_METADATA_RD)
|
||||
|
||||
#define OCF_METADATA_LOCK_WR() \
|
||||
ocf_metadata_lock(cache, OCF_METADATA_WR)
|
||||
|
||||
#define OCF_METADATA_LOCK_WR_TRY() \
|
||||
ocf_metadata_try_lock(cache, OCF_METADATA_WR)
|
||||
|
||||
#define OCF_METADATA_UNLOCK_WR() \
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_WR)
|
||||
|
||||
#define OCF_METADATA_BITS_LOCK_RD() \
|
||||
ocf_metadata_status_bits_lock(cache, OCF_METADATA_RD)
|
||||
ocf_metadata_status_bits_lock(&cache->metadata.lock, \
|
||||
OCF_METADATA_RD)
|
||||
|
||||
#define OCF_METADATA_BITS_UNLOCK_RD() \
|
||||
ocf_metadata_status_bits_unlock(cache, OCF_METADATA_RD)
|
||||
ocf_metadata_status_bits_unlock(&cache->metadata.lock, \
|
||||
OCF_METADATA_RD)
|
||||
|
||||
#define OCF_METADATA_BITS_LOCK_WR() \
|
||||
ocf_metadata_status_bits_lock(cache, OCF_METADATA_WR)
|
||||
ocf_metadata_status_bits_lock(&cache->metadata.lock, \
|
||||
OCF_METADATA_WR)
|
||||
|
||||
#define OCF_METADATA_BITS_UNLOCK_WR() \
|
||||
ocf_metadata_status_bits_unlock(cache, OCF_METADATA_WR)
|
||||
ocf_metadata_status_bits_unlock(&cache->metadata.lock, \
|
||||
OCF_METADATA_WR)
|
||||
|
||||
#define OCF_METADATA_FLUSH_LOCK() \
|
||||
ocf_metadata_flush_lock(cache)
|
||||
void ocf_metadata_hash_lock_rd(struct ocf_metadata_lock *metadata_lock,
|
||||
uint32_t core_id, uint64_t core_line);
|
||||
void ocf_metadata_hash_unlock_rd(struct ocf_metadata_lock *metadata_lock,
|
||||
uint32_t core_id, uint64_t core_line);
|
||||
void ocf_metadata_hash_lock_wr(struct ocf_metadata_lock *metadata_lock,
|
||||
uint32_t core_id, uint64_t core_line);
|
||||
void ocf_metadata_hash_unlock_wr(struct ocf_metadata_lock *metadata_lock,
|
||||
uint32_t core_id, uint64_t core_line);
|
||||
|
||||
#define OCF_METADATA_FLUSH_UNLOCK() \
|
||||
ocf_metadata_flush_unlock(cache)
|
||||
/* lock entire request in deadlock-free manner */
|
||||
void ocf_req_hash_lock_rd(struct ocf_request *req);
|
||||
void ocf_req_hash_unlock_rd(struct ocf_request *req);
|
||||
void ocf_req_hash_lock_wr(struct ocf_request *req);
|
||||
void ocf_req_hash_unlock_wr(struct ocf_request *req);
|
||||
void ocf_req_hash_lock_upgrade(struct ocf_request *req);
|
||||
|
||||
#endif
|
||||
|
@ -37,8 +37,6 @@ struct ocf_io_if {
|
||||
|
||||
int (*write)(struct ocf_request *req);
|
||||
|
||||
void (*resume)(struct ocf_request *req);
|
||||
|
||||
const char *name;
|
||||
};
|
||||
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include "../metadata/metadata.h"
|
||||
#include "../eviction/eviction.h"
|
||||
#include "../promotion/promotion.h"
|
||||
#include "../concurrency/ocf_concurrency.h"
|
||||
|
||||
void ocf_engine_error(struct ocf_request *req,
|
||||
bool stop_cache, const char *msg)
|
||||
@ -306,7 +307,7 @@ static void ocf_engine_map_hndl_error(struct ocf_cache *cache,
|
||||
}
|
||||
}
|
||||
|
||||
void ocf_engine_map(struct ocf_request *req)
|
||||
static void ocf_engine_map(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
uint32_t i;
|
||||
@ -315,18 +316,15 @@ void ocf_engine_map(struct ocf_request *req)
|
||||
int status = LOOKUP_MAPPED;
|
||||
ocf_core_id_t core_id = ocf_core_get_id(req->core);
|
||||
|
||||
if (!ocf_promotion_req_should_promote(cache->promotion_policy, req)) {
|
||||
if (!ocf_engine_unmapped_count(req))
|
||||
return;
|
||||
|
||||
if (ocf_engine_unmapped_count(req) >
|
||||
ocf_freelist_num_free(cache->freelist)) {
|
||||
req->info.mapping_error = 1;
|
||||
return;
|
||||
}
|
||||
|
||||
if (ocf_engine_unmapped_count(req))
|
||||
status = space_managment_evict_do(cache, req,
|
||||
ocf_engine_unmapped_count(req));
|
||||
|
||||
if (req->info.mapping_error)
|
||||
return;
|
||||
|
||||
ocf_req_clear_info(req);
|
||||
req->info.seq_req = true;
|
||||
|
||||
@ -397,6 +395,134 @@ static void _ocf_engine_clean_end(void *private_data, int error)
|
||||
}
|
||||
}
|
||||
|
||||
static int ocf_engine_evict(struct ocf_request *req)
|
||||
{
|
||||
if (!ocf_engine_unmapped_count(req))
|
||||
return 0;
|
||||
|
||||
return space_managment_evict_do(req->cache, req,
|
||||
ocf_engine_unmapped_count(req));
|
||||
}
|
||||
|
||||
static int lock_clines(struct ocf_request *req, enum ocf_engine_lock_type lock,
|
||||
ocf_req_async_lock_cb cb)
|
||||
{
|
||||
switch (lock) {
|
||||
case ocf_engine_lock_write:
|
||||
return ocf_req_async_lock_wr(req, cb);
|
||||
case ocf_engine_lock_read:
|
||||
return ocf_req_async_lock_rd(req, cb);
|
||||
default:
|
||||
return OCF_LOCK_ACQUIRED;
|
||||
}
|
||||
}
|
||||
|
||||
static int trylock_clines(struct ocf_request *req,
|
||||
enum ocf_engine_lock_type lock)
|
||||
{
|
||||
switch (lock) {
|
||||
case ocf_engine_lock_write:
|
||||
return ocf_req_trylock_wr(req);
|
||||
case ocf_engine_lock_read:
|
||||
return ocf_req_trylock_rd(req);
|
||||
default:
|
||||
return OCF_LOCK_ACQUIRED;
|
||||
}
|
||||
}
|
||||
|
||||
int ocf_engine_prepare_clines(struct ocf_request *req,
|
||||
const struct ocf_engine_callbacks *engine_cbs)
|
||||
{
|
||||
bool mapped;
|
||||
bool promote = true;
|
||||
int lock = -ENOENT;
|
||||
enum ocf_engine_lock_type lock_type;
|
||||
struct ocf_metadata_lock *metadata_lock = &req->cache->metadata.lock;
|
||||
|
||||
/* Calculate hashes for hash-bucket locking */
|
||||
ocf_req_hash(req);
|
||||
|
||||
/* Read-lock hash buckets associated with request target core & LBAs
|
||||
* (core lines) to assure that cache mapping for these core lines does
|
||||
* not change during traversation */
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Traverse request to cache if there is hit */
|
||||
ocf_engine_traverse(req);
|
||||
mapped = ocf_engine_is_mapped(req);
|
||||
|
||||
if (mapped) {
|
||||
/* We are holding hash buckets read lock, so we can attempt
|
||||
* per-cacheline locking fast path, which would fail either if
|
||||
* cachelines are already locked without putting request to a
|
||||
* waiter list */
|
||||
lock_type = engine_cbs->get_lock_type(req);
|
||||
lock = trylock_clines(req, lock_type);
|
||||
|
||||
if (lock == OCF_LOCK_ACQUIRED) {
|
||||
/* Cachelines are mapped and locked, we don't need the
|
||||
* hash bucket lock any more */
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
} else {
|
||||
/* Failed to acquire cachelines lock in fast path,
|
||||
* acquire hash-buckets write lock and attempt the lock
|
||||
* again, allowing slow path and async assignment of
|
||||
* the lock. */
|
||||
ocf_req_hash_lock_upgrade(req);
|
||||
lock = lock_clines(req, lock_type, engine_cbs->resume);
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
} else {
|
||||
/* check if request should promote cachelines */
|
||||
promote = ocf_promotion_req_should_promote(
|
||||
req->cache->promotion_policy, req);
|
||||
if (!promote) {
|
||||
req->info.mapping_error = 1;
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
}
|
||||
|
||||
if (!mapped && promote) {
|
||||
/* Need to map (potentially evict) cachelines. Mapping must be
|
||||
* performed holding (at least) hash-bucket write lock */
|
||||
ocf_req_hash_lock_upgrade(req);
|
||||
|
||||
ocf_engine_map(req);
|
||||
if (!req->info.mapping_error) {
|
||||
/* Lock cachelines, potentially putting the request on
|
||||
* waiter list */
|
||||
lock_type = engine_cbs->get_lock_type(req);
|
||||
lock = trylock_clines(req, lock_type);
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
lock = lock_clines(req, lock_type,
|
||||
engine_cbs->resume);
|
||||
}
|
||||
}
|
||||
|
||||
/* At this point the request is mapped or we need to evict,
|
||||
* which is done under global metadata lock */
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
|
||||
if (req->info.mapping_error) {
|
||||
/* Not mapped - evict cachelines */
|
||||
ocf_metadata_start_exclusive_access(metadata_lock);
|
||||
if (ocf_engine_evict(req) == LOOKUP_MAPPED)
|
||||
ocf_engine_map(req);
|
||||
if (!req->info.mapping_error) {
|
||||
lock_type = engine_cbs->get_lock_type(req);
|
||||
lock = trylock_clines(req, lock_type);
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
lock = lock_clines(req, lock_type,
|
||||
engine_cbs->resume);
|
||||
}
|
||||
}
|
||||
ocf_metadata_end_exclusive_access(metadata_lock);
|
||||
}
|
||||
}
|
||||
|
||||
return lock;
|
||||
}
|
||||
|
||||
static int _ocf_engine_clean_getter(struct ocf_cache *cache,
|
||||
void *getter_context, uint32_t item, ocf_cache_line_t *line)
|
||||
{
|
||||
@ -533,15 +659,14 @@ void inc_fallback_pt_error_counter(ocf_cache_t cache)
|
||||
|
||||
static int _ocf_engine_refresh(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
int result;
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
/* Check under metadata RD lock */
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
result = ocf_engine_check(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
|
||||
if (result == 0) {
|
||||
|
||||
|
@ -162,19 +162,48 @@ void ocf_engine_lookup_map_entry(struct ocf_cache *cache,
|
||||
uint64_t core_line);
|
||||
|
||||
/**
|
||||
* @brief Traverse request in order to lookup cache lines If there are misses
|
||||
* need to call eviction. This process is called 'mapping'.
|
||||
* @brief Request cacheline lock type
|
||||
*/
|
||||
enum ocf_engine_lock_type
|
||||
{
|
||||
/** No lock */
|
||||
ocf_engine_lock_none = 0,
|
||||
/** Write lock */
|
||||
ocf_engine_lock_write,
|
||||
/** Read lock */
|
||||
ocf_engine_lock_read,
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Engine-specific callbacks for common request handling rountine
|
||||
*
|
||||
* @note This function CALL EVICTION
|
||||
* TODO(arutk): expand this structure to fit all engines and all steps
|
||||
*/
|
||||
struct ocf_engine_callbacks
|
||||
{
|
||||
/** Specify locking requirements after request is mapped */
|
||||
enum ocf_engine_lock_type (*get_lock_type)(struct ocf_request *req);
|
||||
|
||||
/** Resume handling after acquiring asynchronous lock */
|
||||
ocf_req_async_lock_cb resume;
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Map and lock cachelines
|
||||
*
|
||||
* @param req OCF request
|
||||
*
|
||||
* @returns eviction status
|
||||
* @retval LOOKUP_MAPPED successfully evicted required number of cachelines
|
||||
* @retval LOOKUP_MISS eviction failure
|
||||
*/
|
||||
void ocf_engine_map(struct ocf_request *req);
|
||||
int ocf_engine_prepare_clines(struct ocf_request *req,
|
||||
const struct ocf_engine_callbacks *engine_cbs);
|
||||
|
||||
/**
|
||||
* @brief Traverse OCF request (lookup cache)
|
||||
*
|
||||
* @note This function DO NOT CALL EVICTION. Only lookup in metadata is
|
||||
* @note This function does not evict cachelines. Only lookup in metadata is
|
||||
* performed. Main purpose of this function is to check if there is a HIT.
|
||||
*
|
||||
* @param req OCF request
|
||||
|
@ -22,30 +22,25 @@ static int _ocf_discard_step_do(struct ocf_request *req);
|
||||
static int _ocf_discard_step(struct ocf_request *req);
|
||||
static int _ocf_discard_flush_cache(struct ocf_request *req);
|
||||
static int _ocf_discard_core(struct ocf_request *req);
|
||||
static void _ocf_discard_on_resume(struct ocf_request *req);
|
||||
|
||||
static const struct ocf_io_if _io_if_discard_step = {
|
||||
.read = _ocf_discard_step,
|
||||
.write = _ocf_discard_step,
|
||||
.resume = _ocf_discard_on_resume,
|
||||
};
|
||||
|
||||
static const struct ocf_io_if _io_if_discard_step_resume = {
|
||||
.read = _ocf_discard_step_do,
|
||||
.write = _ocf_discard_step_do,
|
||||
.resume = _ocf_discard_on_resume,
|
||||
};
|
||||
|
||||
static const struct ocf_io_if _io_if_discard_flush_cache = {
|
||||
.read = _ocf_discard_flush_cache,
|
||||
.write = _ocf_discard_flush_cache,
|
||||
.resume = _ocf_discard_on_resume,
|
||||
};
|
||||
|
||||
static const struct ocf_io_if _io_if_discard_core = {
|
||||
.read = _ocf_discard_core,
|
||||
.write = _ocf_discard_core,
|
||||
.resume = _ocf_discard_on_resume,
|
||||
};
|
||||
|
||||
static void _ocf_discard_complete_req(struct ocf_request *req, int error)
|
||||
@ -175,7 +170,7 @@ int _ocf_discard_step_do(struct ocf_request *req)
|
||||
if (ocf_engine_mapped_count(req)) {
|
||||
/* There are mapped cache line, need to remove them */
|
||||
|
||||
OCF_METADATA_LOCK_WR(); /*- Metadata WR access ---------------*/
|
||||
ocf_req_hash_lock_wr(req);
|
||||
|
||||
/* Remove mapped cache lines from metadata */
|
||||
ocf_purge_map_info(req);
|
||||
@ -186,16 +181,16 @@ int _ocf_discard_step_do(struct ocf_request *req)
|
||||
_ocf_discard_step_complete);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_WR(); /*- END Metadata WR access ---------*/
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Even if no cachelines are mapped they could be tracked in promotion
|
||||
* policy. RD lock suffices. */
|
||||
ocf_promotion_req_purge(req->cache->promotion_policy, req);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
|
||||
OCF_DEBUG_RQ(req, "Discard");
|
||||
_ocf_discard_step_complete(req, 0);
|
||||
@ -229,11 +224,12 @@ static int _ocf_discard_step(struct ocf_request *req)
|
||||
req->core_line_count = req->core_line_last - req->core_line_first + 1;
|
||||
req->io_if = &_io_if_discard_step_resume;
|
||||
|
||||
OCF_METADATA_LOCK_RD(); /*- Metadata READ access, No eviction --------*/
|
||||
|
||||
ENV_BUG_ON(env_memset(req->map, sizeof(*req->map) * req->core_line_count,
|
||||
0));
|
||||
|
||||
ocf_req_hash(req);
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Travers to check if request is mapped fully */
|
||||
ocf_engine_traverse(req);
|
||||
|
||||
@ -244,7 +240,13 @@ static int _ocf_discard_step(struct ocf_request *req)
|
||||
lock = OCF_LOCK_ACQUIRED;
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD(); /*- END Metadata READ access----------------*/
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
ocf_req_hash_lock_upgrade(req);
|
||||
lock = ocf_req_async_lock_wr(req, _ocf_discard_on_resume);
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
} else {
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
|
||||
if (lock >= 0) {
|
||||
if (OCF_LOCK_ACQUIRED == lock) {
|
||||
|
@ -58,8 +58,6 @@ static void _ocf_read_fast_complete(struct ocf_request *req, int error)
|
||||
|
||||
static int _ocf_read_fast_do(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
if (ocf_engine_is_miss(req)) {
|
||||
/* It seams that after resume, now request is MISS, do PT */
|
||||
OCF_DEBUG_RQ(req, "Switching to read PT");
|
||||
@ -74,14 +72,14 @@ static int _ocf_read_fast_do(struct ocf_request *req)
|
||||
if (req->info.re_part) {
|
||||
OCF_DEBUG_RQ(req, "Re-Part");
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_req_hash_lock_wr(req);
|
||||
|
||||
/* Probably some cache lines are assigned into wrong
|
||||
* partition. Need to move it to new one
|
||||
*/
|
||||
ocf_part_move(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
|
||||
/* Submit IO */
|
||||
@ -104,14 +102,12 @@ static int _ocf_read_fast_do(struct ocf_request *req)
|
||||
static const struct ocf_io_if _io_if_read_fast_resume = {
|
||||
.read = _ocf_read_fast_do,
|
||||
.write = _ocf_read_fast_do,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
int ocf_read_fast(struct ocf_request *req)
|
||||
{
|
||||
bool hit;
|
||||
int lock = OCF_LOCK_NOT_ACQUIRED;
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
/* Get OCF request - increase reference counter */
|
||||
ocf_req_get(req);
|
||||
@ -121,7 +117,8 @@ int ocf_read_fast(struct ocf_request *req)
|
||||
|
||||
/*- Metadata RD access -----------------------------------------------*/
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_req_hash(req);
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Traverse request to cache if there is hit */
|
||||
ocf_engine_traverse(req);
|
||||
@ -129,10 +126,10 @@ int ocf_read_fast(struct ocf_request *req)
|
||||
hit = ocf_engine_is_hit(req);
|
||||
if (hit) {
|
||||
ocf_io_start(&req->ioi.io);
|
||||
lock = ocf_req_trylock_rd(req);
|
||||
lock = ocf_req_async_lock_rd(req, ocf_engine_on_resume);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
|
||||
if (hit) {
|
||||
OCF_DEBUG_RQ(req, "Fast path success");
|
||||
@ -174,14 +171,12 @@ int ocf_read_fast(struct ocf_request *req)
|
||||
static const struct ocf_io_if _io_if_write_fast_resume = {
|
||||
.read = ocf_write_wb_do,
|
||||
.write = ocf_write_wb_do,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
int ocf_write_fast(struct ocf_request *req)
|
||||
{
|
||||
bool mapped;
|
||||
int lock = OCF_LOCK_NOT_ACQUIRED;
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
/* Get OCF request - increase reference counter */
|
||||
ocf_req_get(req);
|
||||
@ -191,7 +186,8 @@ int ocf_write_fast(struct ocf_request *req)
|
||||
|
||||
/*- Metadata RD access -----------------------------------------------*/
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_req_hash(req);
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Traverse request to cache if there is hit */
|
||||
ocf_engine_traverse(req);
|
||||
@ -200,9 +196,16 @@ int ocf_write_fast(struct ocf_request *req)
|
||||
if (mapped) {
|
||||
ocf_io_start(&req->ioi.io);
|
||||
lock = ocf_req_trylock_wr(req);
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
ocf_req_hash_lock_upgrade(req);
|
||||
lock = ocf_req_async_lock_wr(req, ocf_engine_on_resume);
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
} else {
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
} else {
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
|
||||
if (mapped) {
|
||||
if (lock >= 0) {
|
||||
|
@ -43,9 +43,9 @@ static int _ocf_invalidate_do(struct ocf_request *req)
|
||||
|
||||
ENV_BUG_ON(env_atomic_read(&req->req_remaining));
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_req_hash_lock_wr(req);
|
||||
ocf_purge_map_info(req);
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
|
||||
env_atomic_inc(&req->req_remaining);
|
||||
|
||||
|
@ -52,16 +52,14 @@ static inline void _ocf_read_pt_submit(struct ocf_request *req)
|
||||
|
||||
int ocf_read_pt_do(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
/* Get OCF request - increase reference counter */
|
||||
ocf_req_get(req);
|
||||
|
||||
if (req->info.dirty_any) {
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_req_hash_lock_rd(req);
|
||||
/* Need to clean, start it */
|
||||
ocf_engine_clean(req);
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
|
||||
/* Do not processing, because first we need to clean request */
|
||||
ocf_req_put(req);
|
||||
@ -72,14 +70,14 @@ int ocf_read_pt_do(struct ocf_request *req)
|
||||
if (req->info.re_part) {
|
||||
OCF_DEBUG_RQ(req, "Re-Part");
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_req_hash_lock_wr(req);
|
||||
|
||||
/* Probably some cache lines are assigned into wrong
|
||||
* partition. Need to move it to new one
|
||||
*/
|
||||
ocf_part_move(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
|
||||
/* Submit read IO to the core */
|
||||
@ -99,14 +97,12 @@ int ocf_read_pt_do(struct ocf_request *req)
|
||||
static const struct ocf_io_if _io_if_pt_resume = {
|
||||
.read = ocf_read_pt_do,
|
||||
.write = ocf_read_pt_do,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
int ocf_read_pt(struct ocf_request *req)
|
||||
{
|
||||
bool use_cache = false;
|
||||
int lock = OCF_LOCK_NOT_ACQUIRED;
|
||||
struct ocf_cache *cache = req->cache;
|
||||
int lock = OCF_LOCK_ACQUIRED;
|
||||
|
||||
OCF_DEBUG_TRACE(req->cache);
|
||||
|
||||
@ -118,7 +114,8 @@ int ocf_read_pt(struct ocf_request *req)
|
||||
/* Set resume io_if */
|
||||
req->io_if = &_io_if_pt_resume;
|
||||
|
||||
OCF_METADATA_LOCK_RD(); /*- Metadata RD access -----------------------*/
|
||||
ocf_req_hash(req);
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Traverse request to check if there are mapped cache lines */
|
||||
ocf_engine_traverse(req);
|
||||
@ -131,13 +128,16 @@ int ocf_read_pt(struct ocf_request *req)
|
||||
* lock request for READ access
|
||||
*/
|
||||
lock = ocf_req_trylock_rd(req);
|
||||
} else {
|
||||
/* No mapped cache lines, no need to get lock */
|
||||
lock = OCF_LOCK_ACQUIRED;
|
||||
}
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD(); /*- END Metadata RD access -----------------*/
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
ocf_req_hash_lock_upgrade(req);
|
||||
lock = ocf_req_async_lock_rd(req, ocf_engine_on_resume);
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
} else {
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
|
||||
if (use_cache) {
|
||||
/*
|
||||
|
@ -137,8 +137,6 @@ err_alloc:
|
||||
|
||||
static int _ocf_read_generic_do(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
if (ocf_engine_is_miss(req) && req->map->rd_locked) {
|
||||
/* Miss can be handled only on write locks.
|
||||
* Need to switch to PT
|
||||
@ -153,12 +151,12 @@ static int _ocf_read_generic_do(struct ocf_request *req)
|
||||
|
||||
if (ocf_engine_is_miss(req)) {
|
||||
if (req->info.dirty_any) {
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Request is dirty need to clean request */
|
||||
ocf_engine_clean(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
|
||||
/* We need to clean request before processing, return */
|
||||
ocf_req_put(req);
|
||||
@ -166,25 +164,25 @@ static int _ocf_read_generic_do(struct ocf_request *req)
|
||||
return 0;
|
||||
}
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Set valid status bits map */
|
||||
ocf_set_valid_map_info(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
|
||||
if (req->info.re_part) {
|
||||
OCF_DEBUG_RQ(req, "Re-Part");
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_req_hash_lock_wr(req);
|
||||
|
||||
/* Probably some cache lines are assigned into wrong
|
||||
* partition. Need to move it to new one
|
||||
*/
|
||||
ocf_part_move(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
|
||||
OCF_DEBUG_RQ(req, "Submit");
|
||||
@ -208,12 +206,24 @@ static int _ocf_read_generic_do(struct ocf_request *req)
|
||||
static const struct ocf_io_if _io_if_read_generic_resume = {
|
||||
.read = _ocf_read_generic_do,
|
||||
.write = _ocf_read_generic_do,
|
||||
};
|
||||
|
||||
static enum ocf_engine_lock_type ocf_rd_get_lock_type(struct ocf_request *req)
|
||||
{
|
||||
if (ocf_engine_is_hit(req))
|
||||
return ocf_engine_lock_read;
|
||||
else
|
||||
return ocf_engine_lock_write;
|
||||
}
|
||||
|
||||
static const struct ocf_engine_callbacks _rd_engine_callbacks =
|
||||
{
|
||||
.get_lock_type = ocf_rd_get_lock_type,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
int ocf_read_generic(struct ocf_request *req)
|
||||
{
|
||||
bool mapped;
|
||||
int lock = OCF_LOCK_NOT_ACQUIRED;
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
@ -231,61 +241,7 @@ int ocf_read_generic(struct ocf_request *req)
|
||||
/* Set resume call backs */
|
||||
req->io_if = &_io_if_read_generic_resume;
|
||||
|
||||
/*- Metadata RD access -----------------------------------------------*/
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
|
||||
/* Traverse request to cache if there is hit */
|
||||
ocf_engine_traverse(req);
|
||||
|
||||
mapped = ocf_engine_is_mapped(req);
|
||||
if (mapped) {
|
||||
/* Request is fully mapped, no need to call eviction */
|
||||
if (ocf_engine_is_hit(req)) {
|
||||
/* There is a hit, lock request for READ access */
|
||||
lock = ocf_req_trylock_rd(req);
|
||||
} else {
|
||||
/* All cache line mapped, but some sectors are not valid
|
||||
* and cache insert will be performed - lock for
|
||||
* WRITE is required
|
||||
*/
|
||||
lock = ocf_req_trylock_wr(req);
|
||||
}
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
|
||||
/*- END Metadata RD access -------------------------------------------*/
|
||||
|
||||
if (!mapped) {
|
||||
|
||||
/*- Metadata WR access ---------------------------------------*/
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
|
||||
/* Now there is exclusive access for metadata. May traverse once
|
||||
* again. If there are misses need to call eviction. This
|
||||
* process is called 'mapping'.
|
||||
*/
|
||||
ocf_engine_map(req);
|
||||
|
||||
if (!req->info.mapping_error) {
|
||||
if (ocf_engine_is_hit(req)) {
|
||||
/* After mapping turns out there is hit,
|
||||
* so lock OCF request for read access
|
||||
*/
|
||||
lock = ocf_req_trylock_rd(req);
|
||||
} else {
|
||||
/* Miss, new cache lines were mapped,
|
||||
* need to lock OCF request for write access
|
||||
*/
|
||||
lock = ocf_req_trylock_wr(req);
|
||||
}
|
||||
}
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
|
||||
/*- END Metadata WR access -----------------------------------*/
|
||||
}
|
||||
lock = ocf_engine_prepare_clines(req, &_rd_engine_callbacks);
|
||||
|
||||
if (!req->info.mapping_error) {
|
||||
if (lock >= 0) {
|
||||
|
@ -38,19 +38,19 @@ static void _ocf_read_wa_complete(struct ocf_request *req, int error)
|
||||
|
||||
int ocf_write_wa(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
ocf_io_start(&req->ioi.io);
|
||||
|
||||
/* Get OCF request - increase reference counter */
|
||||
ocf_req_get(req);
|
||||
|
||||
OCF_METADATA_LOCK_RD(); /*- Metadata RD access -----------------------*/
|
||||
ocf_req_hash(req);
|
||||
|
||||
ocf_req_hash_lock_rd(req); /*- Metadata RD access -----------------------*/
|
||||
|
||||
/* Traverse request to check if there are mapped cache lines */
|
||||
ocf_engine_traverse(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD(); /*- END Metadata RD access -----------------*/
|
||||
ocf_req_hash_unlock_rd(req); /*- END Metadata RD access -----------------*/
|
||||
|
||||
if (ocf_engine_is_hit(req)) {
|
||||
ocf_req_clear(req);
|
||||
|
@ -21,28 +21,25 @@
|
||||
static const struct ocf_io_if _io_if_wb_resume = {
|
||||
.read = ocf_write_wb_do,
|
||||
.write = ocf_write_wb_do,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
static void _ocf_write_wb_update_bits(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
if (ocf_engine_is_miss(req)) {
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_req_hash_lock_rd(req);
|
||||
/* Update valid status bits */
|
||||
ocf_set_valid_map_info(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
|
||||
if (!ocf_engine_is_dirty_all(req)) {
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_req_hash_lock_wr(req);
|
||||
|
||||
/* set dirty bits, and mark if metadata flushing is required */
|
||||
ocf_set_dirty_map_info(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
}
|
||||
|
||||
@ -128,14 +125,14 @@ static inline void _ocf_write_wb_submit(struct ocf_request *req)
|
||||
if (req->info.re_part) {
|
||||
OCF_DEBUG_RQ(req, "Re-Part");
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_req_hash_lock_wr(req);
|
||||
|
||||
/* Probably some cache lines are assigned into wrong
|
||||
* partition. Need to move it to new one
|
||||
*/
|
||||
ocf_part_move(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
|
||||
OCF_DEBUG_RQ(req, "Submit Data");
|
||||
@ -166,11 +163,20 @@ int ocf_write_wb_do(struct ocf_request *req)
|
||||
return 0;
|
||||
}
|
||||
|
||||
static enum ocf_engine_lock_type ocf_wb_get_lock_type(struct ocf_request *req)
|
||||
{
|
||||
return ocf_engine_lock_write;
|
||||
}
|
||||
|
||||
static const struct ocf_engine_callbacks _wb_engine_callbacks =
|
||||
{
|
||||
.get_lock_type = ocf_wb_get_lock_type,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
int ocf_write_wb(struct ocf_request *req)
|
||||
{
|
||||
bool mapped;
|
||||
int lock = OCF_LOCK_NOT_ACQUIRED;
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
ocf_io_start(&req->ioi.io);
|
||||
|
||||
@ -182,35 +188,7 @@ int ocf_write_wb(struct ocf_request *req)
|
||||
|
||||
/* TODO: Handle fits into dirty */
|
||||
|
||||
OCF_METADATA_LOCK_RD(); /*- Metadata READ access, No eviction --------*/
|
||||
|
||||
/* Travers to check if request is mapped fully */
|
||||
ocf_engine_traverse(req);
|
||||
|
||||
mapped = ocf_engine_is_mapped(req);
|
||||
if (mapped) {
|
||||
/* All cache line are mapped, lock request for WRITE access */
|
||||
lock = ocf_req_trylock_wr(req);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD(); /*- END Metadata READ access----------------*/
|
||||
|
||||
if (!mapped) {
|
||||
OCF_METADATA_LOCK_WR(); /*- Metadata WR access, eviction -----*/
|
||||
|
||||
/* Now there is exclusive access for metadata. May traverse once
|
||||
* again. If there are misses need to call eviction. This
|
||||
* process is called 'mapping'.
|
||||
*/
|
||||
ocf_engine_map(req);
|
||||
|
||||
if (!req->info.mapping_error) {
|
||||
/* Lock request for WRITE access */
|
||||
lock = ocf_req_trylock_wr(req);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_WR(); /*- END Metadata WR access ---------*/
|
||||
}
|
||||
lock = ocf_engine_prepare_clines(req, &_wb_engine_callbacks);
|
||||
|
||||
if (!req->info.mapping_error) {
|
||||
if (lock >= 0) {
|
||||
|
@ -52,12 +52,12 @@ static int ocf_write_wi_update_and_flush_metadata(struct ocf_request *req)
|
||||
if (ocf_engine_mapped_count(req)) {
|
||||
/* There are mapped cache line, need to remove them */
|
||||
|
||||
OCF_METADATA_LOCK_WR(); /*- Metadata WR access ---------------*/
|
||||
ocf_req_hash_lock_wr(req); /*- Metadata WR access ---------------*/
|
||||
|
||||
/* Remove mapped cache lines from metadata */
|
||||
ocf_purge_map_info(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR(); /*- END Metadata WR access ---------*/
|
||||
ocf_req_hash_unlock_wr(req); /*- END Metadata WR access ---------*/
|
||||
|
||||
if (req->info.flush_metadata) {
|
||||
/* Request was dirty and need to flush metadata */
|
||||
@ -130,13 +130,11 @@ static void _ocf_write_wi_on_resume(struct ocf_request *req)
|
||||
static const struct ocf_io_if _io_if_wi_resume = {
|
||||
.read = _ocf_write_wi_do,
|
||||
.write = _ocf_write_wi_do,
|
||||
.resume = _ocf_write_wi_on_resume,
|
||||
};
|
||||
|
||||
int ocf_write_wi(struct ocf_request *req)
|
||||
{
|
||||
int lock = OCF_LOCK_NOT_ACQUIRED;
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
OCF_DEBUG_TRACE(req->cache);
|
||||
|
||||
@ -148,7 +146,8 @@ int ocf_write_wi(struct ocf_request *req)
|
||||
/* Set resume io_if */
|
||||
req->io_if = &_io_if_wi_resume;
|
||||
|
||||
OCF_METADATA_LOCK_RD(); /*- Metadata READ access, No eviction --------*/
|
||||
ocf_req_hash(req);
|
||||
ocf_req_hash_lock_rd(req); /*- Metadata READ access, No eviction --------*/
|
||||
|
||||
/* Travers to check if request is mapped fully */
|
||||
ocf_engine_traverse(req);
|
||||
@ -160,7 +159,13 @@ int ocf_write_wi(struct ocf_request *req)
|
||||
lock = OCF_LOCK_ACQUIRED;
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD(); /*- END Metadata READ access----------------*/
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
ocf_req_hash_lock_upgrade(req);
|
||||
lock = ocf_req_async_lock_wr(req, _ocf_write_wi_on_resume);
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
} else {
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
|
||||
if (lock >= 0) {
|
||||
if (lock == OCF_LOCK_ACQUIRED) {
|
||||
|
@ -197,12 +197,10 @@ int ocf_read_wo_do(struct ocf_request *req)
|
||||
static const struct ocf_io_if _io_if_wo_resume = {
|
||||
.read = ocf_read_wo_do,
|
||||
.write = ocf_read_wo_do,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
int ocf_read_wo(struct ocf_request *req)
|
||||
{
|
||||
ocf_cache_t cache = req->cache;
|
||||
int lock = OCF_LOCK_ACQUIRED;
|
||||
|
||||
OCF_DEBUG_TRACE(req->cache);
|
||||
@ -215,7 +213,8 @@ int ocf_read_wo(struct ocf_request *req)
|
||||
/* Set resume call backs */
|
||||
req->io_if = &_io_if_wo_resume;
|
||||
|
||||
OCF_METADATA_LOCK_RD(); /*- Metadata RD access -----------------------*/
|
||||
ocf_req_hash(req);
|
||||
ocf_req_hash_lock_rd(req); /*- Metadata RD access -----------------------*/
|
||||
|
||||
/* Traverse request to check if there are mapped cache lines */
|
||||
ocf_engine_traverse(req);
|
||||
@ -227,7 +226,13 @@ int ocf_read_wo(struct ocf_request *req)
|
||||
lock = ocf_req_trylock_rd(req);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD(); /*- END Metadata RD access -----------------*/
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
ocf_req_hash_lock_upgrade(req);
|
||||
lock = ocf_req_async_lock_rd(req, ocf_engine_on_resume);
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
} else {
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
|
||||
if (lock >= 0) {
|
||||
if (lock != OCF_LOCK_ACQUIRED) {
|
||||
|
@ -97,19 +97,17 @@ static inline void _ocf_write_wt_submit(struct ocf_request *req)
|
||||
|
||||
static void _ocf_write_wt_update_bits(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
if (ocf_engine_is_miss(req)) {
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
/* Update valid status bits */
|
||||
ocf_set_valid_map_info(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_req_hash_unlock_rd(req);
|
||||
}
|
||||
|
||||
if (req->info.dirty_any) {
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_req_hash_lock_wr(req);
|
||||
|
||||
/* Writes goes to SDD and HDD, need to update status bits from
|
||||
* dirty to clean
|
||||
@ -117,20 +115,20 @@ static void _ocf_write_wt_update_bits(struct ocf_request *req)
|
||||
|
||||
ocf_set_clean_map_info(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
|
||||
if (req->info.re_part) {
|
||||
OCF_DEBUG_RQ(req, "Re-Part");
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_req_hash_lock_wr(req);
|
||||
|
||||
/* Probably some cache lines are assigned into wrong
|
||||
* partition. Need to move it to new one
|
||||
*/
|
||||
ocf_part_move(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_req_hash_unlock_wr(req);
|
||||
}
|
||||
}
|
||||
|
||||
@ -158,14 +156,22 @@ static int _ocf_write_wt_do(struct ocf_request *req)
|
||||
static const struct ocf_io_if _io_if_wt_resume = {
|
||||
.read = _ocf_write_wt_do,
|
||||
.write = _ocf_write_wt_do,
|
||||
};
|
||||
|
||||
static enum ocf_engine_lock_type ocf_wt_get_lock_type(struct ocf_request *req)
|
||||
{
|
||||
return ocf_engine_lock_write;
|
||||
}
|
||||
|
||||
static const struct ocf_engine_callbacks _wt_engine_callbacks =
|
||||
{
|
||||
.get_lock_type = ocf_wt_get_lock_type,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
int ocf_write_wt(struct ocf_request *req)
|
||||
{
|
||||
bool mapped;
|
||||
int lock = OCF_LOCK_NOT_ACQUIRED;
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
ocf_io_start(&req->ioi.io);
|
||||
|
||||
@ -175,35 +181,7 @@ int ocf_write_wt(struct ocf_request *req)
|
||||
/* Set resume io_if */
|
||||
req->io_if = &_io_if_wt_resume;
|
||||
|
||||
OCF_METADATA_LOCK_RD(); /*- Metadata READ access, No eviction --------*/
|
||||
|
||||
/* Travers to check if request is mapped fully */
|
||||
ocf_engine_traverse(req);
|
||||
|
||||
mapped = ocf_engine_is_mapped(req);
|
||||
if (mapped) {
|
||||
/* All cache line are mapped, lock request for WRITE access */
|
||||
lock = ocf_req_trylock_wr(req);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_RD(); /*- END Metadata READ access----------------*/
|
||||
|
||||
if (!mapped) {
|
||||
OCF_METADATA_LOCK_WR(); /*- Metadata WR access, eviction -----*/
|
||||
|
||||
/* Now there is exclusive access for metadata. May traverse once
|
||||
* again. If there are misses need to call eviction. This
|
||||
* process is called 'mapping'.
|
||||
*/
|
||||
ocf_engine_map(req);
|
||||
|
||||
if (!req->info.mapping_error) {
|
||||
/* Lock request for WRITE access */
|
||||
lock = ocf_req_trylock_wr(req);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_WR(); /*- END Metadata WR access ---------*/
|
||||
}
|
||||
lock = ocf_engine_prepare_clines(req, &_wt_engine_callbacks);
|
||||
|
||||
if (!req->info.mapping_error) {
|
||||
if (lock >= 0) {
|
||||
|
@ -18,19 +18,17 @@
|
||||
|
||||
static int ocf_zero_purge(struct ocf_request *req)
|
||||
{
|
||||
struct ocf_cache *cache = req->cache;
|
||||
|
||||
if (req->error) {
|
||||
ocf_engine_error(req, true, "Failed to discard data on cache");
|
||||
} else {
|
||||
/* There are mapped cache line, need to remove them */
|
||||
|
||||
OCF_METADATA_LOCK_WR(); /*- Metadata WR access ---------------*/
|
||||
ocf_req_hash_lock_wr(req); /*- Metadata WR access ---------------*/
|
||||
|
||||
/* Remove mapped cache lines from metadata */
|
||||
ocf_purge_map_info(req);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR(); /*- END Metadata WR access ---------*/
|
||||
ocf_req_hash_unlock_wr(req); /*- END Metadata WR access ---------*/
|
||||
}
|
||||
|
||||
ocf_req_unlock_wr(req);
|
||||
@ -129,7 +127,6 @@ static int _ocf_zero_do(struct ocf_request *req)
|
||||
static const struct ocf_io_if _io_if_ocf_zero_do = {
|
||||
.read = _ocf_zero_do,
|
||||
.write = _ocf_zero_do,
|
||||
.resume = ocf_engine_on_resume,
|
||||
};
|
||||
|
||||
/**
|
||||
@ -143,6 +140,10 @@ void ocf_engine_zero_line(struct ocf_request *req)
|
||||
|
||||
ENV_BUG_ON(req->core_line_count != 1);
|
||||
|
||||
/* No hash bucket locking here - ocf_engine_zero_line caller must hold
|
||||
* metadata global write lock, so we have exclusive access to all hash
|
||||
* buckets here. */
|
||||
|
||||
/* Traverse to check if request is mapped */
|
||||
ocf_engine_traverse(req);
|
||||
|
||||
@ -151,7 +152,7 @@ void ocf_engine_zero_line(struct ocf_request *req)
|
||||
req->io_if = &_io_if_ocf_zero_do;
|
||||
|
||||
/* Some cache line are mapped, lock request for WRITE access */
|
||||
lock = ocf_req_trylock_wr(req);
|
||||
lock = ocf_req_async_lock_wr(req, ocf_engine_on_resume);
|
||||
|
||||
if (lock >= 0) {
|
||||
ENV_BUG_ON(lock != OCF_LOCK_ACQUIRED);
|
||||
|
@ -39,7 +39,7 @@ int ocf_metadata_init(struct ocf_cache *cache,
|
||||
return ret;
|
||||
}
|
||||
|
||||
ocf_metadata_concurrency_init(cache);
|
||||
ocf_metadata_concurrency_init(&cache->metadata.lock);
|
||||
|
||||
return 0;
|
||||
}
|
||||
@ -73,7 +73,7 @@ void ocf_metadata_deinit(struct ocf_cache *cache)
|
||||
cache->metadata.iface.deinit(cache);
|
||||
}
|
||||
|
||||
ocf_metadata_concurrency_deinit(cache);
|
||||
ocf_metadata_concurrency_deinit(&cache->metadata.lock);
|
||||
|
||||
ocf_metadata_io_deinit(cache);
|
||||
}
|
||||
@ -113,17 +113,17 @@ ocf_cache_line_t ocf_metadata_get_cachelines_count(ocf_cache_t cache)
|
||||
void ocf_metadata_flush_all(ocf_cache_t cache,
|
||||
ocf_metadata_end_t cmpl, void *priv)
|
||||
{
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
cache->metadata.iface.flush_all(cache, cmpl, priv);
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
}
|
||||
|
||||
void ocf_metadata_load_all(ocf_cache_t cache,
|
||||
ocf_metadata_end_t cmpl, void *priv)
|
||||
{
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
cache->metadata.iface.load_all(cache, cmpl, priv);
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
}
|
||||
|
||||
void ocf_metadata_load_recovery(ocf_cache_t cache,
|
||||
|
@ -1903,7 +1903,7 @@ static void _recovery_rebuild_metadata(ocf_pipeline_t pipeline,
|
||||
const uint64_t collision_table_entries =
|
||||
ocf_metadata_collision_table_entries(cache);
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
for (cline = 0; cline < collision_table_entries; cline++) {
|
||||
ocf_metadata_get_core_info(cache, cline, &core_id, &core_line);
|
||||
@ -1923,7 +1923,7 @@ static void _recovery_rebuild_metadata(ocf_pipeline_t pipeline,
|
||||
OCF_COND_RESCHED(step, 128);
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
ocf_pipeline_next(pipeline);
|
||||
}
|
||||
|
@ -226,9 +226,9 @@ static int ocf_restart_meta_io(struct ocf_request *req)
|
||||
int ret;
|
||||
|
||||
/* Fill with the latest metadata. */
|
||||
OCF_METADATA_LOCK_RD();
|
||||
/* TODO: synchronize with concurrent metadata io and hash bucket locks
|
||||
*/
|
||||
metadata_io_req_fill(meta_io_req);
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
|
||||
io = ocf_new_cache_io(cache, req->io_queue,
|
||||
PAGES_TO_BYTES(meta_io_req->page),
|
||||
|
@ -428,6 +428,16 @@ struct ocf_cache_line_settings {
|
||||
uint64_t sector_end;
|
||||
};
|
||||
|
||||
struct ocf_metadata_lock
|
||||
{
|
||||
env_rwsem global; /*!< global metadata lock (GML) */
|
||||
env_rwlock status; /*!< Fast lock for status bits */
|
||||
env_spinlock eviction; /*!< Fast lock for eviction policy */
|
||||
env_rwsem *hash; /*!< Hash bucket locks */
|
||||
uint32_t num_hash_entries; /*!< Hash bucket count */
|
||||
ocf_cache_t cache; /*!< Parent cache object */
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Metadata control structure
|
||||
*/
|
||||
@ -444,11 +454,7 @@ struct ocf_metadata {
|
||||
bool is_volatile;
|
||||
/*!< true if metadata used in volatile mode (RAM only) */
|
||||
|
||||
struct {
|
||||
env_rwsem collision; /*!< lock for collision table */
|
||||
env_rwlock status; /*!< Fast lock for status bits */
|
||||
env_spinlock eviction; /*!< Fast lock for eviction policy */
|
||||
} lock;
|
||||
struct ocf_metadata_lock lock;
|
||||
};
|
||||
|
||||
#endif /* __METADATA_STRUCTS_H__ */
|
||||
|
@ -292,7 +292,6 @@ static ocf_error_t init_attached_data_structures(ocf_cache_t cache,
|
||||
ocf_error_t result;
|
||||
|
||||
/* Lock to ensure consistency */
|
||||
OCF_METADATA_LOCK_WR();
|
||||
|
||||
ocf_metadata_init_hash_table(cache);
|
||||
ocf_metadata_init_collision(cache);
|
||||
@ -303,7 +302,6 @@ static ocf_error_t init_attached_data_structures(ocf_cache_t cache,
|
||||
if (result) {
|
||||
ocf_cache_log(cache, log_err,
|
||||
"Cannot initialize cleaning policy\n");
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
return result;
|
||||
}
|
||||
|
||||
@ -313,24 +311,19 @@ static ocf_error_t init_attached_data_structures(ocf_cache_t cache,
|
||||
ocf_cache_log(cache, log_err,
|
||||
"Cannot initialize promotion policy\n");
|
||||
__deinit_cleaning_policy(cache);
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
return result;
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
static void init_attached_data_structures_recovery(ocf_cache_t cache)
|
||||
{
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_init_hash_table(cache);
|
||||
ocf_metadata_init_collision(cache);
|
||||
__init_partitions_attached(cache);
|
||||
__reset_stats(cache);
|
||||
__init_metadata_version(cache);
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
}
|
||||
|
||||
/****************************************************************
|
||||
@ -986,7 +979,8 @@ static void _ocf_mngt_attach_prepare_metadata(ocf_pipeline_t pipeline,
|
||||
|
||||
context->flags.attached_metadata_inited = true;
|
||||
|
||||
if (ocf_metadata_concurrency_attached_init(cache)) {
|
||||
if (ocf_metadata_concurrency_attached_init(&cache->metadata.lock,
|
||||
cache, cache->device->hash_table_entries)) {
|
||||
ocf_cache_log(cache, log_err, "Failed to initialize attached "
|
||||
"metadata concurrency\n");
|
||||
OCF_PL_FINISH_RET(context->pipeline, -OCF_ERR_START_CACHE_FAIL);
|
||||
@ -1734,6 +1728,7 @@ static void _ocf_mngt_cache_unplug_complete(void *priv, int error)
|
||||
|
||||
ocf_volume_close(&cache->device->volume);
|
||||
|
||||
ocf_metadata_concurrency_attached_deinit(&cache->metadata.lock);
|
||||
ocf_metadata_deinit_variable_size(cache);
|
||||
ocf_concurrency_deinit(cache);
|
||||
ocf_freelist_deinit(cache->freelist);
|
||||
@ -2240,11 +2235,11 @@ int ocf_mngt_cache_promotion_set_policy(ocf_cache_t cache, ocf_promotion_t type)
|
||||
{
|
||||
int result;
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
result = ocf_promotion_set_policy(cache->promotion_policy, type);
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
return result;
|
||||
}
|
||||
@ -2253,11 +2248,11 @@ ocf_promotion_t ocf_mngt_cache_promotion_get_policy(ocf_cache_t cache)
|
||||
{
|
||||
ocf_promotion_t result;
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_metadata_start_shared_access(&cache->metadata.lock);
|
||||
|
||||
result = cache->conf_meta->promotion_policy_type;
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_metadata_end_shared_access(&cache->metadata.lock);
|
||||
|
||||
return result;
|
||||
}
|
||||
@ -2267,12 +2262,12 @@ int ocf_mngt_cache_promotion_get_param(ocf_cache_t cache, uint8_t param_id,
|
||||
{
|
||||
int result;
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_metadata_start_shared_access(&cache->metadata.lock);
|
||||
|
||||
result = ocf_promotion_get_param(cache->promotion_policy, param_id,
|
||||
param_value);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_metadata_end_shared_access(&cache->metadata.lock);
|
||||
|
||||
return result;
|
||||
}
|
||||
@ -2282,12 +2277,12 @@ int ocf_mngt_cache_promotion_set_param(ocf_cache_t cache, uint8_t param_id,
|
||||
{
|
||||
int result;
|
||||
|
||||
OCF_METADATA_LOCK_RD();
|
||||
ocf_metadata_start_shared_access(&cache->metadata.lock);
|
||||
|
||||
result = ocf_promotion_set_param(cache->promotion_policy, param_id,
|
||||
param_value);
|
||||
|
||||
OCF_METADATA_UNLOCK_RD();
|
||||
ocf_metadata_end_shared_access(&cache->metadata.lock);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ void cache_mngt_core_remove_from_cleaning_pol(ocf_core_t core)
|
||||
ocf_core_id_t core_id = ocf_core_get_id(core);
|
||||
ocf_cleaning_t clean_pol_type;
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
clean_pol_type = cache->conf_meta->cleaning_policy_type;
|
||||
if (cache->core[core_id].opened) {
|
||||
@ -49,7 +49,7 @@ void cache_mngt_core_remove_from_cleaning_pol(ocf_core_t core)
|
||||
}
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
}
|
||||
|
||||
/* Deinitialize core metadata in attached metadata */
|
||||
@ -65,7 +65,7 @@ void cache_mngt_core_deinit_attached_meta(ocf_core_t core)
|
||||
if (!core_size)
|
||||
core_size = ~0ULL;
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
clean_pol_type = cache->conf_meta->cleaning_policy_type;
|
||||
while (retry) {
|
||||
@ -82,13 +82,14 @@ void cache_mngt_core_deinit_attached_meta(ocf_core_t core)
|
||||
}
|
||||
|
||||
if (retry) {
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
env_msleep(100);
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(
|
||||
&cache->metadata.lock);
|
||||
}
|
||||
}
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
}
|
||||
|
||||
/* Mark core as removed in metadata */
|
||||
@ -96,7 +97,7 @@ void cache_mngt_core_remove_from_meta(ocf_core_t core)
|
||||
{
|
||||
ocf_cache_t cache = ocf_core_get_cache(core);
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
/* In metadata mark data this core was removed from cache */
|
||||
core->conf_meta->valid = false;
|
||||
@ -105,7 +106,7 @@ void cache_mngt_core_remove_from_meta(ocf_core_t core)
|
||||
ocf_mngt_core_clear_uuid_metadata(core);
|
||||
core->conf_meta->seq_no = OCF_SEQ_NO_INVALID;
|
||||
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
}
|
||||
|
||||
/* Deinit in-memory structures related to this core */
|
||||
|
@ -385,9 +385,9 @@ static int _ofc_flush_container_step(struct ocf_request *req)
|
||||
struct flush_container *fc = req->priv;
|
||||
ocf_cache_t cache = fc->cache;
|
||||
|
||||
ocf_metadata_lock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
_ocf_mngt_flush_portion(fc);
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
return 0;
|
||||
}
|
||||
@ -501,7 +501,7 @@ static void _ocf_mngt_flush_core(
|
||||
return;
|
||||
}
|
||||
|
||||
ocf_metadata_lock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
ret = _ocf_mngt_get_sectors(cache, core_id,
|
||||
&fc->flush_data, &fc->count);
|
||||
@ -509,7 +509,7 @@ static void _ocf_mngt_flush_core(
|
||||
ocf_core_log(core, log_err, "Flushing operation aborted, "
|
||||
"no memory\n");
|
||||
env_vfree(fc);
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
complete(context, -OCF_ERR_NO_MEM);
|
||||
return;
|
||||
}
|
||||
@ -519,7 +519,7 @@ static void _ocf_mngt_flush_core(
|
||||
|
||||
_ocf_mngt_flush_containers(context, fc, 1, complete);
|
||||
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
}
|
||||
|
||||
static void _ocf_mngt_flush_all_cores(
|
||||
@ -538,21 +538,21 @@ static void _ocf_mngt_flush_all_cores(
|
||||
|
||||
env_atomic_set(&cache->flush_in_progress, 1);
|
||||
|
||||
ocf_metadata_lock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
/* Get all 'dirty' sectors for all cores */
|
||||
ret = _ocf_mngt_get_flush_containers(cache, &fctbl, &fcnum);
|
||||
if (ret) {
|
||||
ocf_cache_log(cache, log_err, "Flushing operation aborted, "
|
||||
"no memory\n");
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
complete(context, ret);
|
||||
return;
|
||||
}
|
||||
|
||||
_ocf_mngt_flush_containers(context, fctbl, fcnum, complete);
|
||||
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
}
|
||||
|
||||
static void _ocf_mngt_flush_all_cores_complete(
|
||||
@ -774,10 +774,10 @@ static void _ocf_mngt_cache_invalidate(ocf_pipeline_t pipeline, void *priv,
|
||||
ocf_cache_t cache = context->cache;
|
||||
int result;
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
result = ocf_metadata_sparse_range(cache, context->purge.core_id, 0,
|
||||
context->purge.end_byte);
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
OCF_PL_NEXT_ON_SUCCESS_RET(context->pipeline, result);
|
||||
}
|
||||
@ -907,7 +907,7 @@ int ocf_mngt_cache_cleaning_set_policy(ocf_cache_t cache, ocf_cleaning_t type)
|
||||
return 0;
|
||||
}
|
||||
|
||||
ocf_metadata_lock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
if (cleaning_policy_ops[old_type].deinitialize)
|
||||
cleaning_policy_ops[old_type].deinitialize(cache);
|
||||
@ -925,7 +925,7 @@ int ocf_mngt_cache_cleaning_set_policy(ocf_cache_t cache, ocf_cleaning_t type)
|
||||
|
||||
cache->conf_meta->cleaning_policy_type = type;
|
||||
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
ocf_cache_log(cache, log_info, "Changing cleaning policy from "
|
||||
"%s to %s\n", cleaning_policy_ops[old_type].name,
|
||||
@ -957,12 +957,12 @@ int ocf_mngt_cache_cleaning_set_param(ocf_cache_t cache, ocf_cleaning_t type,
|
||||
if (!cleaning_policy_ops[type].set_cleaning_param)
|
||||
return -OCF_ERR_INVAL;
|
||||
|
||||
ocf_metadata_lock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
ret = cleaning_policy_ops[type].set_cleaning_param(cache,
|
||||
param_id, param_value);
|
||||
|
||||
ocf_metadata_unlock(cache, OCF_METADATA_WR);
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
@ -275,7 +275,7 @@ int ocf_mngt_cache_io_classes_configure(ocf_cache_t cache,
|
||||
if (!old_config)
|
||||
return -OCF_ERR_NO_MEM;
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
result = env_memcpy(old_config, sizeof(cache->user_parts),
|
||||
cache->user_parts, sizeof(cache->user_parts));
|
||||
@ -300,7 +300,7 @@ out_edit:
|
||||
}
|
||||
|
||||
out_cpy:
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
env_free(old_config);
|
||||
|
||||
return result;
|
||||
|
@ -311,3 +311,14 @@ void ocf_req_clear_map(struct ocf_request *req)
|
||||
ENV_BUG_ON(env_memset(req->map,
|
||||
sizeof(req->map[0]) * req->core_line_count, 0));
|
||||
}
|
||||
|
||||
void ocf_req_hash(struct ocf_request *req)
|
||||
{
|
||||
int i;
|
||||
|
||||
for (i = 0; i < req->core_line_count; i++) {
|
||||
req->map[i].hash = ocf_metadata_hash_func(req->cache,
|
||||
req->core_line_first + i,
|
||||
ocf_core_get_id(req->core));
|
||||
}
|
||||
}
|
||||
|
@ -319,6 +319,13 @@ void ocf_req_clear_info(struct ocf_request *req);
|
||||
*/
|
||||
void ocf_req_clear_map(struct ocf_request *req);
|
||||
|
||||
/**
|
||||
* @brief Calculate hashes for all core lines within the request
|
||||
*
|
||||
* @param req - OCF request
|
||||
*/
|
||||
void ocf_req_hash(struct ocf_request *req);
|
||||
|
||||
/**
|
||||
* @brief Clear OCF request
|
||||
*
|
||||
|
@ -197,6 +197,12 @@ static void _ocf_cleaner_complete_req(struct ocf_request *req)
|
||||
cmpl(master->priv, master->error);
|
||||
}
|
||||
|
||||
static void _ocf_cleaner_on_resume(struct ocf_request *req)
|
||||
{
|
||||
OCF_DEBUG_TRACE(req->cache);
|
||||
ocf_engine_push_req_front(req, true);
|
||||
}
|
||||
|
||||
/*
|
||||
* cleaner - Cache line lock, function lock cache lines depends on attributes
|
||||
*/
|
||||
@ -207,7 +213,7 @@ static int _ocf_cleaner_cache_line_lock(struct ocf_request *req)
|
||||
|
||||
OCF_DEBUG_TRACE(req->cache);
|
||||
|
||||
return ocf_req_trylock_rd(req);
|
||||
return ocf_req_async_lock_rd(req, _ocf_cleaner_on_resume);
|
||||
}
|
||||
|
||||
/*
|
||||
@ -314,7 +320,7 @@ static int _ocf_cleaner_update_metadata(struct ocf_request *req)
|
||||
|
||||
OCF_DEBUG_TRACE(req->cache);
|
||||
|
||||
OCF_METADATA_LOCK_WR();
|
||||
ocf_metadata_start_exclusive_access(&cache->metadata.lock);
|
||||
/* Update metadata */
|
||||
for (i = 0; i < req->core_line_count; i++, iter++) {
|
||||
if (iter->status == LOOKUP_MISS)
|
||||
@ -339,7 +345,7 @@ static int _ocf_cleaner_update_metadata(struct ocf_request *req)
|
||||
}
|
||||
|
||||
ocf_metadata_flush_do_asynch(cache, req, _ocf_cleaner_metadata_io_end);
|
||||
OCF_METADATA_UNLOCK_WR();
|
||||
ocf_metadata_end_exclusive_access(&cache->metadata.lock);
|
||||
|
||||
return 0;
|
||||
}
|
||||
@ -697,16 +703,9 @@ static int _ocf_cleaner_fire_cache(struct ocf_request *req)
|
||||
return 0;
|
||||
}
|
||||
|
||||
static void _ocf_cleaner_on_resume(struct ocf_request *req)
|
||||
{
|
||||
OCF_DEBUG_TRACE(req->cache);
|
||||
ocf_engine_push_req_front(req, true);
|
||||
}
|
||||
|
||||
static const struct ocf_io_if _io_if_fire_cache = {
|
||||
.read = _ocf_cleaner_fire_cache,
|
||||
.write = _ocf_cleaner_fire_cache,
|
||||
.resume = _ocf_cleaner_on_resume,
|
||||
};
|
||||
|
||||
static int _ocf_cleaner_fire(struct ocf_request *req)
|
||||
|
@ -0,0 +1,126 @@
|
||||
/*
|
||||
* <tested_file_path>src/concurrency/ocf_metadata_concurrency.c</tested_file_path>
|
||||
* <tested_function>ocf_req_hash_lock_rd</tested_function>
|
||||
* <functions_to_leave>
|
||||
* INSERT HERE LIST OF FUNCTIONS YOU WANT TO LEAVE
|
||||
* ONE FUNCTION PER LINE
|
||||
* </functions_to_leave>
|
||||
*/
|
||||
|
||||
#undef static
|
||||
|
||||
#undef inline
|
||||
|
||||
|
||||
#include <stdarg.h>
|
||||
#include <stddef.h>
|
||||
#include <setjmp.h>
|
||||
#include <cmocka.h>
|
||||
#include "print_desc.h"
|
||||
|
||||
#include "ocf_metadata_concurrency.h"
|
||||
#include "../metadata/metadata_misc.h"
|
||||
|
||||
#include "concurrency/ocf_metadata_concurrency.c/ocf_metadata_concurrency_generated_warps.c"
|
||||
|
||||
void __wrap_ocf_metadata_hash_lock(struct ocf_metadata_lock *metadata_lock,
|
||||
ocf_cache_line_t hash, int rw)
|
||||
{
|
||||
check_expected(hash);
|
||||
function_called();
|
||||
}
|
||||
|
||||
#define MAP_SIZE 16
|
||||
|
||||
static struct ocf_request *alloc_req()
|
||||
{
|
||||
struct ocf_request *req;
|
||||
struct ocf_cache *cache = malloc(sizeof(*cache));
|
||||
|
||||
req = malloc(sizeof(*req) + MAP_SIZE * sizeof(req->map[0]));
|
||||
req->map = req->__map;
|
||||
req->cache = cache;
|
||||
|
||||
return req;
|
||||
}
|
||||
|
||||
static void _test_lock_order(struct ocf_request* req,
|
||||
unsigned hash[], unsigned hash_count,
|
||||
unsigned expected_call[], unsigned expected_call_count)
|
||||
{
|
||||
unsigned i;
|
||||
|
||||
req->core_line_count = hash_count;
|
||||
|
||||
for (i = 0; i < hash_count; i++)
|
||||
req->map[i].hash = hash[i];
|
||||
|
||||
for (i = 0; i < expected_call_count; i++) {
|
||||
expect_function_call(__wrap_ocf_metadata_hash_lock);
|
||||
expect_value(__wrap_ocf_metadata_hash_lock, hash, expected_call[i]);
|
||||
}
|
||||
|
||||
ocf_req_hash_lock_rd(req);
|
||||
|
||||
}
|
||||
|
||||
static void ocf_req_hash_lock_rd_test01(void **state)
|
||||
{
|
||||
struct ocf_request *req = alloc_req();
|
||||
struct {
|
||||
struct {
|
||||
unsigned val[MAP_SIZE];
|
||||
unsigned count;
|
||||
} hash, expected_call;
|
||||
} test_cases[] = {
|
||||
{
|
||||
.hash = {.val = {2}, .count = 1},
|
||||
.expected_call = {.val = {2}, .count = 1}
|
||||
},
|
||||
{
|
||||
.hash = {.val = {2, 3, 4}, .count = 3},
|
||||
.expected_call = {.val = {2, 3, 4}, .count = 3}
|
||||
},
|
||||
{
|
||||
.hash = {.val = {2, 3, 4, 0}, .count = 4},
|
||||
.expected_call = {.val = {0, 2, 3, 4}, .count = 4}
|
||||
},
|
||||
{
|
||||
.hash = {.val = {2, 3, 4, 0, 1, 2, 3, 4, 0, 1}, .count = 10},
|
||||
.expected_call = {.val = {0, 1, 2, 3, 4}, .count = 5}
|
||||
},
|
||||
{
|
||||
.hash = {.val = {4, 0}, .count = 2},
|
||||
.expected_call = {.val = {0, 4}, .count = 2}
|
||||
},
|
||||
{
|
||||
.hash = {.val = {0, 1, 2, 3, 4, 0, 1}, .count = 7},
|
||||
.expected_call = {.val = {0, 1, 2, 3, 4}, .count = 5}
|
||||
},
|
||||
};
|
||||
const unsigned test_case_count = sizeof(test_cases) / sizeof(test_cases[0]);
|
||||
unsigned i;
|
||||
|
||||
req->cache->metadata.lock.num_hash_entries = 5;
|
||||
|
||||
print_test_description("Verify hash locking order\n");
|
||||
|
||||
for (i = 0; i < test_case_count; i++) {
|
||||
_test_lock_order(req, test_cases[i].hash.val, test_cases[i].hash.count,
|
||||
test_cases[i].expected_call.val, test_cases[i].expected_call.count);
|
||||
}
|
||||
|
||||
free(req->cache);
|
||||
free(req);
|
||||
}
|
||||
|
||||
int main(void)
|
||||
{
|
||||
const struct CMUnitTest tests[] = {
|
||||
cmocka_unit_test(ocf_req_hash_lock_rd_test01)
|
||||
};
|
||||
|
||||
print_message("Unit test for ocf_req_hash_lock_rd\n");
|
||||
|
||||
return cmocka_run_group_tests(tests, NULL, NULL);
|
||||
}
|
Loading…
Reference in New Issue
Block a user