[DTrace-devel] [PATCH v2 2/2] proc: rip out waitfd() and hit waitpidding thread with a signal instead

Nick Alcock nick.alcock at oracle.com
Fri Oct 20 17:38:43 UTC 2023


For a long time we have used waitfd() to allow our process-monitoring
threads to wait for messages from DTrace proper (via a pipe) and state
changes from the monitored process (via Pwait(), which calls waitpid())
without having to engage in CPU-chewing polling loops and without inducing
latency in the monitored process or in responses to DTrace. (Both matter:
the proxying mechanism is quite fine-grained. The main DTrace thread can
mess around in libproc as much as it likes, with individual Ptrace() calls
being automatically proxied to the monitoring thread down the pipe. So
single operations from the perspective of DTrace's main thread can involve
dozens of proxy calls. Excessive latency would be bad.)

But waitfd() is ugly enough that it's never going to go upstream: not only
was it already rejected in 2010, but it needs to add really ugly hacks to
the waitqueue mechanism and even disable some assetions because it bloats
some core polling data structures beyond a cacheline (theoretically slowing
them down, though I've never been able to measure anything).

So it would be very nice to get rid of it.

The solution is an old Unix horror: EINTR. Everyone hates it, everyone wraps
long-running syscalls in EINTR loops to evade it... and here it does just
what we want. We find an unused realtime signal (the only WIP: actually hunt
one down rather than just stamping on one) and unmask it in the monitoring
thread, then drop the whole waitfd thing entirely and typically wait in
waitpid() (only waiting blocked on poll() on the proxy pipe if we're
explicitly not listening to the process right now). When we send a proxy
message to a monitoring thread, we hit it with this signal, which causes
waitpid() to exit with EINTR -- and once it does that we can check the proxy
pipe and process any messages, with no polling loops or added latencies.

We use SIGRTMIN as the signal by default: if the caller is one of those few
that actually uses realtime signals for something, we provide a new
dtrace_set_internal_signal API function that the caller can invoke before
calling dtrace_open to reset the signal to some other value that the caller
is not using (specified as a number to be added to SIGRTMIN, since we *do*
require that it's a realtime signal).

This requires an additional hook, analogous to the existing ptrace_lock_hook
(which is used to take out the dpr_lock around Ptrace() calls, so that no
other Ptrace() calls can happen for that process at the same time). This
one, the waitpid_lock_look, is used to *drop* the lock around the call to
waitpid(), because the waitpid() call may now take a long time, and the
proxy calling mechanism has to take out the dpr_lock (because it protects
the dpr_msg_cv condvar that mediates the proxy call). But we can't drop the
dpr_lock around the call to Pwait() as a whole because that call also
invokes all the breakpoint handlers, and *that* work requires the dpr_lock
to have been already taken out by Pwait()s caller. So we need another
hook. Thankfully the two hooks are never nested!

We do have one unfixable race.  We check to see if a proxy message has
arrived immediately before starting waitpid() (which requires some annoying
prototype changes to Pwait() to pass the relevant variable down), but
there is a race here:

		if (return_early && *return_early > 0) {
			if (block && waitpid_lock_hook)
				waitpid_lock_hook(P, P->wrap_arg, 0);
			return 0;
		}
-->
                err = waitpid(P->pid, &status, __WALL | (!block ? WNOHANG : 0));

At the indicated point, if a proxy signal hits, waitpid() will not return
EINTR because it hasn't started yet, but the return_early variable changing
its state (via the signal handler) will not be spotted either because it's
too late. We cannot fix this without having a variant of waitpid() that
unmasks signals after starting to listen for -EINTR, and such a thing does
not (yet) exist. (This sort of race is why ppoll() exists, with its
atomically applied signal mask, but of course the whole point of this is
that we're using waitpid() instead of poll(), and there is no pwaitpid()
analogous to ppoll(). Yet. It seems like something that we could ask about
upstream...)

Signed-off-by: Nick Alcock <nick.alcock at oracle.com>
---

Notes:
    v2: add dtrace_set_internal_signal and use it to pick the signal rather than
        just picking a value out of the air

 include/arm64/platform.h               |  10 +-
 include/i386/platform.h                |  10 +-
 include/port.h                         |   4 -
 libdtrace/dt_impl.h                    |   7 +-
 libdtrace/dt_open.c                    |   2 +
 libdtrace/dt_proc.c                    | 228 +++++++++++++++++--------
 libdtrace/dt_proc.h                    |   3 +-
 libdtrace/dtrace.h                     |  10 ++
 libdtrace/libdtrace.ver                |   1 +
 libport/Build                          |   6 +-
 libport/arm64/waitfd.c                 |  18 --
 libport/i386/waitfd.c                  |  18 --
 libport/sparc/waitfd.c                 |  69 --------
 libproc/Pcontrol.c                     |  79 ++++++---
 libproc/Pcontrol.h                     |   4 +-
 libproc/libproc.h                      |  20 ++-
 libproc/rtld_db.c                      |  20 +--
 libproc/wrap.c                         |  11 +-
 test/triggers/libproc-consistency.c    |   4 +-
 test/triggers/libproc-execing-bkpts.c  |   4 +-
 test/triggers/libproc-lookup-by-name.c |   6 +-
 21 files changed, 280 insertions(+), 254 deletions(-)
 delete mode 100644 libport/arm64/waitfd.c
 delete mode 100644 libport/i386/waitfd.c
 delete mode 100644 libport/sparc/waitfd.c

diff --git a/include/arm64/platform.h b/include/arm64/platform.h
index 5e3160ccc1366..fb173a9c61428 100644
--- a/include/arm64/platform.h
+++ b/include/arm64/platform.h
@@ -1,6 +1,6 @@
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2018, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2018, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -26,13 +26,5 @@ const static unsigned char plat_bkpt[] = { 0x00, 0x00, 0x20, 0xd4 };
  */
 #undef NEED_SOFTWARE_SINGLESTEP
 
-/*
- * Translates waitpid() into a pollable fd.
- */
-
-#ifndef __NR_waitfd
-#define __NR_waitfd 473
-#endif
-
 #endif
 
diff --git a/include/i386/platform.h b/include/i386/platform.h
index cdff3c9f70f17..7e6caea01a497 100644
--- a/include/i386/platform.h
+++ b/include/i386/platform.h
@@ -1,6 +1,6 @@
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2013, 2015, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -27,13 +27,5 @@ const static unsigned char plat_bkpt[] = { 0xcc };
  */
 #undef NEED_SOFTWARE_SINGLESTEP
 
-/*
- * Translates waitpid() into a pollable fd.
- */
-
-#ifndef __NR_waitfd
-#define __NR_waitfd 473
-#endif
-
 #endif
 
diff --git a/include/port.h b/include/port.h
index 75d54fdf431d6..9df5f8dfe8be1 100644
--- a/include/port.h
+++ b/include/port.h
@@ -42,10 +42,6 @@ unsigned long linux_version_code(void);
 #define elf_getshdrnum elf_getshnum
 #endif
 
-#ifndef HAVE_WAITFD
-int waitfd(int which, pid_t upid, int options, int flags);
-#endif
-
 #ifndef HAVE_CLOSE_RANGE
 int close_range(unsigned int first, unsigned int last, unsigned int flags);
 #endif
diff --git a/libdtrace/dt_impl.h b/libdtrace/dt_impl.h
index 05d8ef494737c..92d3b1437ed71 100644
--- a/libdtrace/dt_impl.h
+++ b/libdtrace/dt_impl.h
@@ -19,6 +19,7 @@
 #include <sys/utsname.h>
 #include <sys/compiler.h>
 #include <math.h>
+#include <signal.h>
 #include <string.h>
 #include <stddef.h>
 #include <bpf_asm.h>
@@ -357,8 +358,10 @@ struct dtrace_hdl {
 	dt_htab_t *dt_provs;	/* hash table of dt_provider_t's */
 	const struct dt_provider *dt_prov_pid; /* PID provider */
 	const struct dt_provider *dt_prov_usdt; /* USDT provider */
-	dt_proc_hash_t *dt_procs; /* hash table of grabbed process handles */
-	dt_intdesc_t dt_ints[6]; /* cached integer type descriptions */
+	int dt_proc_signal;	/* signal used to interrupt monitoring threads */
+        struct sigaction dt_proc_oact;
+        dt_proc_hash_t *dt_procs; /* hash table of grabbed process handles */
+        dt_intdesc_t dt_ints[6]; /* cached integer type descriptions */
 	ctf_id_t dt_type_func;	/* cached CTF identifier for function type */
 	ctf_id_t dt_type_fptr;	/* cached CTF identifier for function pointer */
 	ctf_id_t dt_type_str;	/* cached CTF identifier for string type */
diff --git a/libdtrace/dt_open.c b/libdtrace/dt_open.c
index 2aad974af3a5b..eb7892ee7b56d 100644
--- a/libdtrace/dt_open.c
+++ b/libdtrace/dt_open.c
@@ -732,6 +732,7 @@ dt_vopen(int version, int flags, int *errp,
 	dtp->dt_stdout_fd = -1;
 	dtp->dt_poll_fd = -1;
 	dt_proc_hash_create(dtp);
+	dt_proc_signal_init(dtp);
 	dtp->dt_proc_fd = eventfd(0, EFD_CLOEXEC | EFD_NONBLOCK);
 	dtp->dt_nextepid = 1;
 	dtp->dt_maxprobe = 0;
@@ -1215,6 +1216,7 @@ dtrace_close(dtrace_hdl_t *dtp)
 
 	if (dtp->dt_procs != NULL)
 		dt_proc_hash_destroy(dtp);
+	dt_proc_signal_fini(dtp);
 
 	while ((pgp = dt_list_next(&dtp->dt_programs)) != NULL)
 		dt_program_destroy(dtp, pgp);
diff --git a/libdtrace/dt_proc.c b/libdtrace/dt_proc.c
index 737abd78f62d0..bbd1e50483d78 100644
--- a/libdtrace/dt_proc.c
+++ b/libdtrace/dt_proc.c
@@ -105,6 +105,8 @@ static int dt_proc_loop(dt_proc_t *dpr, int awaiting_continue);
 static void dt_main_fail_rendezvous(dt_proc_t *dpr);
 static void dt_proc_ptrace_lock(struct ps_prochandle *P, void *arg,
     int ptracing);
+static void dt_proc_waitpid_lock(struct ps_prochandle *P, void *arg,
+    int waitpidding);
 static long dt_proc_continue(dtrace_hdl_t *dtp, dt_proc_t *dpr);
 
 /*
@@ -116,6 +118,12 @@ static long dt_proc_continue(dtrace_hdl_t *dtp, dt_proc_t *dpr);
 		assert(pthread_equal(dpr->dpr_lock_holder, pthread_self())); \
 	} while (0)
 
+/*
+ * The default internal signal value.
+ */
+static int internal_proc_signal = 0;
+static int proc_initialized;
+
 /*
  * Unwinder pad for libproc setjmp() chains.
  */
@@ -603,6 +611,7 @@ proxy_call(dt_proc_t *dpr, long (*proxy_rq)(), int exec_retry)
 		    "for Pwait(), deadlock is certain: %s\n", strerror(errno));
 		return -1;
 	}
+	pthread_kill(dpr->dpr_tid, dpr->dpr_hdl->dt_proc_signal);
 
 	while (dpr->dpr_proxy_rq != NULL)
 		pthread_cond_wait(&dpr->dpr_msg_cv, &dpr->dpr_lock);
@@ -618,7 +627,8 @@ proxy_call(dt_proc_t *dpr, long (*proxy_rq)(), int exec_retry)
 }
 
 static long
-proxy_pwait(struct ps_prochandle *P, void *arg, boolean_t block)
+proxy_pwait(struct ps_prochandle *P, void *arg, boolean_t block,
+    int *return_early)
 {
 	dt_proc_t *dpr = arg;
 
@@ -626,9 +636,13 @@ proxy_pwait(struct ps_prochandle *P, void *arg, boolean_t block)
 
 	/*
 	 * If we are already in the right thread, pass the call straight on.
+	 *
+	 * Otherwise, proxy it, throwing out the return_early arg because
+	 * it is only used for internal communication between the monitor
+	 * thread and Pwait() itself.
 	 */
 	if (pthread_equal(dpr->dpr_tid, pthread_self()))
-		return Pwait_internal(P, block);
+		return Pwait_internal(P, block, return_early);
 
 	dpr->dpr_proxy_args.dpr_pwait.P = P;
 	dpr->dpr_proxy_args.dpr_pwait.block = block;
@@ -732,6 +746,36 @@ proxy_quit(dt_proc_t *dpr, int err)
 	return proxy_call(dpr, proxy_quit, 0);
 }
 
+static __thread int waitpid_interrupted;
+
+static void
+waitpid_interrupting_handler(int sig)
+{
+	waitpid_interrupted = 1;
+}
+
+/*
+ * Set up and tear down the signal handler (above) used to force waitpid() to
+ * abort with -EINTR.
+ */
+void
+dt_proc_signal_init(dtrace_hdl_t *dtp)
+{
+	struct sigaction act;
+
+        memset(&act, 0, sizeof(act));
+	act.sa_handler = waitpid_interrupting_handler;
+	dtp->dt_proc_signal = SIGRTMIN + internal_proc_signal;
+	sigaction(dtp->dt_proc_signal, &act, &dtp->dt_proc_oact);
+	proc_initialized = 1;
+}
+
+void
+dt_proc_signal_fini(dtrace_hdl_t *dtp)
+{
+	sigaction(dtp->dt_proc_signal, &dtp->dt_proc_oact, NULL);
+}
+
 typedef struct dt_proc_control_data {
 	dtrace_hdl_t *dpcd_hdl;			/* DTrace handle */
 	dt_proc_t *dpcd_proc;			/* process to control */
@@ -777,9 +821,10 @@ dt_proc_control(void *arg)
 
 	/*
 	 * Set up global libproc hooks that must be active before any processes
-	 * are * grabbed or created.
+	 * are grabbed or created.
 	 */
 	Pset_ptrace_lock_hook(dt_proc_ptrace_lock);
+	Pset_waitpid_lock_hook(dt_proc_waitpid_lock);
 	Pset_libproc_unwinder_pad(dt_unwinder_pad);
 
 	/*
@@ -792,7 +837,8 @@ dt_proc_control(void *arg)
 	 * controlling thread and dt_proc_continue() or process destruction.
 	 *
 	 * It is eventually unlocked by dt_proc_control_cleanup(), and
-	 * temporarily unlocked (while waiting) by dt_proc_loop().
+	 * temporarily unlocked (while waiting) by Pwait(), called from
+	 * dt_proc_loop().
 	 */
 	dt_proc_lock(dpr);
 
@@ -874,29 +920,6 @@ dt_proc_control(void *arg)
 	Pset_pwait_wrapper(dpr->dpr_proc, proxy_pwait);
 	Pset_ptrace_wrapper(dpr->dpr_proc, proxy_ptrace);
 
-	/*
-	 * Make a waitfd to this process, and set up polling structures
-	 * appropriately.  WEXITED | WSTOPPED is what Pwait() waits for.
-	 */
-	if ((dpr->dpr_fd = waitfd(P_PID, dpr->dpr_pid, WEXITED | WSTOPPED, 0)) < 0) {
-		dt_proc_error(dtp, dpr, "failed to get waitfd() for pid %li: %s\n",
-		    (long)dpr->dpr_pid, strerror(errno));
-		/*
-		 * Demote this to a mandatorily noninvasive grab: if we
-		 * Pcreate()d it, dpr_created is still set, so it will still get
-		 * killed on dtrace exit.  If even that fails, there's nothing
-		 * we can do but hope.
-		 */
-		Prelease(dpr->dpr_proc, PS_RELEASE_NORMAL);
-		if ((dpr->dpr_proc = Pgrab(dpr->dpr_pid, 2, 0,
-			    dpr, &err)) == NULL) {
-			dt_proc_error(dtp, dpr, "failed to regrab pid %li: %s\n",
-			    (long)dpr->dpr_pid, strerror(err));
-		}
-
-		pthread_exit(NULL);
-	}
-
 	/*
 	 * Detect execve()s from loci in this thread other than proxy calls:
 	 * handle them by destroying and re-grabbing the libproc handle without
@@ -984,53 +1007,45 @@ dt_proc_control(void *arg)
 static int
 dt_proc_loop(dt_proc_t *dpr, int awaiting_continue)
 {
-	volatile struct pollfd pfd[2];
+	volatile struct pollfd pfd;
+	int timeout = 0;
+	int pwait_event_count;
 
 	assert(MUTEX_HELD(&dpr->dpr_lock));
 
 	/*
-	 * We always want to listen on the proxy pipe; we only want to listen on
-	 * the process's waitfd pipe sometimes.
+	 * Check the proxy pipe on every loop.
 	 */
 
-	pfd[0].events = POLLIN;
-	pfd[1].fd = dpr->dpr_proxy_fd[0];
-	pfd[1].events = POLLIN;
+	pfd.fd = dpr->dpr_proxy_fd[0];
+	pfd.events = POLLIN;
 
 	/*
-	 * If we're only proxying while waiting for a dt_proc_continue(),
-	 * avoid waiting on the process's fd.
+	 * If we're only proxying while waiting for a dt_proc_continue(), wait
+	 * on it indefinitely; otherwise, don't wait, because we'll be waiting
+	 * in Pwait() instead.
 	 */
 	if (awaiting_continue)
-		pfd[0].fd = dpr->dpr_fd * -1;
+		timeout = -1;
 
 	/*
-	 * Wait for the process corresponding to this control thread to stop,
-	 * process the event, and then set it running again.  We want to sleep
-	 * with dpr_lock *unheld* so that other parts of libdtrace can send
-	 * requests to us, which is protected by that lock.  It is impossible
-	 * for them, or any thread but this one, to modify the Pstate(), so we
-	 * can call that without grabbing the lock.
+	 * Check for any outstanding events, possibly sleeping to do so if we
+	 * have no process to wait for.  Process any such events, then wait in
+	 * Pwait() to handle any process events (again, unless we are
+	 * awaiting_continue).  We want to sleep with dpr_lock unheld so that
+	 * other parts of libdtrace can send requests to us, which is protected
+	 * by that lock.  It is impossible for them, or any thread but this one,
+	 * to modify the Pstate(), so we can call that without grabbing the
+	 * lock.  We also unlock it around Pwait() so that proxy requests can
+	 * initiate then.
 	 */
 	for (;;) {
 		volatile int did_proxy_pwait = 0;
 
 		dt_proc_unlock(dpr);
 
-		/*
-		 * If we should stop monitoring the process and only listen for
-		 * proxy requests, avoid waiting on its fd.
-		 */
-
-		if (!awaiting_continue) {
-			if (!dpr->dpr_monitoring)
-				pfd[0].fd = dpr->dpr_fd * -1;
-			else
-				pfd[0].fd = dpr->dpr_fd;
-		}
-
-		while (errno = EINTR,
-		    poll((struct pollfd *)pfd, 2, -1) <= 0 && errno == EINTR)
+		while (errno = 0,
+		    poll((struct pollfd *)&pfd, 1, timeout) <= 0 && errno == EINTR)
 			continue;
 
 		/*
@@ -1052,8 +1067,13 @@ dt_proc_loop(dt_proc_t *dpr, int awaiting_continue)
 		 * running breakpoint handlers and the like, which will run in
 		 * the control thread, with their effects visible in the main
 		 * thread, all serialized by dpr_lock).
+		 *
+		 * Since we are about to process any proxy requests, we can
+		 * clear the waitpid-interruption signal flag that sending a
+		 * proxy request sets.
 		 */
 		dt_proc_lock(dpr);
+		waitpid_interrupted = 0;
 
 		/*
 		 * Incoming proxy request.  Drain this byte out of the pipe, and
@@ -1063,13 +1083,13 @@ dt_proc_loop(dt_proc_t *dpr, int awaiting_continue)
 		 * case -- but if they do, it is harmless, because the
 		 * dpr_proxy_rq will be NULL in subsequent calls.)
 		 */
-		if (pfd[1].revents != 0) {
+		if (pfd.revents != 0) {
 			char junk;
 			jmp_buf this_exec_jmp, *old_exec_jmp;
 			volatile int did_exec_retry = 0;
 
 			read(dpr->dpr_proxy_fd[0], &junk, 1);
-			pfd[1].revents = 0;
+			pfd.revents = 0;
 
 			/*
 			 * execve() detected during a proxy request: notify the
@@ -1086,7 +1106,11 @@ dt_proc_loop(dt_proc_t *dpr, int awaiting_continue)
 				unwinder_pad = &this_exec_jmp;
 
 				/*
-				 * Pwait() from another thread.
+				 * Pwait() from another thread.  Only one proxy
+				 * request can be active at once, so thank
+				 * goodness we don't need to worry about the
+				 * possibility of another proxy request coming
+				 * in while we're handling this one.
 				 */
 				if (dpr->dpr_proxy_rq == proxy_pwait) {
 					dt_dprintf("%d: Handling a proxy Pwait(%i)\n",
@@ -1095,7 +1119,8 @@ dt_proc_loop(dt_proc_t *dpr, int awaiting_continue)
 					errno = 0;
 					dpr->dpr_proxy_ret = proxy_pwait
 					    (dpr->dpr_proxy_args.dpr_pwait.P, dpr,
-						dpr->dpr_proxy_args.dpr_pwait.block);
+					         dpr->dpr_proxy_args.dpr_pwait.block,
+						 NULL);
 
 					did_proxy_pwait = 1;
 				/*
@@ -1176,19 +1201,35 @@ dt_proc_loop(dt_proc_t *dpr, int awaiting_continue)
 			unwinder_pad = old_exec_jmp;
 		}
 
-		/*
-		 * The process needs attention. Pwait() for it (which will make
-		 * the waitfd transition back to empty).
-		 */
-		if (pfd[0].revents != 0) {
-			dt_dprintf("%d: Handling a process state change\n",
-			    dpr->dpr_pid);
-			pfd[0].revents = 0;
-			Pwait(dpr->dpr_proc, B_FALSE);
+		if (awaiting_continue)
+			continue;
 
+                /*
+		 * Pwait() for the process, listening for process state
+		 * transitions, handling breakpoints and other problems,
+		 * possibly detecting exec() and longjmping back out, etc.
+		 *
+		 * If a proxy request comes in, Pwait() returns 0. Proxy
+		 * requests cannot come in while the lock is held, so we can be
+		 * sure that the waitpid_interrupted flag is still unset at this
+		 * point.
+		 *
+		 * We do not unlock the dpr_lock at this stage because
+		 * breakpoint invocations, proxied ptraces and the like can all
+		 * require the lock to be held.  Instead, the waitpid_lock_hook
+		 * unblocks it around the call to waitpid itself.
+		 */
+
+		dt_dprintf("%d: Waiting for process state changes\n",
+			   dpr->dpr_pid);
+
+                assert(waitpid_interrupted == 0);
+		assert(MUTEX_HELD(&dpr->dpr_lock));
+		pwait_event_count = Pwait(dpr->dpr_proc, B_TRUE, &waitpid_interrupted);
+
+		if (pwait_event_count > 0) {
 			switch (Pstate(dpr->dpr_proc)) {
 			case PS_STOP:
-
 				/*
 				 * If the process stops showing one of the
 				 * events that we are tracing, perform the
@@ -1301,8 +1342,6 @@ dt_proc_control_cleanup(void *arg)
 	 */
 
 	dpr->dpr_done = B_TRUE;
-	if (dpr->dpr_fd)
-	    close(dpr->dpr_fd);
 
 	if (dpr->dpr_proxy_fd[0])
 	    close(dpr->dpr_proxy_fd[0]);
@@ -1604,6 +1643,7 @@ dt_proc_create_thread(dtrace_hdl_t *dtp, dt_proc_t *dpr, uint_t stop,
 
 	sigfillset(&nset);
 	sigdelset(&nset, SIGABRT);	/* unblocked for assert() */
+	sigdelset(&nset, dtp->dt_proc_signal);	/* unblocked for waitpid */
 
 	data.dpcd_hdl = dtp;
 	data.dpcd_proc = dpr;
@@ -1938,6 +1978,7 @@ dt_proc_continue(dtrace_hdl_t *dtp, dt_proc_t *dpr)
 		dpr->dpr_proxy_rq = dt_proc_continue;
 		errno = 0;
 		while (write(dpr->dpr_proxy_fd[1], &junk, 1) < 0 && errno == EINTR);
+		pthread_kill(dpr->dpr_tid, dtp->dt_proc_signal);
 		if (errno != 0 && errno != EINTR) {
 			dt_proc_error(dpr->dpr_hdl, dpr, "Cannot write to "
 			    "proxy pipe for dt_proc_continue(), deadlock is "
@@ -2016,6 +2057,10 @@ dt_proc_unlock(dt_proc_t *dpr)
 		assert(MUTEX_HELD(&dpr->dpr_lock));
 }
 
+/*
+ * Take the lock around Ptrace() calls, to prevent other threads issuing
+ * Ptrace()s while we are working.
+ */
 static void
 dt_proc_ptrace_lock(struct ps_prochandle *P, void *arg, int ptracing)
 {
@@ -2027,6 +2072,24 @@ dt_proc_ptrace_lock(struct ps_prochandle *P, void *arg, int ptracing)
 		dt_proc_unlock(dpr);
 }
 
+/*
+ * Release the lock around blocking waitpid() calls, so that proxy requests can
+ * come in.  Proxy requests take the lock before hitting the process control
+ * thread with a signal to wake it up: the lock is taken by the caller of the
+ * various dt_Pfunction()s below, while proxy_monitor() invokes proxy_call()
+ * which does the signalling.
+ */
+static void
+dt_proc_waitpid_lock(struct ps_prochandle *P, void *arg, int waitpidding)
+{
+	dt_proc_t *dpr = arg;
+
+	if (waitpidding)
+		dt_proc_unlock(dpr);
+	else
+		dt_proc_lock(dpr);
+}
+
 /*
  * Define the public interface to a libproc function from the rest of DTrace,
  * automatically proxying via the process-control thread and retrying on
@@ -2311,3 +2374,24 @@ dtrace_proc_continue(dtrace_hdl_t *dtp, struct dtrace_proc *proc)
 	if (dpr != NULL)
 		dt_proc_continue(dtp, dpr);
 }
+
+/*
+ * Set the internal signal number used to prod monitoring threads to wake up.
+ */
+int
+dtrace_set_internal_signal(unsigned int sig)
+{
+	if (proc_initialized) {
+		dt_dprintf("Cannot change internal signal after DTrace is initialized.\n");
+		return -1;
+	}
+
+        if (SIGRTMIN + sig > SIGRTMAX) {
+		dt_dprintf("Internal signal %i+%i is greater than the maximum allowed, %i.\n",
+			   SIGRTMIN, sig, SIGRTMAX);
+		return -1;
+	}
+
+	internal_proc_signal = sig;
+	return 0;
+}
diff --git a/libdtrace/dt_proc.h b/libdtrace/dt_proc.h
index a08922bc68483..90ab6c9c8d4c1 100644
--- a/libdtrace/dt_proc.h
+++ b/libdtrace/dt_proc.h
@@ -34,7 +34,6 @@ typedef struct dt_proc {
 	pthread_cond_t dpr_msg_cv;	/* cond for msgs from main thread */
 	pthread_t dpr_tid;		/* control thread (or zero if none) */
 	pid_t dpr_pid;			/* pid of process */
-	int dpr_fd;			/* waitfd for process */
 	int dpr_proxy_fd[2];		/* proxy request pipe from main thread */
 	uint_t dpr_refs;		/* reference count */
 	uint8_t dpr_stop;		/* stop mask: see flag bits below */
@@ -169,6 +168,8 @@ extern ssize_t dt_Pread(dtrace_hdl_t *, pid_t, void *, size_t, uintptr_t);
 
 extern void dt_proc_hash_create(dtrace_hdl_t *);
 extern void dt_proc_hash_destroy(dtrace_hdl_t *);
+extern void dt_proc_signal_init(dtrace_hdl_t *);
+extern void dt_proc_signal_fini(dtrace_hdl_t *);
 
 #ifdef	__cplusplus
 }
diff --git a/libdtrace/dtrace.h b/libdtrace/dtrace.h
index 0568355c00576..8f40a5817d6cf 100644
--- a/libdtrace/dtrace.h
+++ b/libdtrace/dtrace.h
@@ -62,6 +62,16 @@ extern void dtrace_setoptenv(dtrace_hdl_t *dtp, const char *prefix);
 extern int dtrace_update(dtrace_hdl_t *dtp);
 extern int dtrace_ctlfd(dtrace_hdl_t *dtp);
 
+/*
+ * DTrace needs one internal signal for its own use.  By default it uses
+ * SIGRTMIN.  This function (which must be called before dtrace_open(),
+ * and applies to all dtrace handles) lets the consumer pick a different
+ * signal.  The number provided is added to SIGRTMIN.  If the result is
+ * greater than SIGRTMAX, this function returns -1.
+ */
+
+extern int dtrace_set_internal_signal(unsigned int sig);
+
 /*
  * DTrace Program Interface
  *
diff --git a/libdtrace/libdtrace.ver b/libdtrace/libdtrace.ver
index 3886c18e4abd8..58783af90d840 100644
--- a/libdtrace/libdtrace.ver
+++ b/libdtrace/libdtrace.ver
@@ -69,6 +69,7 @@ LIBDTRACE_1.0 {
 	dtrace_program_link;
 	dtrace_program_strcompile;
 	dtrace_provider_modules;
+	dtrace_set_internal_signal;
 	dtrace_setopt;
 	dtrace_setoptenv;
 	dtrace_stability_name;
diff --git a/libport/Build b/libport/Build
index 1b4fca0c52dd4..e043a27efa5b7 100644
--- a/libport/Build
+++ b/libport/Build
@@ -1,5 +1,5 @@
 # Oracle Linux DTrace.
-# Copyright (c) 2011, 2022, Oracle and/or its affiliates. All rights reserved.
+# Copyright (c) 2011, 2023, Oracle and/or its affiliates. All rights reserved.
 # Licensed under the Universal Permissive License v 1.0 as shown at
 # http://oss.oracle.com/licenses/upl.
 
@@ -9,9 +9,9 @@ LIBS += libport
 libport_TARGET = libport
 libport_DIR := $(current-dir)
 ifdef HAVE_CLOSE_RANGE
-libport_SOURCES = gmatch.c linux_version_code.c strlcat.c strlcpy.c p_online.c time.c daemonize.c $(ARCHINC)/waitfd.c
+libport_SOURCES = gmatch.c linux_version_code.c strlcat.c strlcpy.c p_online.c time.c daemonize.c
 else
-libport_SOURCES = gmatch.c linux_version_code.c strlcat.c strlcpy.c p_online.c time.c daemonize.c close_range.c $(ARCHINC)/waitfd.c
+libport_SOURCES = gmatch.c linux_version_code.c strlcat.c strlcpy.c p_online.c time.c daemonize.c close_range.c
 endif
 libport_LIBSOURCES := libport
 libport_CPPFLAGS := -Ilibdtrace
diff --git a/libport/arm64/waitfd.c b/libport/arm64/waitfd.c
deleted file mode 100644
index 944fb66946dd4..0000000000000
--- a/libport/arm64/waitfd.c
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * Licensed under the Universal Permissive License v 1.0 as shown at
- * http://oss.oracle.com/licenses/upl.
- */
-
-#include <config.h>				/* for HAVE_* */
-
-#ifndef HAVE_WAITFD
-#include <unistd.h>				/* for syscall() */
-#include <platform.h>
-
-int
-waitfd(int which, pid_t upid, int options, int flags)
-{
-        return syscall(__NR_waitfd, which, upid, options, flags);
-}
-
-#endif
diff --git a/libport/i386/waitfd.c b/libport/i386/waitfd.c
deleted file mode 100644
index 944fb66946dd4..0000000000000
--- a/libport/i386/waitfd.c
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * Licensed under the Universal Permissive License v 1.0 as shown at
- * http://oss.oracle.com/licenses/upl.
- */
-
-#include <config.h>				/* for HAVE_* */
-
-#ifndef HAVE_WAITFD
-#include <unistd.h>				/* for syscall() */
-#include <platform.h>
-
-int
-waitfd(int which, pid_t upid, int options, int flags)
-{
-        return syscall(__NR_waitfd, which, upid, options, flags);
-}
-
-#endif
diff --git a/libport/sparc/waitfd.c b/libport/sparc/waitfd.c
deleted file mode 100644
index 1358ec1b2b8e4..0000000000000
--- a/libport/sparc/waitfd.c
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Oracle Linux DTrace.
- * Copyright (c) 2011, 2018, Oracle and/or its affiliates. All rights reserved.
- * Licensed under the Universal Permissive License v 1.0 as shown at
- * http://oss.oracle.com/licenses/upl.
- */
-
-#include <config.h>				/* for HAVE_* */
-
-#ifndef HAVE_WAITFD
-#include <errno.h>
-#include <unistd.h>				/* for syscall() */
-#include <linux/version.h>                      /* for KERNEL_VERSION() */
-#include <port.h>                               /* for linux_version_code() */
-#include <dt_debug.h>
-
-/*
- * Translates waitpid() into a pollable fd.
- *
- * The syscall number varies between kernel releases.
- * The version code in this table is the kernel version in which a particular
- * value was introduced (i.e. a lower bound).  Kernels with major/minor numbers
- * not in this list are considered unknown, and we return -ENOSYS.  A syscall
- * number of zero terminates the list.
- */
-
-static struct waitfds_tag {
-        unsigned long linux_version_code;
-        long waitfd;
-} waitfds[] = { { KERNEL_VERSION(4,19,0), 362 },
-		{ KERNEL_VERSION(4,14,0), 361 },
-		{ KERNEL_VERSION(4,13,0), 361 },
-		{ KERNEL_VERSION(4,12,0), 361 },
-		{ KERNEL_VERSION(4,11,0), 361 },
-		{ KERNEL_VERSION(4,10,0), 360 },
-		{ KERNEL_VERSION(4,9,0), 360 },
-		{ KERNEL_VERSION(4,8,0), 360 },
-		{ KERNEL_VERSION(4,6,0), 360 },
-		{ KERNEL_VERSION(4,5,0), 358 },
-		{ KERNEL_VERSION(4,1,4), 351 },
-		{ 0, 0 } };
-
-static long waitfd_nr;
-
-int
-waitfd(int which, pid_t upid, int options, int flags)
-{
-        if (!waitfd_nr) {
-                struct waitfds_tag *walk;
-                unsigned long version = linux_version_code();
-
-                for (walk = waitfds; walk->waitfd; walk++) {
-                        if ((version >= walk->linux_version_code) &&
-                            ((version >> 8) == (walk->linux_version_code >> 8))) {
-                                waitfd_nr = walk->waitfd;
-                                break;
-                        }
-                }
-		if (!waitfd_nr) {
-			dt_dprintf("waitfd() syscall number for this kernel "
-			    "not known.\n");
-			return -ENOSYS;
-		}
-        }
-
-        return syscall(waitfd_nr, which, upid, options, flags);
-}
-
-#endif
diff --git a/libproc/Pcontrol.c b/libproc/Pcontrol.c
index c9a4847cdb949..371b547c5e07b 100644
--- a/libproc/Pcontrol.c
+++ b/libproc/Pcontrol.c
@@ -1,6 +1,6 @@
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2010, 2022, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2010, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -69,6 +69,7 @@ static void delete_bkpt_handler(struct bkpt *bkpt);
 static jmp_buf **single_thread_unwinder_pad(struct ps_prochandle *unused);
 
 static ptrace_lock_hook_fun *ptrace_lock_hook;
+static waitpid_lock_hook_fun *waitpid_lock_hook;
 libproc_unwinder_pad_fun *libproc_unwinder_pad = single_thread_unwinder_pad;
 
 #define LIBPROC_PTRACE_OPTIONS PTRACE_O_TRACEEXEC | \
@@ -623,13 +624,20 @@ unlock_exit:
  * as are necessary to drain the queue of requests and leave the child in a
  * state capable of handling more ptrace() requests -- or dead.)
  *
- * Returns the number of state changes processed, or -1 on error.
+ * The return_early flag is checked right before we wait; if nonzero, an
+ * immediate return is carried out.  (This should almost close the race where
+ * the thread is interrupted by being hit by a signal before the waitpid()
+ * starts.  In the absence of a waitpid_sigunmask() I don't think we can close
+ * it completely...)
+ *
+ * Returns the number of state changes processed, or -1 on error.  0 can be
+ * returned if this thread was hit with a signal.
  *
  * The debugging strings starting "process status change" are relied upon by the
  * libproc/tst.signals.sh test.
  */
 long
-Pwait_internal(struct ps_prochandle *P, boolean_t block)
+Pwait_internal(struct ps_prochandle *P, boolean_t block, int *return_early)
 {
 	long err;
 	long num_waits = 0;
@@ -672,27 +680,45 @@ Pwait_internal(struct ps_prochandle *P, boolean_t block)
 	if (P->state == PS_DEAD)
 		return 0;
 
-	do
-	{
+	do {
 		errno = 0;
-		err = waitpid(P->pid, &status, __WALL | (!block ? WNOHANG : 0));
 
-		switch (err) {
+		if (block && waitpid_lock_hook)
+			waitpid_lock_hook(P, P->wrap_arg, 1);
+
+		/*
+		 * Return at once if so requested.  (We lock and then possibly
+		 * unlock again to minimize the size of the race window in which
+		 * the signal might hit before waitpid() starts.)
+		 */
+		if (return_early && *return_early > 0) {
+			if (block && waitpid_lock_hook)
+				waitpid_lock_hook(P, P->wrap_arg, 0);
+			return 0;
+		}
+
+                err = waitpid(P->pid, &status, __WALL | (!block ? WNOHANG : 0));
+	
+		if (block && waitpid_lock_hook)
+			waitpid_lock_hook(P, P->wrap_arg, 0);
+
+                switch (err) {
 		case 0:
 			return 0;
 		case -1:
+			if (block && errno == EINTR)
+				return 0;
+
 			if (errno == ECHILD) {
 				P->state = PS_DEAD;
 				return 0;
 			}
 
-			if (errno != EINTR) {
-				_dprintf("Pwait: error waiting: %s\n",
-				    strerror(errno));
-				return -1;
-			}
+			_dprintf("Pwait: error waiting: %s\n",
+				 strerror(errno));
+			return -1;
 		}
-	} while (errno == EINTR);
+	} while (block && errno == EINTR);
 
 	if (Pwait_handle_waitpid(P, status) < 0)
 		return -1;
@@ -701,7 +727,7 @@ Pwait_internal(struct ps_prochandle *P, boolean_t block)
 	 * Now repeatedly loop, processing more waits until none remain.
 	 */
 	do {
-		one_wait = Pwait(P, 0);
+		one_wait = Pwait(P, 0, NULL);
 		num_waits += one_wait;
 	} while (one_wait > 0);
 
@@ -1307,7 +1333,7 @@ Ptrace(struct ps_prochandle *P, int stopped)
 		 * that event clears the listening state and makes it possible
 		 * for other ptrace requests to succeed.
 		 */
-		Pwait(P, 0);
+		Pwait(P, 0, NULL);
 		state->state = P->state;
 		if ((!stopped) || (P->state == PS_TRACESTOP))
 			return 0;
@@ -1325,7 +1351,7 @@ Ptrace(struct ps_prochandle *P, int stopped)
 		while (P->pending_stops &&
 		    ((P->state == PS_RUN) ||
 			(listen_interrupt && P->listening)))
-			Pwait(P, 1);
+			Pwait(P, 1, NULL);
 		P->awaiting_pending_stops--;
 
 		return 0;
@@ -1358,7 +1384,7 @@ Ptrace(struct ps_prochandle *P, int stopped)
 		P->pending_stops++;
 		P->awaiting_pending_stops++;
 		while (P->pending_stops && P->state == PS_RUN) {
-			if (Pwait(P, 1) == -1)
+			if (Pwait(P, 1, NULL) == -1)
 				goto err;
 		}
 		P->awaiting_pending_stops--;
@@ -1469,7 +1495,7 @@ Puntrace(struct ps_prochandle *P, int leave_stopped)
 			if (!Pbkpt_continue(P))
 				P->state = PS_RUN;
 			P->ptrace_halted = FALSE;
-			Pwait(P, 0);
+			Pwait(P, 0, NULL);
 		}
 	} else {
 		_dprintf("%i: Detaching.\n", P->pid);
@@ -1756,7 +1782,7 @@ Punbkpt(struct ps_prochandle *P, uintptr_t addr)
 		return;
 	}
 
-	Pwait(P, 0);
+	Pwait(P, 0, NULL);
 	bkpt = bkpt_by_addr(P, addr, TRUE);
 
 	P->num_bkpts--;
@@ -1933,7 +1959,7 @@ bkpt_flush(struct ps_prochandle *P, pid_t pid, int gone) {
 		Puntrace(P, state);
 
 		if (!gone)
-			Pwait(P, 0);
+			Pwait(P, 0, NULL);
 
 		P->bkpt_consume = 0;
 		P->tracing_bkpt = 0;
@@ -2180,7 +2206,7 @@ Pbkpt_continue(struct ps_prochandle *P)
 		/*
 		 * Not stopped at all.  Just do a quick Pwait().
 		 */
-		Pwait(P, 0);
+		Pwait(P, 0, NULL);
 		return 0;
 	} else if (ip == P->tracing_bkpt)
 		/*
@@ -2193,7 +2219,7 @@ Pbkpt_continue(struct ps_prochandle *P)
 		 * a SIGTRAP.
 		 */
 		P->bkpt_consume = 1;
-		Pwait(P, 0);
+		Pwait(P, 0, NULL);
 		P->bkpt_consume = 0;
 		P->state = Pbkpt_continue_internal(P, bkpt, FALSE);
 	}
@@ -2628,6 +2654,15 @@ Pset_ptrace_lock_hook(ptrace_lock_hook_fun *hook)
 	ptrace_lock_hook = hook;
 }
 
+/*
+ * Set the waitpid() lock hook.
+ */
+void
+Pset_waitpid_lock_hook(waitpid_lock_hook_fun *hook)
+{
+	waitpid_lock_hook = hook;
+}
+
 /*
  * Return 1 if the process is invasively grabbed, and thus ptrace()able.
  */
diff --git a/libproc/Pcontrol.h b/libproc/Pcontrol.h
index 77d71d98abf05..7ba792218e374 100644
--- a/libproc/Pcontrol.h
+++ b/libproc/Pcontrol.h
@@ -1,6 +1,6 @@
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2008, 2022, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2008, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -228,7 +228,7 @@ struct ps_prochandle {
 	int	ptrace_count;	/* count of Ptrace() calls */
 	dt_list_t ptrace_states; /* states of higher Ptrace() levels */
 	int	ptrace_halted;	/* true if halted by Ptrace() call */
-	int	pending_stops;	/* number of SIGSTOPs Ptrace() has sent that
+        int	pending_stops;	/* number of SIGSTOPs Ptrace() has sent that
 				   have yet to be consumed */
 	int	awaiting_pending_stops; /* if 1, a pending stop is being waited
 					   for: all blocking Pwait()s when
diff --git a/libproc/libproc.h b/libproc/libproc.h
index 761a0a0bdfff8..9e5c25990b4fb 100644
--- a/libproc/libproc.h
+++ b/libproc/libproc.h
@@ -87,7 +87,8 @@ extern	void	Puntrace(struct ps_prochandle *, int stay_stopped);
 extern	void	Pclose(struct ps_prochandle *);
 
 extern	int	Pmemfd(struct ps_prochandle *);
-extern	long	Pwait(struct ps_prochandle *, boolean_t block);
+extern	long	Pwait(struct ps_prochandle *, boolean_t block,
+    int *return_early);
 extern	int	Pstate(struct ps_prochandle *);
 extern	ssize_t	Pread(struct ps_prochandle *, void *, size_t, uintptr_t);
 extern	ssize_t Pread_string(struct ps_prochandle *, char *, size_t, uintptr_t);
@@ -138,7 +139,8 @@ extern	void	Pset_ptrace_wrapper(struct ps_prochandle *P,
  * A program intending to call libproc functions from threads other than those
  * grabbing the process will typically need to wrap both ptrace() and Pwait().
  */
-typedef long pwait_fun(struct ps_prochandle *P, void *arg, boolean_t block);
+typedef long pwait_fun(struct ps_prochandle *P, void *arg, boolean_t block,
+    int *return_early);
 
 extern	void	Pset_pwait_wrapper(struct ps_prochandle *P, pwait_fun *wrapper);
 
@@ -147,7 +149,8 @@ extern	void	Pset_pwait_wrapper(struct ps_prochandle *P, pwait_fun *wrapper);
  * function should end up calling (somehow, from some thread or other).  Safe to
  * call only from the thread that did Pgrab() or Pcreate().
  */
-extern  long	Pwait_internal(struct ps_prochandle *P, boolean_t block);
+extern	long	Pwait_internal(struct ps_prochandle *P, boolean_t block,
+    int *return_early);
 
 /*
  * Register a function to be called around the outermost layer of Ptrace()/
@@ -166,6 +169,17 @@ typedef	void	ptrace_lock_hook_fun(struct ps_prochandle *P, void *arg,
 
 extern	void	Pset_ptrace_lock_hook(ptrace_lock_hook_fun *hook);
 
+/*
+ * Like the ptrace_lock_hook, but of inverse sign: used to possibly release
+ * locks around long-running blocking waitpid() calls inside Pwait(), while
+ * retaining the lock for the remainder of Pwait() (which may trigger
+ * breakpoints, invoke other wrapped functions etc).
+ */
+typedef	void	waitpid_lock_hook_fun(struct ps_prochandle *P, void *arg,
+    int waitpidding);
+
+extern	void	Pset_waitpid_lock_hook(waitpid_lock_hook_fun *hook);
+
 /*
  * Register a function that returns the address of a per-thread pointer-sized
  * area suitable for storing a jmp_buf, to be called on exec() to register a
diff --git a/libproc/rtld_db.c b/libproc/rtld_db.c
index a155e00841a46..314d3584a5a1b 100644
--- a/libproc/rtld_db.c
+++ b/libproc/rtld_db.c
@@ -4,7 +4,7 @@
 
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2013, 2022, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -1067,11 +1067,11 @@ rd_ldso_consistent_begin(rd_agent_t *rd)
 		 * breakpoint somewhere inside the dynamic linker, we will
 		 * return with inconsistent link maps.  Don't do that.
 		 */
-		Pwait(rd->P, FALSE);
+		Pwait(rd->P, FALSE, NULL);
 		while (!rd->ic_transitioned && (rd->P->state == PS_RUN ||
 			rd->P->group_stopped) &&
 		    rd_ldso_consistency(rd, LM_ID_BASE) != RD_CONSISTENT)
-			Pwait(rd->P, TRUE);
+			Pwait(rd->P, TRUE, NULL);
 
 		rd->stop_on_consistent = 0;
 	}
@@ -1153,7 +1153,7 @@ rd_ldso_nonzero_lmid_consistent_begin(rd_agent_t *rd)
 	 */
 	rd->stop_on_consistent = 1;
 
-	Pwait(rd->P, FALSE);
+	Pwait(rd->P, FALSE, NULL);
 
 	if (rd->P->state == PS_DEAD)
 		return -1;
@@ -1244,7 +1244,7 @@ rd_ldso_nonzero_lmid_consistent_begin(rd_agent_t *rd)
 	 */
 
 	do {
-		Pwait(rd->P, FALSE);
+		Pwait(rd->P, FALSE, NULL);
 	} while (rd->P->state == PS_TRACESTOP);
 
 	timeout_nsec = 1000000;
@@ -1265,7 +1265,7 @@ rd_ldso_nonzero_lmid_consistent_begin(rd_agent_t *rd)
 			return -1;
 		}
 
-		Pwait(rd->P, FALSE);
+		Pwait(rd->P, FALSE, NULL);
 		sane_nanosleep(timeout_nsec);
 		timeout_nsec *= 2;
 	}
@@ -1526,7 +1526,7 @@ rd_new(struct ps_prochandle *P)
 		return NULL;
 	}
 
-	Pwait(P, 0);
+	Pwait(P, 0, NULL);
 
 	rd = calloc(sizeof(struct rd_agent), 1);
 	if (rd == NULL)
@@ -1768,7 +1768,7 @@ rd_loadobj_iter(rd_agent_t *rd, rl_iter_f *fun, void *state)
 		goto spotted_exec;
 	*jmp_pad = &this_exec_jmp;
 
-	Pwait(rd->P, 0);
+	Pwait(rd->P, 0, NULL);
 
 	if (rd->P->state == PS_DEAD) {
 		*jmp_pad = old_exec_jmp;
@@ -1817,7 +1817,7 @@ rd_loadobj_iter(rd_agent_t *rd, rl_iter_f *fun, void *state)
 			    nloaded, lmid);
 		}
 
-		Pwait(rd->P, FALSE);
+		Pwait(rd->P, FALSE, NULL);
 
 		/*
 		 * Read this link map out of the child.  If link map zero cannot
@@ -1945,7 +1945,7 @@ err:
 	 * iteration.  Pwait() to pick that up.
 	 */
 	old_r_brk = r_brk(rd);
-	Pwait(rd->P, FALSE);
+	Pwait(rd->P, FALSE, NULL);
 
 	jmp_pad = libproc_unwinder_pad(rd->P);
 	*jmp_pad = old_exec_jmp;
diff --git a/libproc/wrap.c b/libproc/wrap.c
index c822e8da0175f..a7bd96a48404b 100644
--- a/libproc/wrap.c
+++ b/libproc/wrap.c
@@ -4,7 +4,7 @@
 
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2013, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -54,18 +54,19 @@ wrapped_ptrace(struct ps_prochandle *P, enum __ptrace_request request, pid_t pid
  * Default (degenerate) Pwait() wrapper.
  */
 static long
-default_pwait_wrapper(struct ps_prochandle *P, void *arg, boolean_t block)
+default_pwait_wrapper(struct ps_prochandle *P, void *arg, boolean_t block,
+    int *return_early)
 {
-    return Pwait_internal(P, block);
+	return Pwait_internal(P, block, return_early);
 }
 
 /*
  * Call Pwait_internal() using the wrapper.
  */
 long
-Pwait(struct ps_prochandle *P, boolean_t block)
+Pwait(struct ps_prochandle *P, boolean_t block, int *return_early)
 {
-	return P->pwait_wrap(P, P->wrap_arg, block);
+	return P->pwait_wrap(P, P->wrap_arg, block, return_early);
 }
 
 /*
diff --git a/test/triggers/libproc-consistency.c b/test/triggers/libproc-consistency.c
index 8027f140769c6..2450e3a863644 100644
--- a/test/triggers/libproc-consistency.c
+++ b/test/triggers/libproc-consistency.c
@@ -1,6 +1,6 @@
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2013, 2020, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -177,7 +177,7 @@ int main(int argc, char *argv[])
 			    "long: %li seconds.\n", (long)(b.tv_sec - a.tv_sec));
 			err = 1;
 		}
-		Pwait(P, 0);
+		Pwait(P, 0, NULL);
 	}
 
 	Prelease(P, PS_RELEASE_KILL);
diff --git a/test/triggers/libproc-execing-bkpts.c b/test/triggers/libproc-execing-bkpts.c
index 5af748b202ecd..507d777391f61 100644
--- a/test/triggers/libproc-execing-bkpts.c
+++ b/test/triggers/libproc-execing-bkpts.c
@@ -1,6 +1,6 @@
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2013, 2020, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -96,7 +96,7 @@ main(int argc, char *argv[])
 	P_preserved = P;
 
 	while (Pstate(P) != PS_DEAD) {
-		Pwait(P, 0);
+		Pwait(P, 0, NULL);
 
 		/*
 		 * Look up the name.
diff --git a/test/triggers/libproc-lookup-by-name.c b/test/triggers/libproc-lookup-by-name.c
index 31ff665ff5c5e..d6dc0333fab3e 100644
--- a/test/triggers/libproc-lookup-by-name.c
+++ b/test/triggers/libproc-lookup-by-name.c
@@ -1,6 +1,6 @@
 /*
  * Oracle Linux DTrace.
- * Copyright (c) 2013, 2020, Oracle and/or its affiliates. All rights reserved.
+ * Copyright (c) 2013, 2023, Oracle and/or its affiliates. All rights reserved.
  * Licensed under the Universal Permissive License v 1.0 as shown at
  * http://oss.oracle.com/licenses/upl.
  */
@@ -57,7 +57,7 @@ main(int argc, char *argv[])
 	 * Wait until halted and waiting for a SIGCONT.
 	 */
 	while (Pstate(P) == PS_RUN)
-		Pwait(P, 1);
+		Pwait(P, 1, NULL);
 
 	/*
 	 * Look up the name.
@@ -82,7 +82,7 @@ main(int argc, char *argv[])
 
 	kill(Pgetpid(P), SIGCONT);
 	do
-		Pwait(P, 1);
+		Pwait(P, 1, NULL);
 	while (Pstate(P) == PS_RUN);
 
 	Prelease(P, PS_RELEASE_KILL);
-- 
2.42.0.271.g85384428f1




More information about the DTrace-devel mailing list