Commit d12883aa by bernhard-gatzhammer

Merge branch 'embb509_centralize_spinlocks' into development

parents 8ccb89ef 5bc9ff94
......@@ -46,12 +46,25 @@ extern "C" {
#include <embb/base/c/internal/platform.h>
#include <embb/base/c/errors.h>
#include <embb/base/c/atomic.h>
#ifdef DOXYGEN
/**
* Opaque type representing a mutex.
*/
typedef opaque_type embb_mutex_t;
/**
* Opaque type representing a spinlock.
*/
typedef opaque_type embb_spinlock_t;
#else
/**
* Spinlock type, treat as opaque.
*/
typedef struct {
embb_atomic_int atomic_spin_variable_;
} embb_spinlock_t;
#endif /* DOXYGEN */
/**
......@@ -147,6 +160,86 @@ void embb_mutex_destroy(
/**< [IN/OUT] Pointer to mutex */
);
/**
* Initializes a spinlock
*
* \pre \c spinlock is uninitialized
* \post \c spinlock is initialized
* \return EMBB_SUCCESS if spinlock could be initialized \n
* EMBB_ERROR otherwise
* \memory (Potentially) allocates dynamic memory
* \notthreadsafe
* \see embb_spinlock_destroy()
*/
int embb_spin_init(
embb_spinlock_t* spinlock
/**< [OUT] Pointer to spinlock */
);
/**
* Spins until the spinlock can be locked and locks it.
*
* \pre \c spinlock is initialized \n
* \post If successful, \c spinlock is locked.
* \return EMBB_SUCCESS if spinlock could be locked. \n
* EMBB_ERROR if an error occurred.
* \threadsafe
* \see embb_spinlock_try_lock(), embb_mutex_unlock()
*/
int embb_spin_lock(
embb_spinlock_t* spinlock
/**< [IN/OUT] Pointer to spinlock */
);
/**
* Tries to lock the spinlock and returns if not successful.
*
* \pre \c spinlock is initialized
* \post If successful, \c spinlock is locked
*
* \return EMBB_SUCCESS if spinlock could be locked \n
* EMBB_BUSY if spinlock could not be locked \n
* EMBB_ERROR if an error occurred
* \threadsafe
* \see embb_spin_lock(), embb_spin_unlock()
*/
int embb_spin_try_lock(
embb_spinlock_t* spinlock,
/**< [IN/OUT] Pointer to spinlock */
unsigned int max_number_spins
/**< [IN] Maximal count of spins to perform, trying to acquire lock. Note that
* passing 0 here results in not trying to obtain the lock at all.
*/
);
/**
* Unlocks a locked spinlock.
*
* \pre \c spinlock has been locked by the current thread.
* \post If successful, \c spinlock is unlocked.
* \return EMBB_SUCCESS if the operation was successful \n
* EMBB_ERROR otherwise
* \threadsafe
* \see embb_spin_lock(), embb_spin_try_lock()
*/
int embb_spin_unlock(
embb_spinlock_t* spinlock
/**< [IN/OUT] Pointer to spinlock */
);
/**
* Destroys a spinlock and frees its resources.
*
* \pre \c spinlock has been initialized
* \post \c spinlock is uninitialized
* \notthreadsafe
* \see embb_spin_init()
*/
void embb_spin_destroy(
embb_spinlock_t* spinlock
/**< [IN/OUT] Pointer to spinlock */
);
#ifdef __cplusplus
} /* Close extern "C" { */
#endif
......
......@@ -115,3 +115,53 @@ void embb_mutex_destroy(embb_mutex_t* mutex) {
}
#endif /* EMBB_PLATFORM_THREADING_POSIXTHREADS */
int embb_spin_init(embb_spinlock_t* spinlock) {
// For now, store the initial value. In the future will use atomic init
// function (as soon as available).
embb_atomic_store_int(&spinlock->atomic_spin_variable_, 0);
}
int embb_spin_lock(embb_spinlock_t* spinlock) {
int expected = 0;
// try to swap the
while (0 == embb_atomic_compare_and_swap_int(
&spinlock->atomic_spin_variable_, &expected, 1)) {
// reset expected, as CAS might change it...
expected = 0;
}
return EMBB_SUCCESS;
}
int embb_spin_try_lock(embb_spinlock_t* spinlock,
unsigned int max_number_spins) {
if (max_number_spins == 0)
return EMBB_BUSY;
int expected = 0;
while (0 == embb_atomic_compare_and_swap_int(
&spinlock->atomic_spin_variable_,
&expected, 1)) {
max_number_spins--;
if (0 == max_number_spins) {
return EMBB_BUSY;
}
expected = 0;
}
return EMBB_SUCCESS;
}
int embb_spin_unlock(embb_spinlock_t* spinlock) {
int expected = 1;
return embb_atomic_compare_and_swap_int(&spinlock->atomic_spin_variable_,
&expected, 0) ?
EMBB_SUCCESS : EMBB_ERROR;
}
void embb_spin_destroy(embb_spinlock_t* spinlock) {
// for now, doing nothing here... in future, will call the respective
// destroy function for atomics...
return EMBB_SUCCESS;
}
......@@ -46,6 +46,7 @@ using embb::base::test::DurationTest;
using embb::base::test::TimeTest;
using embb::base::test::CounterTest;
using embb::base::test::MutexTest;
using embb::base::test::SpinLockTest;
using embb::base::test::ThreadIndexTest;
using embb::base::test::CoreSetTest;
using embb::base::test::ConditionVarTest;
......@@ -63,11 +64,11 @@ PT_MAIN("Base C") {
PT_RUN(TimeTest);
PT_RUN(CounterTest);
PT_RUN(MutexTest);
PT_RUN(SpinLockTest);
PT_RUN(ThreadIndexTest);
PT_RUN(CoreSetTest);
PT_RUN(ConditionVarTest);
PT_RUN(ThreadTest);
PT_RUN(ThreadSpecificStorageTest);
PT_EXPECT(embb_get_bytes_allocated() == 0);
}
......@@ -76,6 +76,77 @@ void MutexTest::TestRecursiveMutex() {
embb_mutex_destroy(&mutex);
}
SpinLockTest::SpinLockTest() : counter_(0),
number_threads_(partest::TestSuite::GetDefaultNumThreads()),
number_iterations_(partest::TestSuite::GetDefaultNumIterations()),
counter_iterations_(10000) {
CreateUnit("Protected counter using Lock")
.Pre(&SpinLockTest::PreSpinLockInc, this)
.Add(&SpinLockTest::TestSpinLockIncUseLock, this,
number_threads_,
number_iterations_)
.Post(&SpinLockTest::PostSpinLockInc, this);
CreateUnit("Protected counter using TryLock")
.Pre(&SpinLockTest::PreSpinLockInc, this)
.Add(&SpinLockTest::TestSpinLockIncUseTryLock, this,
number_threads_,
number_iterations_)
.Post(&SpinLockTest::PostSpinLockInc, this);
CreateUnit("Test spinning (too many spins), single thread")
.Add(&SpinLockTest::TestSpinLockTooManySpins, this,
// one thread
1,
// one iteration
1);
}
void SpinLockTest::TestSpinLockTooManySpins() {
embb_spin_init(&spinlock_);
embb_spin_lock(&spinlock_);
int return_code = embb_spin_try_lock(&spinlock_, 100);
PT_ASSERT(return_code == EMBB_BUSY);
embb_spin_unlock(&spinlock_);
return_code = embb_spin_try_lock(&spinlock_, 100);
PT_ASSERT(return_code == EMBB_SUCCESS);
embb_spin_unlock(&spinlock_);
embb_spin_destroy(&spinlock_);
}
void SpinLockTest::PreSpinLockInc() {
embb_spin_init(&spinlock_);
}
void SpinLockTest::TestSpinLockIncUseLock() {
for (unsigned int i = 0; i != counter_iterations_; ++i){
embb_spin_lock(&spinlock_);
counter_++;
embb_spin_unlock(&spinlock_);
}
}
void SpinLockTest::TestSpinLockIncUseTryLock() {
for (unsigned int i = 0; i != counter_iterations_; ++i){
while (embb_spin_try_lock(&spinlock_, 100) != EMBB_SUCCESS) {}
counter_++;
embb_spin_unlock(&spinlock_);
}
}
void SpinLockTest::PostSpinLockInc() {
embb_spin_destroy(&spinlock_);
PT_EXPECT_EQ(counter_, number_iterations_ *
number_threads_*
counter_iterations_);
counter_ = 0;
}
} // namespace test
} // namespace base
} // namespace embb
......@@ -85,6 +85,63 @@ class MutexTest : public partest::TestCase {
size_t number_iterations_;
};
class SpinLockTest : public partest::TestCase {
public:
SpinLockTest();
private:
/**
* Check that the try lock fails, when lock is already set.
*/
void TestSpinLockTooManySpins();
/**
* Prepares TestMutexIncCpp.
*/
void PreSpinLockInc();
/**
* Tests mutex locking and unlocking to protect shared counter.
*/
void TestSpinLockIncUseLock();
/**
* Tests mutex locking and unlocking to protect shared counter using trylock.
*/
void TestSpinLockIncUseTryLock();
/**
* Checks and tears down TestMutexIncCpp.
*/
void PostSpinLockInc();
/**
* Shared counter to check effectiveness of mutex.
*/
size_t counter_;
/**
* Number of threads used to run tests.
*/
size_t number_threads_;
/**
* Number of times the test method is called by each thread.
*/
size_t number_iterations_;
/**
* Number of internal iterations, for incrementing the counter.
*/
size_t counter_iterations_;
/**
* The used spinlock
*/
embb_spinlock_t spinlock_;
};
} // namespace test
} // namespace base
} // namespace embb
......
......@@ -114,6 +114,8 @@ bool UniqueLock<Mutex>::OwnsLock() const {
return locked_;
}
} // namespace base
} // namespace embb
......
......@@ -29,9 +29,63 @@
#include <embb/base/internal/platform.h>
#include <embb/base/exceptions.h>
#include <embb/base/c/mutex.h>
namespace embb {
namespace base {
/**
* \defgroup CPP_CONCEPTS_MUTEX Mutex Concept
*
* \brief Concept for thread synchronization.
*
* \anchor CPP_CONCEPTS_MUTEX_ANCHOR
*
* \ingroup CPP_CONCEPT
* \{
* \par Description
*
* The mutex concept is a concept for thread synchronization. It provides a
* lock. At any point in time, only one thread can exclusively hold the lock and
* the lock is held, until the thread explicitly releases the beforehand
* acquired lock.
*
* \par Requirements
* - Let \c Mutex be the mutex type
* - Let \c m be an object of type \c Mutex.
*
* \par Valid Expressions
*
* <table>
* <tr>
* <th>Expression</th>
* <th>Return type</th>
* <th>Description</th>
* </tr>
* <tr>
* <td>Mutex()</td>
* <td>\c void</td>
* <td>Constructs a mutex.</td>
* </tr>
* <tr>
* <td>m.TryLock()</td>
* <td>\c bool</td>
* <td>Tries to lock the mutex and immediately returns. Returns \c false, if
* the mutex could not be acquired (locked), otherwise \c true.
* </tr>
* <tr>
* <td>m.Lock()</td>
* <td>\c void</td>
* <td>Locks the mutex. When the mutex is already locked, the current thread
* is blocked until the mutex is unlocked.</td>
* </tr>
* <tr>
* <td>m.Unlock()</td>
* <td>\c void</td>
* <td>Unlocks the mutex.</td>
* </tr>
* </table>
* \}
*/
/**
* \defgroup CPP_BASE_MUTEX Mutex and Lock
......@@ -47,7 +101,6 @@ namespace base {
class ConditionVariable;
namespace internal {
/**
* Provides main functionality for mutexes.
*/
......@@ -111,10 +164,82 @@ class MutexBase {
*/
friend class embb::base::ConditionVariable;
};
} // namespace internal
/**
* Spinlock
*
* \concept{CPP_CONCEPTS_MUTEX}
*
* \ingroup CPP_BASE_MUTEX
*/
class Spinlock {
public:
/**
* Creates a spinlock which is in unlocked state.
*
* \notthreadsafe
*/
Spinlock();
/**
* Destructs a spinlock.
*
* \notthreadsafe
*/
~Spinlock();
/**
* Waits until the spinlock can be locked and locks it.
*
* \pre The spinlock is not locked by the current thread.
* \post The spinlock is locked
* \threadsafe
* \see TryLock(), Unlock()
*/
void Lock();
/**
* Tries to lock the spinlock for \c number_spins times and returns.
*
* \pre The spinlock is not locked by the current thread.
* \post If successful, the spinlock is locked.
* \return \c true if the spinlock could be locked, otherwise \c false.
* \threadsafe
* \see Lock(), Unlock()
*/
bool TryLock(
unsigned int number_spins = 1
/**< [IN] Maximal count of spins to perform, trying to acquire lock.
* Note that passing 0 here results in not trying to obtain the lock at all.
* Default parameter is 1.
*/
);
/**
* Unlocks the spinlock.
*
* \pre The spinlock is locked by the current thread
* \post The spinlock is unlocked
* \threadsafe
* \see Lock(), TryLock()
*/
void Unlock();
private:
/**
* Disables copy construction and assignment.
*/
Spinlock(const Spinlock&);
Spinlock& operator=(const Spinlock&);
/**
* Internal spinlock from base_c
*/
embb_spinlock_t spinlock_;
};
/**
* Non-recursive, exclusive mutex.
*
* Mutexes of this type cannot be locked recursively, that is, multiple times
......@@ -123,6 +248,8 @@ class MutexBase {
*
* \see RecursiveMutex
* \ingroup CPP_BASE_MUTEX
*
* \concept{CPP_CONCEPTS_MUTEX}
*/
class Mutex : public internal::MutexBase {
public:
......@@ -182,7 +309,6 @@ class Mutex : public internal::MutexBase {
friend class ConditionVariable;
};
/**
* Recursive, exclusive mutex.
*
......@@ -193,6 +319,8 @@ class Mutex : public internal::MutexBase {
*
* \see Mutex
* \ingroup CPP_BASE_MUTEX
*
* \concept{CPP_CONCEPTS_MUTEX}
*/
class RecursiveMutex : public internal::MutexBase {
public:
......@@ -246,15 +374,16 @@ class RecursiveMutex : public internal::MutexBase {
RecursiveMutex& operator=(const RecursiveMutex&);
};
/**
* Scoped lock (according to the RAII principle) using a mutex.
*
* The mutex is locked on construction and unlocked on leaving the scope of the
* lock.
*
* \tparam Mutex Used mutex type
* \see Mutex, UniqueLock
* \tparam Mutex Used mutex type. Has to fulfil the
* \ref CPP_CONCEPTS_MUTEX_ANCHOR "Mutex Concept".
*
* \see UniqueLock
* \ingroup CPP_BASE_MUTEX
*/
template<typename Mutex = embb::base::Mutex>
......@@ -352,7 +481,8 @@ const AdoptLockTag adopt_lock = AdoptLockTag();
*
* \notthreadsafe
* \see Mutex, LockGuard
* \tparam Mutex Used mutex type
* \tparam Mutex Used mutex type. Has to fulfil the
* \ref CPP_CONCEPTS_MUTEX_ANCHOR "Mutex Concept".
* \ingroup CPP_BASE_MUTEX
*/
template<typename Mutex = embb::base::Mutex>
......@@ -482,7 +612,6 @@ class UniqueLock {
*/
friend class embb::base::ConditionVariable;
};
} // namespace base
} // namespace embb
......
......@@ -60,8 +60,43 @@ Mutex::Mutex() : MutexBase(EMBB_MUTEX_PLAIN) {
RecursiveMutex::RecursiveMutex() : MutexBase(EMBB_MUTEX_RECURSIVE) {
}
} // namespace base
} // namespace embb
Spinlock::Spinlock() {
embb_spin_init(&spinlock_);
}
Spinlock::~Spinlock() {
embb_spin_destroy(&spinlock_);
}
void Spinlock::Lock() {
int status = embb_spin_lock(&spinlock_);
// Currently, embb_spin_lock will always return EMBB_SUCCESS. However,
// This might change.
if (status != EMBB_SUCCESS) {
EMBB_THROW(ErrorException, "Error while locking spinlock");
}
}
bool Spinlock::TryLock(unsigned int number_spins) {
int status = embb_spin_try_lock(&spinlock_, number_spins);
if (status == EMBB_BUSY){
return false;
}
else if (status != EMBB_SUCCESS) {
EMBB_THROW(ErrorException, "Error while try-locking spinlock");
}
return true;
}
void Spinlock::Unlock() {
int status = embb_spin_unlock(&spinlock_);
if (status != EMBB_SUCCESS) {
EMBB_THROW(ErrorException, "Error while unlocking spinlock");
}
}
} // namespace base
} // namespace embb
......@@ -41,6 +41,7 @@ using embb::base::test::CoreSetTest;
using embb::base::test::DurationTest;
using embb::base::test::ConditionVarTest;
using embb::base::test::MutexTest;
using embb::base::test::SpinLockTest;
using embb::base::test::ThreadSpecificStorageTest;
using embb::base::test::AtomicTest;
using embb::base::test::MemoryAllocationTest;
......@@ -50,10 +51,12 @@ PT_MAIN("Base C++") {
unsigned int max_threads =
static_cast<unsigned int>(2 * partest::TestSuite::GetDefaultNumThreads());
embb_thread_set_max_count(max_threads);
PT_RUN(CoreSetTest);
PT_RUN(DurationTest);
PT_RUN(ConditionVarTest);
PT_RUN(MutexTest);
PT_RUN(SpinLockTest);
PT_RUN(ThreadSpecificStorageTest);
PT_RUN(AtomicTest);
PT_RUN(MemoryAllocationTest);
......
......@@ -32,7 +32,6 @@
namespace embb {
namespace base {
namespace test {
MutexTest::MutexTest() : mutex_(), counter_(0),
number_threads_(partest::TestSuite::GetDefaultNumThreads()),
number_iterations_(partest::TestSuite::GetDefaultNumIterations()) {
......@@ -209,6 +208,57 @@ void MutexTest::TestUniqueLock() {
}
}
SpinLockTest::SpinLockTest() : spinlock_(), counter_(0),
number_threads_(partest::TestSuite::GetDefaultNumThreads()),
number_iterations_(partest::TestSuite::GetDefaultNumIterations()),
counter_iterations_(10000) {
CreateUnit("Spinlock protected counter (using Lock)")
.Add(&SpinLockTest::TestSpinlockCountLock, this, number_threads_,
number_iterations_)
.Post(&SpinLockTest::PostSpinlockCount, this);
CreateUnit("Spinlock protected counter (using Trylock)")
.Add(&SpinLockTest::TestSpinlockCountLockTryLock, this, number_threads_,
number_iterations_)
.Post(&SpinLockTest::PostSpinlockCount, this);
CreateUnit("Test spinning (too many spins), single thread")
.Add(&SpinLockTest::TestSpinLockTooManySpins, this, 1, 1);
}
void SpinLockTest::TestSpinlockCountLock() {
for (unsigned int i = 0; i != counter_iterations_; ++i){
spinlock_.Lock();
counter_++;
spinlock_.Unlock();
}
}
void SpinLockTest::TestSpinlockCountLockTryLock() {
for (unsigned int i = 0; i != counter_iterations_; ++i){
while (!spinlock_.TryLock()) {}
counter_++;
spinlock_.Unlock();
}
}
void SpinLockTest::PostSpinlockCount() {
PT_EXPECT_EQ(counter_,
number_iterations_ *
number_threads_*
counter_iterations_);
counter_ = 0;
}
void SpinLockTest::TestSpinLockTooManySpins() {
Spinlock lock;
lock.Lock();
bool success = lock.TryLock(100);
PT_ASSERT(!success);
lock.Unlock();
success = lock.TryLock(100);
PT_ASSERT(success);
}
} // namespace test
} // namespace base
} // namespace embb
......@@ -89,6 +89,49 @@ class MutexTest : public partest::TestCase {
size_t number_iterations_;
};
class SpinLockTest : public partest::TestCase {
public:
SpinLockTest();
private:
/**
* Uses Spinlock to realize multi-threaded counting.
*/
void TestSpinlockCountLock();
void TestSpinlockCountLockTryLock();
void PostSpinlockCount();
/**
* Test that TryLock returns false, if lock is already locked.
*/
void TestSpinLockTooManySpins();
/**
* Spinlock for tests
*/
Spinlock spinlock_;
/**
* Shared counter to check effectiveness of mutex.
*/
size_t counter_;
/**
* Number of threads used to run tests.
*/
size_t number_threads_;
/**
* Number of times the test method is called by each thread.
*/
size_t number_iterations_;
/**
* Number of internal iterations, for incrementing the counter.
*/
size_t counter_iterations_;
};
} // namespace test
} // namespace base
} // namespace embb
......
......@@ -29,7 +29,7 @@
#if EMBB_DATAFLOW_TRACE_SIGNAL_HISTORY
#include <vector>
#include <embb/dataflow/internal/spinlock.h>
#include <embb/base/mutex.h>
#endif
#include <embb/dataflow/internal/signal.h>
......@@ -80,7 +80,7 @@ class In {
ClockListener * listener_;
bool connected_;
#if EMBB_DATAFLOW_TRACE_SIGNAL_HISTORY
SpinLock lock_;
embb::base::Spinlock lock_;
std::vector<SignalType> history_;
#endif
......
/*
* Copyright (c) 2014-2015, Siemens AG. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
*
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef EMBB_DATAFLOW_INTERNAL_SPINLOCK_H_
#define EMBB_DATAFLOW_INTERNAL_SPINLOCK_H_
#include <embb/base/atomic.h>
namespace embb {
namespace dataflow {
namespace internal {
class SpinLock {
public:
SpinLock() : lock_(0) {}
void Lock() {
int expected = 0;
while (!lock_.CompareAndSwap(expected, 1)) {
expected = 0;
}
}
bool TryLock(int spins = 0) {
int expected = 0;
while (spins > 0 && !lock_.CompareAndSwap(expected, 1)) {
expected = 0;
spins--;
}
return spins > 0;
}
void Unlock() {
lock_ = 0;
}
private:
embb::base::Atomic<int> lock_;
};
} // namespace internal
} // namespace dataflow
} // namespace embb
#endif // EMBB_DATAFLOW_INTERNAL_SPINLOCK_H_
......@@ -29,6 +29,7 @@
#include <embb_mtapi_alloc.h>
#include <embb_mtapi_log.h>
#include <embb_mtapi_id_pool_t.h>
#include <embb/base/c/mutex.h>
void embb_mtapi_id_pool_initialize(
embb_mtapi_id_pool_t * that,
......@@ -45,7 +46,7 @@ void embb_mtapi_id_pool_initialize(
that->ids_available = capacity;
that->put_id_position = 0;
that->get_id_position = 1;
embb_mtapi_spinlock_initialize(&that->lock);
embb_spin_init(&that->lock);
}
void embb_mtapi_id_pool_finalize(embb_mtapi_id_pool_t * that) {
......@@ -55,7 +56,7 @@ void embb_mtapi_id_pool_finalize(embb_mtapi_id_pool_t * that) {
that->put_id_position = 0;
embb_mtapi_alloc_deallocate(that->id_buffer);
that->id_buffer = NULL;
embb_mtapi_spinlock_finalize(&that->lock);
embb_spin_destroy(&that->lock);
}
mtapi_uint_t embb_mtapi_id_pool_allocate(embb_mtapi_id_pool_t * that) {
......@@ -63,7 +64,7 @@ mtapi_uint_t embb_mtapi_id_pool_allocate(embb_mtapi_id_pool_t * that) {
assert(MTAPI_NULL != that);
if (embb_mtapi_spinlock_acquire(&that->lock)) {
if (embb_spin_lock(&that->lock) == EMBB_SUCCESS) {
if (0 < that->ids_available) {
/* take away one id */
that->ids_available--;
......@@ -81,7 +82,7 @@ mtapi_uint_t embb_mtapi_id_pool_allocate(embb_mtapi_id_pool_t * that) {
/* make id entry invalid just in case */
that->id_buffer[id_position] = EMBB_MTAPI_IDPOOL_INVALID_ID;
}
embb_mtapi_spinlock_release(&that->lock);
embb_spin_unlock(&that->lock);
}
return id;
......@@ -92,7 +93,7 @@ void embb_mtapi_id_pool_deallocate(
mtapi_uint_t id) {
assert(MTAPI_NULL != that);
if (embb_mtapi_spinlock_acquire(&that->lock)) {
if (embb_spin_lock(&that->lock) == EMBB_SUCCESS) {
if (that->capacity > that->ids_available) {
/* acquire position to put id to */
mtapi_uint_t id_position = that->put_id_position;
......@@ -107,7 +108,7 @@ void embb_mtapi_id_pool_deallocate(
/* make it available */
that->ids_available++;
}
embb_mtapi_spinlock_release(&that->lock);
embb_spin_unlock(&that->lock);
} else {
embb_mtapi_log_error(
"could not acquire lock in embb_mtapi_IdPool_deallocate\n");
......
......@@ -29,8 +29,7 @@
#include <embb/mtapi/c/mtapi.h>
#include <embb/base/c/atomic.h>
#include <embb_mtapi_spinlock_t.h>
#include <embb/base/c/mutex.h>
#ifdef __cplusplus
extern "C" {
......@@ -51,7 +50,7 @@ struct embb_mtapi_id_pool_struct {
mtapi_uint_t ids_available;
mtapi_uint_t get_id_position;
mtapi_uint_t put_id_position;
embb_mtapi_spinlock_t lock;
embb_spinlock_t lock;
};
/**
......
......@@ -41,7 +41,6 @@
static embb_mtapi_node_t* embb_mtapi_node_instance = NULL;
extern embb_atomic_int embb_mtapi_spinlock_spins;
/* ---- CLASS MEMBERS ------------------------------------------------------ */
......@@ -87,8 +86,6 @@ void mtapi_initialize(
/* out of memory! */
local_status = MTAPI_ERR_UNKNOWN;
} else {
embb_atomic_store_int(&embb_mtapi_spinlock_spins, 0);
node = embb_mtapi_node_instance;
node->domain_id = domain_id;
......@@ -184,9 +181,6 @@ void mtapi_finalize(MTAPI_OUT mtapi_status_t* status) {
embb_mtapi_alloc_deallocate(node);
embb_mtapi_node_instance = MTAPI_NULL;
embb_mtapi_log_info("mtapi spinlock spun %d times.\n",
embb_atomic_load_int(&embb_mtapi_spinlock_spins));
local_status = MTAPI_SUCCESS;
} else {
local_status = MTAPI_ERR_NODE_NOTINIT;
......
......@@ -31,7 +31,7 @@
#include <embb/base/c/atomic.h>
#include <embb_mtapi_pool_template.h>
#include <embb_mtapi_spinlock_t.h>
#include <embb/base/c/mutex.h>
#ifdef __cplusplus
extern "C" {
......
/*
* Copyright (c) 2014-2015, Siemens AG. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
*
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include <embb_mtapi_spinlock_t.h>
void embb_mtapi_spinlock_initialize(embb_mtapi_spinlock_t * that) {
embb_atomic_store_int(that, 0);
}
void embb_mtapi_spinlock_finalize(embb_mtapi_spinlock_t * that) {
embb_atomic_store_int(that, 0);
}
embb_atomic_int embb_mtapi_spinlock_spins = { 0 };
mtapi_boolean_t embb_mtapi_spinlock_acquire(embb_mtapi_spinlock_t * that) {
int expected = 0;
while (0 == embb_atomic_compare_and_swap_int(that, &expected, 1)) {
/* empty */
embb_atomic_fetch_and_add_int(&embb_mtapi_spinlock_spins, 1);
expected = 0;
}
return MTAPI_TRUE;
}
mtapi_boolean_t embb_mtapi_spinlock_acquire_with_spincount(
embb_mtapi_spinlock_t * that,
mtapi_uint_t max_spin_count) {
int expected = 0;
mtapi_uint_t spin_count = max_spin_count;
while (0 == embb_atomic_compare_and_swap_int(that, &expected, 1)) {
embb_atomic_fetch_and_add_int(&embb_mtapi_spinlock_spins, 1);
spin_count--;
if (0 == spin_count) {
return MTAPI_FALSE;
}
expected = 0;
}
return MTAPI_TRUE;
}
mtapi_boolean_t embb_mtapi_spinlock_release(embb_mtapi_spinlock_t * that) {
int expected = 1;
return embb_atomic_compare_and_swap_int(that, &expected, 0) ?
MTAPI_TRUE : MTAPI_FALSE;
}
/*
* Copyright (c) 2014-2015, Siemens AG. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
*
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#ifndef MTAPI_C_SRC_EMBB_MTAPI_SPINLOCK_T_H_
#define MTAPI_C_SRC_EMBB_MTAPI_SPINLOCK_T_H_
#include <embb/mtapi/c/mtapi.h>
#include <embb/base/c/atomic.h>
#ifdef __cplusplus
extern "C" {
#endif
/* ---- CLASS DECLARATION -------------------------------------------------- */
typedef embb_atomic_int embb_mtapi_spinlock_t;
void embb_mtapi_spinlock_initialize(embb_mtapi_spinlock_t * that);
void embb_mtapi_spinlock_finalize(embb_mtapi_spinlock_t * that);
mtapi_boolean_t embb_mtapi_spinlock_acquire(embb_mtapi_spinlock_t * that);
mtapi_boolean_t embb_mtapi_spinlock_acquire_with_spincount(
embb_mtapi_spinlock_t * that,
mtapi_uint_t max_spin_count);
mtapi_boolean_t embb_mtapi_spinlock_release(embb_mtapi_spinlock_t * that);
#ifdef __cplusplus
}
#endif
#endif // MTAPI_C_SRC_EMBB_MTAPI_SPINLOCK_T_H_
......@@ -46,7 +46,7 @@ void embb_mtapi_task_queue_initialize(embb_mtapi_task_queue_t* that) {
that->get_task_position = 0;
that->put_task_position = 0;
mtapi_queueattr_init(&that->attributes, MTAPI_NULL);
embb_mtapi_spinlock_initialize(&that->lock);
embb_spin_init(&that->lock);
}
void embb_mtapi_task_queue_initialize_with_capacity(
......@@ -61,7 +61,7 @@ void embb_mtapi_task_queue_initialize_with_capacity(
that->put_task_position = 0;
mtapi_queueattr_init(&that->attributes, MTAPI_NULL);
that->attributes.limit = capacity;
embb_mtapi_spinlock_initialize(&that->lock);
embb_spin_init(&that->lock);
}
void embb_mtapi_task_queue_finalize(embb_mtapi_task_queue_t* that) {
......@@ -70,7 +70,7 @@ void embb_mtapi_task_queue_finalize(embb_mtapi_task_queue_t* that) {
embb_mtapi_task_queue_initialize(that);
embb_mtapi_spinlock_finalize(&that->lock);
embb_spin_destroy(&that->lock);
}
embb_mtapi_task_t * embb_mtapi_task_queue_pop(embb_mtapi_task_queue_t* that) {
......@@ -78,7 +78,7 @@ embb_mtapi_task_t * embb_mtapi_task_queue_pop(embb_mtapi_task_queue_t* that) {
assert(MTAPI_NULL != that);
if (embb_mtapi_spinlock_acquire_with_spincount(&that->lock, 128)) {
if (embb_spin_try_lock(&that->lock, 128) == EMBB_SUCCESS) {
if (0 < that->tasks_available) {
/* take away one task */
that->tasks_available--;
......@@ -96,7 +96,7 @@ embb_mtapi_task_t * embb_mtapi_task_queue_pop(embb_mtapi_task_queue_t* that) {
/* make task entry invalid just in case */
that->task_buffer[task_position] = MTAPI_NULL;
}
embb_mtapi_spinlock_release(&that->lock);
embb_spin_unlock(&that->lock);
}
return task;
......@@ -109,7 +109,7 @@ mtapi_boolean_t embb_mtapi_task_queue_push(
assert(MTAPI_NULL != that);
if (embb_mtapi_spinlock_acquire(&that->lock)) {
if (embb_spin_lock(&that->lock) == EMBB_SUCCESS) {
if (that->attributes.limit > that->tasks_available) {
/* acquire position to put task into */
mtapi_uint_t task_position = that->put_task_position;
......@@ -126,7 +126,7 @@ mtapi_boolean_t embb_mtapi_task_queue_push(
result = MTAPI_TRUE;
}
embb_mtapi_spinlock_release(&that->lock);
embb_spin_unlock(&that->lock);
}
return result;
......@@ -143,7 +143,7 @@ mtapi_boolean_t embb_mtapi_task_queue_process(
assert(MTAPI_NULL != that);
assert(MTAPI_NULL != process);
if (embb_mtapi_spinlock_acquire(&that->lock)) {
if (embb_spin_lock(&that->lock) == EMBB_SUCCESS) {
idx = that->get_task_position;
for (ii = 0; ii < that->tasks_available; ii++) {
result = process(that->task_buffer[ii], user_data);
......@@ -152,7 +152,7 @@ mtapi_boolean_t embb_mtapi_task_queue_process(
}
idx = (idx + 1) % that->attributes.limit;
}
embb_mtapi_spinlock_release(&that->lock);
embb_spin_unlock(&that->lock);
}
return result;
......
......@@ -30,7 +30,7 @@
#include <embb/mtapi/c/mtapi.h>
#include <embb/base/c/atomic.h>
#include <embb_mtapi_spinlock_t.h>
#include <embb/base/c/mutex.h>
#include <embb_mtapi_task_visitor_function_t.h>
#ifdef __cplusplus
......@@ -57,7 +57,7 @@ struct embb_mtapi_task_queue_struct {
mtapi_uint_t get_task_position;
mtapi_uint_t put_task_position;
mtapi_queue_attributes_t attributes;
embb_mtapi_spinlock_t lock;
embb_spinlock_t lock;
};
#include <embb_mtapi_task_queue_t_fwd.h>
......
......@@ -85,14 +85,14 @@ void embb_mtapi_task_initialize(embb_mtapi_task_t* that) {
that->queue.id = EMBB_MTAPI_IDPOOL_INVALID_ID;
that->error_code = MTAPI_SUCCESS;
embb_atomic_store_unsigned_int(&that->current_instance, 0);
embb_mtapi_spinlock_initialize(&that->state_lock);
embb_spin_init(&that->state_lock);
}
void embb_mtapi_task_finalize(embb_mtapi_task_t* that) {
assert(MTAPI_NULL != that);
embb_mtapi_task_initialize(that);
embb_mtapi_spinlock_finalize(&that->state_lock);
embb_spin_destroy(&that->state_lock);
}
mtapi_boolean_t embb_mtapi_task_execute(
......@@ -158,10 +158,10 @@ void embb_mtapi_task_set_state(
mtapi_task_state_t state) {
assert(MTAPI_NULL != that);
embb_mtapi_spinlock_acquire(&that->state_lock);
embb_spin_lock(&that->state_lock);
that->state = state;
embb_atomic_memory_barrier();
embb_mtapi_spinlock_release(&that->state_lock);
embb_spin_unlock(&that->state_lock);
}
static mtapi_task_hndl_t embb_mtapi_task_start(
......
......@@ -31,7 +31,7 @@
#include <embb/base/c/atomic.h>
#include <embb_mtapi_pool_template.h>
#include <embb_mtapi_spinlock_t.h>
#include <embb/base/c/mutex.h>
#ifdef __cplusplus
extern "C" {
......@@ -65,7 +65,7 @@ struct embb_mtapi_task_struct {
mtapi_queue_hndl_t queue;
mtapi_action_hndl_t action;
embb_mtapi_spinlock_t state_lock;
embb_spinlock_t state_lock;
volatile mtapi_task_state_t state;
embb_atomic_unsigned_int current_instance;
embb_atomic_unsigned_int instances_todo;
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or sign in to comment