public inbox for gcc-patches@gcc.gnu.org
 help / color / mirror / Atom feed
* [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes
@ 2018-09-25 13:13 Chung-Lin Tang
  2018-10-05 14:07 ` Tom de Vries
                   ` (2 more replies)
  0 siblings, 3 replies; 9+ messages in thread
From: Chung-Lin Tang @ 2018-09-25 13:13 UTC (permalink / raw)
  To: gcc-patches, Tom de Vries

[-- Attachment #1: Type: text/plain, Size: 3304 bytes --]

Hi Tom,
this patch removes large portions of plugin/plugin-nvptx.c, since a lot of it is
now in oacc-async.c now. The new code is essentially a NVPTX/CUDA-specific implementation
of the new-style goacc_asyncqueues.

Also, some needed functions in cuda-lib.def are added. The cuda.h function has also
been updated to build independently without a CUDA installation.

Thanks,
Chung-Lin

	* plugin/plugin-nvptx.c (struct cuda_map): Remove.
	(struct ptx_stream): Remove.
	(struct nvptx_thread): Remove current_stream field.
	(cuda_map_create): Remove.
	(cuda_map_destroy): Remove.
	(map_init): Remove.
	(map_fini): Remove.
	(map_pop): Remove.
	(map_push): Remove.
	(struct goacc_asyncqueue): Define.
	(struct nvptx_callback): Define.
	(struct ptx_free_block): Define.
	(struct ptx_device): Remove null_stream, active_streams, async_streams,
	stream_lock, and next fields.
	(enum ptx_event_type): Remove.
	(struct ptx_event): Remove.
	(ptx_event_lock): Remove.
	(ptx_events): Remove.
	(init_streams_for_device): Remove.
	(fini_streams_for_device): Remove.
	(select_stream_for_async): Remove.
	(nvptx_init): Remove ptx_events and ptx_event_lock references.
	(nvptx_attach_host_thread_to_device): Remove CUDA_ERROR_NOT_PERMITTED
	case.
	(nvptx_open_device): Add free_blocks initialization, remove
	init_streams_for_device call.
	(nvptx_close_device): Remove fini_streams_for_device call, add
	free_blocks destruct code.
	(event_gc): Remove.
	(event_add): Remove.
	(nvptx_exec): Adjust parameters and code.
	(nvptx_free): Likewise.
	(nvptx_host2dev): Remove.
	(nvptx_dev2host): Remove.
	(nvptx_set_async): Remove.
	(nvptx_async_test): Remove.
	(nvptx_async_test_all): Remove.
	(nvptx_wait): Remove.
	(nvptx_wait_async): Remove.
	(nvptx_wait_all): Remove.
	(nvptx_wait_all_async): Remove.
	(nvptx_get_cuda_stream): Remove.
	(nvptx_set_cuda_stream): Remove.
	(GOMP_OFFLOAD_alloc): Adjust code.
	(GOMP_OFFLOAD_free): Likewise.
	(GOMP_OFFLOAD_openacc_register_async_cleanup): Remove.
	(GOMP_OFFLOAD_openacc_exec): Adjust parameters and code.
	(GOMP_OFFLOAD_openacc_async_test_all): Remove.
	(GOMP_OFFLOAD_openacc_async_wait): Remove.
	(GOMP_OFFLOAD_openacc_async_wait_async): Remove.
	(GOMP_OFFLOAD_openacc_async_wait_all): Remove.
	(GOMP_OFFLOAD_openacc_async_wait_all_async): Remove.
	(GOMP_OFFLOAD_openacc_async_set_async): Remove.
	(cuda_free_argmem): New function.
	(GOMP_OFFLOAD_openacc_async_exec): New plugin hook function.
	(GOMP_OFFLOAD_openacc_create_thread_data): Adjust code.
	(GOMP_OFFLOAD_openacc_cuda_get_stream): Adjust code.
	(GOMP_OFFLOAD_openacc_cuda_set_stream): Adjust code.
	(GOMP_OFFLOAD_openacc_async_construct): New plugin hook function.
	(GOMP_OFFLOAD_openacc_async_destruct): New plugin hook function.
	(GOMP_OFFLOAD_openacc_async_test): Remove and re-implement.
	(GOMP_OFFLOAD_openacc_async_synchronize): New plugin hook function.
	(GOMP_OFFLOAD_openacc_async_serialize): New plugin hook function.
	(GOMP_OFFLOAD_openacc_async_queue_callback): New plugin hook function.
	(cuda_callback_wrapper): New function.
	(cuda_memcpy_sanity_check): New function.
	(GOMP_OFFLOAD_host2dev): Remove and re-implement.
	(GOMP_OFFLOAD_dev2host): Remove and re-implement.
	(GOMP_OFFLOAD_openacc_async_host2dev): New plugin hook function.
	(GOMP_OFFLOAD_openacc_async_dev2host): New plugin hook function.

[-- Attachment #2: async-06.nvptx.patch --]
[-- Type: text/plain, Size: 42696 bytes --]

diff --git a/libgomp/plugin/cuda-lib.def b/libgomp/plugin/cuda-lib.def
index b2a4c21..a16badc 100644
--- a/libgomp/plugin/cuda-lib.def
+++ b/libgomp/plugin/cuda-lib.def
@@ -42,6 +42,7 @@ CUDA_ONE_CALL (cuModuleLoad)
 CUDA_ONE_CALL (cuModuleLoadData)
 CUDA_ONE_CALL (cuModuleUnload)
 CUDA_ONE_CALL_MAYBE_NULL (cuOccupancyMaxPotentialBlockSize)
+CUDA_ONE_CALL (cuStreamAddCallback)
 CUDA_ONE_CALL (cuStreamCreate)
 CUDA_ONE_CALL (cuStreamDestroy)
 CUDA_ONE_CALL (cuStreamQuery)
diff --git a/libgomp/plugin/cuda/cuda.h b/libgomp/plugin/cuda/cuda.h
index b4c1b29..326db54 100644
--- a/libgomp/plugin/cuda/cuda.h
+++ b/libgomp/plugin/cuda/cuda.h
@@ -54,7 +54,11 @@ typedef enum {
   CUDA_ERROR_INVALID_CONTEXT = 201,
   CUDA_ERROR_NOT_FOUND = 500,
   CUDA_ERROR_NOT_READY = 600,
-  CUDA_ERROR_LAUNCH_FAILED = 719
+  CUDA_ERROR_LAUNCH_FAILED = 719,
+  CUDA_ERROR_COOPERATIVE_LAUNCH_TOO_LARGE = 720,
+  CUDA_ERROR_NOT_PERMITTED = 800,
+  CUDA_ERROR_NOT_SUPPORTED = 801,
+  CUDA_ERROR_UNKNOWN = 999
 } CUresult;
 
 typedef enum {
@@ -173,6 +177,8 @@ CUresult cuModuleLoadData (CUmodule *, const void *);
 CUresult cuModuleUnload (CUmodule);
 CUresult cuOccupancyMaxPotentialBlockSize(int *, int *, CUfunction,
 					  CUoccupancyB2DSize, size_t, int);
+typedef void (*CUstreamCallback)(CUstream, CUresult, void *);
+CUresult cuStreamAddCallback(CUstream, CUstreamCallback, void *, unsigned int);
 CUresult cuStreamCreate (CUstream *, unsigned);
 #define cuStreamDestroy cuStreamDestroy_v2
 CUresult cuStreamDestroy (CUstream);
diff --git a/libgomp/plugin/plugin-nvptx.c b/libgomp/plugin/plugin-nvptx.c
index bae1b05..2959df2 100644
--- a/libgomp/plugin/plugin-nvptx.c
+++ b/libgomp/plugin/plugin-nvptx.c
@@ -192,128 +192,30 @@ cuda_error (CUresult r)
 static unsigned int instantiated_devices = 0;
 static pthread_mutex_t ptx_dev_lock = PTHREAD_MUTEX_INITIALIZER;
 
-struct ptx_stream
+/* NVPTX/CUDA specific definition of asynchronous queues.  */
+struct goacc_asyncqueue
 {
-  CUstream stream;
-  pthread_t host_thread;
-  bool multithreaded;
+  CUstream cuda_stream;
+};
 
-  CUdeviceptr d;
-  void *h;
-  void *h_begin;
-  void *h_end;
-  void *h_next;
-  void *h_prev;
-  void *h_tail;
-
-  struct ptx_stream *next;
+struct nvptx_callback
+{
+  void (*fn) (void *);
+  void *ptr;
+  struct goacc_asyncqueue *aq;
+  struct nvptx_callback *next;
 };
 
 /* Thread-specific data for PTX.  */
 
 struct nvptx_thread
 {
-  struct ptx_stream *current_stream;
+  /* We currently have this embedded inside the plugin because libgomp manages
+     devices through integer target_ids.  This might be better if using an
+     opaque target-specific pointer directly from gomp_device_descr.  */
   struct ptx_device *ptx_dev;
 };
 
-static bool
-map_init (struct ptx_stream *s)
-{
-  int size = getpagesize ();
-
-  assert (s);
-  assert (!s->d);
-  assert (!s->h);
-
-  CUDA_CALL (cuMemAllocHost, &s->h, size);
-  CUDA_CALL (cuMemHostGetDevicePointer, &s->d, s->h, 0);
-
-  assert (s->h);
-
-  s->h_begin = s->h;
-  s->h_end = s->h_begin + size;
-  s->h_next = s->h_prev = s->h_tail = s->h_begin;
-
-  assert (s->h_next);
-  assert (s->h_end);
-  return true;
-}
-
-static bool
-map_fini (struct ptx_stream *s)
-{
-  CUDA_CALL (cuMemFreeHost, s->h);
-  return true;
-}
-
-static void
-map_pop (struct ptx_stream *s)
-{
-  assert (s != NULL);
-  assert (s->h_next);
-  assert (s->h_prev);
-  assert (s->h_tail);
-
-  s->h_tail = s->h_next;
-
-  if (s->h_tail >= s->h_end)
-    s->h_tail = s->h_begin + (int) (s->h_tail - s->h_end);
-
-  if (s->h_next == s->h_tail)
-    s->h_prev = s->h_next;
-
-  assert (s->h_next >= s->h_begin);
-  assert (s->h_tail >= s->h_begin);
-  assert (s->h_prev >= s->h_begin);
-
-  assert (s->h_next <= s->h_end);
-  assert (s->h_tail <= s->h_end);
-  assert (s->h_prev <= s->h_end);
-}
-
-static void
-map_push (struct ptx_stream *s, size_t size, void **h, void **d)
-{
-  int left;
-  int offset;
-
-  assert (s != NULL);
-
-  left = s->h_end - s->h_next;
-
-  assert (s->h_prev);
-  assert (s->h_next);
-
-  if (size >= left)
-    {
-      assert (s->h_next == s->h_prev);
-      s->h_next = s->h_prev = s->h_tail = s->h_begin;
-    }
-
-  assert (s->h_next);
-
-  offset = s->h_next - s->h;
-
-  *d = (void *)(s->d + offset);
-  *h = (void *)(s->h + offset);
-
-  s->h_prev = s->h_next;
-  s->h_next += size;
-
-  assert (s->h_prev);
-  assert (s->h_next);
-
-  assert (s->h_next >= s->h_begin);
-  assert (s->h_tail >= s->h_begin);
-  assert (s->h_prev >= s->h_begin);
-  assert (s->h_next <= s->h_end);
-  assert (s->h_tail <= s->h_end);
-  assert (s->h_prev <= s->h_end);
-
-  return;
-}
-
 /* Target data function launch information.  */
 
 struct targ_fn_launch
@@ -365,22 +267,18 @@ struct ptx_image_data
   struct ptx_image_data *next;
 };
 
+struct ptx_free_block
+{
+  void *ptr;
+  struct ptx_free_block *next;
+};
+
 struct ptx_device
 {
   CUcontext ctx;
   bool ctx_shared;
   CUdevice dev;
-  struct ptx_stream *null_stream;
-  /* All non-null streams associated with this device (actually context),
-     either created implicitly or passed in from the user (via
-     acc_set_cuda_stream).  */
-  struct ptx_stream *active_streams;
-  struct {
-    struct ptx_stream **arr;
-    int size;
-  } async_streams;
-  /* A lock for use when manipulating the above stream list and array.  */
-  pthread_mutex_t stream_lock;
+
   int ord;
   bool overlap;
   bool map;
@@ -398,32 +296,13 @@ struct ptx_device
 
   struct ptx_image_data *images;  /* Images loaded on device.  */
   pthread_mutex_t image_lock;     /* Lock for above list.  */
-  
-  struct ptx_device *next;
-};
-
-enum ptx_event_type
-{
-  PTX_EVT_MEM,
-  PTX_EVT_KNL,
-  PTX_EVT_SYNC,
-  PTX_EVT_ASYNC_CLEANUP
-};
 
-struct ptx_event
-{
-  CUevent *evt;
-  int type;
-  void *addr;
-  int ord;
-  int val;
+  struct ptx_free_block *free_blocks;
+  pthread_mutex_t free_blocks_lock;
 
-  struct ptx_event *next;
+  struct ptx_device *next;
 };
 
-static pthread_mutex_t ptx_event_lock;
-static struct ptx_event *ptx_events;
-
 static struct ptx_device **ptx_devices;
 
 static inline struct nvptx_thread *
@@ -432,197 +311,6 @@ nvptx_thread (void)
   return (struct nvptx_thread *) GOMP_PLUGIN_acc_thread ();
 }
 
-static bool
-init_streams_for_device (struct ptx_device *ptx_dev, int concurrency)
-{
-  int i;
-  struct ptx_stream *null_stream
-    = GOMP_PLUGIN_malloc (sizeof (struct ptx_stream));
-
-  null_stream->stream = NULL;
-  null_stream->host_thread = pthread_self ();
-  null_stream->multithreaded = true;
-  null_stream->d = (CUdeviceptr) NULL;
-  null_stream->h = NULL;
-  if (!map_init (null_stream))
-    return false;
-
-  ptx_dev->null_stream = null_stream;
-  ptx_dev->active_streams = NULL;
-  pthread_mutex_init (&ptx_dev->stream_lock, NULL);
-
-  if (concurrency < 1)
-    concurrency = 1;
-
-  /* This is just a guess -- make space for as many async streams as the
-     current device is capable of concurrently executing.  This can grow
-     later as necessary.  No streams are created yet.  */
-  ptx_dev->async_streams.arr
-    = GOMP_PLUGIN_malloc (concurrency * sizeof (struct ptx_stream *));
-  ptx_dev->async_streams.size = concurrency;
-
-  for (i = 0; i < concurrency; i++)
-    ptx_dev->async_streams.arr[i] = NULL;
-
-  return true;
-}
-
-static bool
-fini_streams_for_device (struct ptx_device *ptx_dev)
-{
-  free (ptx_dev->async_streams.arr);
-
-  bool ret = true;
-  while (ptx_dev->active_streams != NULL)
-    {
-      struct ptx_stream *s = ptx_dev->active_streams;
-      ptx_dev->active_streams = ptx_dev->active_streams->next;
-
-      ret &= map_fini (s);
-
-      CUresult r = CUDA_CALL_NOCHECK (cuStreamDestroy, s->stream);
-      if (r != CUDA_SUCCESS)
-	{
-	  GOMP_PLUGIN_error ("cuStreamDestroy error: %s", cuda_error (r));
-	  ret = false;
-	}
-      free (s);
-    }
-
-  ret &= map_fini (ptx_dev->null_stream);
-  free (ptx_dev->null_stream);
-  return ret;
-}
-
-/* Select a stream for (OpenACC-semantics) ASYNC argument for the current
-   thread THREAD (and also current device/context).  If CREATE is true, create
-   the stream if it does not exist (or use EXISTING if it is non-NULL), and
-   associate the stream with the same thread argument.  Returns stream to use
-   as result.  */
-
-static struct ptx_stream *
-select_stream_for_async (int async, pthread_t thread, bool create,
-			 CUstream existing)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  /* Local copy of TLS variable.  */
-  struct ptx_device *ptx_dev = nvthd->ptx_dev;
-  struct ptx_stream *stream = NULL;
-  int orig_async = async;
-
-  /* The special value acc_async_noval (-1) maps (for now) to an
-     implicitly-created stream, which is then handled the same as any other
-     numbered async stream.  Other options are available, e.g. using the null
-     stream for anonymous async operations, or choosing an idle stream from an
-     active set.  But, stick with this for now.  */
-  if (async > acc_async_sync)
-    async++;
-
-  if (create)
-    pthread_mutex_lock (&ptx_dev->stream_lock);
-
-  /* NOTE: AFAICT there's no particular need for acc_async_sync to map to the
-     null stream, and in fact better performance may be obtainable if it doesn't
-     (because the null stream enforces overly-strict synchronisation with
-     respect to other streams for legacy reasons, and that's probably not
-     needed with OpenACC).  Maybe investigate later.  */
-  if (async == acc_async_sync)
-    stream = ptx_dev->null_stream;
-  else if (async >= 0 && async < ptx_dev->async_streams.size
-	   && ptx_dev->async_streams.arr[async] && !(create && existing))
-    stream = ptx_dev->async_streams.arr[async];
-  else if (async >= 0 && create)
-    {
-      if (async >= ptx_dev->async_streams.size)
-	{
-	  int i, newsize = ptx_dev->async_streams.size * 2;
-
-	  if (async >= newsize)
-	    newsize = async + 1;
-
-	  ptx_dev->async_streams.arr
-	    = GOMP_PLUGIN_realloc (ptx_dev->async_streams.arr,
-				   newsize * sizeof (struct ptx_stream *));
-
-	  for (i = ptx_dev->async_streams.size; i < newsize; i++)
-	    ptx_dev->async_streams.arr[i] = NULL;
-
-	  ptx_dev->async_streams.size = newsize;
-	}
-
-      /* Create a new stream on-demand if there isn't one already, or if we're
-	 setting a particular async value to an existing (externally-provided)
-	 stream.  */
-      if (!ptx_dev->async_streams.arr[async] || existing)
-        {
-	  CUresult r;
-	  struct ptx_stream *s
-	    = GOMP_PLUGIN_malloc (sizeof (struct ptx_stream));
-
-	  if (existing)
-	    s->stream = existing;
-	  else
-	    {
-	      r = CUDA_CALL_NOCHECK (cuStreamCreate, &s->stream,
-				     CU_STREAM_DEFAULT);
-	      if (r != CUDA_SUCCESS)
-		{
-		  pthread_mutex_unlock (&ptx_dev->stream_lock);
-		  GOMP_PLUGIN_fatal ("cuStreamCreate error: %s",
-				     cuda_error (r));
-		}
-	    }
-
-	  /* If CREATE is true, we're going to be queueing some work on this
-	     stream.  Associate it with the current host thread.  */
-	  s->host_thread = thread;
-	  s->multithreaded = false;
-
-	  s->d = (CUdeviceptr) NULL;
-	  s->h = NULL;
-	  if (!map_init (s))
-	    {
-	      pthread_mutex_unlock (&ptx_dev->stream_lock);
-	      GOMP_PLUGIN_fatal ("map_init fail");
-	    }
-
-	  s->next = ptx_dev->active_streams;
-	  ptx_dev->active_streams = s;
-	  ptx_dev->async_streams.arr[async] = s;
-	}
-
-      stream = ptx_dev->async_streams.arr[async];
-    }
-  else if (async < 0)
-    {
-      if (create)
-	pthread_mutex_unlock (&ptx_dev->stream_lock);
-      GOMP_PLUGIN_fatal ("bad async %d", async);
-    }
-
-  if (create)
-    {
-      assert (stream != NULL);
-
-      /* If we're trying to use the same stream from different threads
-	 simultaneously, set stream->multithreaded to true.  This affects the
-	 behaviour of acc_async_test_all and acc_wait_all, which are supposed to
-	 only wait for asynchronous launches from the same host thread they are
-	 invoked on.  If multiple threads use the same async value, we make note
-	 of that here and fall back to testing/waiting for all threads in those
-	 functions.  */
-      if (thread != stream->host_thread)
-        stream->multithreaded = true;
-
-      pthread_mutex_unlock (&ptx_dev->stream_lock);
-    }
-  else if (stream && !stream->multithreaded
-	   && !pthread_equal (stream->host_thread, thread))
-    GOMP_PLUGIN_fatal ("async %d used on wrong thread", orig_async);
-
-  return stream;
-}
-
 /* Initialize the device.  Return TRUE on success, else FALSE.  PTX_DEV_LOCK
    should be locked on entry and remains locked on exit.  */
 
@@ -634,9 +322,6 @@ nvptx_init (void)
   if (instantiated_devices != 0)
     return true;
 
-  ptx_events = NULL;
-  pthread_mutex_init (&ptx_event_lock, NULL);
-
   if (!init_cuda_lib ())
     return false;
 
@@ -660,6 +345,11 @@ nvptx_attach_host_thread_to_device (int n)
   CUcontext thd_ctx;
 
   r = CUDA_CALL_NOCHECK (cuCtxGetDevice, &dev);
+  if (r == CUDA_ERROR_NOT_PERMITTED)
+    {
+      /* Assume we're in a CUDA callback, just return true.  */
+      return true;
+    }
   if (r != CUDA_SUCCESS && r != CUDA_ERROR_INVALID_CONTEXT)
     {
       GOMP_PLUGIN_error ("cuCtxGetDevice error: %s", cuda_error (r));
@@ -804,8 +494,8 @@ nvptx_open_device (int n)
   ptx_dev->images = NULL;
   pthread_mutex_init (&ptx_dev->image_lock, NULL);
 
-  if (!init_streams_for_device (ptx_dev, async_engines))
-    return NULL;
+  ptx_dev->free_blocks = NULL;
+  pthread_mutex_init (&ptx_dev->free_blocks_lock, NULL);
 
   return ptx_dev;
 }
@@ -816,9 +506,15 @@ nvptx_close_device (struct ptx_device *ptx_dev)
   if (!ptx_dev)
     return true;
 
-  if (!fini_streams_for_device (ptx_dev))
-    return false;
-  
+  for (struct ptx_free_block *b = ptx_dev->free_blocks; b;)
+    {
+      struct ptx_free_block *b_next = b->next;
+      CUDA_CALL (cuMemFree, (CUdeviceptr) b->ptr);
+      free (b);
+      b = b_next;
+    }
+
+  pthread_mutex_destroy (&ptx_dev->free_blocks_lock);
   pthread_mutex_destroy (&ptx_dev->image_lock);
 
   if (!ptx_dev->ctx_shared)
@@ -998,138 +694,19 @@ link_ptx (CUmodule *module, const struct targ_ptx_obj *ptx_objs,
 }
 
 static void
-event_gc (bool memmap_lockable)
-{
-  struct ptx_event *ptx_event = ptx_events;
-  struct ptx_event *async_cleanups = NULL;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&ptx_event_lock);
-
-  while (ptx_event != NULL)
-    {
-      CUresult r;
-      struct ptx_event *e = ptx_event;
-
-      ptx_event = ptx_event->next;
-
-      if (e->ord != nvthd->ptx_dev->ord)
-	continue;
-
-      r = CUDA_CALL_NOCHECK (cuEventQuery, *e->evt);
-      if (r == CUDA_SUCCESS)
-	{
-	  bool append_async = false;
-	  CUevent *te;
-
-	  te = e->evt;
-
-	  switch (e->type)
-	    {
-	    case PTX_EVT_MEM:
-	    case PTX_EVT_SYNC:
-	      break;
-
-	    case PTX_EVT_KNL:
-	      map_pop (e->addr);
-	      break;
-
-	    case PTX_EVT_ASYNC_CLEANUP:
-	      {
-		/* The function gomp_plugin_async_unmap_vars needs to claim the
-		   memory-map splay tree lock for the current device, so we
-		   can't call it when one of our callers has already claimed
-		   the lock.  In that case, just delay the GC for this event
-		   until later.  */
-		if (!memmap_lockable)
-		  continue;
-
-		append_async = true;
-	      }
-	      break;
-	    }
-
-	  CUDA_CALL_NOCHECK (cuEventDestroy, *te);
-	  free ((void *)te);
-
-	  /* Unlink 'e' from ptx_events list.  */
-	  if (ptx_events == e)
-	    ptx_events = ptx_events->next;
-	  else
-	    {
-	      struct ptx_event *e_ = ptx_events;
-	      while (e_->next != e)
-		e_ = e_->next;
-	      e_->next = e_->next->next;
-	    }
-
-	  if (append_async)
-	    {
-	      e->next = async_cleanups;
-	      async_cleanups = e;
-	    }
-	  else
-	    free (e);
-	}
-    }
-
-  pthread_mutex_unlock (&ptx_event_lock);
-
-  /* We have to do these here, after ptx_event_lock is released.  */
-  while (async_cleanups)
-    {
-      struct ptx_event *e = async_cleanups;
-      async_cleanups = async_cleanups->next;
-
-      GOMP_PLUGIN_async_unmap_vars (e->addr, e->val);
-      free (e);
-    }
-}
-
-static void
-event_add (enum ptx_event_type type, CUevent *e, void *h, int val)
-{
-  struct ptx_event *ptx_event;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  assert (type == PTX_EVT_MEM || type == PTX_EVT_KNL || type == PTX_EVT_SYNC
-	  || type == PTX_EVT_ASYNC_CLEANUP);
-
-  ptx_event = GOMP_PLUGIN_malloc (sizeof (struct ptx_event));
-  ptx_event->type = type;
-  ptx_event->evt = e;
-  ptx_event->addr = h;
-  ptx_event->ord = nvthd->ptx_dev->ord;
-  ptx_event->val = val;
-
-  pthread_mutex_lock (&ptx_event_lock);
-
-  ptx_event->next = ptx_events;
-  ptx_events = ptx_event;
-
-  pthread_mutex_unlock (&ptx_event_lock);
-}
-
-static void
 nvptx_exec (void (*fn), size_t mapnum, void **hostaddrs, void **devaddrs,
-	    int async, unsigned *dims, void *targ_mem_desc)
+	    unsigned *dims, void *targ_mem_desc,
+	    CUdeviceptr dp, CUstream stream)
 {
   struct targ_fn_descriptor *targ_fn = (struct targ_fn_descriptor *) fn;
   CUfunction function;
-  CUresult r;
   int i;
-  struct ptx_stream *dev_str;
   void *kargs[1];
-  void *hp, *dp;
   struct nvptx_thread *nvthd = nvptx_thread ();
   int warp_size = nvthd->ptx_dev->warp_size;
-  const char *maybe_abort_msg = "(perhaps abort was called)";
 
   function = targ_fn->fn;
 
-  dev_str = select_stream_for_async (async, pthread_self (), false, NULL);
-  assert (dev_str == nvthd->current_stream);
-
   /* Initialize the launch dimensions.  Typically this is constant,
      provided by the device compiler, but we must permit runtime
      values.  */
@@ -1292,21 +869,6 @@ nvptx_exec (void (*fn), size_t mapnum, void **hostaddrs, void **devaddrs,
 			 suggest_workers, suggest_workers);
     }
 
-  /* This reserves a chunk of a pre-allocated page of memory mapped on both
-     the host and the device. HP is a host pointer to the new chunk, and DP is
-     the corresponding device pointer.  */
-  map_push (dev_str, mapnum * sizeof (void *), &hp, &dp);
-
-  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
-
-  /* Copy the array of arguments to the mapped page.  */
-  for (i = 0; i < mapnum; i++)
-    ((void **) hp)[i] = devaddrs[i];
-
-  /* Copy the (device) pointers to arguments to the device (dp and hp might in
-     fact have the same value on a unified-memory system).  */
-  CUDA_CALL_ASSERT (cuMemcpy, (CUdeviceptr) dp, (CUdeviceptr) hp,
-		    mapnum * sizeof (void *));
   GOMP_PLUGIN_debug (0, "  %s: kernel %s: launch"
 		     " gangs=%u, workers=%u, vectors=%u\n",
 		     __FUNCTION__, targ_fn->launch->fn, dims[GOMP_DIM_GANG],
@@ -1317,58 +879,14 @@ nvptx_exec (void (*fn), size_t mapnum, void **hostaddrs, void **devaddrs,
   // num_gangs		nctaid.x
   // num_workers	ntid.y
   // vector length	ntid.x
-
   kargs[0] = &dp;
   CUDA_CALL_ASSERT (cuLaunchKernel, function,
 		    dims[GOMP_DIM_GANG], 1, 1,
 		    dims[GOMP_DIM_VECTOR], dims[GOMP_DIM_WORKER], 1,
-		    0, dev_str->stream, kargs, 0);
-
-#ifndef DISABLE_ASYNC
-  if (async < acc_async_noval)
-    {
-      r = CUDA_CALL_NOCHECK (cuStreamSynchronize, dev_str->stream);
-      if (r == CUDA_ERROR_LAUNCH_FAILED)
-	GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s %s\n", cuda_error (r),
-			   maybe_abort_msg);
-      else if (r != CUDA_SUCCESS)
-        GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s", cuda_error (r));
-    }
-  else
-    {
-      CUevent *e;
-
-      e = (CUevent *)GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-      r = CUDA_CALL_NOCHECK (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      if (r == CUDA_ERROR_LAUNCH_FAILED)
-	GOMP_PLUGIN_fatal ("cuEventCreate error: %s %s\n", cuda_error (r),
-			   maybe_abort_msg);
-      else if (r != CUDA_SUCCESS)
-        GOMP_PLUGIN_fatal ("cuEventCreate error: %s", cuda_error (r));
-
-      event_gc (true);
-
-      CUDA_CALL_ASSERT (cuEventRecord, *e, dev_str->stream);
-
-      event_add (PTX_EVT_KNL, e, (void *)dev_str, 0);
-    }
-#else
-  r = CUDA_CALL_NOCHECK (cuCtxSynchronize, );
-  if (r == CUDA_ERROR_LAUNCH_FAILED)
-    GOMP_PLUGIN_fatal ("cuCtxSynchronize error: %s %s\n", cuda_error (r),
-		       maybe_abort_msg);
-  else if (r != CUDA_SUCCESS)
-    GOMP_PLUGIN_fatal ("cuCtxSynchronize error: %s", cuda_error (r));
-#endif
+		    0, stream, kargs, 0);
 
   GOMP_PLUGIN_debug (0, "  %s: kernel %s: finished\n", __FUNCTION__,
 		     targ_fn->launch->fn);
-
-#ifndef DISABLE_ASYNC
-  if (async < acc_async_noval)
-#endif
-    map_pop (dev_str);
 }
 
 void * openacc_get_current_cuda_context (void);
@@ -1383,8 +901,21 @@ nvptx_alloc (size_t s)
 }
 
 static bool
-nvptx_free (void *p)
+nvptx_free (void *p, struct ptx_device *ptx_dev)
 {
+  /* Assume callback context if this is null.  */
+  if (GOMP_PLUGIN_acc_thread () == NULL)
+    {
+      struct ptx_free_block *n
+	= GOMP_PLUGIN_malloc (sizeof (struct ptx_free_block));
+      n->ptr = p;
+      pthread_mutex_lock (&ptx_dev->free_blocks_lock);
+      n->next = ptx_dev->free_blocks;
+      ptx_dev->free_blocks = n;
+      pthread_mutex_unlock (&ptx_dev->free_blocks_lock);
+      return true;
+    }
+
   CUdeviceptr pb;
   size_t ps;
 
@@ -1399,389 +930,27 @@ nvptx_free (void *p)
   return true;
 }
 
-
-static bool
-nvptx_host2dev (void *d, const void *h, size_t s)
+static void *
+nvptx_get_current_cuda_device (void)
 {
-  CUdeviceptr pb;
-  size_t ps;
   struct nvptx_thread *nvthd = nvptx_thread ();
 
-  if (!s)
-    return true;
-  if (!d)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-
-  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
-
-  if (!pb)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-  if (!h)
-    {
-      GOMP_PLUGIN_error ("invalid host address");
-      return false;
-    }
-  if (d == h)
-    {
-      GOMP_PLUGIN_error ("invalid host or device address");
-      return false;
-    }
-  if ((void *)(d + s) > (void *)(pb + ps))
-    {
-      GOMP_PLUGIN_error ("invalid size");
-      return false;
-    }
-
-#ifndef DISABLE_ASYNC
-  if (nvthd && nvthd->current_stream != nvthd->ptx_dev->null_stream)
-    {
-      CUevent *e = (CUevent *)GOMP_PLUGIN_malloc (sizeof (CUevent));
-      CUDA_CALL (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      event_gc (false);
-      CUDA_CALL (cuMemcpyHtoDAsync,
-		 (CUdeviceptr) d, h, s, nvthd->current_stream->stream);
-      CUDA_CALL (cuEventRecord, *e, nvthd->current_stream->stream);
-      event_add (PTX_EVT_MEM, e, (void *)h, 0);
-    }
-  else
-#endif
-    CUDA_CALL (cuMemcpyHtoD, (CUdeviceptr) d, h, s);
+  if (!nvthd || !nvthd->ptx_dev)
+    return NULL;
 
-  return true;
+  return &nvthd->ptx_dev->dev;
 }
 
-static bool
-nvptx_dev2host (void *h, const void *d, size_t s)
+static void *
+nvptx_get_current_cuda_context (void)
 {
-  CUdeviceptr pb;
-  size_t ps;
   struct nvptx_thread *nvthd = nvptx_thread ();
 
-  if (!s)
-    return true;
-  if (!d)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
+  if (!nvthd || !nvthd->ptx_dev)
+    return NULL;
 
-  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
-
-  if (!pb)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-  if (!h)
-    {
-      GOMP_PLUGIN_error ("invalid host address");
-      return false;
-    }
-  if (d == h)
-    {
-      GOMP_PLUGIN_error ("invalid host or device address");
-      return false;
-    }
-  if ((void *)(d + s) > (void *)(pb + ps))
-    {
-      GOMP_PLUGIN_error ("invalid size");
-      return false;
-    }
-
-#ifndef DISABLE_ASYNC
-  if (nvthd && nvthd->current_stream != nvthd->ptx_dev->null_stream)
-    {
-      CUevent *e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-      CUDA_CALL (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      event_gc (false);
-      CUDA_CALL (cuMemcpyDtoHAsync,
-		 h, (CUdeviceptr) d, s, nvthd->current_stream->stream);
-      CUDA_CALL (cuEventRecord, *e, nvthd->current_stream->stream);
-      event_add (PTX_EVT_MEM, e, (void *)h, 0);
-    }
-  else
-#endif
-    CUDA_CALL (cuMemcpyDtoH, h, (CUdeviceptr) d, s);
-
-  return true;
-}
-
-static void
-nvptx_set_async (int async)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  nvthd->current_stream
-    = select_stream_for_async (async, pthread_self (), true, NULL);
-}
-
-static int
-nvptx_async_test (int async)
-{
-  CUresult r;
-  struct ptx_stream *s;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-
-  if (!s)
-    GOMP_PLUGIN_fatal ("unknown async %d", async);
-
-  r = CUDA_CALL_NOCHECK (cuStreamQuery, s->stream);
-  if (r == CUDA_SUCCESS)
-    {
-      /* The oacc-parallel.c:goacc_wait function calls this hook to determine
-	 whether all work has completed on this stream, and if so omits the call
-	 to the wait hook.  If that happens, event_gc might not get called
-	 (which prevents variables from getting unmapped and their associated
-	 device storage freed), so call it here.  */
-      event_gc (true);
-      return 1;
-    }
-  else if (r == CUDA_ERROR_NOT_READY)
-    return 0;
-
-  GOMP_PLUGIN_fatal ("cuStreamQuery error: %s", cuda_error (r));
-
-  return 0;
-}
-
-static int
-nvptx_async_test_all (void)
-{
-  struct ptx_stream *s;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  for (s = nvthd->ptx_dev->active_streams; s != NULL; s = s->next)
-    {
-      if ((s->multithreaded || pthread_equal (s->host_thread, self))
-	  && CUDA_CALL_NOCHECK (cuStreamQuery,
-				s->stream) == CUDA_ERROR_NOT_READY)
-	{
-	  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-	  return 0;
-	}
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  event_gc (true);
-
-  return 1;
-}
-
-static void
-nvptx_wait (int async)
-{
-  struct ptx_stream *s;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-  if (!s)
-    GOMP_PLUGIN_fatal ("unknown async %d", async);
-
-  CUDA_CALL_ASSERT (cuStreamSynchronize, s->stream);
-
-  event_gc (true);
-}
-
-static void
-nvptx_wait_async (int async1, int async2)
-{
-  CUevent *e;
-  struct ptx_stream *s1, *s2;
-  pthread_t self = pthread_self ();
-
-  /* The stream that is waiting (rather than being waited for) doesn't
-     necessarily have to exist already.  */
-  s2 = select_stream_for_async (async2, self, true, NULL);
-
-  s1 = select_stream_for_async (async1, self, false, NULL);
-  if (!s1)
-    GOMP_PLUGIN_fatal ("invalid async 1\n");
-
-  if (s1 == s2)
-    GOMP_PLUGIN_fatal ("identical parameters");
-
-  e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-  CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-
-  event_gc (true);
-
-  CUDA_CALL_ASSERT (cuEventRecord, *e, s1->stream);
-
-  event_add (PTX_EVT_SYNC, e, NULL, 0);
-
-  CUDA_CALL_ASSERT (cuStreamWaitEvent, s2->stream, *e, 0);
-}
-
-static void
-nvptx_wait_all (void)
-{
-  CUresult r;
-  struct ptx_stream *s;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  /* Wait for active streams initiated by this thread (or by multiple threads)
-     to complete.  */
-  for (s = nvthd->ptx_dev->active_streams; s != NULL; s = s->next)
-    {
-      if (s->multithreaded || pthread_equal (s->host_thread, self))
-	{
-	  r = CUDA_CALL_NOCHECK (cuStreamQuery, s->stream);
-	  if (r == CUDA_SUCCESS)
-	    continue;
-	  else if (r != CUDA_ERROR_NOT_READY)
-	    GOMP_PLUGIN_fatal ("cuStreamQuery error: %s", cuda_error (r));
-
-	  CUDA_CALL_ASSERT (cuStreamSynchronize, s->stream);
-	}
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  event_gc (true);
-}
-
-static void
-nvptx_wait_all_async (int async)
-{
-  struct ptx_stream *waiting_stream, *other_stream;
-  CUevent *e;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  pthread_t self = pthread_self ();
-
-  /* The stream doing the waiting.  This could be the first mention of the
-     stream, so create it if necessary.  */
-  waiting_stream
-    = select_stream_for_async (async, pthread_self (), true, NULL);
-
-  /* Launches on the null stream already block on other streams in the
-     context.  */
-  if (!waiting_stream || waiting_stream == nvthd->ptx_dev->null_stream)
-    return;
-
-  event_gc (true);
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  for (other_stream = nvthd->ptx_dev->active_streams;
-       other_stream != NULL;
-       other_stream = other_stream->next)
-    {
-      if (!other_stream->multithreaded
-	  && !pthread_equal (other_stream->host_thread, self))
-	continue;
-
-      e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-      CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-
-      /* Record an event on the waited-for stream.  */
-      CUDA_CALL_ASSERT (cuEventRecord, *e, other_stream->stream);
-
-      event_add (PTX_EVT_SYNC, e, NULL, 0);
-
-      CUDA_CALL_ASSERT (cuStreamWaitEvent, waiting_stream->stream, *e, 0);
-   }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-}
-
-static void *
-nvptx_get_current_cuda_device (void)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!nvthd || !nvthd->ptx_dev)
-    return NULL;
-
-  return &nvthd->ptx_dev->dev;
-}
-
-static void *
-nvptx_get_current_cuda_context (void)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!nvthd || !nvthd->ptx_dev)
-    return NULL;
-
-  return nvthd->ptx_dev->ctx;
-}
-
-static void *
-nvptx_get_cuda_stream (int async)
-{
-  struct ptx_stream *s;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!nvthd || !nvthd->ptx_dev)
-    return NULL;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-
-  return s ? s->stream : NULL;
-}
-
-static int
-nvptx_set_cuda_stream (int async, void *stream)
-{
-  struct ptx_stream *oldstream;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (async < 0)
-    GOMP_PLUGIN_fatal ("bad async %d", async);
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  /* We have a list of active streams and an array mapping async values to
-     entries of that list.  We need to take "ownership" of the passed-in stream,
-     and add it to our list, removing the previous entry also (if there was one)
-     in order to prevent resource leaks.  Note the potential for surprise
-     here: maybe we should keep track of passed-in streams and leave it up to
-     the user to tidy those up, but that doesn't work for stream handles
-     returned from acc_get_cuda_stream above...  */
-
-  oldstream = select_stream_for_async (async, self, false, NULL);
-
-  if (oldstream)
-    {
-      if (nvthd->ptx_dev->active_streams == oldstream)
-	nvthd->ptx_dev->active_streams = nvthd->ptx_dev->active_streams->next;
-      else
-	{
-	  struct ptx_stream *s = nvthd->ptx_dev->active_streams;
-	  while (s->next != oldstream)
-	    s = s->next;
-	  s->next = s->next->next;
-	}
-
-      CUDA_CALL_ASSERT (cuStreamDestroy, oldstream->stream);
-
-      if (!map_fini (oldstream))
-	GOMP_PLUGIN_fatal ("error when freeing host memory");
-
-      free (oldstream);
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  (void) select_stream_for_async (async, self, true, (CUstream) stream);
-
-  return 1;
-}
+  return nvthd->ptx_dev->ctx;
+}
 
 /* Plugin entry points.  */
 
@@ -2016,100 +1185,116 @@ GOMP_OFFLOAD_alloc (int ord, size_t size)
 {
   if (!nvptx_attach_host_thread_to_device (ord))
     return NULL;
-  return nvptx_alloc (size);
-}
 
-bool
-GOMP_OFFLOAD_free (int ord, void *ptr)
-{
-  return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_free (ptr));
-}
+  struct ptx_device *ptx_dev = ptx_devices[ord];
+  struct ptx_free_block *blocks, *tmp;
 
-bool
-GOMP_OFFLOAD_dev2host (int ord, void *dst, const void *src, size_t n)
-{
-  return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_dev2host (dst, src, n));
-}
+  pthread_mutex_lock (&ptx_dev->free_blocks_lock);
+  blocks = ptx_dev->free_blocks;
+  ptx_dev->free_blocks = NULL;
+  pthread_mutex_unlock (&ptx_dev->free_blocks_lock);
 
-bool
-GOMP_OFFLOAD_host2dev (int ord, void *dst, const void *src, size_t n)
-{
-  return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_host2dev (dst, src, n));
+  while (blocks)
+    {
+      tmp = blocks->next;
+      nvptx_free (blocks->ptr, ptx_dev);
+      free (blocks);
+      blocks = tmp;
+    }
+
+  return nvptx_alloc (size);
 }
 
 bool
-GOMP_OFFLOAD_dev2dev (int ord, void *dst, const void *src, size_t n)
+GOMP_OFFLOAD_free (int ord, void *ptr)
 {
-  struct ptx_device *ptx_dev = ptx_devices[ord];
-  CUDA_CALL (cuMemcpyDtoDAsync, (CUdeviceptr) dst, (CUdeviceptr) src, n,
-				ptx_dev->null_stream->stream);
-  return true;
+  return (nvptx_attach_host_thread_to_device (ord)
+	  && nvptx_free (ptr, ptx_devices[ord]));
 }
 
-void (*device_run) (int n, void *fn_ptr, void *vars) = NULL;
-
 void
 GOMP_OFFLOAD_openacc_exec (void (*fn) (void *), size_t mapnum,
 			   void **hostaddrs, void **devaddrs,
-			   int async, unsigned *dims, void *targ_mem_desc)
+			   unsigned *dims, void *targ_mem_desc)
 {
-  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, async, dims, targ_mem_desc);
-}
+  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
 
-void
-GOMP_OFFLOAD_openacc_register_async_cleanup (void *targ_mem_desc, int async)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  CUevent *e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
+  void **hp = NULL;
+  CUdeviceptr dp = 0;
 
-  CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-  CUDA_CALL_ASSERT (cuEventRecord, *e, nvthd->current_stream->stream);
-  event_add (PTX_EVT_ASYNC_CLEANUP, e, targ_mem_desc, async);
-}
+  if (mapnum > 0)
+    {
+      hp = alloca (mapnum * sizeof (void *));
+      for (int i = 0; i < mapnum; i++)
+	hp[i] = (devaddrs[i] ? devaddrs[i] : hostaddrs[i]);
+      CUDA_CALL_ASSERT (cuMemAlloc, &dp, mapnum * sizeof (void *));
+    }
 
-int
-GOMP_OFFLOAD_openacc_async_test (int async)
-{
-  return nvptx_async_test (async);
-}
+  /* Copy the (device) pointers to arguments to the device (dp and hp might in
+     fact have the same value on a unified-memory system).  */
+  if (mapnum > 0)
+    CUDA_CALL_ASSERT (cuMemcpyHtoD, dp, (void *) hp,
+		      mapnum * sizeof (void *));
 
-int
-GOMP_OFFLOAD_openacc_async_test_all (void)
-{
-  return nvptx_async_test_all ();
-}
+  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, dims, targ_mem_desc,
+	      dp, NULL);
 
-void
-GOMP_OFFLOAD_openacc_async_wait (int async)
-{
-  nvptx_wait (async);
+  CUresult r = CUDA_CALL_NOCHECK (cuStreamSynchronize, NULL);
+  const char *maybe_abort_msg = "(perhaps abort was called)";
+  if (r == CUDA_ERROR_LAUNCH_FAILED)
+    GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s %s\n", cuda_error (r),
+		       maybe_abort_msg);
+  else if (r != CUDA_SUCCESS)
+    GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s", cuda_error (r));
+  CUDA_CALL_ASSERT (cuMemFree, dp);
 }
 
-void
-GOMP_OFFLOAD_openacc_async_wait_async (int async1, int async2)
+static void
+cuda_free_argmem (void *ptr)
 {
-  nvptx_wait_async (async1, async2);
+  void **block = (void **) ptr;
+  nvptx_free (block[0], (struct ptx_device *) block[1]);
+  free (block);
 }
 
 void
-GOMP_OFFLOAD_openacc_async_wait_all (void)
+GOMP_OFFLOAD_openacc_async_exec (void (*fn) (void *), size_t mapnum,
+				 void **hostaddrs, void **devaddrs,
+				 unsigned *dims, void *targ_mem_desc,
+				 struct goacc_asyncqueue *aq)
 {
-  nvptx_wait_all ();
-}
+  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
 
-void
-GOMP_OFFLOAD_openacc_async_wait_all_async (int async)
-{
-  nvptx_wait_all_async (async);
-}
+  void **hp = NULL;
+  CUdeviceptr dp = 0;
+  void **block = NULL;
 
-void
-GOMP_OFFLOAD_openacc_async_set_async (int async)
-{
-  nvptx_set_async (async);
+  if (mapnum > 0)
+    {
+      block = (void **) GOMP_PLUGIN_malloc ((mapnum + 2) * sizeof (void *));
+      hp = block + 2;
+      for (int i = 0; i < mapnum; i++)
+	hp[i] = (devaddrs[i] ? devaddrs[i] : hostaddrs[i]);
+      CUDA_CALL_ASSERT (cuMemAlloc, &dp, mapnum * sizeof (void *));
+    }
+
+  /* Copy the (device) pointers to arguments to the device (dp and hp might in
+     fact have the same value on a unified-memory system).  */
+  if (mapnum > 0)
+    {
+      CUDA_CALL_ASSERT (cuMemcpyHtoDAsync, dp, (void *) hp,
+			mapnum * sizeof (void *), aq->cuda_stream);
+      block[0] = (void *) dp;
+
+      struct nvptx_thread *nvthd =
+	(struct nvptx_thread *) GOMP_PLUGIN_acc_thread ();
+      block[1] = (void *) nvthd->ptx_dev;
+    }
+  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, dims, targ_mem_desc,
+	      dp, aq->cuda_stream);
+
+  if (mapnum > 0)
+    GOMP_OFFLOAD_openacc_async_queue_callback (aq, cuda_free_argmem, block);
 }
 
 void *
@@ -2131,7 +1316,6 @@ GOMP_OFFLOAD_openacc_create_thread_data (int ord)
   if (!thd_ctx)
     CUDA_CALL_ASSERT (cuCtxPushCurrent, ptx_dev->ctx);
 
-  nvthd->current_stream = ptx_dev->null_stream;
   nvthd->ptx_dev = ptx_dev;
 
   return (void *) nvthd;
@@ -2156,19 +1340,185 @@ GOMP_OFFLOAD_openacc_cuda_get_current_context (void)
 }
 
 /* NOTE: This returns a CUstream, not a ptx_stream pointer.  */
-
 void *
-GOMP_OFFLOAD_openacc_cuda_get_stream (int async)
+GOMP_OFFLOAD_openacc_cuda_get_stream (struct goacc_asyncqueue *aq)
 {
-  return nvptx_get_cuda_stream (async);
+  return (void *) aq->cuda_stream;
 }
 
 /* NOTE: This takes a CUstream, not a ptx_stream pointer.  */
+int
+GOMP_OFFLOAD_openacc_cuda_set_stream (struct goacc_asyncqueue *aq, void *stream)
+{
+  if (aq->cuda_stream)
+    {
+      CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
+      CUDA_CALL_ASSERT (cuStreamDestroy, aq->cuda_stream);
+    }
+
+  aq->cuda_stream = (CUstream) stream;
+  return 1;
+}
+
+struct goacc_asyncqueue *
+GOMP_OFFLOAD_openacc_async_construct (void)
+{
+  struct goacc_asyncqueue *aq
+    = GOMP_PLUGIN_malloc (sizeof (struct goacc_asyncqueue));
+  aq->cuda_stream = NULL;
+  CUDA_CALL_ASSERT (cuStreamCreate, &aq->cuda_stream, CU_STREAM_DEFAULT);
+  if (aq->cuda_stream == NULL)
+    GOMP_PLUGIN_fatal ("CUDA stream create NULL\n");
+
+  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
+
+
+  return aq;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_destruct (struct goacc_asyncqueue *aq)
+{
+  CUDA_CALL_ERET (false, cuStreamDestroy, aq->cuda_stream);
+  free (aq);
+  return true;
+}
 
 int
-GOMP_OFFLOAD_openacc_cuda_set_stream (int async, void *stream)
+GOMP_OFFLOAD_openacc_async_test (struct goacc_asyncqueue *aq)
 {
-  return nvptx_set_cuda_stream (async, stream);
+  CUresult r = CUDA_CALL_NOCHECK (cuStreamQuery, aq->cuda_stream);
+  if (r == CUDA_SUCCESS)
+    return 1;
+  if (r == CUDA_ERROR_NOT_READY)
+    return 0;
+
+  GOMP_PLUGIN_error ("cuStreamQuery error: %s", cuda_error (r));
+  return -1;
+}
+
+void
+GOMP_OFFLOAD_openacc_async_synchronize (struct goacc_asyncqueue *aq)
+{
+  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
+}
+
+void
+GOMP_OFFLOAD_openacc_async_serialize (struct goacc_asyncqueue *aq1,
+				      struct goacc_asyncqueue *aq2)
+{
+  CUevent e;
+  CUDA_CALL_ASSERT (cuEventCreate, &e, CU_EVENT_DISABLE_TIMING);
+  CUDA_CALL_ASSERT (cuEventRecord, e, aq1->cuda_stream);
+  CUDA_CALL_ASSERT (cuStreamWaitEvent, aq2->cuda_stream, e, 0);
+}
+
+static void
+cuda_callback_wrapper (CUstream stream, CUresult res, void *ptr)
+{
+  if (res != CUDA_SUCCESS)
+    GOMP_PLUGIN_fatal ("%s error: %s", __FUNCTION__, cuda_error (res));
+  struct nvptx_callback *cb = (struct nvptx_callback *) ptr;
+  cb->fn (cb->ptr);
+  free (ptr);
+}
+
+void
+GOMP_OFFLOAD_openacc_async_queue_callback (struct goacc_asyncqueue *aq,
+					   void (*callback_fn)(void *),
+					   void *userptr)
+{
+  struct nvptx_callback *b = GOMP_PLUGIN_malloc (sizeof (*b));
+  b->fn = callback_fn;
+  b->ptr = userptr;
+  b->aq = aq;
+  CUDA_CALL_ASSERT (cuStreamAddCallback, aq->cuda_stream,
+		    cuda_callback_wrapper, (void *) b, 0);
+}
+
+static bool
+cuda_memcpy_sanity_check (const void *h, const void *d, size_t s)
+{
+  CUdeviceptr pb;
+  size_t ps;
+  if (!s)
+    return true;
+  if (!d)
+    {
+      GOMP_PLUGIN_error ("invalid device address");
+      return false;
+    }
+  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
+  if (!pb)
+    {
+      GOMP_PLUGIN_error ("invalid device address");
+      return false;
+    }
+  if (!h)
+    {
+      GOMP_PLUGIN_error ("invalid host address");
+      return false;
+    }
+  if (d == h)
+    {
+      GOMP_PLUGIN_error ("invalid host or device address");
+      return false;
+    }
+  if ((void *)(d + s) > (void *)(pb + ps))
+    {
+      GOMP_PLUGIN_error ("invalid size");
+      return false;
+    }
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_host2dev (int ord, void *dst, const void *src, size_t n)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (src, dst, n))
+    return false;
+  CUDA_CALL (cuMemcpyHtoD, (CUdeviceptr) dst, src, n);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_dev2host (int ord, void *dst, const void *src, size_t n)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (dst, src, n))
+    return false;
+  CUDA_CALL (cuMemcpyDtoH, dst, (CUdeviceptr) src, n);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_dev2dev (int ord, void *dst, const void *src, size_t n)
+{
+  CUDA_CALL (cuMemcpyDtoDAsync, (CUdeviceptr) dst, (CUdeviceptr) src, n, NULL);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_host2dev (int ord, void *dst, const void *src,
+				     size_t n, struct goacc_asyncqueue *aq)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (src, dst, n))
+    return false;
+  CUDA_CALL (cuMemcpyHtoDAsync, (CUdeviceptr) dst, src, n, aq->cuda_stream);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_dev2host (int ord, void *dst, const void *src,
+				     size_t n, struct goacc_asyncqueue *aq)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (dst, src, n))
+    return false;
+  CUDA_CALL (cuMemcpyDtoHAsync, dst, (CUdeviceptr) src, n, aq->cuda_stream);
+  return true;
 }
 
 /* Adjust launch dimensions: pick good values for number of blocks and warps
@@ -2269,8 +1619,7 @@ GOMP_OFFLOAD_run (int ord, void *tgt_fn, void *tgt_vars, void **args)
     CU_LAUNCH_PARAM_END
   };
   r = CUDA_CALL_NOCHECK (cuLaunchKernel, function, teams, 1, 1,
-			 32, threads, 1, 0, ptx_dev->null_stream->stream,
-			 NULL, config);
+			 32, threads, 1, 0, NULL, NULL, config);
   if (r != CUDA_SUCCESS)
     GOMP_PLUGIN_fatal ("cuLaunchKernel error: %s", cuda_error (r));
 

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes
  2018-09-25 13:13 [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes Chung-Lin Tang
@ 2018-10-05 14:07 ` Tom de Vries
  2018-12-06 20:57 ` Thomas Schwinge
  2023-01-12 13:51 ` nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case (was: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes) Thomas Schwinge
  2 siblings, 0 replies; 9+ messages in thread
From: Tom de Vries @ 2018-10-05 14:07 UTC (permalink / raw)
  To: cltang, gcc-patches

On 9/25/18 3:11 PM, Chung-Lin Tang wrote:
> Hi Tom,
> this patch removes large portions of plugin/plugin-nvptx.c, since a lot
> of it is
> now in oacc-async.c now.

Yay!

> The new code is essentially a
> NVPTX/CUDA-specific implementation
> of the new-style goacc_asyncqueues.
> 
> Also, some needed functions in cuda-lib.def are added. The cuda.h
> function has also
> been updated to build independently without a CUDA installation.
> 

I see these formatting issues:
...
$ check_GNU_style.sh async-06.nvptx.patch

There should be exactly one space between function name and parenthesis.
35:+CUresult cuStreamAddCallback(CUstream, CUstreamCallback, void *,
unsigned int);

Trailing operator.
1320:+      struct nvptx_thread *nvthd =
...

Otherwise, OK.

Thanks,
- Tom


> Thanks,
> Chung-Lin
> 
>     * plugin/plugin-nvptx.c (struct cuda_map): Remove.
>     (struct ptx_stream): Remove.
>     (struct nvptx_thread): Remove current_stream field.
>     (cuda_map_create): Remove.
>     (cuda_map_destroy): Remove.
>     (map_init): Remove.
>     (map_fini): Remove.
>     (map_pop): Remove.
>     (map_push): Remove.
>     (struct goacc_asyncqueue): Define.
>     (struct nvptx_callback): Define.
>     (struct ptx_free_block): Define.
>     (struct ptx_device): Remove null_stream, active_streams, async_streams,
>     stream_lock, and next fields.
>     (enum ptx_event_type): Remove.
>     (struct ptx_event): Remove.
>     (ptx_event_lock): Remove.
>     (ptx_events): Remove.
>     (init_streams_for_device): Remove.
>     (fini_streams_for_device): Remove.
>     (select_stream_for_async): Remove.
>     (nvptx_init): Remove ptx_events and ptx_event_lock references.
>     (nvptx_attach_host_thread_to_device): Remove CUDA_ERROR_NOT_PERMITTED
>     case.
>     (nvptx_open_device): Add free_blocks initialization, remove
>     init_streams_for_device call.
>     (nvptx_close_device): Remove fini_streams_for_device call, add
>     free_blocks destruct code.
>     (event_gc): Remove.
>     (event_add): Remove.
>     (nvptx_exec): Adjust parameters and code.
>     (nvptx_free): Likewise.
>     (nvptx_host2dev): Remove.
>     (nvptx_dev2host): Remove.
>     (nvptx_set_async): Remove.
>     (nvptx_async_test): Remove.
>     (nvptx_async_test_all): Remove.
>     (nvptx_wait): Remove.
>     (nvptx_wait_async): Remove.
>     (nvptx_wait_all): Remove.
>     (nvptx_wait_all_async): Remove.
>     (nvptx_get_cuda_stream): Remove.
>     (nvptx_set_cuda_stream): Remove.
>     (GOMP_OFFLOAD_alloc): Adjust code.
>     (GOMP_OFFLOAD_free): Likewise.
>     (GOMP_OFFLOAD_openacc_register_async_cleanup): Remove.
>     (GOMP_OFFLOAD_openacc_exec): Adjust parameters and code.
>     (GOMP_OFFLOAD_openacc_async_test_all): Remove.
>     (GOMP_OFFLOAD_openacc_async_wait): Remove.
>     (GOMP_OFFLOAD_openacc_async_wait_async): Remove.
>     (GOMP_OFFLOAD_openacc_async_wait_all): Remove.
>     (GOMP_OFFLOAD_openacc_async_wait_all_async): Remove.
>     (GOMP_OFFLOAD_openacc_async_set_async): Remove.
>     (cuda_free_argmem): New function.
>     (GOMP_OFFLOAD_openacc_async_exec): New plugin hook function.
>     (GOMP_OFFLOAD_openacc_create_thread_data): Adjust code.
>     (GOMP_OFFLOAD_openacc_cuda_get_stream): Adjust code.
>     (GOMP_OFFLOAD_openacc_cuda_set_stream): Adjust code.
>     (GOMP_OFFLOAD_openacc_async_construct): New plugin hook function.
>     (GOMP_OFFLOAD_openacc_async_destruct): New plugin hook function.
>     (GOMP_OFFLOAD_openacc_async_test): Remove and re-implement.
>     (GOMP_OFFLOAD_openacc_async_synchronize): New plugin hook function.
>     (GOMP_OFFLOAD_openacc_async_serialize): New plugin hook function.
>     (GOMP_OFFLOAD_openacc_async_queue_callback): New plugin hook function.
>     (cuda_callback_wrapper): New function.
>     (cuda_memcpy_sanity_check): New function.
>     (GOMP_OFFLOAD_host2dev): Remove and re-implement.
>     (GOMP_OFFLOAD_dev2host): Remove and re-implement.
>     (GOMP_OFFLOAD_openacc_async_host2dev): New plugin hook function.
>     (GOMP_OFFLOAD_openacc_async_dev2host): New plugin hook function.

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes
  2018-09-25 13:13 [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes Chung-Lin Tang
  2018-10-05 14:07 ` Tom de Vries
@ 2018-12-06 20:57 ` Thomas Schwinge
  2018-12-10 10:02   ` Chung-Lin Tang
  2023-01-12 13:51 ` nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case (was: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes) Thomas Schwinge
  2 siblings, 1 reply; 9+ messages in thread
From: Thomas Schwinge @ 2018-12-06 20:57 UTC (permalink / raw)
  To: Chung-Lin Tang; +Cc: gcc-patches, Tom de Vries

Hi Chung-Lin!

On Tue, 25 Sep 2018 21:11:58 +0800, Chung-Lin Tang <chunglin_tang@mentor.com> wrote:
> Hi Tom,
> this patch removes large portions of plugin/plugin-nvptx.c, since a lot of it is
> now in oacc-async.c now. The new code is essentially a NVPTX/CUDA-specific implementation
> of the new-style goacc_asyncqueues.

> --- a/libgomp/plugin/plugin-nvptx.c
> +++ b/libgomp/plugin/plugin-nvptx.c

> +struct goacc_asyncqueue *
> +GOMP_OFFLOAD_openacc_async_construct (void)
> +{
> +  struct goacc_asyncqueue *aq
> +    = GOMP_PLUGIN_malloc (sizeof (struct goacc_asyncqueue));
> +  aq->cuda_stream = NULL;
> +  CUDA_CALL_ASSERT (cuStreamCreate, &aq->cuda_stream, CU_STREAM_DEFAULT);

Curiously (this was the same in the code before): does this have to be
"CU_STREAM_DEFAULT" instead of "CU_STREAM_NON_BLOCKING", because we want
to block anything from running in parallel with "acc_async_sync" GPU
kernels, that use the "NULL" stream?  (Not asking you to change this now,
but I wonder if this is overly strict?)

> +  if (aq->cuda_stream == NULL)
> +    GOMP_PLUGIN_fatal ("CUDA stream create NULL\n");

Can this actually happen, given the "CUDA_CALL_ASSERT" usage above?

> +  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);

Why is the synchronization needed here?

> +  return aq;
> +}


Grüße
 Thomas

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes
  2018-12-06 20:57 ` Thomas Schwinge
@ 2018-12-10 10:02   ` Chung-Lin Tang
  2018-12-11 13:50     ` [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes (revised, v2) Chung-Lin Tang
  0 siblings, 1 reply; 9+ messages in thread
From: Chung-Lin Tang @ 2018-12-10 10:02 UTC (permalink / raw)
  To: Thomas Schwinge, Chung-Lin Tang; +Cc: gcc-patches, Tom de Vries

On 2018/12/7 04:57 AM, Thomas Schwinge wrote>> --- 
a/libgomp/plugin/plugin-nvptx.c
>> +++ b/libgomp/plugin/plugin-nvptx.c
> 
>> +struct goacc_asyncqueue *
>> +GOMP_OFFLOAD_openacc_async_construct (void)
>> +{
>> +  struct goacc_asyncqueue *aq
>> +    = GOMP_PLUGIN_malloc (sizeof (struct goacc_asyncqueue));
>> +  aq->cuda_stream = NULL;
>> +  CUDA_CALL_ASSERT (cuStreamCreate, &aq->cuda_stream, CU_STREAM_DEFAULT);
> 
> Curiously (this was the same in the code before): does this have to be
> "CU_STREAM_DEFAULT" instead of "CU_STREAM_NON_BLOCKING", because we want
> to block anything from running in parallel with "acc_async_sync" GPU
> kernels, that use the "NULL" stream?  (Not asking you to change this now,
> but I wonder if this is overly strict?)

IIUC, this non-blocking only pertains to the "Legacy Default Stream" in 
CUDA, which we're pretty much ignoring; we should be using the newer 
per-thread default stream model. We could review this issue later.

>> +  if (aq->cuda_stream == NULL)
>> +    GOMP_PLUGIN_fatal ("CUDA stream create NULL\n");
> 
> Can this actually happen, given the "CUDA_CALL_ASSERT" usage above?

Hmm, yeah I think this is superfluous too...

>> +  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
> 
> Why is the synchronization needed here?

I don't remember, could likely be something added during debug.
I'll remove this and test if things are okay.

Thanks,
Chung-Lin

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes (revised, v2)
  2018-12-10 10:02   ` Chung-Lin Tang
@ 2018-12-11 13:50     ` Chung-Lin Tang
  2018-12-18 15:07       ` [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes (revised, v3) Chung-Lin Tang
  0 siblings, 1 reply; 9+ messages in thread
From: Chung-Lin Tang @ 2018-12-11 13:50 UTC (permalink / raw)
  To: Thomas Schwinge, Chung-Lin Tang; +Cc: gcc-patches, Tom de Vries

[-- Attachment #1: Type: text/plain, Size: 1748 bytes --]

On 2018/12/10 6:02 PM, Chung-Lin Tang wrote:
> On 2018/12/7 04:57 AM, Thomas Schwinge wrote>> --- a/libgomp/plugin/plugin-nvptx.c
>>> +++ b/libgomp/plugin/plugin-nvptx.c
>>
>>> +struct goacc_asyncqueue *
>>> +GOMP_OFFLOAD_openacc_async_construct (void)
>>> +{
>>> +  struct goacc_asyncqueue *aq
>>> +    = GOMP_PLUGIN_malloc (sizeof (struct goacc_asyncqueue));
>>> +  aq->cuda_stream = NULL;
>>> +  CUDA_CALL_ASSERT (cuStreamCreate, &aq->cuda_stream, CU_STREAM_DEFAULT);
>>
>> Curiously (this was the same in the code before): does this have to be
>> "CU_STREAM_DEFAULT" instead of "CU_STREAM_NON_BLOCKING", because we want
>> to block anything from running in parallel with "acc_async_sync" GPU
>> kernels, that use the "NULL" stream?  (Not asking you to change this now,
>> but I wonder if this is overly strict?)
> 
> IIUC, this non-blocking only pertains to the "Legacy Default Stream" in CUDA, which we're pretty much ignoring; we should be using the newer per-thread default stream model. We could review this issue later.
> 
>>> +  if (aq->cuda_stream == NULL)
>>> +    GOMP_PLUGIN_fatal ("CUDA stream create NULL\n");
>>
>> Can this actually happen, given the "CUDA_CALL_ASSERT" usage above?
> 
> Hmm, yeah I think this is superfluous too...
> 
>>> +  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
>>
>> Why is the synchronization needed here?
> 
> I don't remember, could likely be something added during debug.
> I'll remove this and test if things are okay.

I have removed the above seemingly unneeded lines and re-tested, appears okay.
Also the formerly attached version seemed to for some reason had many conflicts
with older code, all resolved in this v2 nvptx part.

Thanks,
Chung-Lin

[-- Attachment #2: async-06.nvptx.v2.patch --]
[-- Type: text/plain, Size: 42493 bytes --]

Index: libgomp/plugin/cuda/cuda.h
===================================================================
--- libgomp/plugin/cuda/cuda.h	(revision 266973)
+++ libgomp/plugin/cuda/cuda.h	(working copy)
@@ -54,7 +54,11 @@ typedef enum {
   CUDA_ERROR_INVALID_CONTEXT = 201,
   CUDA_ERROR_NOT_FOUND = 500,
   CUDA_ERROR_NOT_READY = 600,
-  CUDA_ERROR_LAUNCH_FAILED = 719
+  CUDA_ERROR_LAUNCH_FAILED = 719,
+  CUDA_ERROR_COOPERATIVE_LAUNCH_TOO_LARGE = 720,
+  CUDA_ERROR_NOT_PERMITTED = 800,
+  CUDA_ERROR_NOT_SUPPORTED = 801,
+  CUDA_ERROR_UNKNOWN = 999
 } CUresult;
 
 typedef enum {
@@ -173,6 +177,8 @@ CUresult cuModuleLoadData (CUmodule *, const void
 CUresult cuModuleUnload (CUmodule);
 CUresult cuOccupancyMaxPotentialBlockSize(int *, int *, CUfunction,
 					  CUoccupancyB2DSize, size_t, int);
+typedef void (*CUstreamCallback)(CUstream, CUresult, void *);
+CUresult cuStreamAddCallback(CUstream, CUstreamCallback, void *, unsigned int);
 CUresult cuStreamCreate (CUstream *, unsigned);
 #define cuStreamDestroy cuStreamDestroy_v2
 CUresult cuStreamDestroy (CUstream);
Index: libgomp/plugin/cuda-lib.def
===================================================================
--- libgomp/plugin/cuda-lib.def	(revision 266973)
+++ libgomp/plugin/cuda-lib.def	(working copy)
@@ -42,6 +42,7 @@ CUDA_ONE_CALL (cuModuleLoad)
 CUDA_ONE_CALL (cuModuleLoadData)
 CUDA_ONE_CALL (cuModuleUnload)
 CUDA_ONE_CALL_MAYBE_NULL (cuOccupancyMaxPotentialBlockSize)
+CUDA_ONE_CALL (cuStreamAddCallback)
 CUDA_ONE_CALL (cuStreamCreate)
 CUDA_ONE_CALL (cuStreamDestroy)
 CUDA_ONE_CALL (cuStreamQuery)
Index: libgomp/plugin/plugin-nvptx.c
===================================================================
--- libgomp/plugin/plugin-nvptx.c	(revision 266973)
+++ libgomp/plugin/plugin-nvptx.c	(working copy)
@@ -192,21 +192,18 @@ cuda_error (CUresult r)
 static unsigned int instantiated_devices = 0;
 static pthread_mutex_t ptx_dev_lock = PTHREAD_MUTEX_INITIALIZER;
 
-struct cuda_map
+/* NVPTX/CUDA specific definition of asynchronous queues.  */
+struct goacc_asyncqueue
 {
-  CUdeviceptr d;
-  size_t size;
-  bool active;
-  struct cuda_map *next;
+  CUstream cuda_stream;
 };
 
-struct ptx_stream
+struct nvptx_callback
 {
-  CUstream stream;
-  pthread_t host_thread;
-  bool multithreaded;
-  struct cuda_map *map;
-  struct ptx_stream *next;
+  void (*fn) (void *);
+  void *ptr;
+  struct goacc_asyncqueue *aq;
+  struct nvptx_callback *next;
 };
 
 /* Thread-specific data for PTX.  */
@@ -213,120 +210,13 @@ static pthread_mutex_t ptx_dev_lock = PTHREAD_MUTE
 
 struct nvptx_thread
 {
-  struct ptx_stream *current_stream;
+  /* We currently have this embedded inside the plugin because libgomp manages
+     devices through integer target_ids.  This might be better if using an
+     opaque target-specific pointer directly from gomp_device_descr.  */
   struct ptx_device *ptx_dev;
 };
 
-static struct cuda_map *
-cuda_map_create (size_t size)
-{
-  struct cuda_map *map = GOMP_PLUGIN_malloc (sizeof (struct cuda_map));
 
-  assert (map);
-
-  map->next = NULL;
-  map->size = size;
-  map->active = false;
-
-  CUDA_CALL_ERET (NULL, cuMemAlloc, &map->d, size);
-  assert (map->d);
-
-  return map;
-}
-
-static void
-cuda_map_destroy (struct cuda_map *map)
-{
-  CUDA_CALL_ASSERT (cuMemFree, map->d);
-  free (map);
-}
-
-/* The following map_* routines manage the CUDA device memory that
-   contains the data mapping arguments for cuLaunchKernel.  Each
-   asynchronous PTX stream may have multiple pending kernel
-   invocations, which are launched in a FIFO order.  As such, the map
-   routines maintains a queue of cuLaunchKernel arguments.
-
-   Calls to map_push and map_pop must be guarded by ptx_event_lock.
-   Likewise, calls to map_init and map_fini are guarded by
-   ptx_dev_lock inside GOMP_OFFLOAD_init_device and
-   GOMP_OFFLOAD_fini_device, respectively.  */
-
-static bool
-map_init (struct ptx_stream *s)
-{
-  int size = getpagesize ();
-
-  assert (s);
-
-  s->map = cuda_map_create (size);
-
-  return true;
-}
-
-static bool
-map_fini (struct ptx_stream *s)
-{
-  assert (s->map->next == NULL);
-  assert (!s->map->active);
-
-  cuda_map_destroy (s->map);
-
-  return true;
-}
-
-static void
-map_pop (struct ptx_stream *s)
-{
-  struct cuda_map *next;
-
-  assert (s != NULL);
-
-  if (s->map->next == NULL)
-    {
-      s->map->active = false;
-      return;
-    }
-
-  next = s->map->next;
-  cuda_map_destroy (s->map);
-  s->map = next;
-}
-
-static CUdeviceptr
-map_push (struct ptx_stream *s, size_t size)
-{
-  struct cuda_map *map = NULL, *t = NULL;
-
-  assert (s);
-  assert (s->map);
-
-  /* Each PTX stream requires a separate data region to store the
-     launch arguments for cuLaunchKernel.  Allocate a new
-     cuda_map and push it to the end of the list.  */
-  if (s->map->active)
-    {
-      map = cuda_map_create (size);
-
-      for (t = s->map; t->next != NULL; t = t->next)
-	;
-
-      t->next = map;
-    }
-  else if (s->map->size < size)
-    {
-      cuda_map_destroy (s->map);
-      map = cuda_map_create (size);
-    }
-  else
-    map = s->map;
-
-  s->map = map;
-  s->map->active = true;
-
-  return s->map->d;
-}
-
 /* Target data function launch information.  */
 
 struct targ_fn_launch
@@ -378,22 +268,18 @@ struct ptx_image_data
   struct ptx_image_data *next;
 };
 
+struct ptx_free_block
+{
+  void *ptr;
+  struct ptx_free_block *next;
+};
+
 struct ptx_device
 {
   CUcontext ctx;
   bool ctx_shared;
   CUdevice dev;
-  struct ptx_stream *null_stream;
-  /* All non-null streams associated with this device (actually context),
-     either created implicitly or passed in from the user (via
-     acc_set_cuda_stream).  */
-  struct ptx_stream *active_streams;
-  struct {
-    struct ptx_stream **arr;
-    int size;
-  } async_streams;
-  /* A lock for use when manipulating the above stream list and array.  */
-  pthread_mutex_t stream_lock;
+
   int ord;
   bool overlap;
   bool map;
@@ -411,32 +297,13 @@ struct ptx_device
 
   struct ptx_image_data *images;  /* Images loaded on device.  */
   pthread_mutex_t image_lock;     /* Lock for above list.  */
-  
-  struct ptx_device *next;
-};
 
-enum ptx_event_type
-{
-  PTX_EVT_MEM,
-  PTX_EVT_KNL,
-  PTX_EVT_SYNC,
-  PTX_EVT_ASYNC_CLEANUP
-};
+  struct ptx_free_block *free_blocks;
+  pthread_mutex_t free_blocks_lock;
 
-struct ptx_event
-{
-  CUevent *evt;
-  int type;
-  void *addr;
-  int ord;
-  int val;
-
-  struct ptx_event *next;
+  struct ptx_device *next;
 };
 
-static pthread_mutex_t ptx_event_lock;
-static struct ptx_event *ptx_events;
-
 static struct ptx_device **ptx_devices;
 
 static inline struct nvptx_thread *
@@ -445,193 +312,6 @@ nvptx_thread (void)
   return (struct nvptx_thread *) GOMP_PLUGIN_acc_thread ();
 }
 
-static bool
-init_streams_for_device (struct ptx_device *ptx_dev, int concurrency)
-{
-  int i;
-  struct ptx_stream *null_stream
-    = GOMP_PLUGIN_malloc (sizeof (struct ptx_stream));
-
-  null_stream->stream = NULL;
-  null_stream->host_thread = pthread_self ();
-  null_stream->multithreaded = true;
-  if (!map_init (null_stream))
-    return false;
-
-  ptx_dev->null_stream = null_stream;
-  ptx_dev->active_streams = NULL;
-  pthread_mutex_init (&ptx_dev->stream_lock, NULL);
-
-  if (concurrency < 1)
-    concurrency = 1;
-
-  /* This is just a guess -- make space for as many async streams as the
-     current device is capable of concurrently executing.  This can grow
-     later as necessary.  No streams are created yet.  */
-  ptx_dev->async_streams.arr
-    = GOMP_PLUGIN_malloc (concurrency * sizeof (struct ptx_stream *));
-  ptx_dev->async_streams.size = concurrency;
-
-  for (i = 0; i < concurrency; i++)
-    ptx_dev->async_streams.arr[i] = NULL;
-
-  return true;
-}
-
-static bool
-fini_streams_for_device (struct ptx_device *ptx_dev)
-{
-  free (ptx_dev->async_streams.arr);
-
-  bool ret = true;
-  while (ptx_dev->active_streams != NULL)
-    {
-      struct ptx_stream *s = ptx_dev->active_streams;
-      ptx_dev->active_streams = ptx_dev->active_streams->next;
-
-      ret &= map_fini (s);
-
-      CUresult r = CUDA_CALL_NOCHECK (cuStreamDestroy, s->stream);
-      if (r != CUDA_SUCCESS)
-	{
-	  GOMP_PLUGIN_error ("cuStreamDestroy error: %s", cuda_error (r));
-	  ret = false;
-	}
-      free (s);
-    }
-
-  ret &= map_fini (ptx_dev->null_stream);
-  free (ptx_dev->null_stream);
-  return ret;
-}
-
-/* Select a stream for (OpenACC-semantics) ASYNC argument for the current
-   thread THREAD (and also current device/context).  If CREATE is true, create
-   the stream if it does not exist (or use EXISTING if it is non-NULL), and
-   associate the stream with the same thread argument.  Returns stream to use
-   as result.  */
-
-static struct ptx_stream *
-select_stream_for_async (int async, pthread_t thread, bool create,
-			 CUstream existing)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  /* Local copy of TLS variable.  */
-  struct ptx_device *ptx_dev = nvthd->ptx_dev;
-  struct ptx_stream *stream = NULL;
-  int orig_async = async;
-
-  /* The special value acc_async_noval (-1) maps (for now) to an
-     implicitly-created stream, which is then handled the same as any other
-     numbered async stream.  Other options are available, e.g. using the null
-     stream for anonymous async operations, or choosing an idle stream from an
-     active set.  But, stick with this for now.  */
-  if (async > acc_async_sync)
-    async++;
-
-  if (create)
-    pthread_mutex_lock (&ptx_dev->stream_lock);
-
-  /* NOTE: AFAICT there's no particular need for acc_async_sync to map to the
-     null stream, and in fact better performance may be obtainable if it doesn't
-     (because the null stream enforces overly-strict synchronisation with
-     respect to other streams for legacy reasons, and that's probably not
-     needed with OpenACC).  Maybe investigate later.  */
-  if (async == acc_async_sync)
-    stream = ptx_dev->null_stream;
-  else if (async >= 0 && async < ptx_dev->async_streams.size
-	   && ptx_dev->async_streams.arr[async] && !(create && existing))
-    stream = ptx_dev->async_streams.arr[async];
-  else if (async >= 0 && create)
-    {
-      if (async >= ptx_dev->async_streams.size)
-	{
-	  int i, newsize = ptx_dev->async_streams.size * 2;
-
-	  if (async >= newsize)
-	    newsize = async + 1;
-
-	  ptx_dev->async_streams.arr
-	    = GOMP_PLUGIN_realloc (ptx_dev->async_streams.arr,
-				   newsize * sizeof (struct ptx_stream *));
-
-	  for (i = ptx_dev->async_streams.size; i < newsize; i++)
-	    ptx_dev->async_streams.arr[i] = NULL;
-
-	  ptx_dev->async_streams.size = newsize;
-	}
-
-      /* Create a new stream on-demand if there isn't one already, or if we're
-	 setting a particular async value to an existing (externally-provided)
-	 stream.  */
-      if (!ptx_dev->async_streams.arr[async] || existing)
-        {
-	  CUresult r;
-	  struct ptx_stream *s
-	    = GOMP_PLUGIN_malloc (sizeof (struct ptx_stream));
-
-	  if (existing)
-	    s->stream = existing;
-	  else
-	    {
-	      r = CUDA_CALL_NOCHECK (cuStreamCreate, &s->stream,
-				     CU_STREAM_DEFAULT);
-	      if (r != CUDA_SUCCESS)
-		{
-		  pthread_mutex_unlock (&ptx_dev->stream_lock);
-		  GOMP_PLUGIN_fatal ("cuStreamCreate error: %s",
-				     cuda_error (r));
-		}
-	    }
-
-	  /* If CREATE is true, we're going to be queueing some work on this
-	     stream.  Associate it with the current host thread.  */
-	  s->host_thread = thread;
-	  s->multithreaded = false;
-
-	  if (!map_init (s))
-	    {
-	      pthread_mutex_unlock (&ptx_dev->stream_lock);
-	      GOMP_PLUGIN_fatal ("map_init fail");
-	    }
-
-	  s->next = ptx_dev->active_streams;
-	  ptx_dev->active_streams = s;
-	  ptx_dev->async_streams.arr[async] = s;
-	}
-
-      stream = ptx_dev->async_streams.arr[async];
-    }
-  else if (async < 0)
-    {
-      if (create)
-	pthread_mutex_unlock (&ptx_dev->stream_lock);
-      GOMP_PLUGIN_fatal ("bad async %d", async);
-    }
-
-  if (create)
-    {
-      assert (stream != NULL);
-
-      /* If we're trying to use the same stream from different threads
-	 simultaneously, set stream->multithreaded to true.  This affects the
-	 behaviour of acc_async_test_all and acc_wait_all, which are supposed to
-	 only wait for asynchronous launches from the same host thread they are
-	 invoked on.  If multiple threads use the same async value, we make note
-	 of that here and fall back to testing/waiting for all threads in those
-	 functions.  */
-      if (thread != stream->host_thread)
-        stream->multithreaded = true;
-
-      pthread_mutex_unlock (&ptx_dev->stream_lock);
-    }
-  else if (stream && !stream->multithreaded
-	   && !pthread_equal (stream->host_thread, thread))
-    GOMP_PLUGIN_fatal ("async %d used on wrong thread", orig_async);
-
-  return stream;
-}
-
 /* Initialize the device.  Return TRUE on success, else FALSE.  PTX_DEV_LOCK
    should be locked on entry and remains locked on exit.  */
 
@@ -643,9 +323,6 @@ nvptx_init (void)
   if (instantiated_devices != 0)
     return true;
 
-  ptx_events = NULL;
-  pthread_mutex_init (&ptx_event_lock, NULL);
-
   if (!init_cuda_lib ())
     return false;
 
@@ -669,6 +346,11 @@ nvptx_attach_host_thread_to_device (int n)
   CUcontext thd_ctx;
 
   r = CUDA_CALL_NOCHECK (cuCtxGetDevice, &dev);
+  if (r == CUDA_ERROR_NOT_PERMITTED)
+    {
+      /* Assume we're in a CUDA callback, just return true.  */
+      return true;
+    }
   if (r != CUDA_SUCCESS && r != CUDA_ERROR_INVALID_CONTEXT)
     {
       GOMP_PLUGIN_error ("cuCtxGetDevice error: %s", cuda_error (r));
@@ -813,8 +495,8 @@ nvptx_open_device (int n)
   ptx_dev->images = NULL;
   pthread_mutex_init (&ptx_dev->image_lock, NULL);
 
-  if (!init_streams_for_device (ptx_dev, async_engines))
-    return NULL;
+  ptx_dev->free_blocks = NULL;
+  pthread_mutex_init (&ptx_dev->free_blocks_lock, NULL);
 
   return ptx_dev;
 }
@@ -825,9 +507,15 @@ nvptx_close_device (struct ptx_device *ptx_dev)
   if (!ptx_dev)
     return true;
 
-  if (!fini_streams_for_device (ptx_dev))
-    return false;
-  
+  for (struct ptx_free_block *b = ptx_dev->free_blocks; b;)
+    {
+      struct ptx_free_block *b_next = b->next;
+      CUDA_CALL (cuMemFree, (CUdeviceptr) b->ptr);
+      free (b);
+      b = b_next;
+    }
+
+  pthread_mutex_destroy (&ptx_dev->free_blocks_lock);
   pthread_mutex_destroy (&ptx_dev->image_lock);
 
   if (!ptx_dev->ctx_shared)
@@ -1007,139 +695,19 @@ link_ptx (CUmodule *module, const struct targ_ptx_
 }
 
 static void
-event_gc (bool memmap_lockable)
-{
-  struct ptx_event *ptx_event = ptx_events;
-  struct ptx_event *async_cleanups = NULL;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&ptx_event_lock);
-
-  while (ptx_event != NULL)
-    {
-      CUresult r;
-      struct ptx_event *e = ptx_event;
-
-      ptx_event = ptx_event->next;
-
-      if (e->ord != nvthd->ptx_dev->ord)
-	continue;
-
-      r = CUDA_CALL_NOCHECK (cuEventQuery, *e->evt);
-      if (r == CUDA_SUCCESS)
-	{
-	  bool append_async = false;
-	  CUevent *te;
-
-	  te = e->evt;
-
-	  switch (e->type)
-	    {
-	    case PTX_EVT_MEM:
-	    case PTX_EVT_SYNC:
-	      break;
-
-	    case PTX_EVT_KNL:
-	      map_pop (e->addr);
-	      break;
-
-	    case PTX_EVT_ASYNC_CLEANUP:
-	      {
-		/* The function gomp_plugin_async_unmap_vars needs to claim the
-		   memory-map splay tree lock for the current device, so we
-		   can't call it when one of our callers has already claimed
-		   the lock.  In that case, just delay the GC for this event
-		   until later.  */
-		if (!memmap_lockable)
-		  continue;
-
-		append_async = true;
-	      }
-	      break;
-	    }
-
-	  CUDA_CALL_NOCHECK (cuEventDestroy, *te);
-	  free ((void *)te);
-
-	  /* Unlink 'e' from ptx_events list.  */
-	  if (ptx_events == e)
-	    ptx_events = ptx_events->next;
-	  else
-	    {
-	      struct ptx_event *e_ = ptx_events;
-	      while (e_->next != e)
-		e_ = e_->next;
-	      e_->next = e_->next->next;
-	    }
-
-	  if (append_async)
-	    {
-	      e->next = async_cleanups;
-	      async_cleanups = e;
-	    }
-	  else
-	    free (e);
-	}
-    }
-
-  pthread_mutex_unlock (&ptx_event_lock);
-
-  /* We have to do these here, after ptx_event_lock is released.  */
-  while (async_cleanups)
-    {
-      struct ptx_event *e = async_cleanups;
-      async_cleanups = async_cleanups->next;
-
-      GOMP_PLUGIN_async_unmap_vars (e->addr, e->val);
-      free (e);
-    }
-}
-
-static void
-event_add (enum ptx_event_type type, CUevent *e, void *h, int val)
-{
-  struct ptx_event *ptx_event;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  assert (type == PTX_EVT_MEM || type == PTX_EVT_KNL || type == PTX_EVT_SYNC
-	  || type == PTX_EVT_ASYNC_CLEANUP);
-
-  ptx_event = GOMP_PLUGIN_malloc (sizeof (struct ptx_event));
-  ptx_event->type = type;
-  ptx_event->evt = e;
-  ptx_event->addr = h;
-  ptx_event->ord = nvthd->ptx_dev->ord;
-  ptx_event->val = val;
-
-  pthread_mutex_lock (&ptx_event_lock);
-
-  ptx_event->next = ptx_events;
-  ptx_events = ptx_event;
-
-  pthread_mutex_unlock (&ptx_event_lock);
-}
-
-static void
 nvptx_exec (void (*fn), size_t mapnum, void **hostaddrs, void **devaddrs,
-	    int async, unsigned *dims, void *targ_mem_desc)
+	    unsigned *dims, void *targ_mem_desc,
+	    CUdeviceptr dp, CUstream stream)
 {
   struct targ_fn_descriptor *targ_fn = (struct targ_fn_descriptor *) fn;
   CUfunction function;
-  CUresult r;
   int i;
-  struct ptx_stream *dev_str;
   void *kargs[1];
-  void *hp;
-  CUdeviceptr dp;
   struct nvptx_thread *nvthd = nvptx_thread ();
   int warp_size = nvthd->ptx_dev->warp_size;
-  const char *maybe_abort_msg = "(perhaps abort was called)";
 
   function = targ_fn->fn;
 
-  dev_str = select_stream_for_async (async, pthread_self (), false, NULL);
-  assert (dev_str == nvthd->current_stream);
-
   /* Initialize the launch dimensions.  Typically this is constant,
      provided by the device compiler, but we must permit runtime
      values.  */
@@ -1302,23 +870,6 @@ nvptx_exec (void (*fn), size_t mapnum, void **host
 			 suggest_workers, suggest_workers);
     }
 
-  /* This reserves a chunk of a pre-allocated page of memory mapped on both
-     the host and the device. HP is a host pointer to the new chunk, and DP is
-     the corresponding device pointer.  */
-  pthread_mutex_lock (&ptx_event_lock);
-  dp = map_push (dev_str, mapnum * sizeof (void *));
-  pthread_mutex_unlock (&ptx_event_lock);
-
-  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
-
-  /* Copy the array of arguments to the mapped page.  */
-  hp = alloca(sizeof(void *) * mapnum);
-  for (i = 0; i < mapnum; i++)
-    ((void **) hp)[i] = devaddrs[i];
-
-  /* Copy the (device) pointers to arguments to the device */
-  CUDA_CALL_ASSERT (cuMemcpyHtoD, dp, hp,
-		    mapnum * sizeof (void *));
   GOMP_PLUGIN_debug (0, "  %s: kernel %s: launch"
 		     " gangs=%u, workers=%u, vectors=%u\n",
 		     __FUNCTION__, targ_fn->launch->fn, dims[GOMP_DIM_GANG],
@@ -1329,58 +880,14 @@ nvptx_exec (void (*fn), size_t mapnum, void **host
   // num_gangs		nctaid.x
   // num_workers	ntid.y
   // vector length	ntid.x
-
   kargs[0] = &dp;
   CUDA_CALL_ASSERT (cuLaunchKernel, function,
 		    dims[GOMP_DIM_GANG], 1, 1,
 		    dims[GOMP_DIM_VECTOR], dims[GOMP_DIM_WORKER], 1,
-		    0, dev_str->stream, kargs, 0);
+		    0, stream, kargs, 0);
 
-#ifndef DISABLE_ASYNC
-  if (async < acc_async_noval)
-    {
-      r = CUDA_CALL_NOCHECK (cuStreamSynchronize, dev_str->stream);
-      if (r == CUDA_ERROR_LAUNCH_FAILED)
-	GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s %s\n", cuda_error (r),
-			   maybe_abort_msg);
-      else if (r != CUDA_SUCCESS)
-        GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s", cuda_error (r));
-    }
-  else
-    {
-      CUevent *e;
-
-      e = (CUevent *)GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-      r = CUDA_CALL_NOCHECK (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      if (r == CUDA_ERROR_LAUNCH_FAILED)
-	GOMP_PLUGIN_fatal ("cuEventCreate error: %s %s\n", cuda_error (r),
-			   maybe_abort_msg);
-      else if (r != CUDA_SUCCESS)
-        GOMP_PLUGIN_fatal ("cuEventCreate error: %s", cuda_error (r));
-
-      event_gc (true);
-
-      CUDA_CALL_ASSERT (cuEventRecord, *e, dev_str->stream);
-
-      event_add (PTX_EVT_KNL, e, (void *)dev_str, 0);
-    }
-#else
-  r = CUDA_CALL_NOCHECK (cuCtxSynchronize, );
-  if (r == CUDA_ERROR_LAUNCH_FAILED)
-    GOMP_PLUGIN_fatal ("cuCtxSynchronize error: %s %s\n", cuda_error (r),
-		       maybe_abort_msg);
-  else if (r != CUDA_SUCCESS)
-    GOMP_PLUGIN_fatal ("cuCtxSynchronize error: %s", cuda_error (r));
-#endif
-
   GOMP_PLUGIN_debug (0, "  %s: kernel %s: finished\n", __FUNCTION__,
 		     targ_fn->launch->fn);
-
-#ifndef DISABLE_ASYNC
-  if (async < acc_async_noval)
-#endif
-    map_pop (dev_str);
 }
 
 void * openacc_get_current_cuda_context (void);
@@ -1395,8 +902,21 @@ nvptx_alloc (size_t s)
 }
 
 static bool
-nvptx_free (void *p)
+nvptx_free (void *p, struct ptx_device *ptx_dev)
 {
+  /* Assume callback context if this is null.  */
+  if (GOMP_PLUGIN_acc_thread () == NULL)
+    {
+      struct ptx_free_block *n
+	= GOMP_PLUGIN_malloc (sizeof (struct ptx_free_block));
+      n->ptr = p;
+      pthread_mutex_lock (&ptx_dev->free_blocks_lock);
+      n->next = ptx_dev->free_blocks;
+      ptx_dev->free_blocks = n;
+      pthread_mutex_unlock (&ptx_dev->free_blocks_lock);
+      return true;
+    }
+
   CUdeviceptr pb;
   size_t ps;
 
@@ -1411,305 +931,6 @@ static bool
   return true;
 }
 
-
-static bool
-nvptx_host2dev (void *d, const void *h, size_t s)
-{
-  CUdeviceptr pb;
-  size_t ps;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!s)
-    return true;
-  if (!d)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-
-  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
-
-  if (!pb)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-  if (!h)
-    {
-      GOMP_PLUGIN_error ("invalid host address");
-      return false;
-    }
-  if (d == h)
-    {
-      GOMP_PLUGIN_error ("invalid host or device address");
-      return false;
-    }
-  if ((void *)(d + s) > (void *)(pb + ps))
-    {
-      GOMP_PLUGIN_error ("invalid size");
-      return false;
-    }
-
-#ifndef DISABLE_ASYNC
-  if (nvthd && nvthd->current_stream != nvthd->ptx_dev->null_stream)
-    {
-      CUevent *e = (CUevent *)GOMP_PLUGIN_malloc (sizeof (CUevent));
-      CUDA_CALL (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      event_gc (false);
-      CUDA_CALL (cuMemcpyHtoDAsync,
-		 (CUdeviceptr) d, h, s, nvthd->current_stream->stream);
-      CUDA_CALL (cuEventRecord, *e, nvthd->current_stream->stream);
-      event_add (PTX_EVT_MEM, e, (void *)h, 0);
-    }
-  else
-#endif
-    CUDA_CALL (cuMemcpyHtoD, (CUdeviceptr) d, h, s);
-
-  return true;
-}
-
-static bool
-nvptx_dev2host (void *h, const void *d, size_t s)
-{
-  CUdeviceptr pb;
-  size_t ps;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!s)
-    return true;
-  if (!d)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-
-  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
-
-  if (!pb)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-  if (!h)
-    {
-      GOMP_PLUGIN_error ("invalid host address");
-      return false;
-    }
-  if (d == h)
-    {
-      GOMP_PLUGIN_error ("invalid host or device address");
-      return false;
-    }
-  if ((void *)(d + s) > (void *)(pb + ps))
-    {
-      GOMP_PLUGIN_error ("invalid size");
-      return false;
-    }
-
-#ifndef DISABLE_ASYNC
-  if (nvthd && nvthd->current_stream != nvthd->ptx_dev->null_stream)
-    {
-      CUevent *e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-      CUDA_CALL (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      event_gc (false);
-      CUDA_CALL (cuMemcpyDtoHAsync,
-		 h, (CUdeviceptr) d, s, nvthd->current_stream->stream);
-      CUDA_CALL (cuEventRecord, *e, nvthd->current_stream->stream);
-      event_add (PTX_EVT_MEM, e, (void *)h, 0);
-    }
-  else
-#endif
-    CUDA_CALL (cuMemcpyDtoH, h, (CUdeviceptr) d, s);
-
-  return true;
-}
-
-static void
-nvptx_set_async (int async)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  nvthd->current_stream
-    = select_stream_for_async (async, pthread_self (), true, NULL);
-}
-
-static int
-nvptx_async_test (int async)
-{
-  CUresult r;
-  struct ptx_stream *s;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-
-  if (!s)
-    GOMP_PLUGIN_fatal ("unknown async %d", async);
-
-  r = CUDA_CALL_NOCHECK (cuStreamQuery, s->stream);
-  if (r == CUDA_SUCCESS)
-    {
-      /* The oacc-parallel.c:goacc_wait function calls this hook to determine
-	 whether all work has completed on this stream, and if so omits the call
-	 to the wait hook.  If that happens, event_gc might not get called
-	 (which prevents variables from getting unmapped and their associated
-	 device storage freed), so call it here.  */
-      event_gc (true);
-      return 1;
-    }
-  else if (r == CUDA_ERROR_NOT_READY)
-    return 0;
-
-  GOMP_PLUGIN_fatal ("cuStreamQuery error: %s", cuda_error (r));
-
-  return 0;
-}
-
-static int
-nvptx_async_test_all (void)
-{
-  struct ptx_stream *s;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  for (s = nvthd->ptx_dev->active_streams; s != NULL; s = s->next)
-    {
-      if ((s->multithreaded || pthread_equal (s->host_thread, self))
-	  && CUDA_CALL_NOCHECK (cuStreamQuery,
-				s->stream) == CUDA_ERROR_NOT_READY)
-	{
-	  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-	  return 0;
-	}
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  event_gc (true);
-
-  return 1;
-}
-
-static void
-nvptx_wait (int async)
-{
-  struct ptx_stream *s;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-  if (!s)
-    GOMP_PLUGIN_fatal ("unknown async %d", async);
-
-  CUDA_CALL_ASSERT (cuStreamSynchronize, s->stream);
-
-  event_gc (true);
-}
-
-static void
-nvptx_wait_async (int async1, int async2)
-{
-  CUevent *e;
-  struct ptx_stream *s1, *s2;
-  pthread_t self = pthread_self ();
-
-  /* The stream that is waiting (rather than being waited for) doesn't
-     necessarily have to exist already.  */
-  s2 = select_stream_for_async (async2, self, true, NULL);
-
-  s1 = select_stream_for_async (async1, self, false, NULL);
-  if (!s1)
-    GOMP_PLUGIN_fatal ("invalid async 1\n");
-
-  if (s1 == s2)
-    GOMP_PLUGIN_fatal ("identical parameters");
-
-  e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-  CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-
-  event_gc (true);
-
-  CUDA_CALL_ASSERT (cuEventRecord, *e, s1->stream);
-
-  event_add (PTX_EVT_SYNC, e, NULL, 0);
-
-  CUDA_CALL_ASSERT (cuStreamWaitEvent, s2->stream, *e, 0);
-}
-
-static void
-nvptx_wait_all (void)
-{
-  CUresult r;
-  struct ptx_stream *s;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  /* Wait for active streams initiated by this thread (or by multiple threads)
-     to complete.  */
-  for (s = nvthd->ptx_dev->active_streams; s != NULL; s = s->next)
-    {
-      if (s->multithreaded || pthread_equal (s->host_thread, self))
-	{
-	  r = CUDA_CALL_NOCHECK (cuStreamQuery, s->stream);
-	  if (r == CUDA_SUCCESS)
-	    continue;
-	  else if (r != CUDA_ERROR_NOT_READY)
-	    GOMP_PLUGIN_fatal ("cuStreamQuery error: %s", cuda_error (r));
-
-	  CUDA_CALL_ASSERT (cuStreamSynchronize, s->stream);
-	}
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  event_gc (true);
-}
-
-static void
-nvptx_wait_all_async (int async)
-{
-  struct ptx_stream *waiting_stream, *other_stream;
-  CUevent *e;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  pthread_t self = pthread_self ();
-
-  /* The stream doing the waiting.  This could be the first mention of the
-     stream, so create it if necessary.  */
-  waiting_stream
-    = select_stream_for_async (async, pthread_self (), true, NULL);
-
-  /* Launches on the null stream already block on other streams in the
-     context.  */
-  if (!waiting_stream || waiting_stream == nvthd->ptx_dev->null_stream)
-    return;
-
-  event_gc (true);
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  for (other_stream = nvthd->ptx_dev->active_streams;
-       other_stream != NULL;
-       other_stream = other_stream->next)
-    {
-      if (!other_stream->multithreaded
-	  && !pthread_equal (other_stream->host_thread, self))
-	continue;
-
-      e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-      CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-
-      /* Record an event on the waited-for stream.  */
-      CUDA_CALL_ASSERT (cuEventRecord, *e, other_stream->stream);
-
-      event_add (PTX_EVT_SYNC, e, NULL, 0);
-
-      CUDA_CALL_ASSERT (cuStreamWaitEvent, waiting_stream->stream, *e, 0);
-   }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-}
-
 static void *
 nvptx_get_current_cuda_device (void)
 {
@@ -1732,69 +953,6 @@ nvptx_get_current_cuda_context (void)
   return nvthd->ptx_dev->ctx;
 }
 
-static void *
-nvptx_get_cuda_stream (int async)
-{
-  struct ptx_stream *s;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!nvthd || !nvthd->ptx_dev)
-    return NULL;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-
-  return s ? s->stream : NULL;
-}
-
-static int
-nvptx_set_cuda_stream (int async, void *stream)
-{
-  struct ptx_stream *oldstream;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (async < 0)
-    GOMP_PLUGIN_fatal ("bad async %d", async);
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  /* We have a list of active streams and an array mapping async values to
-     entries of that list.  We need to take "ownership" of the passed-in stream,
-     and add it to our list, removing the previous entry also (if there was one)
-     in order to prevent resource leaks.  Note the potential for surprise
-     here: maybe we should keep track of passed-in streams and leave it up to
-     the user to tidy those up, but that doesn't work for stream handles
-     returned from acc_get_cuda_stream above...  */
-
-  oldstream = select_stream_for_async (async, self, false, NULL);
-
-  if (oldstream)
-    {
-      if (nvthd->ptx_dev->active_streams == oldstream)
-	nvthd->ptx_dev->active_streams = nvthd->ptx_dev->active_streams->next;
-      else
-	{
-	  struct ptx_stream *s = nvthd->ptx_dev->active_streams;
-	  while (s->next != oldstream)
-	    s = s->next;
-	  s->next = s->next->next;
-	}
-
-      CUDA_CALL_ASSERT (cuStreamDestroy, oldstream->stream);
-
-      if (!map_fini (oldstream))
-	GOMP_PLUGIN_fatal ("error when freeing host memory");
-
-      free (oldstream);
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  (void) select_stream_for_async (async, self, true, (CUstream) stream);
-
-  return 1;
-}
-
 /* Plugin entry points.  */
 
 const char *
@@ -2028,6 +1186,23 @@ GOMP_OFFLOAD_alloc (int ord, size_t size)
 {
   if (!nvptx_attach_host_thread_to_device (ord))
     return NULL;
+
+  struct ptx_device *ptx_dev = ptx_devices[ord];
+  struct ptx_free_block *blocks, *tmp;
+
+  pthread_mutex_lock (&ptx_dev->free_blocks_lock);
+  blocks = ptx_dev->free_blocks;
+  ptx_dev->free_blocks = NULL;
+  pthread_mutex_unlock (&ptx_dev->free_blocks_lock);
+
+  while (blocks)
+    {
+      tmp = blocks->next;
+      nvptx_free (blocks->ptr, ptx_dev);
+      free (blocks);
+      blocks = tmp;
+    }
+
   return nvptx_alloc (size);
 }
 
@@ -2035,93 +1210,92 @@ bool
 GOMP_OFFLOAD_free (int ord, void *ptr)
 {
   return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_free (ptr));
+	  && nvptx_free (ptr, ptx_devices[ord]));
 }
 
-bool
-GOMP_OFFLOAD_dev2host (int ord, void *dst, const void *src, size_t n)
-{
-  return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_dev2host (dst, src, n));
-}
-
-bool
-GOMP_OFFLOAD_host2dev (int ord, void *dst, const void *src, size_t n)
-{
-  return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_host2dev (dst, src, n));
-}
-
-bool
-GOMP_OFFLOAD_dev2dev (int ord, void *dst, const void *src, size_t n)
-{
-  struct ptx_device *ptx_dev = ptx_devices[ord];
-  CUDA_CALL (cuMemcpyDtoDAsync, (CUdeviceptr) dst, (CUdeviceptr) src, n,
-				ptx_dev->null_stream->stream);
-  return true;
-}
-
-void (*device_run) (int n, void *fn_ptr, void *vars) = NULL;
-
 void
 GOMP_OFFLOAD_openacc_exec (void (*fn) (void *), size_t mapnum,
 			   void **hostaddrs, void **devaddrs,
-			   int async, unsigned *dims, void *targ_mem_desc)
+			   unsigned *dims, void *targ_mem_desc)
 {
-  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, async, dims, targ_mem_desc);
-}
+  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
 
-void
-GOMP_OFFLOAD_openacc_register_async_cleanup (void *targ_mem_desc, int async)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  CUevent *e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
+  void **hp = NULL;
+  CUdeviceptr dp = 0;
 
-  CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-  CUDA_CALL_ASSERT (cuEventRecord, *e, nvthd->current_stream->stream);
-  event_add (PTX_EVT_ASYNC_CLEANUP, e, targ_mem_desc, async);
-}
+  if (mapnum > 0)
+    {
+      hp = alloca (mapnum * sizeof (void *));
+      for (int i = 0; i < mapnum; i++)
+	hp[i] = (devaddrs[i] ? devaddrs[i] : hostaddrs[i]);
+      CUDA_CALL_ASSERT (cuMemAlloc, &dp, mapnum * sizeof (void *));
+    }
 
-int
-GOMP_OFFLOAD_openacc_async_test (int async)
-{
-  return nvptx_async_test (async);
-}
+  /* Copy the (device) pointers to arguments to the device (dp and hp might in
+     fact have the same value on a unified-memory system).  */
+  if (mapnum > 0)
+    CUDA_CALL_ASSERT (cuMemcpyHtoD, dp, (void *) hp,
+		      mapnum * sizeof (void *));
 
-int
-GOMP_OFFLOAD_openacc_async_test_all (void)
-{
-  return nvptx_async_test_all ();
-}
+  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, dims, targ_mem_desc,
+	      dp, NULL);
 
-void
-GOMP_OFFLOAD_openacc_async_wait (int async)
-{
-  nvptx_wait (async);
+  CUresult r = CUDA_CALL_NOCHECK (cuStreamSynchronize, NULL);
+  const char *maybe_abort_msg = "(perhaps abort was called)";
+  if (r == CUDA_ERROR_LAUNCH_FAILED)
+    GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s %s\n", cuda_error (r),
+		       maybe_abort_msg);
+  else if (r != CUDA_SUCCESS)
+    GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s", cuda_error (r));
+  CUDA_CALL_ASSERT (cuMemFree, dp);
 }
 
-void
-GOMP_OFFLOAD_openacc_async_wait_async (int async1, int async2)
+static void
+cuda_free_argmem (void *ptr)
 {
-  nvptx_wait_async (async1, async2);
+  void **block = (void **) ptr;
+  nvptx_free (block[0], (struct ptx_device *) block[1]);
+  free (block);
 }
 
 void
-GOMP_OFFLOAD_openacc_async_wait_all (void)
+GOMP_OFFLOAD_openacc_async_exec (void (*fn) (void *), size_t mapnum,
+				 void **hostaddrs, void **devaddrs,
+				 unsigned *dims, void *targ_mem_desc,
+				 struct goacc_asyncqueue *aq)
 {
-  nvptx_wait_all ();
-}
+  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
 
-void
-GOMP_OFFLOAD_openacc_async_wait_all_async (int async)
-{
-  nvptx_wait_all_async (async);
-}
+  void **hp = NULL;
+  CUdeviceptr dp = 0;
+  void **block = NULL;
 
-void
-GOMP_OFFLOAD_openacc_async_set_async (int async)
-{
-  nvptx_set_async (async);
+  if (mapnum > 0)
+    {
+      block = (void **) GOMP_PLUGIN_malloc ((mapnum + 2) * sizeof (void *));
+      hp = block + 2;
+      for (int i = 0; i < mapnum; i++)
+	hp[i] = (devaddrs[i] ? devaddrs[i] : hostaddrs[i]);
+      CUDA_CALL_ASSERT (cuMemAlloc, &dp, mapnum * sizeof (void *));
+    }
+
+  /* Copy the (device) pointers to arguments to the device (dp and hp might in
+     fact have the same value on a unified-memory system).  */
+  if (mapnum > 0)
+    {
+      CUDA_CALL_ASSERT (cuMemcpyHtoDAsync, dp, (void *) hp,
+			mapnum * sizeof (void *), aq->cuda_stream);
+      block[0] = (void *) dp;
+
+      struct nvptx_thread *nvthd =
+	(struct nvptx_thread *) GOMP_PLUGIN_acc_thread ();
+      block[1] = (void *) nvthd->ptx_dev;
+    }
+  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, dims, targ_mem_desc,
+	      dp, aq->cuda_stream);
+
+  if (mapnum > 0)
+    GOMP_OFFLOAD_openacc_async_queue_callback (aq, cuda_free_argmem, block);
 }
 
 void *
@@ -2143,7 +1317,6 @@ GOMP_OFFLOAD_openacc_create_thread_data (int ord)
   if (!thd_ctx)
     CUDA_CALL_ASSERT (cuCtxPushCurrent, ptx_dev->ctx);
 
-  nvthd->current_stream = ptx_dev->null_stream;
   nvthd->ptx_dev = ptx_dev;
 
   return (void *) nvthd;
@@ -2168,21 +1341,182 @@ GOMP_OFFLOAD_openacc_cuda_get_current_context (voi
 }
 
 /* NOTE: This returns a CUstream, not a ptx_stream pointer.  */
-
 void *
-GOMP_OFFLOAD_openacc_cuda_get_stream (int async)
+GOMP_OFFLOAD_openacc_cuda_get_stream (struct goacc_asyncqueue *aq)
 {
-  return nvptx_get_cuda_stream (async);
+  return (void *) aq->cuda_stream;
 }
 
 /* NOTE: This takes a CUstream, not a ptx_stream pointer.  */
+int
+GOMP_OFFLOAD_openacc_cuda_set_stream (struct goacc_asyncqueue *aq, void *stream)
+{
+  if (aq->cuda_stream)
+    {
+      CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
+      CUDA_CALL_ASSERT (cuStreamDestroy, aq->cuda_stream);
+    }
 
+  aq->cuda_stream = (CUstream) stream;
+  return 1;
+}
+
+struct goacc_asyncqueue *
+GOMP_OFFLOAD_openacc_async_construct (void)
+{
+  struct goacc_asyncqueue *aq
+    = GOMP_PLUGIN_malloc (sizeof (struct goacc_asyncqueue));
+  aq->cuda_stream = NULL;
+  CUDA_CALL_ASSERT (cuStreamCreate, &aq->cuda_stream, CU_STREAM_DEFAULT);
+  //CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
+  return aq;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_destruct (struct goacc_asyncqueue *aq)
+{
+  CUDA_CALL_ERET (false, cuStreamDestroy, aq->cuda_stream);
+  free (aq);
+  return true;
+}
+
 int
-GOMP_OFFLOAD_openacc_cuda_set_stream (int async, void *stream)
+GOMP_OFFLOAD_openacc_async_test (struct goacc_asyncqueue *aq)
 {
-  return nvptx_set_cuda_stream (async, stream);
+  CUresult r = CUDA_CALL_NOCHECK (cuStreamQuery, aq->cuda_stream);
+  if (r == CUDA_SUCCESS)
+    return 1;
+  if (r == CUDA_ERROR_NOT_READY)
+    return 0;
+
+  GOMP_PLUGIN_error ("cuStreamQuery error: %s", cuda_error (r));
+  return -1;
 }
 
+void
+GOMP_OFFLOAD_openacc_async_synchronize (struct goacc_asyncqueue *aq)
+{
+  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
+}
+
+void
+GOMP_OFFLOAD_openacc_async_serialize (struct goacc_asyncqueue *aq1,
+				      struct goacc_asyncqueue *aq2)
+{
+  CUevent e;
+  CUDA_CALL_ASSERT (cuEventCreate, &e, CU_EVENT_DISABLE_TIMING);
+  CUDA_CALL_ASSERT (cuEventRecord, e, aq1->cuda_stream);
+  CUDA_CALL_ASSERT (cuStreamWaitEvent, aq2->cuda_stream, e, 0);
+}
+
+static void
+cuda_callback_wrapper (CUstream stream, CUresult res, void *ptr)
+{
+  if (res != CUDA_SUCCESS)
+    GOMP_PLUGIN_fatal ("%s error: %s", __FUNCTION__, cuda_error (res));
+  struct nvptx_callback *cb = (struct nvptx_callback *) ptr;
+  cb->fn (cb->ptr);
+  free (ptr);
+}
+
+void
+GOMP_OFFLOAD_openacc_async_queue_callback (struct goacc_asyncqueue *aq,
+					   void (*callback_fn)(void *),
+					   void *userptr)
+{
+  struct nvptx_callback *b = GOMP_PLUGIN_malloc (sizeof (*b));
+  b->fn = callback_fn;
+  b->ptr = userptr;
+  b->aq = aq;
+  CUDA_CALL_ASSERT (cuStreamAddCallback, aq->cuda_stream,
+		    cuda_callback_wrapper, (void *) b, 0);
+}
+
+static bool
+cuda_memcpy_sanity_check (const void *h, const void *d, size_t s)
+{
+  CUdeviceptr pb;
+  size_t ps;
+  if (!s)
+    return true;
+  if (!d)
+    {
+      GOMP_PLUGIN_error ("invalid device address");
+      return false;
+    }
+  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
+  if (!pb)
+    {
+      GOMP_PLUGIN_error ("invalid device address");
+      return false;
+    }
+  if (!h)
+    {
+      GOMP_PLUGIN_error ("invalid host address");
+      return false;
+    }
+  if (d == h)
+    {
+      GOMP_PLUGIN_error ("invalid host or device address");
+      return false;
+    }
+  if ((void *)(d + s) > (void *)(pb + ps))
+    {
+      GOMP_PLUGIN_error ("invalid size");
+      return false;
+    }
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_host2dev (int ord, void *dst, const void *src, size_t n)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (src, dst, n))
+    return false;
+  CUDA_CALL (cuMemcpyHtoD, (CUdeviceptr) dst, src, n);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_dev2host (int ord, void *dst, const void *src, size_t n)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (dst, src, n))
+    return false;
+  CUDA_CALL (cuMemcpyDtoH, dst, (CUdeviceptr) src, n);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_dev2dev (int ord, void *dst, const void *src, size_t n)
+{
+  CUDA_CALL (cuMemcpyDtoDAsync, (CUdeviceptr) dst, (CUdeviceptr) src, n, NULL);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_host2dev (int ord, void *dst, const void *src,
+				     size_t n, struct goacc_asyncqueue *aq)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (src, dst, n))
+    return false;
+  CUDA_CALL (cuMemcpyHtoDAsync, (CUdeviceptr) dst, src, n, aq->cuda_stream);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_dev2host (int ord, void *dst, const void *src,
+				     size_t n, struct goacc_asyncqueue *aq)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (dst, src, n))
+    return false;
+  CUDA_CALL (cuMemcpyDtoHAsync, dst, (CUdeviceptr) src, n, aq->cuda_stream);
+  return true;
+}
+
 /* Adjust launch dimensions: pick good values for number of blocks and warps
    and ensure that number of warps does not exceed CUDA limits as well as GCC's
    own limits.  */
@@ -2281,8 +1615,7 @@ GOMP_OFFLOAD_run (int ord, void *tgt_fn, void *tgt
     CU_LAUNCH_PARAM_END
   };
   r = CUDA_CALL_NOCHECK (cuLaunchKernel, function, teams, 1, 1,
-			 32, threads, 1, 0, ptx_dev->null_stream->stream,
-			 NULL, config);
+			 32, threads, 1, 0, NULL, NULL, config);
   if (r != CUDA_SUCCESS)
     GOMP_PLUGIN_fatal ("cuLaunchKernel error: %s", cuda_error (r));
 

^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes (revised, v3)
  2018-12-11 13:50     ` [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes (revised, v2) Chung-Lin Tang
@ 2018-12-18 15:07       ` Chung-Lin Tang
  0 siblings, 0 replies; 9+ messages in thread
From: Chung-Lin Tang @ 2018-12-18 15:07 UTC (permalink / raw)
  To: Thomas Schwinge, Chung-Lin Tang; +Cc: gcc-patches, Tom de Vries

[-- Attachment #1: Type: text/plain, Size: 2541 bytes --]

On 2018/12/11 9:50 PM, Chung-Lin Tang wrote:
> On 2018/12/10 6:02 PM, Chung-Lin Tang wrote:
>> On 2018/12/7 04:57 AM, Thomas Schwinge wrote>> --- a/libgomp/plugin/plugin-nvptx.c
>>>> +++ b/libgomp/plugin/plugin-nvptx.c
>>>
>>>> +struct goacc_asyncqueue *
>>>> +GOMP_OFFLOAD_openacc_async_construct (void)
>>>> +{
>>>> +  struct goacc_asyncqueue *aq
>>>> +    = GOMP_PLUGIN_malloc (sizeof (struct goacc_asyncqueue));
>>>> +  aq->cuda_stream = NULL;
>>>> +  CUDA_CALL_ASSERT (cuStreamCreate, &aq->cuda_stream, CU_STREAM_DEFAULT);
>>>
>>> Curiously (this was the same in the code before): does this have to be
>>> "CU_STREAM_DEFAULT" instead of "CU_STREAM_NON_BLOCKING", because we want
>>> to block anything from running in parallel with "acc_async_sync" GPU
>>> kernels, that use the "NULL" stream?  (Not asking you to change this now,
>>> but I wonder if this is overly strict?)
>>
>> IIUC, this non-blocking only pertains to the "Legacy Default Stream" in CUDA, which we're pretty much ignoring; we should be using the newer per-thread default stream model. We could review this issue later.
>>
>>>> +  if (aq->cuda_stream == NULL)
>>>> +    GOMP_PLUGIN_fatal ("CUDA stream create NULL\n");
>>>
>>> Can this actually happen, given the "CUDA_CALL_ASSERT" usage above?
>>
>> Hmm, yeah I think this is superfluous too...
>>
>>>> +  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
>>>
>>> Why is the synchronization needed here?
>>
>> I don't remember, could likely be something added during debug.
>> I'll remove this and test if things are okay.
> 
> I have removed the above seemingly unneeded lines and re-tested, appears okay.
> Also the formerly attached version seemed to for some reason had many conflicts
> with older code, all resolved in this v2 nvptx part.

GOMP_OFFLOAD_openacc_async_construct is updated to return NULL for failure,
there's also some adjustments in oacc-async.c, coming next.

Chung-Lin




[-- Attachment #2: 06.nvptx.v2-v3.diff --]
[-- Type: text/plain, Size: 846 bytes --]

diff -ru trunk-orig/libgomp/plugin/plugin-nvptx.c trunk-work/libgomp/plugin/plugin-nvptx.c
--- trunk-orig/libgomp/plugin/plugin-nvptx.c	2018-12-18 18:16:57.804871502 +0800
+++ trunk-work/libgomp/plugin/plugin-nvptx.c	2018-12-18 22:07:43.483068743 +0800
@@ -1364,16 +1364,12 @@
 struct goacc_asyncqueue *
 GOMP_OFFLOAD_openacc_async_construct (void)
 {
+  CUstream stream = NULL;
+  CUDA_CALL_ERET (NULL, cuStreamCreate, &stream, CU_STREAM_DEFAULT);
+
   struct goacc_asyncqueue *aq
     = GOMP_PLUGIN_malloc (sizeof (struct goacc_asyncqueue));
-  aq->cuda_stream = NULL;
-  CUDA_CALL_ASSERT (cuStreamCreate, &aq->cuda_stream, CU_STREAM_DEFAULT);
-  if (aq->cuda_stream == NULL)
-    GOMP_PLUGIN_fatal ("CUDA stream create NULL\n");
-
-  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
-
-
+  aq->cuda_stream = stream;
   return aq;
 }
 

[-- Attachment #3: async-06.nvptx.v3.patch --]
[-- Type: text/plain, Size: 42627 bytes --]

Index: libgomp/plugin/cuda/cuda.h
===================================================================
--- libgomp/plugin/cuda/cuda.h	(revision 267226)
+++ libgomp/plugin/cuda/cuda.h	(working copy)
@@ -54,7 +54,11 @@ typedef enum {
   CUDA_ERROR_INVALID_CONTEXT = 201,
   CUDA_ERROR_NOT_FOUND = 500,
   CUDA_ERROR_NOT_READY = 600,
-  CUDA_ERROR_LAUNCH_FAILED = 719
+  CUDA_ERROR_LAUNCH_FAILED = 719,
+  CUDA_ERROR_COOPERATIVE_LAUNCH_TOO_LARGE = 720,
+  CUDA_ERROR_NOT_PERMITTED = 800,
+  CUDA_ERROR_NOT_SUPPORTED = 801,
+  CUDA_ERROR_UNKNOWN = 999
 } CUresult;
 
 typedef enum {
@@ -173,6 +177,8 @@ CUresult cuModuleLoadData (CUmodule *, const void
 CUresult cuModuleUnload (CUmodule);
 CUresult cuOccupancyMaxPotentialBlockSize(int *, int *, CUfunction,
 					  CUoccupancyB2DSize, size_t, int);
+typedef void (*CUstreamCallback)(CUstream, CUresult, void *);
+CUresult cuStreamAddCallback(CUstream, CUstreamCallback, void *, unsigned int);
 CUresult cuStreamCreate (CUstream *, unsigned);
 #define cuStreamDestroy cuStreamDestroy_v2
 CUresult cuStreamDestroy (CUstream);
Index: libgomp/plugin/cuda-lib.def
===================================================================
--- libgomp/plugin/cuda-lib.def	(revision 267226)
+++ libgomp/plugin/cuda-lib.def	(working copy)
@@ -42,6 +42,7 @@ CUDA_ONE_CALL (cuModuleLoad)
 CUDA_ONE_CALL (cuModuleLoadData)
 CUDA_ONE_CALL (cuModuleUnload)
 CUDA_ONE_CALL_MAYBE_NULL (cuOccupancyMaxPotentialBlockSize)
+CUDA_ONE_CALL (cuStreamAddCallback)
 CUDA_ONE_CALL (cuStreamCreate)
 CUDA_ONE_CALL (cuStreamDestroy)
 CUDA_ONE_CALL (cuStreamQuery)
Index: libgomp/plugin/plugin-nvptx.c
===================================================================
--- libgomp/plugin/plugin-nvptx.c	(revision 267226)
+++ libgomp/plugin/plugin-nvptx.c	(working copy)
@@ -192,21 +192,18 @@ cuda_error (CUresult r)
 static unsigned int instantiated_devices = 0;
 static pthread_mutex_t ptx_dev_lock = PTHREAD_MUTEX_INITIALIZER;
 
-struct cuda_map
+/* NVPTX/CUDA specific definition of asynchronous queues.  */
+struct goacc_asyncqueue
 {
-  CUdeviceptr d;
-  size_t size;
-  bool active;
-  struct cuda_map *next;
+  CUstream cuda_stream;
 };
 
-struct ptx_stream
+struct nvptx_callback
 {
-  CUstream stream;
-  pthread_t host_thread;
-  bool multithreaded;
-  struct cuda_map *map;
-  struct ptx_stream *next;
+  void (*fn) (void *);
+  void *ptr;
+  struct goacc_asyncqueue *aq;
+  struct nvptx_callback *next;
 };
 
 /* Thread-specific data for PTX.  */
@@ -213,120 +210,13 @@ static pthread_mutex_t ptx_dev_lock = PTHREAD_MUTE
 
 struct nvptx_thread
 {
-  struct ptx_stream *current_stream;
+  /* We currently have this embedded inside the plugin because libgomp manages
+     devices through integer target_ids.  This might be better if using an
+     opaque target-specific pointer directly from gomp_device_descr.  */
   struct ptx_device *ptx_dev;
 };
 
-static struct cuda_map *
-cuda_map_create (size_t size)
-{
-  struct cuda_map *map = GOMP_PLUGIN_malloc (sizeof (struct cuda_map));
 
-  assert (map);
-
-  map->next = NULL;
-  map->size = size;
-  map->active = false;
-
-  CUDA_CALL_ERET (NULL, cuMemAlloc, &map->d, size);
-  assert (map->d);
-
-  return map;
-}
-
-static void
-cuda_map_destroy (struct cuda_map *map)
-{
-  CUDA_CALL_ASSERT (cuMemFree, map->d);
-  free (map);
-}
-
-/* The following map_* routines manage the CUDA device memory that
-   contains the data mapping arguments for cuLaunchKernel.  Each
-   asynchronous PTX stream may have multiple pending kernel
-   invocations, which are launched in a FIFO order.  As such, the map
-   routines maintains a queue of cuLaunchKernel arguments.
-
-   Calls to map_push and map_pop must be guarded by ptx_event_lock.
-   Likewise, calls to map_init and map_fini are guarded by
-   ptx_dev_lock inside GOMP_OFFLOAD_init_device and
-   GOMP_OFFLOAD_fini_device, respectively.  */
-
-static bool
-map_init (struct ptx_stream *s)
-{
-  int size = getpagesize ();
-
-  assert (s);
-
-  s->map = cuda_map_create (size);
-
-  return true;
-}
-
-static bool
-map_fini (struct ptx_stream *s)
-{
-  assert (s->map->next == NULL);
-  assert (!s->map->active);
-
-  cuda_map_destroy (s->map);
-
-  return true;
-}
-
-static void
-map_pop (struct ptx_stream *s)
-{
-  struct cuda_map *next;
-
-  assert (s != NULL);
-
-  if (s->map->next == NULL)
-    {
-      s->map->active = false;
-      return;
-    }
-
-  next = s->map->next;
-  cuda_map_destroy (s->map);
-  s->map = next;
-}
-
-static CUdeviceptr
-map_push (struct ptx_stream *s, size_t size)
-{
-  struct cuda_map *map = NULL, *t = NULL;
-
-  assert (s);
-  assert (s->map);
-
-  /* Each PTX stream requires a separate data region to store the
-     launch arguments for cuLaunchKernel.  Allocate a new
-     cuda_map and push it to the end of the list.  */
-  if (s->map->active)
-    {
-      map = cuda_map_create (size);
-
-      for (t = s->map; t->next != NULL; t = t->next)
-	;
-
-      t->next = map;
-    }
-  else if (s->map->size < size)
-    {
-      cuda_map_destroy (s->map);
-      map = cuda_map_create (size);
-    }
-  else
-    map = s->map;
-
-  s->map = map;
-  s->map->active = true;
-
-  return s->map->d;
-}
-
 /* Target data function launch information.  */
 
 struct targ_fn_launch
@@ -378,22 +268,18 @@ struct ptx_image_data
   struct ptx_image_data *next;
 };
 
+struct ptx_free_block
+{
+  void *ptr;
+  struct ptx_free_block *next;
+};
+
 struct ptx_device
 {
   CUcontext ctx;
   bool ctx_shared;
   CUdevice dev;
-  struct ptx_stream *null_stream;
-  /* All non-null streams associated with this device (actually context),
-     either created implicitly or passed in from the user (via
-     acc_set_cuda_stream).  */
-  struct ptx_stream *active_streams;
-  struct {
-    struct ptx_stream **arr;
-    int size;
-  } async_streams;
-  /* A lock for use when manipulating the above stream list and array.  */
-  pthread_mutex_t stream_lock;
+
   int ord;
   bool overlap;
   bool map;
@@ -411,32 +297,13 @@ struct ptx_device
 
   struct ptx_image_data *images;  /* Images loaded on device.  */
   pthread_mutex_t image_lock;     /* Lock for above list.  */
-  
-  struct ptx_device *next;
-};
 
-enum ptx_event_type
-{
-  PTX_EVT_MEM,
-  PTX_EVT_KNL,
-  PTX_EVT_SYNC,
-  PTX_EVT_ASYNC_CLEANUP
-};
+  struct ptx_free_block *free_blocks;
+  pthread_mutex_t free_blocks_lock;
 
-struct ptx_event
-{
-  CUevent *evt;
-  int type;
-  void *addr;
-  int ord;
-  int val;
-
-  struct ptx_event *next;
+  struct ptx_device *next;
 };
 
-static pthread_mutex_t ptx_event_lock;
-static struct ptx_event *ptx_events;
-
 static struct ptx_device **ptx_devices;
 
 static inline struct nvptx_thread *
@@ -445,193 +312,6 @@ nvptx_thread (void)
   return (struct nvptx_thread *) GOMP_PLUGIN_acc_thread ();
 }
 
-static bool
-init_streams_for_device (struct ptx_device *ptx_dev, int concurrency)
-{
-  int i;
-  struct ptx_stream *null_stream
-    = GOMP_PLUGIN_malloc (sizeof (struct ptx_stream));
-
-  null_stream->stream = NULL;
-  null_stream->host_thread = pthread_self ();
-  null_stream->multithreaded = true;
-  if (!map_init (null_stream))
-    return false;
-
-  ptx_dev->null_stream = null_stream;
-  ptx_dev->active_streams = NULL;
-  pthread_mutex_init (&ptx_dev->stream_lock, NULL);
-
-  if (concurrency < 1)
-    concurrency = 1;
-
-  /* This is just a guess -- make space for as many async streams as the
-     current device is capable of concurrently executing.  This can grow
-     later as necessary.  No streams are created yet.  */
-  ptx_dev->async_streams.arr
-    = GOMP_PLUGIN_malloc (concurrency * sizeof (struct ptx_stream *));
-  ptx_dev->async_streams.size = concurrency;
-
-  for (i = 0; i < concurrency; i++)
-    ptx_dev->async_streams.arr[i] = NULL;
-
-  return true;
-}
-
-static bool
-fini_streams_for_device (struct ptx_device *ptx_dev)
-{
-  free (ptx_dev->async_streams.arr);
-
-  bool ret = true;
-  while (ptx_dev->active_streams != NULL)
-    {
-      struct ptx_stream *s = ptx_dev->active_streams;
-      ptx_dev->active_streams = ptx_dev->active_streams->next;
-
-      ret &= map_fini (s);
-
-      CUresult r = CUDA_CALL_NOCHECK (cuStreamDestroy, s->stream);
-      if (r != CUDA_SUCCESS)
-	{
-	  GOMP_PLUGIN_error ("cuStreamDestroy error: %s", cuda_error (r));
-	  ret = false;
-	}
-      free (s);
-    }
-
-  ret &= map_fini (ptx_dev->null_stream);
-  free (ptx_dev->null_stream);
-  return ret;
-}
-
-/* Select a stream for (OpenACC-semantics) ASYNC argument for the current
-   thread THREAD (and also current device/context).  If CREATE is true, create
-   the stream if it does not exist (or use EXISTING if it is non-NULL), and
-   associate the stream with the same thread argument.  Returns stream to use
-   as result.  */
-
-static struct ptx_stream *
-select_stream_for_async (int async, pthread_t thread, bool create,
-			 CUstream existing)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  /* Local copy of TLS variable.  */
-  struct ptx_device *ptx_dev = nvthd->ptx_dev;
-  struct ptx_stream *stream = NULL;
-  int orig_async = async;
-
-  /* The special value acc_async_noval (-1) maps (for now) to an
-     implicitly-created stream, which is then handled the same as any other
-     numbered async stream.  Other options are available, e.g. using the null
-     stream for anonymous async operations, or choosing an idle stream from an
-     active set.  But, stick with this for now.  */
-  if (async > acc_async_sync)
-    async++;
-
-  if (create)
-    pthread_mutex_lock (&ptx_dev->stream_lock);
-
-  /* NOTE: AFAICT there's no particular need for acc_async_sync to map to the
-     null stream, and in fact better performance may be obtainable if it doesn't
-     (because the null stream enforces overly-strict synchronisation with
-     respect to other streams for legacy reasons, and that's probably not
-     needed with OpenACC).  Maybe investigate later.  */
-  if (async == acc_async_sync)
-    stream = ptx_dev->null_stream;
-  else if (async >= 0 && async < ptx_dev->async_streams.size
-	   && ptx_dev->async_streams.arr[async] && !(create && existing))
-    stream = ptx_dev->async_streams.arr[async];
-  else if (async >= 0 && create)
-    {
-      if (async >= ptx_dev->async_streams.size)
-	{
-	  int i, newsize = ptx_dev->async_streams.size * 2;
-
-	  if (async >= newsize)
-	    newsize = async + 1;
-
-	  ptx_dev->async_streams.arr
-	    = GOMP_PLUGIN_realloc (ptx_dev->async_streams.arr,
-				   newsize * sizeof (struct ptx_stream *));
-
-	  for (i = ptx_dev->async_streams.size; i < newsize; i++)
-	    ptx_dev->async_streams.arr[i] = NULL;
-
-	  ptx_dev->async_streams.size = newsize;
-	}
-
-      /* Create a new stream on-demand if there isn't one already, or if we're
-	 setting a particular async value to an existing (externally-provided)
-	 stream.  */
-      if (!ptx_dev->async_streams.arr[async] || existing)
-        {
-	  CUresult r;
-	  struct ptx_stream *s
-	    = GOMP_PLUGIN_malloc (sizeof (struct ptx_stream));
-
-	  if (existing)
-	    s->stream = existing;
-	  else
-	    {
-	      r = CUDA_CALL_NOCHECK (cuStreamCreate, &s->stream,
-				     CU_STREAM_DEFAULT);
-	      if (r != CUDA_SUCCESS)
-		{
-		  pthread_mutex_unlock (&ptx_dev->stream_lock);
-		  GOMP_PLUGIN_fatal ("cuStreamCreate error: %s",
-				     cuda_error (r));
-		}
-	    }
-
-	  /* If CREATE is true, we're going to be queueing some work on this
-	     stream.  Associate it with the current host thread.  */
-	  s->host_thread = thread;
-	  s->multithreaded = false;
-
-	  if (!map_init (s))
-	    {
-	      pthread_mutex_unlock (&ptx_dev->stream_lock);
-	      GOMP_PLUGIN_fatal ("map_init fail");
-	    }
-
-	  s->next = ptx_dev->active_streams;
-	  ptx_dev->active_streams = s;
-	  ptx_dev->async_streams.arr[async] = s;
-	}
-
-      stream = ptx_dev->async_streams.arr[async];
-    }
-  else if (async < 0)
-    {
-      if (create)
-	pthread_mutex_unlock (&ptx_dev->stream_lock);
-      GOMP_PLUGIN_fatal ("bad async %d", async);
-    }
-
-  if (create)
-    {
-      assert (stream != NULL);
-
-      /* If we're trying to use the same stream from different threads
-	 simultaneously, set stream->multithreaded to true.  This affects the
-	 behaviour of acc_async_test_all and acc_wait_all, which are supposed to
-	 only wait for asynchronous launches from the same host thread they are
-	 invoked on.  If multiple threads use the same async value, we make note
-	 of that here and fall back to testing/waiting for all threads in those
-	 functions.  */
-      if (thread != stream->host_thread)
-        stream->multithreaded = true;
-
-      pthread_mutex_unlock (&ptx_dev->stream_lock);
-    }
-  else if (stream && !stream->multithreaded
-	   && !pthread_equal (stream->host_thread, thread))
-    GOMP_PLUGIN_fatal ("async %d used on wrong thread", orig_async);
-
-  return stream;
-}
-
 /* Initialize the device.  Return TRUE on success, else FALSE.  PTX_DEV_LOCK
    should be locked on entry and remains locked on exit.  */
 
@@ -643,9 +323,6 @@ nvptx_init (void)
   if (instantiated_devices != 0)
     return true;
 
-  ptx_events = NULL;
-  pthread_mutex_init (&ptx_event_lock, NULL);
-
   if (!init_cuda_lib ())
     return false;
 
@@ -669,6 +346,11 @@ nvptx_attach_host_thread_to_device (int n)
   CUcontext thd_ctx;
 
   r = CUDA_CALL_NOCHECK (cuCtxGetDevice, &dev);
+  if (r == CUDA_ERROR_NOT_PERMITTED)
+    {
+      /* Assume we're in a CUDA callback, just return true.  */
+      return true;
+    }
   if (r != CUDA_SUCCESS && r != CUDA_ERROR_INVALID_CONTEXT)
     {
       GOMP_PLUGIN_error ("cuCtxGetDevice error: %s", cuda_error (r));
@@ -813,8 +495,8 @@ nvptx_open_device (int n)
   ptx_dev->images = NULL;
   pthread_mutex_init (&ptx_dev->image_lock, NULL);
 
-  if (!init_streams_for_device (ptx_dev, async_engines))
-    return NULL;
+  ptx_dev->free_blocks = NULL;
+  pthread_mutex_init (&ptx_dev->free_blocks_lock, NULL);
 
   return ptx_dev;
 }
@@ -825,9 +507,15 @@ nvptx_close_device (struct ptx_device *ptx_dev)
   if (!ptx_dev)
     return true;
 
-  if (!fini_streams_for_device (ptx_dev))
-    return false;
-  
+  for (struct ptx_free_block *b = ptx_dev->free_blocks; b;)
+    {
+      struct ptx_free_block *b_next = b->next;
+      CUDA_CALL (cuMemFree, (CUdeviceptr) b->ptr);
+      free (b);
+      b = b_next;
+    }
+
+  pthread_mutex_destroy (&ptx_dev->free_blocks_lock);
   pthread_mutex_destroy (&ptx_dev->image_lock);
 
   if (!ptx_dev->ctx_shared)
@@ -1007,139 +695,19 @@ link_ptx (CUmodule *module, const struct targ_ptx_
 }
 
 static void
-event_gc (bool memmap_lockable)
-{
-  struct ptx_event *ptx_event = ptx_events;
-  struct ptx_event *async_cleanups = NULL;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&ptx_event_lock);
-
-  while (ptx_event != NULL)
-    {
-      CUresult r;
-      struct ptx_event *e = ptx_event;
-
-      ptx_event = ptx_event->next;
-
-      if (e->ord != nvthd->ptx_dev->ord)
-	continue;
-
-      r = CUDA_CALL_NOCHECK (cuEventQuery, *e->evt);
-      if (r == CUDA_SUCCESS)
-	{
-	  bool append_async = false;
-	  CUevent *te;
-
-	  te = e->evt;
-
-	  switch (e->type)
-	    {
-	    case PTX_EVT_MEM:
-	    case PTX_EVT_SYNC:
-	      break;
-
-	    case PTX_EVT_KNL:
-	      map_pop (e->addr);
-	      break;
-
-	    case PTX_EVT_ASYNC_CLEANUP:
-	      {
-		/* The function gomp_plugin_async_unmap_vars needs to claim the
-		   memory-map splay tree lock for the current device, so we
-		   can't call it when one of our callers has already claimed
-		   the lock.  In that case, just delay the GC for this event
-		   until later.  */
-		if (!memmap_lockable)
-		  continue;
-
-		append_async = true;
-	      }
-	      break;
-	    }
-
-	  CUDA_CALL_NOCHECK (cuEventDestroy, *te);
-	  free ((void *)te);
-
-	  /* Unlink 'e' from ptx_events list.  */
-	  if (ptx_events == e)
-	    ptx_events = ptx_events->next;
-	  else
-	    {
-	      struct ptx_event *e_ = ptx_events;
-	      while (e_->next != e)
-		e_ = e_->next;
-	      e_->next = e_->next->next;
-	    }
-
-	  if (append_async)
-	    {
-	      e->next = async_cleanups;
-	      async_cleanups = e;
-	    }
-	  else
-	    free (e);
-	}
-    }
-
-  pthread_mutex_unlock (&ptx_event_lock);
-
-  /* We have to do these here, after ptx_event_lock is released.  */
-  while (async_cleanups)
-    {
-      struct ptx_event *e = async_cleanups;
-      async_cleanups = async_cleanups->next;
-
-      GOMP_PLUGIN_async_unmap_vars (e->addr, e->val);
-      free (e);
-    }
-}
-
-static void
-event_add (enum ptx_event_type type, CUevent *e, void *h, int val)
-{
-  struct ptx_event *ptx_event;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  assert (type == PTX_EVT_MEM || type == PTX_EVT_KNL || type == PTX_EVT_SYNC
-	  || type == PTX_EVT_ASYNC_CLEANUP);
-
-  ptx_event = GOMP_PLUGIN_malloc (sizeof (struct ptx_event));
-  ptx_event->type = type;
-  ptx_event->evt = e;
-  ptx_event->addr = h;
-  ptx_event->ord = nvthd->ptx_dev->ord;
-  ptx_event->val = val;
-
-  pthread_mutex_lock (&ptx_event_lock);
-
-  ptx_event->next = ptx_events;
-  ptx_events = ptx_event;
-
-  pthread_mutex_unlock (&ptx_event_lock);
-}
-
-static void
 nvptx_exec (void (*fn), size_t mapnum, void **hostaddrs, void **devaddrs,
-	    int async, unsigned *dims, void *targ_mem_desc)
+	    unsigned *dims, void *targ_mem_desc,
+	    CUdeviceptr dp, CUstream stream)
 {
   struct targ_fn_descriptor *targ_fn = (struct targ_fn_descriptor *) fn;
   CUfunction function;
-  CUresult r;
   int i;
-  struct ptx_stream *dev_str;
   void *kargs[1];
-  void *hp;
-  CUdeviceptr dp;
   struct nvptx_thread *nvthd = nvptx_thread ();
   int warp_size = nvthd->ptx_dev->warp_size;
-  const char *maybe_abort_msg = "(perhaps abort was called)";
 
   function = targ_fn->fn;
 
-  dev_str = select_stream_for_async (async, pthread_self (), false, NULL);
-  assert (dev_str == nvthd->current_stream);
-
   /* Initialize the launch dimensions.  Typically this is constant,
      provided by the device compiler, but we must permit runtime
      values.  */
@@ -1302,23 +870,6 @@ nvptx_exec (void (*fn), size_t mapnum, void **host
 			 suggest_workers, suggest_workers);
     }
 
-  /* This reserves a chunk of a pre-allocated page of memory mapped on both
-     the host and the device. HP is a host pointer to the new chunk, and DP is
-     the corresponding device pointer.  */
-  pthread_mutex_lock (&ptx_event_lock);
-  dp = map_push (dev_str, mapnum * sizeof (void *));
-  pthread_mutex_unlock (&ptx_event_lock);
-
-  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
-
-  /* Copy the array of arguments to the mapped page.  */
-  hp = alloca(sizeof(void *) * mapnum);
-  for (i = 0; i < mapnum; i++)
-    ((void **) hp)[i] = devaddrs[i];
-
-  /* Copy the (device) pointers to arguments to the device */
-  CUDA_CALL_ASSERT (cuMemcpyHtoD, dp, hp,
-		    mapnum * sizeof (void *));
   GOMP_PLUGIN_debug (0, "  %s: kernel %s: launch"
 		     " gangs=%u, workers=%u, vectors=%u\n",
 		     __FUNCTION__, targ_fn->launch->fn, dims[GOMP_DIM_GANG],
@@ -1329,58 +880,14 @@ nvptx_exec (void (*fn), size_t mapnum, void **host
   // num_gangs		nctaid.x
   // num_workers	ntid.y
   // vector length	ntid.x
-
   kargs[0] = &dp;
   CUDA_CALL_ASSERT (cuLaunchKernel, function,
 		    dims[GOMP_DIM_GANG], 1, 1,
 		    dims[GOMP_DIM_VECTOR], dims[GOMP_DIM_WORKER], 1,
-		    0, dev_str->stream, kargs, 0);
+		    0, stream, kargs, 0);
 
-#ifndef DISABLE_ASYNC
-  if (async < acc_async_noval)
-    {
-      r = CUDA_CALL_NOCHECK (cuStreamSynchronize, dev_str->stream);
-      if (r == CUDA_ERROR_LAUNCH_FAILED)
-	GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s %s\n", cuda_error (r),
-			   maybe_abort_msg);
-      else if (r != CUDA_SUCCESS)
-        GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s", cuda_error (r));
-    }
-  else
-    {
-      CUevent *e;
-
-      e = (CUevent *)GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-      r = CUDA_CALL_NOCHECK (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      if (r == CUDA_ERROR_LAUNCH_FAILED)
-	GOMP_PLUGIN_fatal ("cuEventCreate error: %s %s\n", cuda_error (r),
-			   maybe_abort_msg);
-      else if (r != CUDA_SUCCESS)
-        GOMP_PLUGIN_fatal ("cuEventCreate error: %s", cuda_error (r));
-
-      event_gc (true);
-
-      CUDA_CALL_ASSERT (cuEventRecord, *e, dev_str->stream);
-
-      event_add (PTX_EVT_KNL, e, (void *)dev_str, 0);
-    }
-#else
-  r = CUDA_CALL_NOCHECK (cuCtxSynchronize, );
-  if (r == CUDA_ERROR_LAUNCH_FAILED)
-    GOMP_PLUGIN_fatal ("cuCtxSynchronize error: %s %s\n", cuda_error (r),
-		       maybe_abort_msg);
-  else if (r != CUDA_SUCCESS)
-    GOMP_PLUGIN_fatal ("cuCtxSynchronize error: %s", cuda_error (r));
-#endif
-
   GOMP_PLUGIN_debug (0, "  %s: kernel %s: finished\n", __FUNCTION__,
 		     targ_fn->launch->fn);
-
-#ifndef DISABLE_ASYNC
-  if (async < acc_async_noval)
-#endif
-    map_pop (dev_str);
 }
 
 void * openacc_get_current_cuda_context (void);
@@ -1395,8 +902,21 @@ nvptx_alloc (size_t s)
 }
 
 static bool
-nvptx_free (void *p)
+nvptx_free (void *p, struct ptx_device *ptx_dev)
 {
+  /* Assume callback context if this is null.  */
+  if (GOMP_PLUGIN_acc_thread () == NULL)
+    {
+      struct ptx_free_block *n
+	= GOMP_PLUGIN_malloc (sizeof (struct ptx_free_block));
+      n->ptr = p;
+      pthread_mutex_lock (&ptx_dev->free_blocks_lock);
+      n->next = ptx_dev->free_blocks;
+      ptx_dev->free_blocks = n;
+      pthread_mutex_unlock (&ptx_dev->free_blocks_lock);
+      return true;
+    }
+
   CUdeviceptr pb;
   size_t ps;
 
@@ -1411,305 +931,6 @@ static bool
   return true;
 }
 
-
-static bool
-nvptx_host2dev (void *d, const void *h, size_t s)
-{
-  CUdeviceptr pb;
-  size_t ps;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!s)
-    return true;
-  if (!d)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-
-  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
-
-  if (!pb)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-  if (!h)
-    {
-      GOMP_PLUGIN_error ("invalid host address");
-      return false;
-    }
-  if (d == h)
-    {
-      GOMP_PLUGIN_error ("invalid host or device address");
-      return false;
-    }
-  if ((void *)(d + s) > (void *)(pb + ps))
-    {
-      GOMP_PLUGIN_error ("invalid size");
-      return false;
-    }
-
-#ifndef DISABLE_ASYNC
-  if (nvthd && nvthd->current_stream != nvthd->ptx_dev->null_stream)
-    {
-      CUevent *e = (CUevent *)GOMP_PLUGIN_malloc (sizeof (CUevent));
-      CUDA_CALL (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      event_gc (false);
-      CUDA_CALL (cuMemcpyHtoDAsync,
-		 (CUdeviceptr) d, h, s, nvthd->current_stream->stream);
-      CUDA_CALL (cuEventRecord, *e, nvthd->current_stream->stream);
-      event_add (PTX_EVT_MEM, e, (void *)h, 0);
-    }
-  else
-#endif
-    CUDA_CALL (cuMemcpyHtoD, (CUdeviceptr) d, h, s);
-
-  return true;
-}
-
-static bool
-nvptx_dev2host (void *h, const void *d, size_t s)
-{
-  CUdeviceptr pb;
-  size_t ps;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!s)
-    return true;
-  if (!d)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-
-  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
-
-  if (!pb)
-    {
-      GOMP_PLUGIN_error ("invalid device address");
-      return false;
-    }
-  if (!h)
-    {
-      GOMP_PLUGIN_error ("invalid host address");
-      return false;
-    }
-  if (d == h)
-    {
-      GOMP_PLUGIN_error ("invalid host or device address");
-      return false;
-    }
-  if ((void *)(d + s) > (void *)(pb + ps))
-    {
-      GOMP_PLUGIN_error ("invalid size");
-      return false;
-    }
-
-#ifndef DISABLE_ASYNC
-  if (nvthd && nvthd->current_stream != nvthd->ptx_dev->null_stream)
-    {
-      CUevent *e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-      CUDA_CALL (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-      event_gc (false);
-      CUDA_CALL (cuMemcpyDtoHAsync,
-		 h, (CUdeviceptr) d, s, nvthd->current_stream->stream);
-      CUDA_CALL (cuEventRecord, *e, nvthd->current_stream->stream);
-      event_add (PTX_EVT_MEM, e, (void *)h, 0);
-    }
-  else
-#endif
-    CUDA_CALL (cuMemcpyDtoH, h, (CUdeviceptr) d, s);
-
-  return true;
-}
-
-static void
-nvptx_set_async (int async)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  nvthd->current_stream
-    = select_stream_for_async (async, pthread_self (), true, NULL);
-}
-
-static int
-nvptx_async_test (int async)
-{
-  CUresult r;
-  struct ptx_stream *s;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-  if (!s)
-    return 1;
-
-  r = CUDA_CALL_NOCHECK (cuStreamQuery, s->stream);
-  if (r == CUDA_SUCCESS)
-    {
-      /* The oacc-parallel.c:goacc_wait function calls this hook to determine
-	 whether all work has completed on this stream, and if so omits the call
-	 to the wait hook.  If that happens, event_gc might not get called
-	 (which prevents variables from getting unmapped and their associated
-	 device storage freed), so call it here.  */
-      event_gc (true);
-      return 1;
-    }
-  else if (r == CUDA_ERROR_NOT_READY)
-    return 0;
-
-  GOMP_PLUGIN_fatal ("cuStreamQuery error: %s", cuda_error (r));
-
-  return 0;
-}
-
-static int
-nvptx_async_test_all (void)
-{
-  struct ptx_stream *s;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  for (s = nvthd->ptx_dev->active_streams; s != NULL; s = s->next)
-    {
-      if ((s->multithreaded || pthread_equal (s->host_thread, self))
-	  && CUDA_CALL_NOCHECK (cuStreamQuery,
-				s->stream) == CUDA_ERROR_NOT_READY)
-	{
-	  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-	  return 0;
-	}
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  event_gc (true);
-
-  return 1;
-}
-
-static void
-nvptx_wait (int async)
-{
-  struct ptx_stream *s;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-  if (!s)
-    return;
-
-  CUDA_CALL_ASSERT (cuStreamSynchronize, s->stream);
-
-  event_gc (true);
-}
-
-static void
-nvptx_wait_async (int async1, int async2)
-{
-  CUevent *e;
-  struct ptx_stream *s1, *s2;
-  pthread_t self = pthread_self ();
-
-  s1 = select_stream_for_async (async1, self, false, NULL);
-  if (!s1)
-    return;
-
-  /* The stream that is waiting (rather than being waited for) doesn't
-     necessarily have to exist already.  */
-  s2 = select_stream_for_async (async2, self, true, NULL);
-
-  /* A stream is always synchronized with itself.  */
-  if (s1 == s2)
-    return;
-
-  e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-  CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-
-  event_gc (true);
-
-  CUDA_CALL_ASSERT (cuEventRecord, *e, s1->stream);
-
-  event_add (PTX_EVT_SYNC, e, NULL, 0);
-
-  CUDA_CALL_ASSERT (cuStreamWaitEvent, s2->stream, *e, 0);
-}
-
-static void
-nvptx_wait_all (void)
-{
-  CUresult r;
-  struct ptx_stream *s;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  /* Wait for active streams initiated by this thread (or by multiple threads)
-     to complete.  */
-  for (s = nvthd->ptx_dev->active_streams; s != NULL; s = s->next)
-    {
-      if (s->multithreaded || pthread_equal (s->host_thread, self))
-	{
-	  r = CUDA_CALL_NOCHECK (cuStreamQuery, s->stream);
-	  if (r == CUDA_SUCCESS)
-	    continue;
-	  else if (r != CUDA_ERROR_NOT_READY)
-	    GOMP_PLUGIN_fatal ("cuStreamQuery error: %s", cuda_error (r));
-
-	  CUDA_CALL_ASSERT (cuStreamSynchronize, s->stream);
-	}
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  event_gc (true);
-}
-
-static void
-nvptx_wait_all_async (int async)
-{
-  struct ptx_stream *waiting_stream, *other_stream;
-  CUevent *e;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  pthread_t self = pthread_self ();
-
-  /* The stream doing the waiting.  This could be the first mention of the
-     stream, so create it if necessary.  */
-  waiting_stream
-    = select_stream_for_async (async, pthread_self (), true, NULL);
-
-  /* Launches on the null stream already block on other streams in the
-     context.  */
-  if (!waiting_stream || waiting_stream == nvthd->ptx_dev->null_stream)
-    return;
-
-  event_gc (true);
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  for (other_stream = nvthd->ptx_dev->active_streams;
-       other_stream != NULL;
-       other_stream = other_stream->next)
-    {
-      if (!other_stream->multithreaded
-	  && !pthread_equal (other_stream->host_thread, self))
-	continue;
-
-      e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
-
-      CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-
-      /* Record an event on the waited-for stream.  */
-      CUDA_CALL_ASSERT (cuEventRecord, *e, other_stream->stream);
-
-      event_add (PTX_EVT_SYNC, e, NULL, 0);
-
-      CUDA_CALL_ASSERT (cuStreamWaitEvent, waiting_stream->stream, *e, 0);
-   }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-}
-
 static void *
 nvptx_get_current_cuda_device (void)
 {
@@ -1732,75 +953,6 @@ nvptx_get_current_cuda_context (void)
   return nvthd->ptx_dev->ctx;
 }
 
-static void *
-nvptx_get_cuda_stream (int async)
-{
-  struct ptx_stream *s;
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  if (!nvthd || !nvthd->ptx_dev)
-    return NULL;
-
-  s = select_stream_for_async (async, pthread_self (), false, NULL);
-
-  return s ? s->stream : NULL;
-}
-
-static int
-nvptx_set_cuda_stream (int async, void *stream)
-{
-  struct ptx_stream *oldstream;
-  pthread_t self = pthread_self ();
-  struct nvptx_thread *nvthd = nvptx_thread ();
-
-  /* Due to the "null_stream" usage for "acc_async_sync", this cannot be used
-     to change the stream handle associated with "acc_async_sync".  */
-  if (async == acc_async_sync)
-    {
-      GOMP_PLUGIN_debug (0, "Refusing request to set CUDA stream associated"
-			 " with \"acc_async_sync\"\n");
-      return 0;
-    }
-
-  pthread_mutex_lock (&nvthd->ptx_dev->stream_lock);
-
-  /* We have a list of active streams and an array mapping async values to
-     entries of that list.  We need to take "ownership" of the passed-in stream,
-     and add it to our list, removing the previous entry also (if there was one)
-     in order to prevent resource leaks.  Note the potential for surprise
-     here: maybe we should keep track of passed-in streams and leave it up to
-     the user to tidy those up, but that doesn't work for stream handles
-     returned from acc_get_cuda_stream above...  */
-
-  oldstream = select_stream_for_async (async, self, false, NULL);
-
-  if (oldstream)
-    {
-      if (nvthd->ptx_dev->active_streams == oldstream)
-	nvthd->ptx_dev->active_streams = nvthd->ptx_dev->active_streams->next;
-      else
-	{
-	  struct ptx_stream *s = nvthd->ptx_dev->active_streams;
-	  while (s->next != oldstream)
-	    s = s->next;
-	  s->next = s->next->next;
-	}
-
-      CUDA_CALL_ASSERT (cuStreamDestroy, oldstream->stream);
-
-      if (!map_fini (oldstream))
-	GOMP_PLUGIN_fatal ("error when freeing host memory");
-
-      free (oldstream);
-    }
-
-  pthread_mutex_unlock (&nvthd->ptx_dev->stream_lock);
-
-  (void) select_stream_for_async (async, self, true, (CUstream) stream);
-
-  return 1;
-}
-
 /* Plugin entry points.  */
 
 const char *
@@ -2034,6 +1186,23 @@ GOMP_OFFLOAD_alloc (int ord, size_t size)
 {
   if (!nvptx_attach_host_thread_to_device (ord))
     return NULL;
+
+  struct ptx_device *ptx_dev = ptx_devices[ord];
+  struct ptx_free_block *blocks, *tmp;
+
+  pthread_mutex_lock (&ptx_dev->free_blocks_lock);
+  blocks = ptx_dev->free_blocks;
+  ptx_dev->free_blocks = NULL;
+  pthread_mutex_unlock (&ptx_dev->free_blocks_lock);
+
+  while (blocks)
+    {
+      tmp = blocks->next;
+      nvptx_free (blocks->ptr, ptx_dev);
+      free (blocks);
+      blocks = tmp;
+    }
+
   return nvptx_alloc (size);
 }
 
@@ -2041,93 +1210,92 @@ bool
 GOMP_OFFLOAD_free (int ord, void *ptr)
 {
   return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_free (ptr));
+	  && nvptx_free (ptr, ptx_devices[ord]));
 }
 
-bool
-GOMP_OFFLOAD_dev2host (int ord, void *dst, const void *src, size_t n)
-{
-  return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_dev2host (dst, src, n));
-}
-
-bool
-GOMP_OFFLOAD_host2dev (int ord, void *dst, const void *src, size_t n)
-{
-  return (nvptx_attach_host_thread_to_device (ord)
-	  && nvptx_host2dev (dst, src, n));
-}
-
-bool
-GOMP_OFFLOAD_dev2dev (int ord, void *dst, const void *src, size_t n)
-{
-  struct ptx_device *ptx_dev = ptx_devices[ord];
-  CUDA_CALL (cuMemcpyDtoDAsync, (CUdeviceptr) dst, (CUdeviceptr) src, n,
-				ptx_dev->null_stream->stream);
-  return true;
-}
-
-void (*device_run) (int n, void *fn_ptr, void *vars) = NULL;
-
 void
 GOMP_OFFLOAD_openacc_exec (void (*fn) (void *), size_t mapnum,
 			   void **hostaddrs, void **devaddrs,
-			   int async, unsigned *dims, void *targ_mem_desc)
+			   unsigned *dims, void *targ_mem_desc)
 {
-  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, async, dims, targ_mem_desc);
-}
+  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
 
-void
-GOMP_OFFLOAD_openacc_register_async_cleanup (void *targ_mem_desc, int async)
-{
-  struct nvptx_thread *nvthd = nvptx_thread ();
-  CUevent *e = (CUevent *) GOMP_PLUGIN_malloc (sizeof (CUevent));
+  void **hp = NULL;
+  CUdeviceptr dp = 0;
 
-  CUDA_CALL_ASSERT (cuEventCreate, e, CU_EVENT_DISABLE_TIMING);
-  CUDA_CALL_ASSERT (cuEventRecord, *e, nvthd->current_stream->stream);
-  event_add (PTX_EVT_ASYNC_CLEANUP, e, targ_mem_desc, async);
-}
+  if (mapnum > 0)
+    {
+      hp = alloca (mapnum * sizeof (void *));
+      for (int i = 0; i < mapnum; i++)
+	hp[i] = (devaddrs[i] ? devaddrs[i] : hostaddrs[i]);
+      CUDA_CALL_ASSERT (cuMemAlloc, &dp, mapnum * sizeof (void *));
+    }
 
-int
-GOMP_OFFLOAD_openacc_async_test (int async)
-{
-  return nvptx_async_test (async);
-}
+  /* Copy the (device) pointers to arguments to the device (dp and hp might in
+     fact have the same value on a unified-memory system).  */
+  if (mapnum > 0)
+    CUDA_CALL_ASSERT (cuMemcpyHtoD, dp, (void *) hp,
+		      mapnum * sizeof (void *));
 
-int
-GOMP_OFFLOAD_openacc_async_test_all (void)
-{
-  return nvptx_async_test_all ();
-}
+  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, dims, targ_mem_desc,
+	      dp, NULL);
 
-void
-GOMP_OFFLOAD_openacc_async_wait (int async)
-{
-  nvptx_wait (async);
+  CUresult r = CUDA_CALL_NOCHECK (cuStreamSynchronize, NULL);
+  const char *maybe_abort_msg = "(perhaps abort was called)";
+  if (r == CUDA_ERROR_LAUNCH_FAILED)
+    GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s %s\n", cuda_error (r),
+		       maybe_abort_msg);
+  else if (r != CUDA_SUCCESS)
+    GOMP_PLUGIN_fatal ("cuStreamSynchronize error: %s", cuda_error (r));
+  CUDA_CALL_ASSERT (cuMemFree, dp);
 }
 
-void
-GOMP_OFFLOAD_openacc_async_wait_async (int async1, int async2)
+static void
+cuda_free_argmem (void *ptr)
 {
-  nvptx_wait_async (async1, async2);
+  void **block = (void **) ptr;
+  nvptx_free (block[0], (struct ptx_device *) block[1]);
+  free (block);
 }
 
 void
-GOMP_OFFLOAD_openacc_async_wait_all (void)
+GOMP_OFFLOAD_openacc_async_exec (void (*fn) (void *), size_t mapnum,
+				 void **hostaddrs, void **devaddrs,
+				 unsigned *dims, void *targ_mem_desc,
+				 struct goacc_asyncqueue *aq)
 {
-  nvptx_wait_all ();
-}
+  GOMP_PLUGIN_debug (0, "  %s: prepare mappings\n", __FUNCTION__);
 
-void
-GOMP_OFFLOAD_openacc_async_wait_all_async (int async)
-{
-  nvptx_wait_all_async (async);
-}
+  void **hp = NULL;
+  CUdeviceptr dp = 0;
+  void **block = NULL;
 
-void
-GOMP_OFFLOAD_openacc_async_set_async (int async)
-{
-  nvptx_set_async (async);
+  if (mapnum > 0)
+    {
+      block = (void **) GOMP_PLUGIN_malloc ((mapnum + 2) * sizeof (void *));
+      hp = block + 2;
+      for (int i = 0; i < mapnum; i++)
+	hp[i] = (devaddrs[i] ? devaddrs[i] : hostaddrs[i]);
+      CUDA_CALL_ASSERT (cuMemAlloc, &dp, mapnum * sizeof (void *));
+    }
+
+  /* Copy the (device) pointers to arguments to the device (dp and hp might in
+     fact have the same value on a unified-memory system).  */
+  if (mapnum > 0)
+    {
+      CUDA_CALL_ASSERT (cuMemcpyHtoDAsync, dp, (void *) hp,
+			mapnum * sizeof (void *), aq->cuda_stream);
+      block[0] = (void *) dp;
+
+      struct nvptx_thread *nvthd =
+	(struct nvptx_thread *) GOMP_PLUGIN_acc_thread ();
+      block[1] = (void *) nvthd->ptx_dev;
+    }
+  nvptx_exec (fn, mapnum, hostaddrs, devaddrs, dims, targ_mem_desc,
+	      dp, aq->cuda_stream);
+
+  if (mapnum > 0)
+    GOMP_OFFLOAD_openacc_async_queue_callback (aq, cuda_free_argmem, block);
 }
 
 void *
@@ -2149,7 +1317,6 @@ GOMP_OFFLOAD_openacc_create_thread_data (int ord)
   if (!thd_ctx)
     CUDA_CALL_ASSERT (cuCtxPushCurrent, ptx_dev->ctx);
 
-  nvthd->current_stream = ptx_dev->null_stream;
   nvthd->ptx_dev = ptx_dev;
 
   return (void *) nvthd;
@@ -2174,21 +1341,183 @@ GOMP_OFFLOAD_openacc_cuda_get_current_context (voi
 }
 
 /* NOTE: This returns a CUstream, not a ptx_stream pointer.  */
-
 void *
-GOMP_OFFLOAD_openacc_cuda_get_stream (int async)
+GOMP_OFFLOAD_openacc_cuda_get_stream (struct goacc_asyncqueue *aq)
 {
-  return nvptx_get_cuda_stream (async);
+  return (void *) aq->cuda_stream;
 }
 
 /* NOTE: This takes a CUstream, not a ptx_stream pointer.  */
+int
+GOMP_OFFLOAD_openacc_cuda_set_stream (struct goacc_asyncqueue *aq, void *stream)
+{
+  if (aq->cuda_stream)
+    {
+      CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
+      CUDA_CALL_ASSERT (cuStreamDestroy, aq->cuda_stream);
+    }
 
+  aq->cuda_stream = (CUstream) stream;
+  return 1;
+}
+
+struct goacc_asyncqueue *
+GOMP_OFFLOAD_openacc_async_construct (void)
+{
+  CUstream stream = NULL;
+  CUDA_CALL_ERET (NULL, cuStreamCreate, &stream, CU_STREAM_DEFAULT);
+
+  struct goacc_asyncqueue *aq
+    = GOMP_PLUGIN_malloc (sizeof (struct goacc_asyncqueue));
+  aq->cuda_stream = stream;
+  return aq;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_destruct (struct goacc_asyncqueue *aq)
+{
+  CUDA_CALL_ERET (false, cuStreamDestroy, aq->cuda_stream);
+  free (aq);
+  return true;
+}
+
 int
-GOMP_OFFLOAD_openacc_cuda_set_stream (int async, void *stream)
+GOMP_OFFLOAD_openacc_async_test (struct goacc_asyncqueue *aq)
 {
-  return nvptx_set_cuda_stream (async, stream);
+  CUresult r = CUDA_CALL_NOCHECK (cuStreamQuery, aq->cuda_stream);
+  if (r == CUDA_SUCCESS)
+    return 1;
+  if (r == CUDA_ERROR_NOT_READY)
+    return 0;
+
+  GOMP_PLUGIN_error ("cuStreamQuery error: %s", cuda_error (r));
+  return -1;
 }
 
+void
+GOMP_OFFLOAD_openacc_async_synchronize (struct goacc_asyncqueue *aq)
+{
+  CUDA_CALL_ASSERT (cuStreamSynchronize, aq->cuda_stream);
+}
+
+void
+GOMP_OFFLOAD_openacc_async_serialize (struct goacc_asyncqueue *aq1,
+				      struct goacc_asyncqueue *aq2)
+{
+  CUevent e;
+  CUDA_CALL_ASSERT (cuEventCreate, &e, CU_EVENT_DISABLE_TIMING);
+  CUDA_CALL_ASSERT (cuEventRecord, e, aq1->cuda_stream);
+  CUDA_CALL_ASSERT (cuStreamWaitEvent, aq2->cuda_stream, e, 0);
+}
+
+static void
+cuda_callback_wrapper (CUstream stream, CUresult res, void *ptr)
+{
+  if (res != CUDA_SUCCESS)
+    GOMP_PLUGIN_fatal ("%s error: %s", __FUNCTION__, cuda_error (res));
+  struct nvptx_callback *cb = (struct nvptx_callback *) ptr;
+  cb->fn (cb->ptr);
+  free (ptr);
+}
+
+void
+GOMP_OFFLOAD_openacc_async_queue_callback (struct goacc_asyncqueue *aq,
+					   void (*callback_fn)(void *),
+					   void *userptr)
+{
+  struct nvptx_callback *b = GOMP_PLUGIN_malloc (sizeof (*b));
+  b->fn = callback_fn;
+  b->ptr = userptr;
+  b->aq = aq;
+  CUDA_CALL_ASSERT (cuStreamAddCallback, aq->cuda_stream,
+		    cuda_callback_wrapper, (void *) b, 0);
+}
+
+static bool
+cuda_memcpy_sanity_check (const void *h, const void *d, size_t s)
+{
+  CUdeviceptr pb;
+  size_t ps;
+  if (!s)
+    return true;
+  if (!d)
+    {
+      GOMP_PLUGIN_error ("invalid device address");
+      return false;
+    }
+  CUDA_CALL (cuMemGetAddressRange, &pb, &ps, (CUdeviceptr) d);
+  if (!pb)
+    {
+      GOMP_PLUGIN_error ("invalid device address");
+      return false;
+    }
+  if (!h)
+    {
+      GOMP_PLUGIN_error ("invalid host address");
+      return false;
+    }
+  if (d == h)
+    {
+      GOMP_PLUGIN_error ("invalid host or device address");
+      return false;
+    }
+  if ((void *)(d + s) > (void *)(pb + ps))
+    {
+      GOMP_PLUGIN_error ("invalid size");
+      return false;
+    }
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_host2dev (int ord, void *dst, const void *src, size_t n)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (src, dst, n))
+    return false;
+  CUDA_CALL (cuMemcpyHtoD, (CUdeviceptr) dst, src, n);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_dev2host (int ord, void *dst, const void *src, size_t n)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (dst, src, n))
+    return false;
+  CUDA_CALL (cuMemcpyDtoH, dst, (CUdeviceptr) src, n);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_dev2dev (int ord, void *dst, const void *src, size_t n)
+{
+  CUDA_CALL (cuMemcpyDtoDAsync, (CUdeviceptr) dst, (CUdeviceptr) src, n, NULL);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_host2dev (int ord, void *dst, const void *src,
+				     size_t n, struct goacc_asyncqueue *aq)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (src, dst, n))
+    return false;
+  CUDA_CALL (cuMemcpyHtoDAsync, (CUdeviceptr) dst, src, n, aq->cuda_stream);
+  return true;
+}
+
+bool
+GOMP_OFFLOAD_openacc_async_dev2host (int ord, void *dst, const void *src,
+				     size_t n, struct goacc_asyncqueue *aq)
+{
+  if (!nvptx_attach_host_thread_to_device (ord)
+      || !cuda_memcpy_sanity_check (dst, src, n))
+    return false;
+  CUDA_CALL (cuMemcpyDtoHAsync, dst, (CUdeviceptr) src, n, aq->cuda_stream);
+  return true;
+}
+
 /* Adjust launch dimensions: pick good values for number of blocks and warps
    and ensure that number of warps does not exceed CUDA limits as well as GCC's
    own limits.  */
@@ -2287,8 +1616,7 @@ GOMP_OFFLOAD_run (int ord, void *tgt_fn, void *tgt
     CU_LAUNCH_PARAM_END
   };
   r = CUDA_CALL_NOCHECK (cuLaunchKernel, function, teams, 1, 1,
-			 32, threads, 1, 0, ptx_dev->null_stream->stream,
-			 NULL, config);
+			 32, threads, 1, 0, NULL, NULL, config);
   if (r != CUDA_SUCCESS)
     GOMP_PLUGIN_fatal ("cuLaunchKernel error: %s", cuda_error (r));
 

^ permalink raw reply	[flat|nested] 9+ messages in thread

* nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case (was: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes)
  2018-09-25 13:13 [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes Chung-Lin Tang
  2018-10-05 14:07 ` Tom de Vries
  2018-12-06 20:57 ` Thomas Schwinge
@ 2023-01-12 13:51 ` Thomas Schwinge
  2023-01-13 13:17   ` Chung-Lin Tang
  2 siblings, 1 reply; 9+ messages in thread
From: Thomas Schwinge @ 2023-01-12 13:51 UTC (permalink / raw)
  To: gcc-patches, Chung-Lin Tang, Tom de Vries

[-- Attachment #1: Type: text/plain, Size: 2597 bytes --]

Hi Chung-Lin, Tom!

It's been a while:

On 2018-09-25T21:11:58+0800, Chung-Lin Tang <chunglin_tang@mentor.com> wrote:
> [...] NVPTX/CUDA-specific implementation
> of the new-style goacc_asyncqueues.

In an OpenACC 'async' setting, where the device kernel (expectedly)
crashes because of "an illegal memory access was encountered", I'm
running into a deadlock here:

> --- a/libgomp/plugin/plugin-nvptx.c
> +++ b/libgomp/plugin/plugin-nvptx.c

> +static void
> +cuda_callback_wrapper (CUstream stream, CUresult res, void *ptr)
> +{
> +  if (res != CUDA_SUCCESS)
> +    GOMP_PLUGIN_fatal ("%s error: %s", __FUNCTION__, cuda_error (res));
> +  struct nvptx_callback *cb = (struct nvptx_callback *) ptr;
> +  cb->fn (cb->ptr);
> +  free (ptr);
> +}
> +
> +void
> +GOMP_OFFLOAD_openacc_async_queue_callback (struct goacc_asyncqueue *aq,
> +                                        void (*callback_fn)(void *),
> +                                        void *userptr)
> +{
> +  struct nvptx_callback *b = GOMP_PLUGIN_malloc (sizeof (*b));
> +  b->fn = callback_fn;
> +  b->ptr = userptr;
> +  b->aq = aq;
> +  CUDA_CALL_ASSERT (cuStreamAddCallback, aq->cuda_stream,
> +                 cuda_callback_wrapper, (void *) b, 0);
> +}

In my case, 'cuda_callback_wrapper' (expectedly) gets invoked with
'res != CUDA_SUCCESS' ("an illegal memory access was encountered").
When we invoke 'GOMP_PLUGIN_fatal', this attempts to shut down the device
(..., which deadlocks); that's generally problematic: per
<https://docs.nvidia.com/cuda/cuda-driver-api/group__CUDA__STREAM.html#group__CUDA__STREAM_1g613d97a277d7640f4cb1c03bd51c2483>
"'cuStreamAddCallback' [...] Callbacks must not make any CUDA API calls".

Given that eventually we must reach a host/device synchronization point
(latest when the device is shut down at program termination), and the
non-'CUDA_SUCCESS' will be upheld until then, it does seem safe to
replace this 'GOMP_PLUGIN_fatal' with 'GOMP_PLUGIN_error' as per the
"nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case"
attached.  OK to push?

(Might we even skip 'GOMP_PLUGIN_error' here, understanding that the
error will be caught and reported at the next host/device synchronization
point?  But I've not verified that.)


Grüße
 Thomas


-----------------
Siemens Electronic Design Automation GmbH; Anschrift: Arnulfstraße 201, 80634 München; Gesellschaft mit beschränkter Haftung; Geschäftsführer: Thomas Heurung, Frank Thürauf; Sitz der Gesellschaft: München; Registergericht München, HRB 106955

[-- Warning: decoded text below may be mangled, UTF-8 assumed --]
[-- Attachment #2: 0001-nvptx-Avoid-deadlock-in-cuStreamAddCallback-callback.patch --]
[-- Type: text/x-diff, Size: 1634 bytes --]

From b7ddcc0807967750e3c884326ed4c53c05cde81f Mon Sep 17 00:00:00 2001
From: Thomas Schwinge <thomas@codesourcery.com>
Date: Thu, 12 Jan 2023 14:39:46 +0100
Subject: [PATCH] nvptx: Avoid deadlock in 'cuStreamAddCallback' callback,
 error case

When we invoke 'GOMP_PLUGIN_fatal', this attempts to shut down the device
(..., which may deadlock); that's generally problematic: per
<https://docs.nvidia.com/cuda/cuda-driver-api/group__CUDA__STREAM.html#group__CUDA__STREAM_1g613d97a277d7640f4cb1c03bd51c2483>
"'cuStreamAddCallback' [...] Callbacks must not make any CUDA API calls".

Given that eventually we must reach a host/device synchronization point
(latest when the device is shut down at program termination), and the
non-'CUDA_SUCCESS' will be upheld until then, it does seem safe to
replace this 'GOMP_PLUGIN_fatal' with 'GOMP_PLUGIN_error'.

	libgomp/
	* plugin/plugin-nvptx.c (cuda_callback_wrapper): Invoke
	'GOMP_PLUGIN_error' instead of 'GOMP_PLUGIN_fatal'.
---
 libgomp/plugin/plugin-nvptx.c | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/libgomp/plugin/plugin-nvptx.c b/libgomp/plugin/plugin-nvptx.c
index 395639537e83..cdb3d435bdc8 100644
--- a/libgomp/plugin/plugin-nvptx.c
+++ b/libgomp/plugin/plugin-nvptx.c
@@ -1927,7 +1927,7 @@ static void
 cuda_callback_wrapper (CUstream stream, CUresult res, void *ptr)
 {
   if (res != CUDA_SUCCESS)
-    GOMP_PLUGIN_fatal ("%s error: %s", __FUNCTION__, cuda_error (res));
+    GOMP_PLUGIN_error ("%s error: %s", __FUNCTION__, cuda_error (res));
   struct nvptx_callback *cb = (struct nvptx_callback *) ptr;
   cb->fn (cb->ptr);
   free (ptr);
-- 
2.39.0


^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case (was: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes)
  2023-01-12 13:51 ` nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case (was: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes) Thomas Schwinge
@ 2023-01-13 13:17   ` Chung-Lin Tang
  2023-01-13 13:59     ` Thomas Schwinge
  0 siblings, 1 reply; 9+ messages in thread
From: Chung-Lin Tang @ 2023-01-13 13:17 UTC (permalink / raw)
  To: Thomas Schwinge, gcc-patches, Chung-Lin Tang, Tom de Vries

Hi Thomas,

On 2023/1/12 9:51 PM, Thomas Schwinge wrote:
> In my case, 'cuda_callback_wrapper' (expectedly) gets invoked with
> 'res != CUDA_SUCCESS' ("an illegal memory access was encountered").
> When we invoke 'GOMP_PLUGIN_fatal', this attempts to shut down the device
> (..., which deadlocks); that's generally problematic: per
> https://docs.nvidia.com/cuda/cuda-driver-api/group__CUDA__STREAM.html#group__CUDA__STREAM_1g613d97a277d7640f4cb1c03bd51c2483
> "'cuStreamAddCallback' [...] Callbacks must not make any CUDA API calls".

I remember running into this myself when first creating this async support
(IIRC in my case it was cuFree()-ing something) yet you've found another mistake here! :) 

> Given that eventually we must reach a host/device synchronization point
> (latest when the device is shut down at program termination), and the
> non-'CUDA_SUCCESS' will be upheld until then, it does seem safe to
> replace this 'GOMP_PLUGIN_fatal' with 'GOMP_PLUGIN_error' as per the
> "nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case"
> attached.  OK to push?

I think this patch is fine. Actual approval powers are your's or Tom's :)

> 
> (Might we even skip 'GOMP_PLUGIN_error' here, understanding that the
> error will be caught and reported at the next host/device synchronization
> point?  But I've not verified that.)

Actually, the CUDA driver API docs are a bit vague on what exactly this
CUresult arg to the callback actually means. The 'res != CUDA_SUCCESS' handling
here was basically just generic handling. I am not really sure what is the
true right thing to do here (is the error still retained by CUDA after the callback
completes?)

Chung-Lin


^ permalink raw reply	[flat|nested] 9+ messages in thread

* Re: nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case (was: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes)
  2023-01-13 13:17   ` Chung-Lin Tang
@ 2023-01-13 13:59     ` Thomas Schwinge
  0 siblings, 0 replies; 9+ messages in thread
From: Thomas Schwinge @ 2023-01-13 13:59 UTC (permalink / raw)
  To: Chung-Lin Tang, Tom de Vries; +Cc: gcc-patches

Hi!

On 2023-01-13T21:17:43+0800, Chung-Lin Tang <chunglin.tang@siemens.com> wrote:
> On 2023/1/12 9:51 PM, Thomas Schwinge wrote:
>> In my case, 'cuda_callback_wrapper' (expectedly) gets invoked with
>> 'res != CUDA_SUCCESS' ("an illegal memory access was encountered").
>> When we invoke 'GOMP_PLUGIN_fatal', this attempts to shut down the device
>> (..., which deadlocks); that's generally problematic: per
>> https://docs.nvidia.com/cuda/cuda-driver-api/group__CUDA__STREAM.html#group__CUDA__STREAM_1g613d97a277d7640f4cb1c03bd51c2483
>> "'cuStreamAddCallback' [...] Callbacks must not make any CUDA API calls".
>
> I remember running into this myself when first creating this async support
> (IIRC in my case it was cuFree()-ing something) yet you've found another mistake here! :)

;-)

>> Given that eventually we must reach a host/device synchronization point
>> (latest when the device is shut down at program termination), and the
>> non-'CUDA_SUCCESS' will be upheld until then, it does seem safe to
>> replace this 'GOMP_PLUGIN_fatal' with 'GOMP_PLUGIN_error' as per the
>> "nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case"
>> attached.  OK to push?
>
> I think this patch is fine. Actual approval powers are your's or Tom's :)

ACK.  I'll let it sit for some more time before 'git push'.


>> (Might we even skip 'GOMP_PLUGIN_error' here, understanding that the
>> error will be caught and reported at the next host/device synchronization
>> point?  But I've not verified that.)
>
> Actually, the CUDA driver API docs are a bit vague on what exactly this
> CUresult arg to the callback actually means. The 'res != CUDA_SUCCESS' handling
> here was basically just generic handling.

I suppose this really is just for its own use: for example, skip certain
things in presence of pre-existing error?

> I am not really sure what is the
> true right thing to do here (is the error still retained by CUDA after the callback
> completes?)

Indeed the latter is what I do observe:

      GOMP_OFFLOAD_openacc_async_exec: prepare mappings
      nvptx_exec: kernel main$_omp_fn$0: launch gangs=1, workers=1, vectors=32
      nvptx_exec: kernel main$_omp_fn$0: finished

    libgomp: cuMemcpyDtoHAsync_v2 error: an illegal memory access was encountered

    libgomp:
    libgomp: Copying of dev object [0x7f9a45000000..0x7f9a45000028) to host object [0x1d89350..0x1d89378) failed
    cuda_callback_wrapper error: an illegal memory access was encountered

    libgomp: cuStreamDestroy error: an illegal memory access was encountered

    libgomp: cuMemFree_v2 error: an illegal memory access was encountered

    libgomp: device finalization failed

Here, after the 'async' OpenACC 'parallel' a 'copyout' gets enqueued,
thus 'cuMemcpyDtoHAsync_v2', which is where we first get the device-side
fault reported (all as expected).  Then -- CUDA-internally
multi-threaded, I suppose (thus the mangled printing) -- we print the
'Copying [...] failed' error plus get 'cuda_callback_wrapper' invoked.
This receives the previous 'CUresult' as seen, and then the error is
still visible at device shut-down, as shown by the following reports.
(This makes sense, as the 'CUcontext' does not magically recover.)

Also, per
<https://docs.nvidia.com/cuda/cuda-driver-api/group__CUDA__STREAM.html#group__CUDA__STREAM_1g613d97a277d7640f4cb1c03bd51c2483>,
"In the event of a device error, all subsequently executed callbacks will
receive an appropriate 'CUresult'".

But again: I'm perfectly fine with the repeated error reporting.


Grüße
 Thomas
-----------------
Siemens Electronic Design Automation GmbH; Anschrift: Arnulfstraße 201, 80634 München; Gesellschaft mit beschränkter Haftung; Geschäftsführer: Thomas Heurung, Frank Thürauf; Sitz der Gesellschaft: München; Registergericht München, HRB 106955

^ permalink raw reply	[flat|nested] 9+ messages in thread

end of thread, other threads:[~2023-01-13 13:59 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-09-25 13:13 [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes Chung-Lin Tang
2018-10-05 14:07 ` Tom de Vries
2018-12-06 20:57 ` Thomas Schwinge
2018-12-10 10:02   ` Chung-Lin Tang
2018-12-11 13:50     ` [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes (revised, v2) Chung-Lin Tang
2018-12-18 15:07       ` [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes (revised, v3) Chung-Lin Tang
2023-01-12 13:51 ` nvptx: Avoid deadlock in 'cuStreamAddCallback' callback, error case (was: [PATCH 6/6, OpenACC, libgomp] Async re-work, nvptx changes) Thomas Schwinge
2023-01-13 13:17   ` Chung-Lin Tang
2023-01-13 13:59     ` Thomas Schwinge

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for read-only IMAP folder(s) and NNTP newsgroup(s).