Merge pull request #2810 from kumpera/fix_hazard_free
[mono.git] / mono / utils / hazard-pointer.c
index 46ca7779ffb68003ae31142f2b4bb7f7d5b6e56d..efe6b649b3039450cbdc176f0373cd55754a69be 100644 (file)
  * hazard-pointer.c: Hazard pointer related code.
  *
  * (C) Copyright 2011 Novell, Inc
+ * Licensed under the MIT license. See LICENSE file in the project root for full license information.
  */
 
 #include <config.h>
 
-#include <mono/metadata/class-internals.h>
+#include <string.h>
+
 #include <mono/utils/hazard-pointer.h>
 #include <mono/utils/mono-membar.h>
-#include <mono/utils/mono-mmap.h>
+#include <mono/utils/mono-memory-model.h>
 #include <mono/utils/monobitset.h>
+#include <mono/utils/lock-free-array-queue.h>
+#include <mono/utils/atomic.h>
+#include <mono/utils/mono-os-mutex.h>
+#ifdef SGEN_WITHOUT_MONO
+#include <mono/sgen/sgen-gc.h>
+#include <mono/sgen/sgen-client.h>
+#else
+#include <mono/utils/mono-mmap.h>
 #include <mono/utils/mono-threads.h>
+#include <mono/utils/mono-counters.h>
 #include <mono/io-layer/io-layer.h>
+#endif
 
 typedef struct {
        gpointer p;
        MonoHazardousFreeFunc free_func;
+       HazardFreeLocking locking;
 } DelayedFreeItem;
 
-enum {
-       DFE_STATE_FREE,
-       DFE_STATE_USED,
-       DFE_STATE_BUSY
-};
-
-typedef struct {
-       gint32 state;
-       DelayedFreeItem item;
-} DelayedFreeEntry;
-
-typedef struct _DelayedFreeChunk DelayedFreeChunk;
-struct _DelayedFreeChunk {
-       DelayedFreeChunk *next;
-       gint32 num_entries;
-       DelayedFreeEntry entries [MONO_ZERO_LEN_ARRAY];
-};
-
 /* The hazard table */
 #if MONO_SMALL_CONFIG
 #define HAZARD_TABLE_MAX_SIZE  256
+#define HAZARD_TABLE_OVERFLOW  4
 #else
 #define HAZARD_TABLE_MAX_SIZE  16384 /* There cannot be more threads than this number. */
+#define HAZARD_TABLE_OVERFLOW  64
 #endif
 
 static volatile int hazard_table_size = 0;
 static MonoThreadHazardPointers * volatile hazard_table = NULL;
+static MonoHazardFreeQueueSizeCallback queue_size_cb;
+
+/*
+ * Each entry is either 0 or 1, indicating whether that overflow small
+ * ID is busy.
+ */
+static volatile gint32 overflow_busy [HAZARD_TABLE_OVERFLOW];
 
 /* The table where we keep pointers to blocks to be freed but that
    have to wait because they're guarded by a hazard pointer. */
-static volatile gint32 num_used_delayed_free_entries;
-static DelayedFreeChunk *delayed_free_chunk_list;
+static MonoLockFreeArrayQueue delayed_free_queue = MONO_LOCK_FREE_ARRAY_QUEUE_INIT (sizeof (DelayedFreeItem));
 
 /* The table for small ID assignment */
-static CRITICAL_SECTION small_id_mutex;
+static mono_mutex_t small_id_mutex;
 static int small_id_next;
 static int highest_small_id = -1;
 static MonoBitSet *small_id_table;
-
-/*
- * Delayed free table
- *
- * The table is a linked list of arrays (chunks).  Chunks are never
- * removed from the list, only added to the end, in a lock-free manner.
- *
- * Adding or removing an entry in the table is only possible at the end.
- * To do so, the thread first has to increment or decrement
- * num_used_delayed_free_entries.  The entry thus added or removed now
- * "belongs" to that thread.  It first CASes the state to BUSY,
- * writes/reads the entry, and then sets the state to USED or FREE.
- *
- * Note that it's possible that there is contention.  Some thread will
- * always make progress, though.
- *
- * The simplest case of contention is one thread pushing and another
- * thread popping the same entry.  The state is FREE at first, so the
- * pushing thread succeeds in setting it to BUSY.  The popping thread
- * will only succeed with its CAS once the state is USED, which is the
- * case after the pushing thread has finished pushing.
- */
-
-static DelayedFreeChunk*
-alloc_delayed_free_chunk (void)
-{
-       int size = mono_pagesize ();
-       int num_entries = (size - (sizeof (DelayedFreeChunk) - sizeof (DelayedFreeEntry) * MONO_ZERO_LEN_ARRAY)) / sizeof (DelayedFreeEntry);
-       DelayedFreeChunk *chunk = mono_valloc (0, size, MONO_MMAP_READ | MONO_MMAP_WRITE);
-       chunk->num_entries = num_entries;
-       return chunk;
-}
-
-static void
-free_delayed_free_chunk (DelayedFreeChunk *chunk)
-{
-       mono_vfree (chunk, mono_pagesize ());
-}
-
-static DelayedFreeEntry*
-get_delayed_free_entry (int index)
-{
-       DelayedFreeChunk *chunk;
-
-       g_assert (index >= 0);
-
-       if (!delayed_free_chunk_list) {
-               chunk = alloc_delayed_free_chunk ();
-               mono_memory_write_barrier ();
-               if (InterlockedCompareExchangePointer ((volatile gpointer *)&delayed_free_chunk_list, chunk, NULL) != NULL)
-                       free_delayed_free_chunk (chunk);
-       }
-
-       chunk = delayed_free_chunk_list;
-       g_assert (chunk);
-
-       while (index >= chunk->num_entries) {
-               DelayedFreeChunk *next = chunk->next;
-               if (!next) {
-                       next = alloc_delayed_free_chunk ();
-                       mono_memory_write_barrier ();
-                       if (InterlockedCompareExchangePointer ((volatile gpointer *) &chunk->next, next, NULL) != NULL) {
-                               free_delayed_free_chunk (next);
-                               next = chunk->next;
-                               g_assert (next);
-                       }
-               }
-               index -= chunk->num_entries;
-               chunk = next;
-       }
-
-       return &chunk->entries [index];
-}
-
-static void
-delayed_free_push (DelayedFreeItem item)
-{
-       int index, num_used;
-       DelayedFreeEntry *entry;
-
-       do {
-               index = InterlockedIncrement (&num_used_delayed_free_entries) - 1;
-               entry = get_delayed_free_entry (index);
-       } while (InterlockedCompareExchange (&entry->state, DFE_STATE_BUSY, DFE_STATE_FREE) != DFE_STATE_FREE);
-
-       mono_memory_write_barrier ();
-
-       entry->item = item;
-
-       mono_memory_write_barrier ();
-
-       entry->state = DFE_STATE_USED;
-
-       mono_memory_write_barrier ();
-
-       do {
-               num_used = num_used_delayed_free_entries;
-               if (num_used > index)
-                       break;
-       } while (InterlockedCompareExchange (&num_used_delayed_free_entries, index + 1, num_used) != num_used);
-}
-
-static gboolean
-delayed_free_pop (DelayedFreeItem *item)
-{
-       int index;
-       DelayedFreeEntry *entry;
-
-       do {
-               do {
-                       index = num_used_delayed_free_entries;
-                       if (index == 0)
-                               return FALSE;
-               } while (InterlockedCompareExchange (&num_used_delayed_free_entries, index - 1, index) != index);
-
-               entry = get_delayed_free_entry (index - 1);
-       } while (InterlockedCompareExchange (&entry->state, DFE_STATE_BUSY, DFE_STATE_USED) != DFE_STATE_USED);
-
-       mono_memory_barrier ();
-
-       *item = entry->item;
-
-       mono_memory_barrier ();
-
-       entry->state = DFE_STATE_FREE;
-
-       return TRUE;
-}
+static int hazardous_pointer_count;
 
 /*
  * Allocate a small thread id.
@@ -197,12 +73,12 @@ mono_thread_small_id_alloc (void)
 {
        int i, id = -1;
 
-       EnterCriticalSection (&small_id_mutex);
+       mono_os_mutex_lock (&small_id_mutex);
 
        if (!small_id_table)
                small_id_table = mono_bitset_new (1, 0);
 
-       id = mono_bitset_find_first_unset (small_id_table, small_id_next);
+       id = mono_bitset_find_first_unset (small_id_table, small_id_next - 1);
        if (id == -1)
                id = mono_bitset_find_first_unset (small_id_table, -1);
 
@@ -235,7 +111,7 @@ mono_thread_small_id_alloc (void)
                int num_pages = (hazard_table_size * sizeof (MonoThreadHazardPointers) + pagesize - 1) / pagesize;
 
                if (hazard_table == NULL) {
-                       hazard_table = mono_valloc (NULL,
+                       hazard_table = (MonoThreadHazardPointers *volatile) mono_valloc (NULL,
                                sizeof (MonoThreadHazardPointers) * HAZARD_TABLE_MAX_SIZE,
                                MONO_MMAP_NONE);
                }
@@ -259,7 +135,7 @@ mono_thread_small_id_alloc (void)
                mono_memory_write_barrier ();
        }
 
-       LeaveCriticalSection (&small_id_mutex);
+       mono_os_mutex_unlock (&small_id_mutex);
 
        return id;
 }
@@ -268,27 +144,29 @@ void
 mono_thread_small_id_free (int id)
 {
        /* MonoBitSet operations are not atomic. */
-       EnterCriticalSection (&small_id_mutex);
+       mono_os_mutex_lock (&small_id_mutex);
 
        g_assert (id >= 0 && id < small_id_table->size);
        g_assert (mono_bitset_test_fast (small_id_table, id));
        mono_bitset_clear_fast (small_id_table, id);
 
-       LeaveCriticalSection (&small_id_mutex);
+       mono_os_mutex_unlock (&small_id_mutex);
 }
 
 static gboolean
 is_pointer_hazardous (gpointer p)
 {
-       int i;
+       int i, j;
        int highest = highest_small_id;
 
        g_assert (highest < hazard_table_size);
 
        for (i = 0; i <= highest; ++i) {
-               if (hazard_table [i].hazard_pointers [0] == p
-                               || hazard_table [i].hazard_pointers [1] == p)
-                       return TRUE;
+               for (j = 0; j < HAZARD_POINTER_COUNT; ++j) {
+                       if (hazard_table [i].hazard_pointers [j] == p)
+                               return TRUE;
+                       LOAD_LOAD_FENCE;
+               }
        }
 
        return FALSE;
@@ -297,21 +175,14 @@ is_pointer_hazardous (gpointer p)
 MonoThreadHazardPointers*
 mono_hazard_pointer_get (void)
 {
-       MonoThreadInfo *current_thread = mono_thread_info_current ();
+       int small_id = mono_thread_info_get_small_id ();
 
-       if (!(current_thread && current_thread->small_id >= 0)) {
+       if (small_id < 0) {
                static MonoThreadHazardPointers emerg_hazard_table;
-               g_warning ("Thread %p may have been prematurely finalized", current_thread);
+               g_warning ("Thread %p may have been prematurely finalized", (gpointer) (gsize) mono_native_thread_id_get ());
                return &emerg_hazard_table;
        }
 
-       return &hazard_table [current_thread->small_id];
-}
-
-MonoThreadHazardPointers*
-mono_hazard_pointer_get_by_id (int small_id)
-{
-       g_assert (small_id >= 0 && small_id <= highest_small_id);
        return &hazard_table [small_id];
 }
 
@@ -345,17 +216,89 @@ get_hazardous_pointer (gpointer volatile *pp, MonoThreadHazardPointers *hp, int
        return p;
 }
 
+int
+mono_hazard_pointer_save_for_signal_handler (void)
+{
+       int small_id, i;
+       MonoThreadHazardPointers *hp = mono_hazard_pointer_get ();
+       MonoThreadHazardPointers *hp_overflow;
+
+       for (i = 0; i < HAZARD_POINTER_COUNT; ++i)
+               if (hp->hazard_pointers [i])
+                       goto search;
+       return -1;
+
+ search:
+       for (small_id = 0; small_id < HAZARD_TABLE_OVERFLOW; ++small_id) {
+               if (!overflow_busy [small_id])
+                       break;
+       }
+
+       /*
+        * If this assert fails we don't have enough overflow slots.
+        * We should contemplate adding them dynamically.  If we can
+        * make mono_thread_small_id_alloc() lock-free we can just
+        * allocate them on-demand.
+        */
+       g_assert (small_id < HAZARD_TABLE_OVERFLOW);
+
+       if (InterlockedCompareExchange (&overflow_busy [small_id], 1, 0) != 0)
+               goto search;
+
+       hp_overflow = &hazard_table [small_id];
+
+       for (i = 0; i < HAZARD_POINTER_COUNT; ++i)
+               g_assert (!hp_overflow->hazard_pointers [i]);
+       *hp_overflow = *hp;
+
+       mono_memory_write_barrier ();
+
+       memset (hp, 0, sizeof (MonoThreadHazardPointers));
+
+       return small_id;
+}
+
+void
+mono_hazard_pointer_restore_for_signal_handler (int small_id)
+{
+       MonoThreadHazardPointers *hp = mono_hazard_pointer_get ();
+       MonoThreadHazardPointers *hp_overflow;
+       int i;
+
+       if (small_id < 0)
+               return;
+
+       g_assert (small_id < HAZARD_TABLE_OVERFLOW);
+       g_assert (overflow_busy [small_id]);
+
+       for (i = 0; i < HAZARD_POINTER_COUNT; ++i)
+               g_assert (!hp->hazard_pointers [i]);
+
+       hp_overflow = &hazard_table [small_id];
+
+       *hp = *hp_overflow;
+
+       mono_memory_write_barrier ();
+
+       memset (hp_overflow, 0, sizeof (MonoThreadHazardPointers));
+
+       mono_memory_write_barrier ();
+
+       overflow_busy [small_id] = 0;
+}
+
 static gboolean
-try_free_delayed_free_item (void)
+try_free_delayed_free_item (HazardFreeContext context)
 {
        DelayedFreeItem item;
-       gboolean popped = delayed_free_pop (&item);
+       gboolean popped = mono_lock_free_array_queue_pop (&delayed_free_queue, &item);
 
        if (!popped)
                return FALSE;
 
-       if (is_pointer_hazardous (item.p)) {
-               delayed_free_push (item);
+       if ((context == HAZARD_FREE_ASYNC_CTX && item.locking == HAZARD_FREE_MAY_LOCK) ||
+           (is_pointer_hazardous (item.p))) {
+               mono_lock_free_array_queue_push (&delayed_free_queue, &item);
                return FALSE;
        }
 
@@ -364,56 +307,100 @@ try_free_delayed_free_item (void)
        return TRUE;
 }
 
+/**
+ * mono_thread_hazardous_try_free:
+ * @p: the pointer to free
+ * @free_func: the function that can free the pointer
+ *
+ * If @p is not a hazardous pointer it will be immediately freed by calling @free_func.
+ * Otherwise it will be queued for later.
+ *
+ * Use this function if @free_func can ALWAYS be called in the context where this function is being called.
+ *
+ * This function doesn't pump the free queue so try to accommodate a call at an appropriate time.
+ * See mono_thread_hazardous_try_free_some for when it's appropriate.
+ *
+ * Return: TRUE if @p was free or FALSE if it was queued.
+ */
+gboolean
+mono_thread_hazardous_try_free (gpointer p, MonoHazardousFreeFunc free_func)
+{
+       if (!is_pointer_hazardous (p)) {
+               free_func (p);
+               return TRUE;
+       } else {
+               mono_thread_hazardous_queue_free (p, free_func);
+               return FALSE;
+       }
+}
+
+/**
+ * mono_thread_hazardous_queue_free:
+ * @p: the pointer to free
+ * @free_func: the function that can free the pointer
+ *
+ * Queue @p to be freed later. @p will be freed once the hazard free queue is pumped.
+ *
+ * This function doesn't pump the free queue so try to accommodate a call at an appropriate time.
+ * See mono_thread_hazardous_try_free_some for when it's appropriate.
+ *
+ */
 void
-mono_thread_hazardous_free_or_queue (gpointer p, MonoHazardousFreeFunc free_func)
+mono_thread_hazardous_queue_free (gpointer p, MonoHazardousFreeFunc free_func)
 {
-       int i;
+       DelayedFreeItem item = { p, free_func, HAZARD_FREE_MAY_LOCK };
 
-       /* First try to free a few entries in the delayed free
-          table. */
-       for (i = 0; i < 3; ++i)
-               try_free_delayed_free_item ();
+       InterlockedIncrement (&hazardous_pointer_count);
 
-       /* Now see if the pointer we're freeing is hazardous.  If it
-          isn't, free it.  Otherwise put it in the delay list. */
-       if (is_pointer_hazardous (p)) {
-               DelayedFreeItem item = { p, free_func };
+       mono_lock_free_array_queue_push (&delayed_free_queue, &item);
 
-               ++mono_stats.hazardous_pointer_count;
+       guint32 queue_size = delayed_free_queue.num_used_entries;
+       if (queue_size && queue_size_cb)
+               queue_size_cb (queue_size);
+}
 
-               delayed_free_push (item);
-       } else {
-               free_func (p);
-       }
+
+void
+mono_hazard_pointer_install_free_queue_size_callback (MonoHazardFreeQueueSizeCallback cb)
+{
+       queue_size_cb = cb;
 }
 
 void
 mono_thread_hazardous_try_free_all (void)
 {
-       while (try_free_delayed_free_item ())
+       while (try_free_delayed_free_item (HAZARD_FREE_SAFE_CTX))
                ;
 }
 
+void
+mono_thread_hazardous_try_free_some (void)
+{
+       int i;
+       for (i = 0; i < 10; ++i)
+               try_free_delayed_free_item (HAZARD_FREE_SAFE_CTX);
+}
+
 void
 mono_thread_smr_init (void)
 {
-       InitializeCriticalSection(&small_id_mutex);
+       int i;
+
+       mono_os_mutex_init_recursive(&small_id_mutex);
+       mono_counters_register ("Hazardous pointers", MONO_COUNTER_JIT | MONO_COUNTER_INT, &hazardous_pointer_count);
+
+       for (i = 0; i < HAZARD_TABLE_OVERFLOW; ++i) {
+               int small_id = mono_thread_small_id_alloc ();
+               g_assert (small_id == i);
+       }
 }
 
 void
 mono_thread_smr_cleanup (void)
 {
-       DelayedFreeChunk *chunk;
-
        mono_thread_hazardous_try_free_all ();
 
-       chunk = delayed_free_chunk_list;
-       delayed_free_chunk_list = NULL;
-       while (chunk) {
-               DelayedFreeChunk *next = chunk->next;
-               free_delayed_free_chunk (chunk);
-               chunk = next;
-       }
+       mono_lock_free_array_queue_cleanup (&delayed_free_queue);
 
        /*FIXME, can't we release the small id table here?*/
 }