Skip to content
Snippets Groups Projects
  1. Jul 25, 2013
    • Nadav Har'El's avatar
      Condvar: Allow condvar->wait() to take mutex reference · 9a085f36
      Nadav Har'El authored
      As a convenience, overload condvar->wait() to also take a mutex reference,
      not just a mutex pointer.
      9a085f36
    • Nadav Har'El's avatar
      Condvar: Add some tracepoints · f7d6e269
      Nadav Har'El authored
      f7d6e269
    • Nadav Har'El's avatar
      Condvar: Order the condvar_wake_all wakeups by CPU · ac76d75a
      Nadav Har'El authored
      Because of wait morphing, we now have full control of the thread wakeup
      order on condvar_wake_all() - they are all be queued in the mutex's
      wait queue in a certain order, and woken up one by one in that order.
      
      Posix Threads leaves this order undetermined, saying that "the scheduling
      policy shall determine the order". In this patch we improve wakeup
      performance by ordering the wakeups by CPU: When a thread on some CPU
      wakes up (by its unlock()) another thread on the same CPU, it is faster
      than waking up a thread on another CPU.
      ac76d75a
    • Nadav Har'El's avatar
      Condvar: Make thread::tcpu() a const function · ccc911b6
      Nadav Har'El authored
      thread::tcpu() doesn't change the thread object, so let's mark it const,
      so it can be used on const sched::thread objects.
      
      We'll need it in the following patch, when we use it on wait_record.thread().
      ccc911b6
    • Nadav Har'El's avatar
      Condvar: Add wait morphing to condvar · aa3a6244
      Nadav Har'El authored
      This patch adds wait morphing to condvar:
      
      1. condvar->wake*() doesn't wake the thread to take the user mutex. Rather,
      it attempts to grab the lock for the sleeping thread, and if the lock is
      already taken, move the sleeping thread to wait on the mutex's queue,
      without waking the thread up.
      
      2. condvar->wait() now assumes that when it is woken up, it already has
      the mutex.
      
      Wait morphing reduces unnecessary context switches, and therefore improves
      performance, in two case:
      
      1. The "thundering herd" problem - when there are many threads waiting on
      the condvar, if  condvar->wake_all() wakes all of them, all will race to get
      the mutex and likely many of them will go back to sleep.
      
      2. The "locked wakeup" problem - when condvar_>wake*() is done with the user
      mutex locked (as it is very often does), if we wake a waiter to take the
      lock, it may find the lock already held (by the waker) and go back to sleep.
      aa3a6244
    • Nadav Har'El's avatar
      Condvar: Remember the mutex that the user associated with a condvar · 45dc4dcc
      Nadav Har'El authored
      Until now, we allowed each condvar->wait() call to specify a different
      mutex. To support "wait morphing", we need the condvar's waker to know
      which mutex the waiter wants to lock; If it can be a different mutex
      for each wait, we'll need to remember the mutex in wait_record. Adding
      a field to condvar's wait_record but not to mutex's wait_record is possible
      but quite ugly. This patch implements a simpler solution:
      
      In practice, condvar users normally "associate" a single mutex with a
      condvar, and use just it when wait()ing on the condvar. Posix threads
      even officially supports only this use case, and pthread_cond_wait(3p)
      states that "using more than one mutex for concurrent ... pthread_cond_wait
      operations on the same condition variable is undefined".
      
      So this patch remembers for a condvar the last mutex used in condvar->wait(),
      and in a later patch we will use it to implement wait morphing: a wake()
      will take this mutex, instead of waking up the thread to take it. We add
      assertions that verify that this assumption is not broken by the user.
      
      The price we pay for this simplicity is the new assumption on the single
      mutex per condvar, and adding 8 more bytes to the size of a condvar.
      45dc4dcc
    • Nadav Har'El's avatar
      Condvar: Move unlock of user mutex · 7351461a
      Nadav Har'El authored
      Move the unlocking of the user's mutex in condvar_wait() a bit earlier,
      while we still hold the condvar's internal mutex.
      
      This does not change correctness, but it is needed for the wait morphing
      protocol, where we assume that once condvar_wake() finds this thread's
      wait_record (which can happen as soon as we release the internal mutex),
      we are no longer holding the user mutex.
      7351461a
    • Nadav Har'El's avatar
      Condvar: Add mutex->send_lock(wait_record *) method · 5ec8e6d0
      Nadav Har'El authored
      Add a mutex->send_lock(wait_record *) which is similar to lock(), but
      rather than taking the mutex for the current thread it takes it for a
      different thread which is already waiting on the given wait_record.
      
      The thread waiting on wait_record is woken with the lock taken for it,
      and needs to accept the lock by calling mutex->receive_lock().
      
      This feature will be used in a later patch to enable "wait morphing" -
      moving a waiter from a condvar's wait queue to a mutex's wait queue.
      5ec8e6d0
    • Nadav Har'El's avatar
      Condvar: Use wait_record in condvar · eceaefaf
      Nadav Har'El authored
      Use wait_record in condvar, instead of ccondvar_waiter.
      
      We use wait_record's methods, wake() and wait(), instead of including
      their tricky code in condvar.cc.
      
      Unfortunately, this patch also contains a bunch of non-iteresting changes,
      replacing the name of ccondvar_waiter's "newer" field with wait_record's
      "next".
      eceaefaf
    • Nadav Har'El's avatar
      Condvar: Use wait_record in lockfree::mutex · 7bbd7a00
      Nadav Har'El authored
      Use wait_record in lockfree::mutex, instead of linked_item<thread *>.
      
      We use wait_record's methods, wake() and wait(), instead of including
      their tricky code in lfmutex.cc.
      7bbd7a00
    • Nadav Har'El's avatar
      Condvar: Make linked_item type unnecessary · 6c44f99d
      Nadav Har'El authored
      The lock-free queue, queue_mpsc, used to assume that the record stored in
      the queue has a type linked_item<T>. The template doesn't *really* need to
      assume this type - all it really needs is that the queued record has inside
      it a "next" pointer.
      
      In this patch, we allow queue_mpsc to take any type LT which has a field
      "LT *next". linked_item<T> is left just as an example implementation of LT,
      but more importantly, the "struct wait_record" defined in the previous
      patch can also be used in queue_mpsc because it has a "next" pointer.
      6c44f99d
    • Nadav Har'El's avatar
      Condvar: New "wait_record" type · a00aa4ab
      Nadav Har'El authored
      Both mutex and condvar have a wait queue - which is a linked list of
      wait records, each containing a thread pointer to wake and a "next" pointer.
      Unfortunately, mutex and condvar each used a different type: mutex used
      linked_item<thread*>, while condvar used struct ccondvar_waiter.
      
      We want both mutex and condvar to use the same wait_record structure,
      so we can add in a later patch the "wait morphing" feature (moving a
      waiter from the condvar's queue to a mutex's queue).
      
      This patch defines a single type, "struct wait_record", suitable for
      both uses. In particular, it is a struct, not a template, so that pointers
      to it can be used in C code (see <osv/condvar.h>).
      
      wait_record is a structure containing a "waiter" and a "next" pointer.
      The "waiter" is just a thread pointer, which together with a few methods
      becomes a simple synchronization mechanism which we always used but now
      for the first time we encapsulate it in a type.
      a00aa4ab
    • Avi Kivity's avatar
      percpu: fix failure with gcc 4.7.2 · a0263334
      Avi Kivity authored
      The lambda captures 'this', but at runtime, it turns out to be zero.
      Fails with gcc 4.7.2, works with gcc 4.8.1.
      
      Replace with std::bind() and a helper.
      a0263334
  2. Jul 24, 2013
    • Dor Laor's avatar
      Fix memcached networking issue that was caused by unsecure, parallel device... · 5f1e97d6
      Dor Laor authored
      Fix memcached networking issue that was caused by unsecure, parallel device invocation. I wasn't aware that the a device lock has to be held on the tx callback. Will look into it deeper tomorrow. The patch solves the issue
      5f1e97d6
    • Dor Laor's avatar
      Rename _lock to _tx_gc_lock · e2ba7581
      Dor Laor authored
      e2ba7581
    • Dor Laor's avatar
      Use wake_with scheme in order not to wake w/ the lock held · 2379771f
      Dor Laor authored
      This way it's possible to wake a thread while holding the lock
      that protects the thread pointer of going away. The lock itself
      won't be held by the waker and thus the wakee will be able to
      use it immedietly w/o ctx. Suggested by Nadav.
      2379771f
    • Nadav Har'El's avatar
      Add another test to pipe · 2fc78575
      Nadav Har'El authored
      I wasn't sure that read() and write() on pipe correctly avoided poll_wake()
      when the other side of the pipe was closed, so I added this test. Turns
      out it already works correctly - because poll_wake() checks for a zero
      file pointer and ignores it, so it's fine to give it a zero file pointer.
      2fc78575
  3. Jul 21, 2013
    • Avi Kivity's avatar
      Merge branch 'netperf' · 96a56d83
      Avi Kivity authored
      Scheduler and allocator improvements.
      96a56d83
    • Avi Kivity's avatar
      memory: lockless page allocation · c5e88254
      Avi Kivity authored
      Since the memory pools are backed by the page allocator, we need a fast
      page allocator, particularly for pools of large objects (with 1-2 objects per
      page, a page is exhausted very quickly).
      
      This patch adds a per-cpu cache of allocated pages.  Pages are allocated
      from (and freed to) the cache without locking; the buffer is filled or drained
      when it is empty or full, taking the page range lock.
      c5e88254
    • Avi Kivity's avatar
      mempool: add hysteresis · c549e0e8
      Avi Kivity authored
      If we allocate and free just one object in an empty pool, we will
      continuously allocate a page, format it for the pool, then free it.
      
      This is wastefull, so allow the pool to keep one empty page.  The page is kept
      at the back of the free list, so it won't get fragemented needlessly.
      c549e0e8
    • Avi Kivity's avatar
      mempool: switch to dynamic_percpu · a76e1813
      Avi Kivity authored
      Instead of an array of 64 free lists, let dynamic_percpu<> manage the
      allocations for us.  This reduces waste since we no longer require cache line
      alignment.
      a76e1813
    • Avi Kivity's avatar
      per_cpu_counter: switch to dynamic_percpu · 03a711c7
      Avi Kivity authored
      Instead of managing the counters manually, use the generic infrastructure.
      03a711c7
    • Avi Kivity's avatar
      percpu: introduce dynamic_percpu<> · 44bd271f
      Avi Kivity authored
      dynamic_percpu<T> allocates and initializes an object of type T on all cpus
      (if a cpu is later hotplugged, it will also get an instance).  Unlike ordinary
      percpu variables, dynamic_percpu objects can be used in a dynamic scope, that
      is, in objects that are not in static scope (one the stack or heap).
      44bd271f
    • Avi Kivity's avatar
      mempool: make the early allocator not depend on mempools · c148b754
      Avi Kivity authored
      With dynamic percpu allocations, the allocator won't be available until
      the first cpu is created.  This creates a circular dependency, since the
      first cpu itself needs to be allocated.
      
      Use a simple and wasteful allocator in that time until we're ready.  Objects
      allocated by the simple allocator are marked by having a page offset of 8.
      c148b754
  4. Jul 20, 2013
  5. Jul 19, 2013
  6. Jul 18, 2013
    • Avi Kivity's avatar
      sched: initialize tls earlier · 6d2448f1
      Avi Kivity authored
      tls is needed for per-cpu storage, so initialize it before the rest of the
      scheduler.
      6d2448f1
    • Avi Kivity's avatar
      Reorganize startup order · 5984eb5d
      Avi Kivity authored
      Make the early allocator available earlier to support the dynamic
      per-cpu allocator.
      5984eb5d
    • Avi Kivity's avatar
      sched: make cpu::current() not depend on the current thread · 7f7df848
      Avi Kivity authored
      Depending on the current thread causes a circular dependency with later
      patches.
      
      Use a per-thread variable instead, which is maintained on migrations similarly
      to percpu_base.  A small speedup is a nice side effect.
      7f7df848
    • Avi Kivity's avatar
      decf07ba
    • Avi Kivity's avatar
      memory: move page allocation functions to its own header · 7a4cf22f
      Avi Kivity authored
      Avoid a #include loop with later patches.
      7a4cf22f
    • Avi Kivity's avatar
      sched: penalize threads that preempt too much · e2f0c5aa
      Avi Kivity authored
      A preemption is expensive, both in the cycles spent in the scheduler, and
      in cache lines being evicted by the new thread.
      
      Penalize threads that cause preemption by adding a small preemption tax
      to their vruntime; this will decrease their relative priority.  Threads
      that sleep a long time will be relatively unaffected and retain low latency;
      threads that wake up very often, such us those in a wait/wake loop with
      another thread, will be penalized a lot and avoid excessive wakes.
      e2f0c5aa
    • Avi Kivity's avatar
      sched: limit vruntime backlog accrued to a sleeping thread · b0e7f721
      Avi Kivity authored
      With the current implementation, a sleeping thread can accrue a large vruntime
      backlog by sleeping.  This will result in this thread preempting anything that
      moves for a while.
      
      The borrow mechanism attempts to correct for this, but isn't working well.
      
      Reduce the backlog by limiting the vruntime difference to a single round
      trip of all currently queued threads.  The borrow mechanism is removed.
      
      This is similar to Guy's patch, except vruntime only moves forward, so it
      is capped only in the negative (minimum) direction, not forward.  It is also
      similar to Linux cfs.
      b0e7f721
    • Avi Kivity's avatar
      sched: more reasonable initial thread vruntime · 71d4c88a
      Avi Kivity authored
      
      Currently we initialize a new thread's vruntime to the clock time.  However,
      as only acquire vruntime as they run, while the clock always runs, this is
      unreasonably high.
      
      Initialize it instead to the parent thread's vruntime.  Since the parent thread
      is running now, its vruntime represents fairly high priority; we may want to
      tune that later.
      
      Signed-off-by: default avatarAvi Kivity <avi@cloudius-systems.com>
      71d4c88a
    • Avi Kivity's avatar
      x64: add an optimized memset() implementation · 20b15e78
      Avi Kivity authored
      20b15e78
    • Dor Laor's avatar
      d3421d25
    • Nadav Har'El's avatar
      Micro-benchmark for waking condvar on which no-one is waiting · 0080ee69
      Nadav Har'El authored
      This patch adds to tst-condvar two benchmark for measuring
      condvar::wake_all() on a condvar that nobody is waiting on.
      
      The first benchmark does these wakes from a single thread, measuring
      26ns before commit 3509b19b, and
      only 3ns after it.
      
      The second benchmark does wake_all() loops from two threads on two
      different CPUs. Before the aforementioned commit, this frequently
      involved a contented mutex and context switches, with as much as
      30,000 ns delay. After that commit, this benchmark measures 3ns,
      the same as the single-threaded benchmark.
      0080ee69
    • Nadav Har'El's avatar
      Improve performance of unwaited condvar_wake_one()/all() · 3509b19b
      Nadav Har'El authored
      Previously, condvar_wake_one()/all() took the condvar's internal lock
      before testing if anyone is waiting; A condvar_wake when nobody was
      waiting was mutex_lock()+mutex_unlock() time (on my machine, 26 ns)
      when there is no contention, but much much higher (involving a context
      switch) when several CPUs are trying condvar_wake concurrently.
      
      In this patch, we first test if the queue head is null before
      acquiring the lock, and only acquire the lock if it isn't.
      Now the condvar_wake-on-an-empty-queue micro-benchmark (see next patch)
      takes less than 3ns - regardless of how many CPUs are doing it
      concurrently.
      
      Note that the queue head we test is NOT atomic, and we do not
      use any memory fences. If we read the queue head and see there 0,
      it is safe to decide nobody is waiting and do nothing. But if we
      read the queue head and see != 0, we can't do anything with the
      value we read - it might be only half-set (if the pointer is not
      atomic on this architecture) or be set but the value it points
      to is not (we didn't use a memory fence to enforce any ordering).
      So if we see the head is != 0, we need to acquire the lock (which
      also imposes the required memory visibility and ordering) and try
      again.
      3509b19b
  7. Jul 17, 2013
Loading