Merge pull request #258 from arutk/metadata_lock_per_bucket

Improve metadata lock granularity
This commit is contained in:
Jan Musiał 2019-09-20 15:33:37 +02:00 committed by GitHub
commit ed1ae71992
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 925 additions and 492 deletions

View File

@ -392,7 +392,7 @@ static ocf_cache_line_t _acp_trylock_dirty(struct ocf_cache *cache,
struct ocf_map_info info; struct ocf_map_info info;
bool locked = false; 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, ocf_engine_lookup_map_entry(cache, &info, core_id,
core_line); core_line);
@ -403,7 +403,7 @@ static ocf_cache_line_t _acp_trylock_dirty(struct ocf_cache *cache,
locked = true; 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; return locked ? info.coll_idx : cache->device->collision_table_entries;
} }

View File

@ -779,7 +779,7 @@ static void alru_clean(struct alru_flush_ctx *fctx)
return; return;
} }
if (OCF_METADATA_LOCK_WR_TRY()) { if (ocf_metadata_try_start_exclusive_access(&cache->metadata.lock)) {
alru_clean_complete(fctx, 0); alru_clean_complete(fctx, 0);
return; return;
} }
@ -797,7 +797,7 @@ static void alru_clean(struct alru_flush_ctx *fctx)
fctx->flush_perfomed = true; fctx->flush_perfomed = true;
ocf_cleaner_do_flush_data_async(cache, fctx->flush_data, to_clean, ocf_cleaner_do_flush_data_async(cache, fctx->flush_data, to_clean,
&fctx->attribs); &fctx->attribs);
OCF_METADATA_UNLOCK_WR(); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
return; return;
} }
@ -806,7 +806,7 @@ static void alru_clean(struct alru_flush_ctx *fctx)
env_ticks_to_secs(env_get_tick_count()); env_ticks_to_secs(env_get_tick_count());
end: end:
OCF_METADATA_UNLOCK_WR(); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
alru_clean_complete(fctx, 0); alru_clean_complete(fctx, 0);
} }

View File

@ -35,14 +35,11 @@
#define _WAITERS_LIST_ITEM(cache_line) ((cache_line) % _WAITERS_LIST_ENTRIES) #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 { struct __waiter {
ocf_cache_line_t line; ocf_cache_line_t line;
void *ctx; void *ctx;
uint32_t ctx_id; uint32_t ctx_id;
__on_lock on_lock; ocf_req_async_lock_cb cb;
struct list_head item; struct list_head item;
int rw; int rw;
}; };
@ -53,7 +50,6 @@ struct __waiters_list {
}; };
struct ocf_cache_line_concurrency { struct ocf_cache_line_concurrency {
env_rwlock lock;
env_atomic *access; env_atomic *access;
env_atomic waiting; env_atomic waiting;
size_t access_limit; 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_spinlock_init(&c->waiters_lsts[i].lock);
} }
env_rwlock_init(&c->lock);
return 0; return 0;
ocf_cache_line_concurrency_init: 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; concurrency = cache->device->concurrency.cache_line;
env_rwlock_destroy(&concurrency->lock);
for (i = 0; i < _WAITERS_LIST_ENTRIES; i++) for (i = 0; i < _WAITERS_LIST_ENTRIES; i++)
env_spinlock_destroy(&concurrency->waiters_lsts[i].lock); 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; 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, 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) void *ctx, uint32_t ctx_id)
{ {
struct __waiter *waiter; 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)) { if (__try_lock_wr(c, line)) {
/* No activity before look get */ /* No activity before look get */
if (on_lock) if (cb)
on_lock(ctx, ctx_id, line, OCF_WRITE); _req_on_lock(ctx, cb, ctx_id, line, OCF_WRITE);
return true; return true;
} }
@ -382,7 +400,7 @@ static inline bool __lock_cache_line_wr(struct ocf_cache_line_concurrency *c,
locked = true; locked = true;
} else { } else {
waiter = NULL; waiter = NULL;
if (on_lock != NULL) { if (cb != NULL) {
/* Need to create waiters and add it into list */ /* Need to create waiters and add it into list */
waiter = env_allocator_new(c->allocator); 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->line = line;
waiter->ctx = ctx; waiter->ctx = ctx;
waiter->ctx_id = ctx_id; waiter->ctx_id = ctx_id;
waiter->on_lock = on_lock; waiter->cb = cb;
waiter->rw = OCF_WRITE; waiter->rw = OCF_WRITE;
INIT_LIST_HEAD(&waiter->item); 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); __unlock_waiters_list(c, line, flags);
if (locked && on_lock) if (locked && cb)
on_lock(ctx, ctx_id, line, OCF_WRITE); _req_on_lock(ctx, cb, ctx_id, line, OCF_WRITE);
return locked || waiting; 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. * 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, 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) void *ctx, uint32_t ctx_id)
{ {
struct __waiter *waiter; 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)) { if (__try_lock_rd_idle(c, line)) {
/* No activity before look get, it is first reader */ /* No activity before look get, it is first reader */
if (on_lock) if (cb)
on_lock(ctx, ctx_id, line, OCF_READ); _req_on_lock(ctx, cb, ctx_id, line, OCF_READ);
return true; return true;
} }
@ -444,7 +462,7 @@ static inline bool __lock_cache_line_rd(struct ocf_cache_line_concurrency *c,
if (!locked) { if (!locked) {
waiter = NULL; waiter = NULL;
if (on_lock) { if (cb) {
/* Need to create waiters and add it into list */ /* Need to create waiters and add it into list */
waiter = env_allocator_new(c->allocator); 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->line = line;
waiter->ctx = ctx; waiter->ctx = ctx;
waiter->ctx_id = ctx_id; waiter->ctx_id = ctx_id;
waiter->on_lock = on_lock; waiter->cb = cb;
waiter->rw = OCF_READ; waiter->rw = OCF_READ;
INIT_LIST_HEAD(&waiter->item); 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); __unlock_waiters_list(c, line, flags);
if (locked && on_lock) if (locked && cb)
on_lock(ctx, ctx_id, line, OCF_READ); _req_on_lock(ctx, cb, ctx_id, line, OCF_READ);
return locked || waiting; return locked || waiting;
} }
@ -520,8 +538,8 @@ static inline void __unlock_cache_line_rd_common(struct ocf_cache_line_concurren
exchanged = false; exchanged = false;
list_del(iter); list_del(iter);
waiter->on_lock(waiter->ctx, waiter->ctx_id, line, _req_on_lock(waiter->ctx, waiter->cb, waiter->ctx_id,
waiter->rw); line, waiter->rw);
env_allocator_del(c->allocator, waiter); env_allocator_del(c->allocator, waiter);
} else { } else {
@ -601,7 +619,7 @@ static inline void __unlock_cache_line_wr_common(struct ocf_cache_line_concurren
exchanged = false; exchanged = false;
list_del(iter); 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); waiter->rw);
env_allocator_del(c->allocator, waiter); 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); __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 /* Try to read-lock request without adding waiters. Function should be called
* under read lock, multiple threads may attempt to acquire the lock * under read lock, multiple threads may attempt to acquire the lock
* concurrently. */ * concurrently. */
static int _ocf_req_trylock_rd(struct ocf_request *req) int ocf_req_trylock_rd(struct ocf_request *req)
{ {
int32_t i; int32_t i;
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency. 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 * Asynchronously read-lock request cache lines. Must be called under cacheline
* write lock. * 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; int32_t i;
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency. struct ocf_cache_line_concurrency *c = req->cache->device->concurrency.
cache_line; cache_line;
ocf_cache_line_t line; ocf_cache_line_t line;
__on_lock on_lock = _req_on_lock;
int ret = OCF_LOCK_NOT_ACQUIRED; int ret = OCF_LOCK_NOT_ACQUIRED;
ENV_BUG_ON(env_atomic_read(&req->lock_remaining)); 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].rd_locked);
ENV_BUG_ON(req->map[i].wr_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 */ /* lock not acquired and not added to wait list */
ret = -OCF_ERR_NO_MEM; ret = -OCF_ERR_NO_MEM;
goto err; 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 /* Try to write-lock request without adding waiters. Function should be called
* under read lock, multiple threads may attempt to acquire the lock * under read lock, multiple threads may attempt to acquire the lock
* concurrently. */ * concurrently. */
static int _ocf_req_trylock_wr(struct ocf_request *req) int ocf_req_trylock_wr(struct ocf_request *req)
{ {
int32_t i; int32_t i;
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency. 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 * Asynchronously write-lock request cache lines. Must be called under cacheline
* write lock. * 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; int32_t i;
struct ocf_cache_line_concurrency *c = req->cache->device->concurrency. struct ocf_cache_line_concurrency *c = req->cache->device->concurrency.
cache_line; cache_line;
ocf_cache_line_t line; ocf_cache_line_t line;
__on_lock on_lock = _req_on_lock;
int ret = OCF_LOCK_NOT_ACQUIRED; int ret = OCF_LOCK_NOT_ACQUIRED;
ENV_BUG_ON(env_atomic_read(&req->lock_remaining)); 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_inc(&c->waiting);
env_atomic_set(&req->lock_remaining, req->core_line_count); 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].rd_locked);
ENV_BUG_ON(req->map[i].wr_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 */ /* lock not acquired and not added to wait list */
ret = -OCF_ERR_NO_MEM; ret = -OCF_ERR_NO_MEM;
goto err; goto err;
@ -932,26 +904,6 @@ err:
return ret; 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;
}
/* /*
* *
*/ */

View File

@ -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); 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 * @param req - OCF request
* *
* @returns lock acquisition status
* @retval OCF_LOCK_ACQUIRED - OCF request has been locked and can be processed * @retval OCF_LOCK_ACQUIRED - OCF request has been locked and can be processed
* * @retval OCF_LOCK_NOT_ACQUIRED - OCF request lock not acquired
* @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
*/ */
int ocf_req_trylock_wr(struct ocf_request *req); 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 * @param req - OCF request
* *
* @returns lock acquisition status
* @retval OCF_LOCK_ACQUIRED - OCF request has been locked and can be processed * @retval OCF_LOCK_ACQUIRED - OCF request has been locked and can be processed
* * @retval OCF_LOCK_NOT_ACQUIRED - OCF request lock not acquired
* @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
*/ */
int ocf_req_trylock_rd(struct ocf_request *req); int ocf_req_trylock_rd(struct ocf_request *req);

View File

@ -4,22 +4,265 @@
*/ */
#include "ocf_metadata_concurrency.h" #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_spinlock_init(&metadata_lock->eviction);
env_rwlock_init(&cache->metadata.lock.status); env_rwlock_init(&metadata_lock->status);
env_rwsem_init(&cache->metadata.lock.collision); 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_spinlock_destroy(&metadata_lock->eviction);
env_rwlock_destroy(&cache->metadata.lock.status); env_rwlock_destroy(&metadata_lock->status);
env_rwsem_destroy(&cache->metadata.lock.collision); 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; 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);
}

View File

@ -10,125 +10,105 @@
#define OCF_METADATA_RD 0 #define OCF_METADATA_RD 0
#define OCF_METADATA_WR 1 #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() \ #define OCF_METADATA_EVICTION_LOCK() \
ocf_metadata_eviction_lock(cache) ocf_metadata_eviction_lock(&cache->metadata.lock)
#define OCF_METADATA_EVICTION_UNLOCK() \ #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) void ocf_metadata_start_exclusive_access(
{ struct ocf_metadata_lock *metadata_lock);
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();
}
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) void ocf_metadata_end_exclusive_access(
{ struct ocf_metadata_lock *metadata_lock);
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();
}
static inline int ocf_metadata_try_lock(struct ocf_cache *cache, int rw) int ocf_metadata_try_start_shared_access(
{ struct ocf_metadata_lock *metadata_lock);
int result = 0;
if (rw == OCF_METADATA_WR) { void ocf_metadata_start_shared_access(
result = env_rwsem_down_write_trylock( struct ocf_metadata_lock *metadata_lock);
&cache->metadata.lock.collision);
} else if (rw == OCF_METADATA_RD) {
result = env_rwsem_down_read_trylock(
&cache->metadata.lock.collision);
} else {
ENV_BUG();
}
if (result) void ocf_metadata_end_shared_access(
return -1; struct ocf_metadata_lock *metadata_lock);
return 0;
}
static inline void ocf_metadata_status_bits_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) 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) else if (rw == OCF_METADATA_RD)
env_rwlock_read_lock(&cache->metadata.lock.status); env_rwlock_read_lock(&metadata_lock->status);
else else
ENV_BUG(); ENV_BUG();
} }
static inline void ocf_metadata_status_bits_unlock( 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) 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) else if (rw == OCF_METADATA_RD)
env_rwlock_read_unlock(&cache->metadata.lock.status); env_rwlock_read_unlock(&metadata_lock->status);
else else
ENV_BUG(); 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() \ #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() \ #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() \ #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() \ #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() \ void ocf_metadata_hash_lock_rd(struct ocf_metadata_lock *metadata_lock,
ocf_metadata_flush_lock(cache) 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() \ /* lock entire request in deadlock-free manner */
ocf_metadata_flush_unlock(cache) 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 #endif

View File

@ -37,8 +37,6 @@ struct ocf_io_if {
int (*write)(struct ocf_request *req); int (*write)(struct ocf_request *req);
void (*resume)(struct ocf_request *req);
const char *name; const char *name;
}; };

View File

@ -17,6 +17,7 @@
#include "../metadata/metadata.h" #include "../metadata/metadata.h"
#include "../eviction/eviction.h" #include "../eviction/eviction.h"
#include "../promotion/promotion.h" #include "../promotion/promotion.h"
#include "../concurrency/ocf_concurrency.h"
void ocf_engine_error(struct ocf_request *req, void ocf_engine_error(struct ocf_request *req,
bool stop_cache, const char *msg) 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; struct ocf_cache *cache = req->cache;
uint32_t i; uint32_t i;
@ -315,18 +316,15 @@ void ocf_engine_map(struct ocf_request *req)
int status = LOOKUP_MAPPED; int status = LOOKUP_MAPPED;
ocf_core_id_t core_id = ocf_core_get_id(req->core); 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; req->info.mapping_error = 1;
return; 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); ocf_req_clear_info(req);
req->info.seq_req = true; 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, static int _ocf_engine_clean_getter(struct ocf_cache *cache,
void *getter_context, uint32_t item, ocf_cache_line_t *line) 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) static int _ocf_engine_refresh(struct ocf_request *req)
{ {
struct ocf_cache *cache = req->cache;
int result; int result;
OCF_METADATA_LOCK_RD();
/* Check under metadata RD lock */ /* Check under metadata RD lock */
ocf_req_hash_lock_rd(req);
result = ocf_engine_check(req); result = ocf_engine_check(req);
OCF_METADATA_UNLOCK_RD(); ocf_req_hash_unlock_rd(req);
if (result == 0) { if (result == 0) {

View File

@ -162,19 +162,48 @@ void ocf_engine_lookup_map_entry(struct ocf_cache *cache,
uint64_t core_line); uint64_t core_line);
/** /**
* @brief Traverse request in order to lookup cache lines If there are misses * @brief Request cacheline lock type
* need to call eviction. This process is called 'mapping'. */
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 * @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) * @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. * performed. Main purpose of this function is to check if there is a HIT.
* *
* @param req OCF request * @param req OCF request

View File

@ -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_step(struct ocf_request *req);
static int _ocf_discard_flush_cache(struct ocf_request *req); static int _ocf_discard_flush_cache(struct ocf_request *req);
static int _ocf_discard_core(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 = { static const struct ocf_io_if _io_if_discard_step = {
.read = _ocf_discard_step, .read = _ocf_discard_step,
.write = _ocf_discard_step, .write = _ocf_discard_step,
.resume = _ocf_discard_on_resume,
}; };
static const struct ocf_io_if _io_if_discard_step_resume = { static const struct ocf_io_if _io_if_discard_step_resume = {
.read = _ocf_discard_step_do, .read = _ocf_discard_step_do,
.write = _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 = { static const struct ocf_io_if _io_if_discard_flush_cache = {
.read = _ocf_discard_flush_cache, .read = _ocf_discard_flush_cache,
.write = _ocf_discard_flush_cache, .write = _ocf_discard_flush_cache,
.resume = _ocf_discard_on_resume,
}; };
static const struct ocf_io_if _io_if_discard_core = { static const struct ocf_io_if _io_if_discard_core = {
.read = _ocf_discard_core, .read = _ocf_discard_core,
.write = _ocf_discard_core, .write = _ocf_discard_core,
.resume = _ocf_discard_on_resume,
}; };
static void _ocf_discard_complete_req(struct ocf_request *req, int error) 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)) { if (ocf_engine_mapped_count(req)) {
/* There are mapped cache line, need to remove them */ /* 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 */ /* Remove mapped cache lines from metadata */
ocf_purge_map_info(req); ocf_purge_map_info(req);
@ -186,16 +181,16 @@ int _ocf_discard_step_do(struct ocf_request *req)
_ocf_discard_step_complete); _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 /* Even if no cachelines are mapped they could be tracked in promotion
* policy. RD lock suffices. */ * policy. RD lock suffices. */
ocf_promotion_req_purge(req->cache->promotion_policy, req); 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_DEBUG_RQ(req, "Discard");
_ocf_discard_step_complete(req, 0); _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->core_line_count = req->core_line_last - req->core_line_first + 1;
req->io_if = &_io_if_discard_step_resume; 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, ENV_BUG_ON(env_memset(req->map, sizeof(*req->map) * req->core_line_count,
0)); 0));
ocf_req_hash(req);
ocf_req_hash_lock_rd(req);
/* Travers to check if request is mapped fully */ /* Travers to check if request is mapped fully */
ocf_engine_traverse(req); ocf_engine_traverse(req);
@ -244,7 +240,13 @@ static int _ocf_discard_step(struct ocf_request *req)
lock = OCF_LOCK_ACQUIRED; 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 (lock >= 0) {
if (OCF_LOCK_ACQUIRED == lock) { if (OCF_LOCK_ACQUIRED == lock) {

View File

@ -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) static int _ocf_read_fast_do(struct ocf_request *req)
{ {
struct ocf_cache *cache = req->cache;
if (ocf_engine_is_miss(req)) { if (ocf_engine_is_miss(req)) {
/* It seams that after resume, now request is MISS, do PT */ /* It seams that after resume, now request is MISS, do PT */
OCF_DEBUG_RQ(req, "Switching to read 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) { if (req->info.re_part) {
OCF_DEBUG_RQ(req, "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 /* Probably some cache lines are assigned into wrong
* partition. Need to move it to new one * partition. Need to move it to new one
*/ */
ocf_part_move(req); ocf_part_move(req);
OCF_METADATA_UNLOCK_WR(); ocf_req_hash_unlock_wr(req);
} }
/* Submit IO */ /* 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 = { static const struct ocf_io_if _io_if_read_fast_resume = {
.read = _ocf_read_fast_do, .read = _ocf_read_fast_do,
.write = _ocf_read_fast_do, .write = _ocf_read_fast_do,
.resume = ocf_engine_on_resume,
}; };
int ocf_read_fast(struct ocf_request *req) int ocf_read_fast(struct ocf_request *req)
{ {
bool hit; bool hit;
int lock = OCF_LOCK_NOT_ACQUIRED; int lock = OCF_LOCK_NOT_ACQUIRED;
struct ocf_cache *cache = req->cache;
/* Get OCF request - increase reference counter */ /* Get OCF request - increase reference counter */
ocf_req_get(req); ocf_req_get(req);
@ -121,7 +117,8 @@ int ocf_read_fast(struct ocf_request *req)
/*- Metadata RD access -----------------------------------------------*/ /*- 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 */ /* Traverse request to cache if there is hit */
ocf_engine_traverse(req); ocf_engine_traverse(req);
@ -129,10 +126,10 @@ int ocf_read_fast(struct ocf_request *req)
hit = ocf_engine_is_hit(req); hit = ocf_engine_is_hit(req);
if (hit) { if (hit) {
ocf_io_start(&req->ioi.io); 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) { if (hit) {
OCF_DEBUG_RQ(req, "Fast path success"); 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 = { static const struct ocf_io_if _io_if_write_fast_resume = {
.read = ocf_write_wb_do, .read = ocf_write_wb_do,
.write = ocf_write_wb_do, .write = ocf_write_wb_do,
.resume = ocf_engine_on_resume,
}; };
int ocf_write_fast(struct ocf_request *req) int ocf_write_fast(struct ocf_request *req)
{ {
bool mapped; bool mapped;
int lock = OCF_LOCK_NOT_ACQUIRED; int lock = OCF_LOCK_NOT_ACQUIRED;
struct ocf_cache *cache = req->cache;
/* Get OCF request - increase reference counter */ /* Get OCF request - increase reference counter */
ocf_req_get(req); ocf_req_get(req);
@ -191,7 +186,8 @@ int ocf_write_fast(struct ocf_request *req)
/*- Metadata RD access -----------------------------------------------*/ /*- 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 */ /* Traverse request to cache if there is hit */
ocf_engine_traverse(req); ocf_engine_traverse(req);
@ -200,9 +196,16 @@ int ocf_write_fast(struct ocf_request *req)
if (mapped) { if (mapped) {
ocf_io_start(&req->ioi.io); ocf_io_start(&req->ioi.io);
lock = ocf_req_trylock_wr(req); 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 (mapped) {
if (lock >= 0) { if (lock >= 0) {

View File

@ -43,9 +43,9 @@ static int _ocf_invalidate_do(struct ocf_request *req)
ENV_BUG_ON(env_atomic_read(&req->req_remaining)); 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_purge_map_info(req);
OCF_METADATA_UNLOCK_WR(); ocf_req_hash_unlock_wr(req);
env_atomic_inc(&req->req_remaining); env_atomic_inc(&req->req_remaining);

View File

@ -52,16 +52,14 @@ static inline void _ocf_read_pt_submit(struct ocf_request *req)
int ocf_read_pt_do(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 */ /* Get OCF request - increase reference counter */
ocf_req_get(req); ocf_req_get(req);
if (req->info.dirty_any) { if (req->info.dirty_any) {
OCF_METADATA_LOCK_RD(); ocf_req_hash_lock_rd(req);
/* Need to clean, start it */ /* Need to clean, start it */
ocf_engine_clean(req); ocf_engine_clean(req);
OCF_METADATA_UNLOCK_RD(); ocf_req_hash_unlock_rd(req);
/* Do not processing, because first we need to clean request */ /* Do not processing, because first we need to clean request */
ocf_req_put(req); ocf_req_put(req);
@ -72,14 +70,14 @@ int ocf_read_pt_do(struct ocf_request *req)
if (req->info.re_part) { if (req->info.re_part) {
OCF_DEBUG_RQ(req, "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 /* Probably some cache lines are assigned into wrong
* partition. Need to move it to new one * partition. Need to move it to new one
*/ */
ocf_part_move(req); ocf_part_move(req);
OCF_METADATA_UNLOCK_WR(); ocf_req_hash_unlock_wr(req);
} }
/* Submit read IO to the core */ /* 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 = { static const struct ocf_io_if _io_if_pt_resume = {
.read = ocf_read_pt_do, .read = ocf_read_pt_do,
.write = ocf_read_pt_do, .write = ocf_read_pt_do,
.resume = ocf_engine_on_resume,
}; };
int ocf_read_pt(struct ocf_request *req) int ocf_read_pt(struct ocf_request *req)
{ {
bool use_cache = false; bool use_cache = false;
int lock = OCF_LOCK_NOT_ACQUIRED; int lock = OCF_LOCK_ACQUIRED;
struct ocf_cache *cache = req->cache;
OCF_DEBUG_TRACE(req->cache); OCF_DEBUG_TRACE(req->cache);
@ -118,7 +114,8 @@ int ocf_read_pt(struct ocf_request *req)
/* Set resume io_if */ /* Set resume io_if */
req->io_if = &_io_if_pt_resume; 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 */ /* Traverse request to check if there are mapped cache lines */
ocf_engine_traverse(req); ocf_engine_traverse(req);
@ -131,13 +128,16 @@ int ocf_read_pt(struct ocf_request *req)
* lock request for READ access * lock request for READ access
*/ */
lock = ocf_req_trylock_rd(req); 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) { if (use_cache) {
/* /*

View File

@ -137,8 +137,6 @@ err_alloc:
static int _ocf_read_generic_do(struct ocf_request *req) 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) { if (ocf_engine_is_miss(req) && req->map->rd_locked) {
/* Miss can be handled only on write locks. /* Miss can be handled only on write locks.
* Need to switch to PT * 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 (ocf_engine_is_miss(req)) {
if (req->info.dirty_any) { if (req->info.dirty_any) {
OCF_METADATA_LOCK_RD(); ocf_req_hash_lock_rd(req);
/* Request is dirty need to clean request */ /* Request is dirty need to clean request */
ocf_engine_clean(req); ocf_engine_clean(req);
OCF_METADATA_UNLOCK_RD(); ocf_req_hash_unlock_rd(req);
/* We need to clean request before processing, return */ /* We need to clean request before processing, return */
ocf_req_put(req); ocf_req_put(req);
@ -166,25 +164,25 @@ static int _ocf_read_generic_do(struct ocf_request *req)
return 0; return 0;
} }
OCF_METADATA_LOCK_RD(); ocf_req_hash_lock_rd(req);
/* Set valid status bits map */ /* Set valid status bits map */
ocf_set_valid_map_info(req); ocf_set_valid_map_info(req);
OCF_METADATA_UNLOCK_RD(); ocf_req_hash_unlock_rd(req);
} }
if (req->info.re_part) { if (req->info.re_part) {
OCF_DEBUG_RQ(req, "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 /* Probably some cache lines are assigned into wrong
* partition. Need to move it to new one * partition. Need to move it to new one
*/ */
ocf_part_move(req); ocf_part_move(req);
OCF_METADATA_UNLOCK_WR(); ocf_req_hash_unlock_wr(req);
} }
OCF_DEBUG_RQ(req, "Submit"); 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 = { static const struct ocf_io_if _io_if_read_generic_resume = {
.read = _ocf_read_generic_do, .read = _ocf_read_generic_do,
.write = _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, .resume = ocf_engine_on_resume,
}; };
int ocf_read_generic(struct ocf_request *req) int ocf_read_generic(struct ocf_request *req)
{ {
bool mapped;
int lock = OCF_LOCK_NOT_ACQUIRED; int lock = OCF_LOCK_NOT_ACQUIRED;
struct ocf_cache *cache = req->cache; struct ocf_cache *cache = req->cache;
@ -231,61 +241,7 @@ int ocf_read_generic(struct ocf_request *req)
/* Set resume call backs */ /* Set resume call backs */
req->io_if = &_io_if_read_generic_resume; req->io_if = &_io_if_read_generic_resume;
/*- Metadata RD access -----------------------------------------------*/ lock = ocf_engine_prepare_clines(req, &_rd_engine_callbacks);
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 -----------------------------------*/
}
if (!req->info.mapping_error) { if (!req->info.mapping_error) {
if (lock >= 0) { if (lock >= 0) {

View File

@ -38,19 +38,19 @@ static void _ocf_read_wa_complete(struct ocf_request *req, int error)
int ocf_write_wa(struct ocf_request *req) int ocf_write_wa(struct ocf_request *req)
{ {
struct ocf_cache *cache = req->cache;
ocf_io_start(&req->ioi.io); ocf_io_start(&req->ioi.io);
/* Get OCF request - increase reference counter */ /* Get OCF request - increase reference counter */
ocf_req_get(req); 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 */ /* Traverse request to check if there are mapped cache lines */
ocf_engine_traverse(req); 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)) { if (ocf_engine_is_hit(req)) {
ocf_req_clear(req); ocf_req_clear(req);

View File

@ -21,28 +21,25 @@
static const struct ocf_io_if _io_if_wb_resume = { static const struct ocf_io_if _io_if_wb_resume = {
.read = ocf_write_wb_do, .read = ocf_write_wb_do,
.write = 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) static void _ocf_write_wb_update_bits(struct ocf_request *req)
{ {
struct ocf_cache *cache = req->cache;
if (ocf_engine_is_miss(req)) { if (ocf_engine_is_miss(req)) {
OCF_METADATA_LOCK_RD(); ocf_req_hash_lock_rd(req);
/* Update valid status bits */ /* Update valid status bits */
ocf_set_valid_map_info(req); ocf_set_valid_map_info(req);
OCF_METADATA_UNLOCK_RD(); ocf_req_hash_unlock_rd(req);
} }
if (!ocf_engine_is_dirty_all(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 */ /* set dirty bits, and mark if metadata flushing is required */
ocf_set_dirty_map_info(req); 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) { if (req->info.re_part) {
OCF_DEBUG_RQ(req, "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 /* Probably some cache lines are assigned into wrong
* partition. Need to move it to new one * partition. Need to move it to new one
*/ */
ocf_part_move(req); ocf_part_move(req);
OCF_METADATA_UNLOCK_WR(); ocf_req_hash_unlock_wr(req);
} }
OCF_DEBUG_RQ(req, "Submit Data"); OCF_DEBUG_RQ(req, "Submit Data");
@ -166,11 +163,20 @@ int ocf_write_wb_do(struct ocf_request *req)
return 0; 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) int ocf_write_wb(struct ocf_request *req)
{ {
bool mapped;
int lock = OCF_LOCK_NOT_ACQUIRED; int lock = OCF_LOCK_NOT_ACQUIRED;
struct ocf_cache *cache = req->cache;
ocf_io_start(&req->ioi.io); ocf_io_start(&req->ioi.io);
@ -182,35 +188,7 @@ int ocf_write_wb(struct ocf_request *req)
/* TODO: Handle fits into dirty */ /* TODO: Handle fits into dirty */
OCF_METADATA_LOCK_RD(); /*- Metadata READ access, No eviction --------*/ lock = ocf_engine_prepare_clines(req, &_wb_engine_callbacks);
/* 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 ---------*/
}
if (!req->info.mapping_error) { if (!req->info.mapping_error) {
if (lock >= 0) { if (lock >= 0) {

View File

@ -52,12 +52,12 @@ static int ocf_write_wi_update_and_flush_metadata(struct ocf_request *req)
if (ocf_engine_mapped_count(req)) { if (ocf_engine_mapped_count(req)) {
/* There are mapped cache line, need to remove them */ /* 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 */ /* Remove mapped cache lines from metadata */
ocf_purge_map_info(req); 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) { if (req->info.flush_metadata) {
/* Request was dirty and need to 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 = { static const struct ocf_io_if _io_if_wi_resume = {
.read = _ocf_write_wi_do, .read = _ocf_write_wi_do,
.write = _ocf_write_wi_do, .write = _ocf_write_wi_do,
.resume = _ocf_write_wi_on_resume,
}; };
int ocf_write_wi(struct ocf_request *req) int ocf_write_wi(struct ocf_request *req)
{ {
int lock = OCF_LOCK_NOT_ACQUIRED; int lock = OCF_LOCK_NOT_ACQUIRED;
struct ocf_cache *cache = req->cache;
OCF_DEBUG_TRACE(req->cache); OCF_DEBUG_TRACE(req->cache);
@ -148,7 +146,8 @@ int ocf_write_wi(struct ocf_request *req)
/* Set resume io_if */ /* Set resume io_if */
req->io_if = &_io_if_wi_resume; 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 */ /* Travers to check if request is mapped fully */
ocf_engine_traverse(req); ocf_engine_traverse(req);
@ -160,7 +159,13 @@ int ocf_write_wi(struct ocf_request *req)
lock = OCF_LOCK_ACQUIRED; 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 >= 0) {
if (lock == OCF_LOCK_ACQUIRED) { if (lock == OCF_LOCK_ACQUIRED) {

View File

@ -197,12 +197,10 @@ int ocf_read_wo_do(struct ocf_request *req)
static const struct ocf_io_if _io_if_wo_resume = { static const struct ocf_io_if _io_if_wo_resume = {
.read = ocf_read_wo_do, .read = ocf_read_wo_do,
.write = ocf_read_wo_do, .write = ocf_read_wo_do,
.resume = ocf_engine_on_resume,
}; };
int ocf_read_wo(struct ocf_request *req) int ocf_read_wo(struct ocf_request *req)
{ {
ocf_cache_t cache = req->cache;
int lock = OCF_LOCK_ACQUIRED; int lock = OCF_LOCK_ACQUIRED;
OCF_DEBUG_TRACE(req->cache); OCF_DEBUG_TRACE(req->cache);
@ -215,7 +213,8 @@ int ocf_read_wo(struct ocf_request *req)
/* Set resume call backs */ /* Set resume call backs */
req->io_if = &_io_if_wo_resume; 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 */ /* Traverse request to check if there are mapped cache lines */
ocf_engine_traverse(req); ocf_engine_traverse(req);
@ -227,7 +226,13 @@ int ocf_read_wo(struct ocf_request *req)
lock = ocf_req_trylock_rd(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 >= 0) {
if (lock != OCF_LOCK_ACQUIRED) { if (lock != OCF_LOCK_ACQUIRED) {

View File

@ -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) static void _ocf_write_wt_update_bits(struct ocf_request *req)
{ {
struct ocf_cache *cache = req->cache;
if (ocf_engine_is_miss(req)) { if (ocf_engine_is_miss(req)) {
OCF_METADATA_LOCK_RD(); ocf_req_hash_lock_rd(req);
/* Update valid status bits */ /* Update valid status bits */
ocf_set_valid_map_info(req); ocf_set_valid_map_info(req);
OCF_METADATA_UNLOCK_RD(); ocf_req_hash_unlock_rd(req);
} }
if (req->info.dirty_any) { 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 /* Writes goes to SDD and HDD, need to update status bits from
* dirty to clean * 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_set_clean_map_info(req);
OCF_METADATA_UNLOCK_WR(); ocf_req_hash_unlock_wr(req);
} }
if (req->info.re_part) { if (req->info.re_part) {
OCF_DEBUG_RQ(req, "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 /* Probably some cache lines are assigned into wrong
* partition. Need to move it to new one * partition. Need to move it to new one
*/ */
ocf_part_move(req); 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 = { static const struct ocf_io_if _io_if_wt_resume = {
.read = _ocf_write_wt_do, .read = _ocf_write_wt_do,
.write = _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, .resume = ocf_engine_on_resume,
}; };
int ocf_write_wt(struct ocf_request *req) int ocf_write_wt(struct ocf_request *req)
{ {
bool mapped;
int lock = OCF_LOCK_NOT_ACQUIRED; int lock = OCF_LOCK_NOT_ACQUIRED;
struct ocf_cache *cache = req->cache;
ocf_io_start(&req->ioi.io); ocf_io_start(&req->ioi.io);
@ -175,35 +181,7 @@ int ocf_write_wt(struct ocf_request *req)
/* Set resume io_if */ /* Set resume io_if */
req->io_if = &_io_if_wt_resume; req->io_if = &_io_if_wt_resume;
OCF_METADATA_LOCK_RD(); /*- Metadata READ access, No eviction --------*/ lock = ocf_engine_prepare_clines(req, &_wt_engine_callbacks);
/* 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 ---------*/
}
if (!req->info.mapping_error) { if (!req->info.mapping_error) {
if (lock >= 0) { if (lock >= 0) {

View File

@ -18,19 +18,17 @@
static int ocf_zero_purge(struct ocf_request *req) static int ocf_zero_purge(struct ocf_request *req)
{ {
struct ocf_cache *cache = req->cache;
if (req->error) { if (req->error) {
ocf_engine_error(req, true, "Failed to discard data on cache"); ocf_engine_error(req, true, "Failed to discard data on cache");
} else { } else {
/* There are mapped cache line, need to remove them */ /* 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 */ /* Remove mapped cache lines from metadata */
ocf_purge_map_info(req); 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); 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 = { static const struct ocf_io_if _io_if_ocf_zero_do = {
.read = _ocf_zero_do, .read = _ocf_zero_do,
.write = _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); 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 */ /* Traverse to check if request is mapped */
ocf_engine_traverse(req); 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; req->io_if = &_io_if_ocf_zero_do;
/* Some cache line are mapped, lock request for WRITE access */ /* 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) { if (lock >= 0) {
ENV_BUG_ON(lock != OCF_LOCK_ACQUIRED); ENV_BUG_ON(lock != OCF_LOCK_ACQUIRED);

View File

@ -39,7 +39,7 @@ int ocf_metadata_init(struct ocf_cache *cache,
return ret; return ret;
} }
ocf_metadata_concurrency_init(cache); ocf_metadata_concurrency_init(&cache->metadata.lock);
return 0; return 0;
} }
@ -73,7 +73,7 @@ void ocf_metadata_deinit(struct ocf_cache *cache)
cache->metadata.iface.deinit(cache); cache->metadata.iface.deinit(cache);
} }
ocf_metadata_concurrency_deinit(cache); ocf_metadata_concurrency_deinit(&cache->metadata.lock);
ocf_metadata_io_deinit(cache); 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, void ocf_metadata_flush_all(ocf_cache_t cache,
ocf_metadata_end_t cmpl, void *priv) 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); 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, void ocf_metadata_load_all(ocf_cache_t cache,
ocf_metadata_end_t cmpl, void *priv) 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); 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, void ocf_metadata_load_recovery(ocf_cache_t cache,

View File

@ -1903,7 +1903,7 @@ static void _recovery_rebuild_metadata(ocf_pipeline_t pipeline,
const uint64_t collision_table_entries = const uint64_t collision_table_entries =
ocf_metadata_collision_table_entries(cache); 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++) { for (cline = 0; cline < collision_table_entries; cline++) {
ocf_metadata_get_core_info(cache, cline, &core_id, &core_line); 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_COND_RESCHED(step, 128);
} }
OCF_METADATA_UNLOCK_WR(); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
ocf_pipeline_next(pipeline); ocf_pipeline_next(pipeline);
} }

View File

@ -226,9 +226,9 @@ static int ocf_restart_meta_io(struct ocf_request *req)
int ret; int ret;
/* Fill with the latest metadata. */ /* 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); metadata_io_req_fill(meta_io_req);
OCF_METADATA_UNLOCK_RD();
io = ocf_new_cache_io(cache, req->io_queue, io = ocf_new_cache_io(cache, req->io_queue,
PAGES_TO_BYTES(meta_io_req->page), PAGES_TO_BYTES(meta_io_req->page),

View File

@ -428,6 +428,16 @@ struct ocf_cache_line_settings {
uint64_t sector_end; 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 * @brief Metadata control structure
*/ */
@ -444,11 +454,7 @@ struct ocf_metadata {
bool is_volatile; bool is_volatile;
/*!< true if metadata used in volatile mode (RAM only) */ /*!< true if metadata used in volatile mode (RAM only) */
struct { struct ocf_metadata_lock lock;
env_rwsem collision; /*!< lock for collision table */
env_rwlock status; /*!< Fast lock for status bits */
env_spinlock eviction; /*!< Fast lock for eviction policy */
} lock;
}; };
#endif /* __METADATA_STRUCTS_H__ */ #endif /* __METADATA_STRUCTS_H__ */

View File

@ -292,7 +292,6 @@ static ocf_error_t init_attached_data_structures(ocf_cache_t cache,
ocf_error_t result; ocf_error_t result;
/* Lock to ensure consistency */ /* Lock to ensure consistency */
OCF_METADATA_LOCK_WR();
ocf_metadata_init_hash_table(cache); ocf_metadata_init_hash_table(cache);
ocf_metadata_init_collision(cache); ocf_metadata_init_collision(cache);
@ -303,7 +302,6 @@ static ocf_error_t init_attached_data_structures(ocf_cache_t cache,
if (result) { if (result) {
ocf_cache_log(cache, log_err, ocf_cache_log(cache, log_err,
"Cannot initialize cleaning policy\n"); "Cannot initialize cleaning policy\n");
OCF_METADATA_UNLOCK_WR();
return result; return result;
} }
@ -313,24 +311,19 @@ static ocf_error_t init_attached_data_structures(ocf_cache_t cache,
ocf_cache_log(cache, log_err, ocf_cache_log(cache, log_err,
"Cannot initialize promotion policy\n"); "Cannot initialize promotion policy\n");
__deinit_cleaning_policy(cache); __deinit_cleaning_policy(cache);
OCF_METADATA_UNLOCK_WR();
return result; return result;
} }
OCF_METADATA_UNLOCK_WR();
return 0; return 0;
} }
static void init_attached_data_structures_recovery(ocf_cache_t cache) static void init_attached_data_structures_recovery(ocf_cache_t cache)
{ {
OCF_METADATA_LOCK_WR();
ocf_metadata_init_hash_table(cache); ocf_metadata_init_hash_table(cache);
ocf_metadata_init_collision(cache); ocf_metadata_init_collision(cache);
__init_partitions_attached(cache); __init_partitions_attached(cache);
__reset_stats(cache); __reset_stats(cache);
__init_metadata_version(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; 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 " ocf_cache_log(cache, log_err, "Failed to initialize attached "
"metadata concurrency\n"); "metadata concurrency\n");
OCF_PL_FINISH_RET(context->pipeline, -OCF_ERR_START_CACHE_FAIL); 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_volume_close(&cache->device->volume);
ocf_metadata_concurrency_attached_deinit(&cache->metadata.lock);
ocf_metadata_deinit_variable_size(cache); ocf_metadata_deinit_variable_size(cache);
ocf_concurrency_deinit(cache); ocf_concurrency_deinit(cache);
ocf_freelist_deinit(cache->freelist); 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; int result;
OCF_METADATA_LOCK_WR(); ocf_metadata_start_exclusive_access(&cache->metadata.lock);
result = ocf_promotion_set_policy(cache->promotion_policy, type); result = ocf_promotion_set_policy(cache->promotion_policy, type);
OCF_METADATA_UNLOCK_WR(); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
return result; return result;
} }
@ -2253,11 +2248,11 @@ ocf_promotion_t ocf_mngt_cache_promotion_get_policy(ocf_cache_t cache)
{ {
ocf_promotion_t result; ocf_promotion_t result;
OCF_METADATA_LOCK_RD(); ocf_metadata_start_shared_access(&cache->metadata.lock);
result = cache->conf_meta->promotion_policy_type; result = cache->conf_meta->promotion_policy_type;
OCF_METADATA_UNLOCK_RD(); ocf_metadata_end_shared_access(&cache->metadata.lock);
return result; return result;
} }
@ -2267,12 +2262,12 @@ int ocf_mngt_cache_promotion_get_param(ocf_cache_t cache, uint8_t param_id,
{ {
int result; int result;
OCF_METADATA_LOCK_RD(); ocf_metadata_start_shared_access(&cache->metadata.lock);
result = ocf_promotion_get_param(cache->promotion_policy, param_id, result = ocf_promotion_get_param(cache->promotion_policy, param_id,
param_value); param_value);
OCF_METADATA_UNLOCK_RD(); ocf_metadata_end_shared_access(&cache->metadata.lock);
return result; return result;
} }
@ -2282,12 +2277,12 @@ int ocf_mngt_cache_promotion_set_param(ocf_cache_t cache, uint8_t param_id,
{ {
int result; int result;
OCF_METADATA_LOCK_RD(); ocf_metadata_start_shared_access(&cache->metadata.lock);
result = ocf_promotion_set_param(cache->promotion_policy, param_id, result = ocf_promotion_set_param(cache->promotion_policy, param_id,
param_value); param_value);
OCF_METADATA_UNLOCK_RD(); ocf_metadata_end_shared_access(&cache->metadata.lock);
return result; return result;
} }

View File

@ -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_core_id_t core_id = ocf_core_get_id(core);
ocf_cleaning_t clean_pol_type; 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; clean_pol_type = cache->conf_meta->cleaning_policy_type;
if (cache->core[core_id].opened) { 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 */ /* Deinitialize core metadata in attached metadata */
@ -65,7 +65,7 @@ void cache_mngt_core_deinit_attached_meta(ocf_core_t core)
if (!core_size) if (!core_size)
core_size = ~0ULL; core_size = ~0ULL;
OCF_METADATA_LOCK_WR(); ocf_metadata_start_exclusive_access(&cache->metadata.lock);
clean_pol_type = cache->conf_meta->cleaning_policy_type; clean_pol_type = cache->conf_meta->cleaning_policy_type;
while (retry) { while (retry) {
@ -82,13 +82,14 @@ void cache_mngt_core_deinit_attached_meta(ocf_core_t core)
} }
if (retry) { if (retry) {
OCF_METADATA_UNLOCK_WR(); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
env_msleep(100); 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 */ /* 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_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 */ /* In metadata mark data this core was removed from cache */
core->conf_meta->valid = false; 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); ocf_mngt_core_clear_uuid_metadata(core);
core->conf_meta->seq_no = OCF_SEQ_NO_INVALID; 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 */ /* Deinit in-memory structures related to this core */

View File

@ -385,9 +385,9 @@ static int _ofc_flush_container_step(struct ocf_request *req)
struct flush_container *fc = req->priv; struct flush_container *fc = req->priv;
ocf_cache_t cache = fc->cache; 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_mngt_flush_portion(fc);
ocf_metadata_unlock(cache, OCF_METADATA_WR); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
return 0; return 0;
} }
@ -501,7 +501,7 @@ static void _ocf_mngt_flush_core(
return; return;
} }
ocf_metadata_lock(cache, OCF_METADATA_WR); ocf_metadata_start_exclusive_access(&cache->metadata.lock);
ret = _ocf_mngt_get_sectors(cache, core_id, ret = _ocf_mngt_get_sectors(cache, core_id,
&fc->flush_data, &fc->count); &fc->flush_data, &fc->count);
@ -509,7 +509,7 @@ static void _ocf_mngt_flush_core(
ocf_core_log(core, log_err, "Flushing operation aborted, " ocf_core_log(core, log_err, "Flushing operation aborted, "
"no memory\n"); "no memory\n");
env_vfree(fc); env_vfree(fc);
ocf_metadata_unlock(cache, OCF_METADATA_WR); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
complete(context, -OCF_ERR_NO_MEM); complete(context, -OCF_ERR_NO_MEM);
return; return;
} }
@ -519,7 +519,7 @@ static void _ocf_mngt_flush_core(
_ocf_mngt_flush_containers(context, fc, 1, complete); _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( 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); 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 */ /* Get all 'dirty' sectors for all cores */
ret = _ocf_mngt_get_flush_containers(cache, &fctbl, &fcnum); ret = _ocf_mngt_get_flush_containers(cache, &fctbl, &fcnum);
if (ret) { if (ret) {
ocf_cache_log(cache, log_err, "Flushing operation aborted, " ocf_cache_log(cache, log_err, "Flushing operation aborted, "
"no memory\n"); "no memory\n");
ocf_metadata_unlock(cache, OCF_METADATA_WR); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
complete(context, ret); complete(context, ret);
return; return;
} }
_ocf_mngt_flush_containers(context, fctbl, fcnum, complete); _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( 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; ocf_cache_t cache = context->cache;
int result; 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, result = ocf_metadata_sparse_range(cache, context->purge.core_id, 0,
context->purge.end_byte); 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); 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; return 0;
} }
ocf_metadata_lock(cache, OCF_METADATA_WR); ocf_metadata_start_exclusive_access(&cache->metadata.lock);
if (cleaning_policy_ops[old_type].deinitialize) if (cleaning_policy_ops[old_type].deinitialize)
cleaning_policy_ops[old_type].deinitialize(cache); 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; 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 " ocf_cache_log(cache, log_info, "Changing cleaning policy from "
"%s to %s\n", cleaning_policy_ops[old_type].name, "%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) if (!cleaning_policy_ops[type].set_cleaning_param)
return -OCF_ERR_INVAL; 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, ret = cleaning_policy_ops[type].set_cleaning_param(cache,
param_id, param_value); param_id, param_value);
ocf_metadata_unlock(cache, OCF_METADATA_WR); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
return ret; return ret;
} }

View File

@ -275,7 +275,7 @@ int ocf_mngt_cache_io_classes_configure(ocf_cache_t cache,
if (!old_config) if (!old_config)
return -OCF_ERR_NO_MEM; 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), result = env_memcpy(old_config, sizeof(cache->user_parts),
cache->user_parts, sizeof(cache->user_parts)); cache->user_parts, sizeof(cache->user_parts));
@ -300,7 +300,7 @@ out_edit:
} }
out_cpy: out_cpy:
OCF_METADATA_UNLOCK_WR(); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
env_free(old_config); env_free(old_config);
return result; return result;

View File

@ -311,3 +311,14 @@ void ocf_req_clear_map(struct ocf_request *req)
ENV_BUG_ON(env_memset(req->map, ENV_BUG_ON(env_memset(req->map,
sizeof(req->map[0]) * req->core_line_count, 0)); 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));
}
}

View File

@ -319,6 +319,13 @@ void ocf_req_clear_info(struct ocf_request *req);
*/ */
void ocf_req_clear_map(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 * @brief Clear OCF request
* *

View File

@ -197,6 +197,12 @@ static void _ocf_cleaner_complete_req(struct ocf_request *req)
cmpl(master->priv, master->error); 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 * 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); 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_DEBUG_TRACE(req->cache);
OCF_METADATA_LOCK_WR(); ocf_metadata_start_exclusive_access(&cache->metadata.lock);
/* Update metadata */ /* Update metadata */
for (i = 0; i < req->core_line_count; i++, iter++) { for (i = 0; i < req->core_line_count; i++, iter++) {
if (iter->status == LOOKUP_MISS) 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_flush_do_asynch(cache, req, _ocf_cleaner_metadata_io_end);
OCF_METADATA_UNLOCK_WR(); ocf_metadata_end_exclusive_access(&cache->metadata.lock);
return 0; return 0;
} }
@ -697,16 +703,9 @@ static int _ocf_cleaner_fire_cache(struct ocf_request *req)
return 0; 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 = { static const struct ocf_io_if _io_if_fire_cache = {
.read = _ocf_cleaner_fire_cache, .read = _ocf_cleaner_fire_cache,
.write = _ocf_cleaner_fire_cache, .write = _ocf_cleaner_fire_cache,
.resume = _ocf_cleaner_on_resume,
}; };
static int _ocf_cleaner_fire(struct ocf_request *req) static int _ocf_cleaner_fire(struct ocf_request *req)

View File

@ -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);
}