From feb641cc6e69ae21e3c804979b3335f1b4c6d6cc Mon Sep 17 00:00:00 2001
From: Joel Jacobson <joel@compiler.org>
Date: Sat, 27 Dec 2025 08:07:03 +0100
Subject: [PATCH 2/2] Optimize LISTEN/NOTIFY with shared channel map and direct
 advancement

This patch reworks the LISTEN/NOTIFY signaling path to avoid the
long-standing inefficiency where every commit wakes all listening
backends in the same database, even those that are listening on
completely different channels.

Problem
-------

At present, SignalBackends has no central knowledge of which backend
listens on which channel. When a backend commits a transaction that
issued NOTIFY, it simply iterates over all registered listeners in the
same database and sends each one a PROCSIG_NOTIFY_INTERRUPT signal.

That behavior is fine when all listeners are on the same channel, but
when many backends are listening on different channels, each NOTIFY
triggers a storm of unnecessary wakeups and context switches. As the
number of idle listeners grows, this often becomes the bottleneck and
throughput drops sharply.

Overview of the solution
------------------------

This patch introduces a lazily-created dynamic shared hash (dshash)
backed by dynamic shared memory (DSA) that maps (dboid, channel) to
arrays of listening backends (ProcNumbers). This allows the sender to
target only those backends actually listening on the channels for which
it has queued notifications.

LISTEN state tracking
---------------------

LISTEN state is tracked at three levels:

  - pendingListenChannels: per-transaction pending changes
  - listenChannelsHash: per-backend committed state cache
  - channelHash: cluster-wide shared state

The first two are local hash tables, the third is a dshash in shared
memory. PreCommit_Notify updates all three (doing any allocations before
clog commit for OOM safety), and AtCommit_Notify finalizes the changes.

Using a hash table for pendingListenChannels provides automatic
deduplication: LISTEN foo; UNLISTEN foo; LISTEN foo collapses to one
entry storing the final state, which we just apply at commit.

For LISTEN, PreCommit_Notify pre-allocates entries in both the local
listenChannelsHash and the shared channelHash (with listening=false).
AtCommit_Notify then sets listening=true.

For UNLISTEN, PreCommit_Notify only records the intent in
pendingListenChannels. AtCommit_Notify removes the entry from
channelHash and listenChannelsHash.

On abort, entries with listening=false (staged but never committed) are
removed from channelHash and listenChannelsHash.

Signal arrays for sending notifications are also preallocated in
PreCommit_Notify to avoid allocation failures after committing to clog.

Direct advancement
------------------

A further optimization avoids signaling idle backends that are not
listening on any of the channels notified within the transaction.

While queuing notifications, PreCommit_Notify records the queue head
position both before and after writing its notifications. Because all
writers are serialized by the existing cluster-wide heavyweight lock on
"database 0", no backend (from any database) can insert entries between
those two points. This guarantees that the region [oldHead, newHead)
contains only the entries written by our commit.

SignalBackends uses this fact to directly advance any backend still
positioned at oldHead up to newHead, avoiding a needless wakeup for
listeners that would otherwise not find any notifies of interest.

To handle advancing backends correctly, each backend's entry tracks both
whether it is currently advancing (isAdvancing) and the target position
it is advancing to (advancingPos). This allows SignalBackends to signal
advancing backends only when their target position would leave them
behind the new queue head, while safely direct-advancing idle backends
that would not be interested in the newly written notifications.
Idle backends that are stationary at a position before the old queue
head are signaled, since they might be interested in the notifications
in between their current position and the old queue head.

Other notes
-----------

The listenChannelsHash provides fast lock-free lookups during
notification processing, avoiding contention on the shared hash during
the high-frequency IsListeningOn checks that occur for every
notification read from the queue.

This patch adds LWLock tranche NOTIFY_CHANNEL_HASH and wait event
NotifyChannelHash for visibility.

There are no user-visible behavioral changes; this is an internal
optimization only.
---
 src/backend/commands/async.c                  | 1055 ++++++++++++++---
 .../utils/activity/wait_event_names.txt       |    1 +
 src/include/storage/lwlocklist.h              |    1 +
 src/tools/pgindent/typedefs.list              |    3 +
 4 files changed, 864 insertions(+), 196 deletions(-)

diff --git a/src/backend/commands/async.c b/src/backend/commands/async.c
index eb86402cae4..50fb17ad887 100644
--- a/src/backend/commands/async.c
+++ b/src/backend/commands/async.c
@@ -24,8 +24,10 @@
  *	  All notification messages are placed in the queue and later read out
  *	  by listening backends.
  *
- *	  There is no central knowledge of which backend listens on which channel;
- *	  every backend has its own list of interesting channels.
+ *	  We also maintain a dynamic shared hash table (dshash) that maps channel
+ *	  names to the set of backends listening on each channel. This table is
+ *	  created lazily on the first LISTEN command and grows dynamically as
+ *	  needed.
  *
  *	  Although there is only one queue, notifications are treated as being
  *	  database-local; this is done by including the sender's database OID
@@ -64,20 +66,33 @@
  *	  notifications, we can still call elog(ERROR, ...) and the transaction
  *	  will roll back.
  *
+ *	  PreCommit_Notify() also stages any pending LISTEN/UNLISTEN actions.
+ *	  LISTEN operations pre-allocate entries in both the per-backend
+ *	  listenChannelsHash and the shared channelHash (with listening=false).
+ *	  All allocations happen before committing to clog so failures safely abort.
+ *
  *	  Once we have put all of the notifications into the queue, we return to
  *	  CommitTransaction() which will then do the actual transaction commit.
  *
  *	  After commit we are called another time (AtCommit_Notify()). Here we
- *	  make any actual updates to the effective listen state (listenChannels).
- *	  Then we signal any backends that may be interested in our messages
- *	  (including our own backend, if listening).  This is done by
- *	  SignalBackends(), which scans the list of listening backends and sends a
- *	  PROCSIG_NOTIFY_INTERRUPT signal to every listening backend (we don't
- *	  know which backend is listening on which channel so we must signal them
- *	  all).  We can exclude backends that are already up to date, though, and
- *	  we can also exclude backends that are in other databases (unless they
- *	  are way behind and should be kicked to make them advance their
- *	  pointers).
+ *	  commit the staged listen/unlisten changes by setting listening=true for
+ *	  staged LISTENs, or removing entries for UNLISTENs.  Then we signal any backends
+ *	  that may be interested in our messages (including our own backend,
+ *	  if listening).  This is done by SignalBackends(), which consults the
+ *	  shared channel hash table to identify listeners for the channels that
+ *	  have pending notifications in the current database.  Each selected
+ *	  backend is marked as having a wakeup pending to avoid duplicate signals,
+ *	  and a PROCSIG_NOTIFY_INTERRUPT signal is sent to it.
+ *
+ *	  When writing notifications, PreCommit_Notify() records the queue head
+ *	  position both before and after the write.  Because all writers serialize
+ *	  on a cluster-wide heavyweight lock, no backend can insert entries between
+ *	  these two points.  SignalBackends() uses this fact to directly advance any
+ *	  backend that is still positioned at the old head, or within the range
+ *	  written, avoiding unnecessary wakeups for idle listeners that have
+ *	  nothing to read.  Backends that cannot be direct advanced are signaled
+ *	  if they are stuck behind the old queue head, or advancing to a position
+ *	  before the new queue head, since otherwise notifications could be delayed.
  *
  *	  Finally, after we are out of the transaction altogether and about to go
  *	  idle, we scan the queue for messages that need to be sent to our
@@ -137,14 +152,17 @@
 #include "commands/async.h"
 #include "common/hashfn.h"
 #include "funcapi.h"
+#include "lib/dshash.h"
 #include "libpq/libpq.h"
 #include "libpq/pqformat.h"
 #include "miscadmin.h"
+#include "storage/dsm_registry.h"
 #include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "storage/procsignal.h"
 #include "tcop/tcopprot.h"
 #include "utils/builtins.h"
+#include "utils/dsa.h"
 #include "utils/guc_hooks.h"
 #include "utils/memutils.h"
 #include "utils/ps_status.h"
@@ -162,6 +180,36 @@
  */
 #define NOTIFY_PAYLOAD_MAX_LENGTH	(BLCKSZ - NAMEDATALEN - 128)
 
+/*
+ * Channel hash table definitions
+ *
+ * This hash table maps (database OID, channel name) keys to arrays of
+ * ProcNumbers representing the backends listening on each channel.
+ */
+
+#define INITIAL_LISTENERS_ARRAY_SIZE 4
+
+typedef struct ChannelNameKey
+{
+	Oid			dboid;
+	char		channel[NAMEDATALEN];
+} ChannelHashKey;
+
+
+typedef struct ListenerEntry
+{
+	ProcNumber	procNo;
+	bool		listening;		/* true if committed listener */
+} ListenerEntry;
+
+typedef struct ChannelListeners
+{
+	ChannelHashKey key;
+	dsa_pointer listenersArray; /* DSA pointer to ListenerEntry array */
+	int			numListeners;	/* Number of listeners currently stored */
+	int			allocatedListeners; /* Allocated size of array */
+} ChannelListeners;
+
 /*
  * Struct representing an entry in the global notify queue
  *
@@ -224,11 +272,14 @@ typedef struct QueuePosition
 	 (x).page != (y).page ? (x) : \
 	 (x).offset > (y).offset ? (x) : (y))
 
+/* returns true if x comes before y in queue order */
+#define QUEUE_POS_PRECEDES(x,y) \
+	(asyncQueuePagePrecedes((x).page, (y).page) || \
+	 ((x).page == (y).page && (x).offset < (y).offset))
+
 /*
  * Parameter determining how often we try to advance the tail pointer:
- * we do that after every QUEUE_CLEANUP_DELAY pages of NOTIFY data.  This is
- * also the distance by which a backend in another database needs to be
- * behind before we'll decide we need to wake it up to advance its pointer.
+ * we do that after every QUEUE_CLEANUP_DELAY pages of NOTIFY data.
  *
  * Resist the temptation to make this really large.  While that would save
  * work in some places, it would add cost in others.  In particular, this
@@ -246,6 +297,9 @@ typedef struct QueueBackendStatus
 	Oid			dboid;			/* backend's database OID, or InvalidOid */
 	ProcNumber	nextListener;	/* id of next listener, or INVALID_PROC_NUMBER */
 	QueuePosition pos;			/* backend has read queue up to here */
+	bool		wakeupPending;	/* signal sent but not yet processed */
+	bool		isAdvancing;	/* backend is advancing its position */
+	QueuePosition advancingPos; /* target position backend is advancing to */
 } QueueBackendStatus;
 
 /*
@@ -260,14 +314,16 @@ typedef struct QueueBackendStatus
  * (since no other backend will inspect it).
  *
  * When holding NotifyQueueLock in EXCLUSIVE mode, backends can inspect the
- * entries of other backends and also change the head pointer. When holding
- * both NotifyQueueLock and NotifyQueueTailLock in EXCLUSIVE mode, backends
- * can change the tail pointers.
+ * entries of other backends and also change the head pointer. They can
+ * also advance other backends' queue positions, unless they are not
+ * in the process of doing that themselves. When holding both NotifyQueueLock and
+ * NotifyQueueTailLock in EXCLUSIVE mode, backends can change the tail pointers.
  *
  * SLRU buffer pool is divided in banks and bank wise SLRU lock is used as
  * the control lock for the pg_notify SLRU buffers.
  * In order to avoid deadlocks, whenever we need multiple locks, we first get
- * NotifyQueueTailLock, then NotifyQueueLock, and lastly SLRU bank lock.
+ * NotifyQueueTailLock, then NotifyQueueLock, then SLRU bank lock, and lastly
+ * channelHash partition locks.
  *
  * Each backend uses the backend[] array entry with index equal to its
  * ProcNumber.  We rely on this to make SendProcSignal fast.
@@ -288,11 +344,16 @@ typedef struct AsyncQueueControl
 	ProcNumber	firstListener;	/* id of first listener, or
 								 * INVALID_PROC_NUMBER */
 	TimestampTz lastQueueFillWarn;	/* time of last queue-full msg */
+	dsa_handle	channelHashDSA;
+	dshash_table_handle channelHashDSH;
 	QueueBackendStatus backend[FLEXIBLE_ARRAY_MEMBER];
 } AsyncQueueControl;
 
 static AsyncQueueControl *asyncQueueControl;
 
+static dsa_area *channelDSA = NULL;
+static dshash_table *channelHash = NULL;
+
 #define QUEUE_HEAD					(asyncQueueControl->head)
 #define QUEUE_TAIL					(asyncQueueControl->tail)
 #define QUEUE_STOP_PAGE				(asyncQueueControl->stopPage)
@@ -301,6 +362,9 @@ static AsyncQueueControl *asyncQueueControl;
 #define QUEUE_BACKEND_DBOID(i)		(asyncQueueControl->backend[i].dboid)
 #define QUEUE_NEXT_LISTENER(i)		(asyncQueueControl->backend[i].nextListener)
 #define QUEUE_BACKEND_POS(i)		(asyncQueueControl->backend[i].pos)
+#define QUEUE_BACKEND_WAKEUP_PENDING(i)	(asyncQueueControl->backend[i].wakeupPending)
+#define QUEUE_BACKEND_IS_ADVANCING(i)	(asyncQueueControl->backend[i].isAdvancing)
+#define QUEUE_BACKEND_ADVANCING_POS(i)	(asyncQueueControl->backend[i].advancingPos)
 
 /*
  * The SLRU buffer area through which we access the notification queue
@@ -313,16 +377,19 @@ static SlruCtlData NotifyCtlData;
 #define QUEUE_FULL_WARN_INTERVAL	5000	/* warn at most once every 5s */
 
 /*
- * listenChannels identifies the channels we are actually listening to
- * (ie, have committed a LISTEN on).  It is a simple list of channel names,
- * allocated in TopMemoryContext.
+ * listenChannelsHash caches the channels this backend is listening on.
+ * Used by IsListeningOn() for fast lookups when reading notifications.
+ * Entries are pre-allocated during PreCommit_Notify (before clog commit)
+ * so allocation failures safely abort.  On abort, staged entries are removed.
+ * Allocated in TopMemoryContext so it persists across transactions.
  */
-static List *listenChannels = NIL;	/* list of C strings */
+static HTAB *listenChannelsHash = NULL;
 
 /*
  * State for pending LISTEN/UNLISTEN actions consists of an ordered list of
- * all actions requested in the current transaction.  As explained above,
- * we don't actually change listenChannels until we reach transaction commit.
+ * all actions requested in the current transaction.  During PreCommit_Notify,
+ * we stage these changes in listenChannelsHash and the shared channelHash.
+ * On abort, AtAbort_Notify cleans up any staged-but-uncommitted entries.
  *
  * The list is kept in CurTransactionContext.  In subtransactions, each
  * subtransaction has its own list in its own CurTransactionContext, but
@@ -391,6 +458,7 @@ typedef struct NotificationList
 	int			nestingLevel;	/* current transaction nesting depth */
 	List	   *events;			/* list of Notification structs */
 	HTAB	   *hashtab;		/* hash of NotificationHash structs, or NULL */
+	HTAB	   *channelSet;		/* hash of unique channel names, or NULL */
 	struct NotificationList *upper; /* details for upper transaction levels */
 } NotificationList;
 
@@ -401,6 +469,18 @@ struct NotificationHash
 	Notification *event;		/* => the actual Notification struct */
 };
 
+struct ChannelName
+{
+	char		channel[NAMEDATALEN];
+};
+
+/* Entry for pendingListenChannels hash table */
+struct PendingListenEntry
+{
+	char		channel[NAMEDATALEN];	/* hash key */
+	bool		listening;		/* true = LISTEN, false = UNLISTEN */
+};
+
 static NotificationList *pendingNotifies = NULL;
 
 /*
@@ -418,6 +498,37 @@ static bool unlistenExitRegistered = false;
 /* True if we're currently registered as a listener in asyncQueueControl */
 static bool amRegisteredListener = false;
 
+/*
+ * Queue head positions for direct advancement.
+ * These are captured during PreCommit_Notify while holding the heavyweight
+ * lock on database 0, ensuring no other backend can insert notifications
+ * between them.  SignalBackends uses these to advance idle backends.
+ */
+static QueuePosition queueHeadBeforeWrite;
+static QueuePosition queueHeadAfterWrite;
+
+/*
+ * List of channels with pending notifications in the current transaction.
+ */
+static List *pendingNotifyChannels = NIL;
+
+/*
+ * Hash table of pending listen/unlisten changes in the current transaction.
+ * Key is channel name, value is boolean (true = LISTEN, false = UNLISTEN).
+ * Provides automatic deduplication of repeated LISTEN/UNLISTEN on same channel.
+ * Populated during PreCommit_Notify and used by AtCommit_Notify/AtAbort_Notify.
+ */
+static HTAB *pendingListenChannels = NULL;
+
+/*
+ * Preallocated arrays for SignalBackends to avoid memory allocation after
+ * committing to clog.  Allocated in PreCommit_Notify when there are pending
+ * notifications.
+ */
+static int32 *signalPids = NULL;
+static ProcNumber *signalProcnos = NULL;
+
+
 /* have we advanced to a page that's a multiple of QUEUE_CLEANUP_DELAY? */
 static bool tryAdvanceTail = false;
 
@@ -428,14 +539,14 @@ bool		Trace_notify = false;
 int			max_notify_queue_pages = 1048576;
 
 /* local function prototypes */
-static inline int64 asyncQueuePageDiff(int64 p, int64 q);
 static inline bool asyncQueuePagePrecedes(int64 p, int64 q);
 static void queue_listen(ListenActionKind action, const char *channel);
 static void Async_UnlistenOnExit(int code, Datum arg);
 static void Exec_ListenPreCommit(void);
-static void Exec_ListenCommit(const char *channel);
-static void Exec_UnlistenCommit(const char *channel);
-static void Exec_UnlistenAllCommit(void);
+static void Exec_ListenPreCommitStage(const char *channel);
+static void Exec_UnlistenPreCommitStage(const char *channel);
+static void Exec_UnlistenAllPreCommitStage(void);
+static void CleanupListenersOnExit(void);
 static bool IsListeningOn(const char *channel);
 static void asyncQueueUnregister(void);
 static bool asyncQueueIsFull(void);
@@ -456,16 +567,9 @@ static void AddEventToPendingNotifies(Notification *n);
 static uint32 notification_hash(const void *key, Size keysize);
 static int	notification_match(const void *key1, const void *key2, Size keysize);
 static void ClearPendingActionsAndNotifies(void);
-
-/*
- * Compute the difference between two queue page numbers.
- * Previously this function accounted for a wraparound.
- */
-static inline int64
-asyncQueuePageDiff(int64 p, int64 q)
-{
-	return p - q;
-}
+static inline void ChannelHashPrepareKey(ChannelHashKey *key, Oid dboid, const char *channel);
+static dshash_hash channelHashFunc(const void *key, size_t size, void *arg);
+static void initChannelHash(void);
 
 /*
  * Determines whether p precedes q.
@@ -477,6 +581,131 @@ asyncQueuePagePrecedes(int64 p, int64 q)
 	return p < q;
 }
 
+/*
+ * channelHashFunc
+ *		Hash function for channel keys.
+ */
+static dshash_hash
+channelHashFunc(const void *key, size_t size, void *arg)
+{
+	const ChannelHashKey *k = (const ChannelHashKey *) key;
+	dshash_hash h;
+
+	h = DatumGetUInt32(hash_uint32(k->dboid));
+	h ^= DatumGetUInt32(hash_any((const unsigned char *) k->channel,
+								 strnlen(k->channel, NAMEDATALEN)));
+
+	return h;
+}
+
+/* parameters for the channel hash table */
+static const dshash_parameters channelDSHParams = {
+	sizeof(ChannelHashKey),
+	sizeof(ChannelListeners),
+	dshash_memcmp,
+	channelHashFunc,
+	dshash_memcpy,
+	LWTRANCHE_NOTIFY_CHANNEL_HASH
+};
+
+/*
+ * initChannelHash
+ *		Lazy initialization of the channel hash table.
+ */
+static void
+initChannelHash(void)
+{
+	MemoryContext oldcontext;
+
+	/* Quick exit if we already did this */
+	if (asyncQueueControl->channelHashDSH != DSHASH_HANDLE_INVALID &&
+		channelHash != NULL)
+		return;
+
+	/* Otherwise, use a lock to ensure only one process creates the table */
+	LWLockAcquire(NotifyQueueLock, LW_EXCLUSIVE);
+
+	/* Be sure any local memory allocated by DSA routines is persistent */
+	oldcontext = MemoryContextSwitchTo(TopMemoryContext);
+
+	if (asyncQueueControl->channelHashDSH == DSHASH_HANDLE_INVALID)
+	{
+		/* Initialize dynamic shared hash table for channel hash */
+		channelDSA = dsa_create(LWTRANCHE_NOTIFY_CHANNEL_HASH);
+		dsa_pin(channelDSA);
+		dsa_pin_mapping(channelDSA);
+		channelHash = dshash_create(channelDSA, &channelDSHParams, NULL);
+
+		/* Store handles in shared memory for other backends to use */
+		asyncQueueControl->channelHashDSA = dsa_get_handle(channelDSA);
+		asyncQueueControl->channelHashDSH =
+			dshash_get_hash_table_handle(channelHash);
+	}
+	else if (!channelHash)
+	{
+		/* Attach to existing dynamic shared hash table */
+		channelDSA = dsa_attach(asyncQueueControl->channelHashDSA);
+		dsa_pin_mapping(channelDSA);
+		channelHash = dshash_attach(channelDSA, &channelDSHParams,
+									asyncQueueControl->channelHashDSH,
+									NULL);
+	}
+
+	MemoryContextSwitchTo(oldcontext);
+	LWLockRelease(NotifyQueueLock);
+}
+
+/*
+ * initListenChannelsHash
+ *		Lazy initialization of the local listen channels hash table.
+ */
+static void
+initListenChannelsHash(void)
+{
+	HASHCTL		hash_ctl;
+
+	/* Quick exit if we already did this */
+	if (listenChannelsHash != NULL)
+		return;
+
+	/* Initialize local hash table for this backend's listened channels */
+	memset(&hash_ctl, 0, sizeof(hash_ctl));
+	hash_ctl.keysize = NAMEDATALEN;
+	hash_ctl.entrysize = sizeof(struct ChannelName);
+
+	listenChannelsHash =
+		hash_create("Listen Channels",
+					64,
+					&hash_ctl,
+					HASH_ELEM | HASH_STRINGS);
+}
+
+/*
+ * initPendingListenChannels
+ *		Lazy initialization of the pending listen channels hash table.
+ *		This is allocated in CurTransactionContext and destroyed at
+ *		transaction end.
+ */
+static void
+initPendingListenChannels(void)
+{
+	HASHCTL		hash_ctl;
+
+	if (pendingListenChannels != NULL)
+		return;
+
+	memset(&hash_ctl, 0, sizeof(hash_ctl));
+	hash_ctl.keysize = NAMEDATALEN;
+	hash_ctl.entrysize = sizeof(struct PendingListenEntry);
+	hash_ctl.hcxt = CurTransactionContext;
+
+	pendingListenChannels =
+		hash_create("Pending Listen Channels",
+					16,
+					&hash_ctl,
+					HASH_ELEM | HASH_STRINGS | HASH_CONTEXT);
+}
+
 /*
  * Report space needed for our shared memory area
  */
@@ -520,12 +749,17 @@ AsyncShmemInit(void)
 		QUEUE_STOP_PAGE = 0;
 		QUEUE_FIRST_LISTENER = INVALID_PROC_NUMBER;
 		asyncQueueControl->lastQueueFillWarn = 0;
+		asyncQueueControl->channelHashDSA = DSA_HANDLE_INVALID;
+		asyncQueueControl->channelHashDSH = DSHASH_HANDLE_INVALID;
 		for (int i = 0; i < MaxBackends; i++)
 		{
 			QUEUE_BACKEND_PID(i) = InvalidPid;
 			QUEUE_BACKEND_DBOID(i) = InvalidOid;
 			QUEUE_NEXT_LISTENER(i) = INVALID_PROC_NUMBER;
 			SET_QUEUE_POS(QUEUE_BACKEND_POS(i), 0, 0);
+			SET_QUEUE_POS(QUEUE_BACKEND_ADVANCING_POS(i), 0, 0);
+			QUEUE_BACKEND_WAKEUP_PENDING(i) = false;
+			QUEUE_BACKEND_IS_ADVANCING(i) = false;
 		}
 	}
 
@@ -656,6 +890,7 @@ Async_Notify(const char *channel, const char *payload)
 		notifies->events = list_make1(n);
 		/* We certainly don't need a hashtable yet */
 		notifies->hashtab = NULL;
+		notifies->channelSet = NULL;
 		notifies->upper = pendingNotifies;
 		pendingNotifies = notifies;
 	}
@@ -682,8 +917,8 @@ Async_Notify(const char *channel, const char *payload)
  *		Common code for listen, unlisten, unlisten all commands.
  *
  *		Adds the request to the list of pending actions.
- *		Actual update of the listenChannels list happens during transaction
- *		commit.
+ *		Actual update of listenChannelsHash and channelHash happens during
+ *		PreCommit_Notify, with staged changes committed in AtCommit_Notify.
  */
 static void
 queue_listen(ListenActionKind action, const char *channel)
@@ -782,30 +1017,49 @@ Async_UnlistenAll(void)
  * SQL function: return a set of the channel names this backend is actively
  * listening to.
  *
- * Note: this coding relies on the fact that the listenChannels list cannot
+ * Note: this coding relies on the fact that the listenChannelsHash cannot
  * change within a transaction.
  */
 Datum
 pg_listening_channels(PG_FUNCTION_ARGS)
 {
 	FuncCallContext *funcctx;
+	HASH_SEQ_STATUS *status;
 
 	/* stuff done only on the first call of the function */
 	if (SRF_IS_FIRSTCALL())
 	{
+		MemoryContext oldcontext;
+
 		/* create a function context for cross-call persistence */
 		funcctx = SRF_FIRSTCALL_INIT();
+
+		/* Initialize hash table iteration if we have any channels */
+		if (listenChannelsHash != NULL)
+		{
+			oldcontext = MemoryContextSwitchTo(funcctx->multi_call_memory_ctx);
+			status = (HASH_SEQ_STATUS *) palloc(sizeof(HASH_SEQ_STATUS));
+			hash_seq_init(status, listenChannelsHash);
+			funcctx->user_fctx = status;
+			MemoryContextSwitchTo(oldcontext);
+		}
+		else
+		{
+			funcctx->user_fctx = NULL;
+		}
 	}
 
 	/* stuff done on every call of the function */
 	funcctx = SRF_PERCALL_SETUP();
+	status = (HASH_SEQ_STATUS *) funcctx->user_fctx;
 
-	if (funcctx->call_cntr < list_length(listenChannels))
+	if (status != NULL)
 	{
-		char	   *channel = (char *) list_nth(listenChannels,
-												funcctx->call_cntr);
+		struct ChannelName *entry;
 
-		SRF_RETURN_NEXT(funcctx, CStringGetTextDatum(channel));
+		entry = (struct ChannelName *) hash_seq_search(status);
+		if (entry != NULL)
+			SRF_RETURN_NEXT(funcctx, CStringGetTextDatum(entry->channel));
 	}
 
 	SRF_RETURN_DONE(funcctx);
@@ -821,7 +1075,7 @@ pg_listening_channels(PG_FUNCTION_ARGS)
 static void
 Async_UnlistenOnExit(int code, Datum arg)
 {
-	Exec_UnlistenAllCommit();
+	CleanupListenersOnExit();
 	asyncQueueUnregister();
 }
 
@@ -868,8 +1122,25 @@ PreCommit_Notify(void)
 		elog(DEBUG1, "PreCommit_Notify");
 
 	/* Preflight for any pending listen/unlisten actions */
+	if (pendingNotifies != NULL || pendingActions != NULL)
+		initChannelHash();
+
+	if (pendingNotifies != NULL)
+	{
+		if (signalPids == NULL)
+			signalPids = MemoryContextAlloc(TopMemoryContext,
+											MaxBackends * sizeof(int32));
+
+		if (signalProcnos == NULL)
+			signalProcnos = MemoryContextAlloc(TopMemoryContext,
+											   MaxBackends * sizeof(ProcNumber));
+	}
+
 	if (pendingActions != NULL)
 	{
+		initListenChannelsHash();
+		initPendingListenChannels();
+
 		foreach(p, pendingActions->actions)
 		{
 			ListenAction *actrec = (ListenAction *) lfirst(p);
@@ -878,12 +1149,13 @@ PreCommit_Notify(void)
 			{
 				case LISTEN_LISTEN:
 					Exec_ListenPreCommit();
+					Exec_ListenPreCommitStage(actrec->channel);
 					break;
 				case LISTEN_UNLISTEN:
-					/* there is no Exec_UnlistenPreCommit() */
+					Exec_UnlistenPreCommitStage(actrec->channel);
 					break;
 				case LISTEN_UNLISTEN_ALL:
-					/* there is no Exec_UnlistenAllPreCommit() */
+					Exec_UnlistenAllPreCommitStage();
 					break;
 			}
 		}
@@ -893,6 +1165,36 @@ PreCommit_Notify(void)
 	if (pendingNotifies)
 	{
 		ListCell   *nextNotify;
+		bool		firstIteration = true;
+
+		/*
+		 * Build list of unique channels for SignalBackends().
+		 *
+		 * If we have a channelSet, use it to efficiently get the unique
+		 * channels.  Otherwise, fall back to the linear approach.
+		 */
+		pendingNotifyChannels = NIL;
+		if (pendingNotifies->channelSet != NULL)
+		{
+			HASH_SEQ_STATUS status;
+			struct ChannelName *channelEntry;
+
+			hash_seq_init(&status, pendingNotifies->channelSet);
+			while ((channelEntry = (struct ChannelName *) hash_seq_search(&status)) != NULL)
+				pendingNotifyChannels = lappend(pendingNotifyChannels, channelEntry->channel);
+		}
+		else
+		{
+			/* Linear approach for small number of notifications */
+			foreach_ptr(Notification, n, pendingNotifies->events)
+			{
+				char	   *channel = n->data;
+
+				/* Add if not already in list */
+				if (!list_member_ptr(pendingNotifyChannels, channel))
+					pendingNotifyChannels = lappend(pendingNotifyChannels, channel);
+			}
+		}
 
 		/*
 		 * Make sure that we have an XID assigned to the current transaction.
@@ -921,6 +1223,22 @@ PreCommit_Notify(void)
 		LockSharedObject(DatabaseRelationId, InvalidOid, 0,
 						 AccessExclusiveLock);
 
+		/*
+		 * For the direct advancement optimization in SignalBackends(), we
+		 * need to ensure that no other backend can insert queue entries
+		 * between queueHeadBeforeWrite and queueHeadAfterWrite.  The
+		 * heavyweight lock above provides this guarantee, since it serializes
+		 * all writers.
+		 *
+		 * Note: if the heavyweight lock were ever removed for scalability
+		 * reasons, we could achieve the same guarantee by holding
+		 * NotifyQueueLock in EXCLUSIVE mode across all our insertions, rather
+		 * than releasing and reacquiring it for each page as we do below.
+		 */
+
+		/* Initialize queueHeadBeforeWrite to a safe default */
+		SET_QUEUE_POS(queueHeadBeforeWrite, 0, 0);
+
 		/* Now push the notifications into the queue */
 		nextNotify = list_head(pendingNotifies->events);
 		while (nextNotify != NULL)
@@ -938,12 +1256,20 @@ PreCommit_Notify(void)
 			 * point in time we can still roll the transaction back.
 			 */
 			LWLockAcquire(NotifyQueueLock, LW_EXCLUSIVE);
+			if (firstIteration)
+			{
+				queueHeadBeforeWrite = QUEUE_HEAD;
+				firstIteration = false;
+			}
+
 			asyncQueueFillWarning();
 			if (asyncQueueIsFull())
 				ereport(ERROR,
 						(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
 						 errmsg("too many notifications in the NOTIFY queue")));
 			nextNotify = asyncQueueAddEntries(nextNotify);
+			queueHeadAfterWrite = QUEUE_HEAD;
+
 			LWLockRelease(NotifyQueueLock);
 		}
 
@@ -956,7 +1282,7 @@ PreCommit_Notify(void)
  *
  *		This is called at transaction commit, after committing to clog.
  *
- *		Update listenChannels and clear transaction-local state.
+ *		Update listenChannelsHash and clear transaction-local state.
  *
  *		If we issued any notifications in the transaction, send signals to
  *		listening backends (possibly including ourselves) to process them.
@@ -966,8 +1292,6 @@ PreCommit_Notify(void)
 void
 AtCommit_Notify(void)
 {
-	ListCell   *p;
-
 	/*
 	 * Allow transactions that have not executed LISTEN/UNLISTEN/NOTIFY to
 	 * return as soon as possible
@@ -978,30 +1302,69 @@ AtCommit_Notify(void)
 	if (Trace_notify)
 		elog(DEBUG1, "AtCommit_Notify");
 
-	/* Perform any pending listen/unlisten actions */
-	if (pendingActions != NULL)
+	/* Commit staged listen/unlisten changes */
+	if (pendingListenChannels != NULL)
 	{
-		foreach(p, pendingActions->actions)
+		HASH_SEQ_STATUS seq;
+		struct PendingListenEntry *pending;
+
+		hash_seq_init(&seq, pendingListenChannels);
+		while ((pending = (struct PendingListenEntry *) hash_seq_search(&seq)) != NULL)
 		{
-			ListenAction *actrec = (ListenAction *) lfirst(p);
+			ChannelHashKey key;
+			ChannelListeners *entry;
+			ListenerEntry *listeners;
 
-			switch (actrec->action)
+			ChannelHashPrepareKey(&key, MyDatabaseId, pending->channel);
+			entry = dshash_find(channelHash, &key, true);
+			if (entry == NULL)
+				continue;
+
+			listeners = (ListenerEntry *) dsa_get_address(channelDSA, entry->listenersArray);
+
+			for (int i = 0; i < entry->numListeners; i++)
 			{
-				case LISTEN_LISTEN:
-					Exec_ListenCommit(actrec->channel);
-					break;
-				case LISTEN_UNLISTEN:
-					Exec_UnlistenCommit(actrec->channel);
-					break;
-				case LISTEN_UNLISTEN_ALL:
-					Exec_UnlistenAllCommit();
+				if (listeners[i].procNo == MyProcNumber)
+				{
+					if (pending->listening)
+					{
+						/*
+						 * LISTEN being committed: set listening=true.
+						 * listenChannelsHash was pre-allocated in PreCommit.
+						 */
+						listeners[i].listening = true;
+					}
+					else
+					{
+						/* UNLISTEN being committed: remove from channelHash */
+						entry->numListeners--;
+						if (i < entry->numListeners)
+							memmove(&listeners[i], &listeners[i + 1],
+									sizeof(ListenerEntry) * (entry->numListeners - i));
+
+						/* Remove from local cache */
+						(void) hash_search(listenChannelsHash, pending->channel,
+										   HASH_REMOVE, NULL);
+
+						if (entry->numListeners == 0)
+						{
+							dsa_free(channelDSA, entry->listenersArray);
+							dshash_delete_entry(channelHash, entry);
+							entry = NULL;
+						}
+					}
 					break;
+				}
 			}
+
+			if (entry != NULL)
+				dshash_release_lock(channelHash, entry);
 		}
 	}
 
 	/* If no longer listening to anything, get out of listener array */
-	if (amRegisteredListener && listenChannels == NIL)
+	if (amRegisteredListener &&
+		(listenChannelsHash == NULL || hash_get_num_entries(listenChannelsHash) == 0))
 		asyncQueueUnregister();
 
 	/*
@@ -1098,6 +1461,9 @@ Exec_ListenPreCommit(void)
 	QUEUE_BACKEND_POS(MyProcNumber) = max;
 	QUEUE_BACKEND_PID(MyProcNumber) = MyProcPid;
 	QUEUE_BACKEND_DBOID(MyProcNumber) = MyDatabaseId;
+	QUEUE_BACKEND_WAKEUP_PENDING(MyProcNumber) = false;
+	QUEUE_BACKEND_IS_ADVANCING(MyProcNumber) = false;
+	QUEUE_BACKEND_ADVANCING_POS(MyProcNumber) = max;
 	/* Insert backend into list of listeners at correct position */
 	if (prevListener != INVALID_PROC_NUMBER)
 	{
@@ -1127,99 +1493,220 @@ Exec_ListenPreCommit(void)
 }
 
 /*
- * Exec_ListenCommit --- subroutine for AtCommit_Notify
+ * Exec_ListenPreCommitStage --- subroutine for PreCommit_Notify
  *
- * Add the channel to the list of channels we are listening on.
+ * Stage a LISTEN by recording it in pendingListenChannels, pre-allocating
+ * an entry in listenChannelsHash, and pre-allocating an entry in the shared
+ * channelHash with listening=false.  The listening flag is set to true in
+ * AtCommit_Notify.  On abort, the pre-allocated entries are removed.
  */
 static void
-Exec_ListenCommit(const char *channel)
+Exec_ListenPreCommitStage(const char *channel)
 {
-	MemoryContext oldcontext;
+	ChannelHashKey key;
+	ChannelListeners *entry;
+	bool		found;
+	ListenerEntry *listeners;
+	struct PendingListenEntry *pending;
 
-	/* Do nothing if we are already listening on this channel */
-	if (IsListeningOn(channel))
+	/* Record in local pending hash that we want to LISTEN */
+	pending = (struct PendingListenEntry *)
+		hash_search(pendingListenChannels, channel, HASH_ENTER, &found);
+	pending->listening = true;
+
+	/* Pre-allocate in local cache (OOM-safe: before clog commit) */
+	(void) hash_search(listenChannelsHash, channel, HASH_ENTER, NULL);
+
+	/* Pre-allocate entry in shared channelHash with listening=false */
+	ChannelHashPrepareKey(&key, MyDatabaseId, channel);
+	entry = dshash_find_or_insert(channelHash, &key, &found);
+
+	if (!found)
+	{
+		entry->listenersArray = InvalidDsaPointer;
+		entry->numListeners = 0;
+		entry->allocatedListeners = 0;
+	}
+
+	if (!DsaPointerIsValid(entry->listenersArray))
+	{
+		entry->listenersArray = dsa_allocate(channelDSA,
+											 sizeof(ListenerEntry) * INITIAL_LISTENERS_ARRAY_SIZE);
+		entry->allocatedListeners = INITIAL_LISTENERS_ARRAY_SIZE;
+	}
+
+	listeners = (ListenerEntry *) dsa_get_address(channelDSA, entry->listenersArray);
+
+	/*
+	 * Check if we already have an entry (possibly from earlier in this
+	 * transaction)
+	 */
+	for (int i = 0; i < entry->numListeners; i++)
+	{
+		if (listeners[i].procNo == MyProcNumber)
+		{
+			/* Already have an entry; listening flag stays as-is until commit */
+			dshash_release_lock(channelHash, entry);
+			return;
+		}
+	}
+
+	/* Need to add a new entry; grow array if necessary */
+	if (entry->numListeners >= entry->allocatedListeners)
+	{
+		int			new_size = entry->allocatedListeners * 2;
+		dsa_pointer new_array = dsa_allocate(channelDSA,
+											 sizeof(ListenerEntry) * new_size);
+		ListenerEntry *new_listeners = (ListenerEntry *) dsa_get_address(channelDSA, new_array);
+
+		memcpy(new_listeners, listeners, sizeof(ListenerEntry) * entry->numListeners);
+		dsa_free(channelDSA, entry->listenersArray);
+		entry->listenersArray = new_array;
+		entry->allocatedListeners = new_size;
+		listeners = new_listeners;
+	}
+
+	listeners[entry->numListeners].procNo = MyProcNumber;
+	listeners[entry->numListeners].listening = false;	/* staged, not yet
+														 * committed */
+	entry->numListeners++;
+
+	dshash_release_lock(channelHash, entry);
+}
+
+/*
+ * Exec_UnlistenPreCommitStage --- subroutine for PreCommit_Notify
+ *
+ * Stage an UNLISTEN by recording it in pendingListenChannels.  We don't
+ * touch channelHash yet - the listener keeps receiving signals until
+ * commit, when the entry is removed.
+ */
+static void
+Exec_UnlistenPreCommitStage(const char *channel)
+{
+	struct PendingListenEntry *pending;
+	bool		found;
+
+	/*
+	 * Record in local pending hash that we want to UNLISTEN. Don't touch
+	 * listenChannelsHash or channelHash yet - we keep receiving signals until
+	 * commit.
+	 */
+	pending = (struct PendingListenEntry *)
+		hash_search(pendingListenChannels, channel, HASH_ENTER, &found);
+	pending->listening = false;
+}
+
+/*
+ * Exec_UnlistenAllPreCommitStage --- subroutine for PreCommit_Notify
+ *
+ * Stage UNLISTEN * by recording all listened channels in pendingListenChannels
+ * with listening=false.
+ */
+static void
+Exec_UnlistenAllPreCommitStage(void)
+{
+	HASH_SEQ_STATUS seq;
+	struct ChannelName *channelEntry;
+	struct PendingListenEntry *pending;
+
+	/*
+	 * First, set all existing entries in pendingListenChannels to false. This
+	 * handles the case of LISTEN foo; UNLISTEN ALL - foo needs to be marked
+	 * as unlisten even though it's not in listenChannelsHash yet.
+	 */
+	hash_seq_init(&seq, pendingListenChannels);
+	while ((pending = (struct PendingListenEntry *) hash_seq_search(&seq)) != NULL)
+		pending->listening = false;
+
+	/*
+	 * Then scan listenChannelsHash (committed channels) and add any that
+	 * aren't already in pendingListenChannels.
+	 */
+	if (listenChannelsHash != NULL)
+	{
+		hash_seq_init(&seq, listenChannelsHash);
+		while ((channelEntry = (struct ChannelName *) hash_seq_search(&seq)) != NULL)
+		{
+			bool		found;
+
+			pending = (struct PendingListenEntry *)
+				hash_search(pendingListenChannels, channelEntry->channel, HASH_ENTER, &found);
+			pending->listening = false;
+		}
+	}
+}
+
+/*
+ * CleanupListenersOnExit --- called from Async_UnlistenOnExit
+ *
+ *		Remove this backend from all channels in the shared hash.
+ */
+static void
+CleanupListenersOnExit(void)
+{
+	dshash_seq_status status;
+	ChannelListeners *entry;
+
+	if (Trace_notify)
+		elog(DEBUG1, "CleanupListenersOnExit(%d)", MyProcPid);
+
+	/* Clear our local cache */
+	if (listenChannelsHash != NULL)
+	{
+		hash_destroy(listenChannelsHash);
+		listenChannelsHash = NULL;
+	}
+
+	/* Now remove from the shared channelHash */
+	if (channelHash == NULL)
 		return;
 
-	/*
-	 * Add the new channel name to listenChannels.
-	 *
-	 * XXX It is theoretically possible to get an out-of-memory failure here,
-	 * which would be bad because we already committed.  For the moment it
-	 * doesn't seem worth trying to guard against that, but maybe improve this
-	 * later.
-	 */
-	oldcontext = MemoryContextSwitchTo(TopMemoryContext);
-	listenChannels = lappend(listenChannels, pstrdup(channel));
-	MemoryContextSwitchTo(oldcontext);
-}
-
-/*
- * Exec_UnlistenCommit --- subroutine for AtCommit_Notify
- *
- * Remove the specified channel name from listenChannels.
- */
-static void
-Exec_UnlistenCommit(const char *channel)
-{
-	ListCell   *q;
-
-	if (Trace_notify)
-		elog(DEBUG1, "Exec_UnlistenCommit(%s,%d)", channel, MyProcPid);
-
-	foreach(q, listenChannels)
+	dshash_seq_init(&status, channelHash, true);
+	while ((entry = dshash_seq_next(&status)) != NULL)
 	{
-		char	   *lchan = (char *) lfirst(q);
-
-		if (strcmp(lchan, channel) == 0)
+		if (entry->key.dboid == MyDatabaseId)
 		{
-			listenChannels = foreach_delete_current(listenChannels, q);
-			pfree(lchan);
-			break;
+			ListenerEntry *listeners;
+			int			i;
+
+			listeners = (ListenerEntry *) dsa_get_address(channelDSA,
+														  entry->listenersArray);
+
+			for (i = 0; i < entry->numListeners; i++)
+			{
+				if (listeners[i].procNo == MyProcNumber)
+				{
+					entry->numListeners--;
+					if (i < entry->numListeners)
+						memmove(&listeners[i], &listeners[i + 1],
+								sizeof(ListenerEntry) * (entry->numListeners - i));
+
+					if (entry->numListeners == 0)
+					{
+						dsa_free(channelDSA, entry->listenersArray);
+						dshash_delete_current(&status);
+					}
+					break;
+				}
+			}
 		}
 	}
-
-	/*
-	 * We do not complain about unlistening something not being listened;
-	 * should we?
-	 */
-}
-
-/*
- * Exec_UnlistenAllCommit --- subroutine for AtCommit_Notify
- *
- *		Unlisten on all channels for this backend.
- */
-static void
-Exec_UnlistenAllCommit(void)
-{
-	if (Trace_notify)
-		elog(DEBUG1, "Exec_UnlistenAllCommit(%d)", MyProcPid);
-
-	list_free_deep(listenChannels);
-	listenChannels = NIL;
+	dshash_seq_term(&status);
 }
 
 /*
  * Test whether we are actively listening on the given channel name.
  *
  * Note: this function is executed for every notification found in the queue.
- * Perhaps it is worth further optimization, eg convert the list to a sorted
- * array so we can binary-search it.  In practice the list is likely to be
- * fairly short, though.
  */
 static bool
 IsListeningOn(const char *channel)
 {
-	ListCell   *p;
+	if (listenChannelsHash == NULL)
+		return false;
 
-	foreach(p, listenChannels)
-	{
-		char	   *lchan = (char *) lfirst(p);
-
-		if (strcmp(lchan, channel) == 0)
-			return true;
-	}
-	return false;
+	return (hash_search(listenChannelsHash, channel, HASH_FIND, NULL) != NULL);
 }
 
 /*
@@ -1229,7 +1716,7 @@ IsListeningOn(const char *channel)
 static void
 asyncQueueUnregister(void)
 {
-	Assert(listenChannels == NIL);	/* else caller error */
+	Assert(listenChannelsHash == NULL || hash_get_num_entries(listenChannelsHash) == 0);	/* else caller error */
 
 	if (!amRegisteredListener)	/* nothing to do */
 		return;
@@ -1241,6 +1728,9 @@ asyncQueueUnregister(void)
 	/* Mark our entry as invalid */
 	QUEUE_BACKEND_PID(MyProcNumber) = InvalidPid;
 	QUEUE_BACKEND_DBOID(MyProcNumber) = InvalidOid;
+	QUEUE_BACKEND_WAKEUP_PENDING(MyProcNumber) = false;
+	QUEUE_BACKEND_IS_ADVANCING(MyProcNumber) = false;
+	SET_QUEUE_POS(QUEUE_BACKEND_ADVANCING_POS(MyProcNumber), 0, 0);
 	/* and remove it from the list */
 	if (QUEUE_FIRST_LISTENER == MyProcNumber)
 		QUEUE_FIRST_LISTENER = QUEUE_NEXT_LISTENER(MyProcNumber);
@@ -1565,12 +2055,21 @@ asyncQueueFillWarning(void)
 /*
  * Send signals to listening backends.
  *
- * Normally we signal only backends in our own database, since only those
- * backends could be interested in notifies we send.  However, if there's
- * notify traffic in our database but no traffic in another database that
- * does have listener(s), those listeners will fall further and further
- * behind.  Waken them anyway if they're far enough behind, so that they'll
- * advance their queue position pointers, allowing the global tail to advance.
+ * Normally we signal only backends in our own database, that are
+ * listening on the channels with pending notifies, since only those
+ * backends are interested in notifies we send.
+ *
+ * Backends that are not interested in our notifies, that are known
+ * to still be positioned at the old queue head, or anywhere in the
+ * queue region we just wrote, can be safely advanced directly to the
+ * new head, since that region is known to contain only our own
+ * notifications.  This avoids unnecessary wakeups when there is
+ * nothing of interest to them.
+ *
+ * Backends that are not interested in our notifies, that are advancing
+ * to a target position before the new queue head, or that are not
+ * advancing and are stationary at a position before the old queue head
+ * needs to be signaled since notifications could otherwise be delayed.
  *
  * Since we know the ProcNumber and the Pid the signaling is quite cheap.
  *
@@ -1580,60 +2079,106 @@ asyncQueueFillWarning(void)
 static void
 SignalBackends(void)
 {
-	int32	   *pids;
-	ProcNumber *procnos;
 	int			count;
+	ListCell   *lc;
 
-	/*
-	 * Identify backends that we need to signal.  We don't want to send
-	 * signals while holding the NotifyQueueLock, so this loop just builds a
-	 * list of target PIDs.
-	 *
-	 * XXX in principle these pallocs could fail, which would be bad. Maybe
-	 * preallocate the arrays?  They're not that large, though.
-	 */
-	pids = (int32 *) palloc(MaxBackends * sizeof(int32));
-	procnos = (ProcNumber *) palloc(MaxBackends * sizeof(ProcNumber));
+	Assert(signalPids != NULL && signalProcnos != NULL);
 	count = 0;
 
 	LWLockAcquire(NotifyQueueLock, LW_EXCLUSIVE);
-	for (ProcNumber i = QUEUE_FIRST_LISTENER; i != INVALID_PROC_NUMBER; i = QUEUE_NEXT_LISTENER(i))
+	foreach(lc, pendingNotifyChannels)
 	{
-		int32		pid = QUEUE_BACKEND_PID(i);
-		QueuePosition pos;
+		char	   *channel = (char *) lfirst(lc);
+		ChannelListeners *entry = NULL;
+		ListenerEntry *listeners;
 
-		Assert(pid != InvalidPid);
-		pos = QUEUE_BACKEND_POS(i);
-		if (QUEUE_BACKEND_DBOID(i) == MyDatabaseId)
+		if (channelHash != NULL)
 		{
-			/*
-			 * Always signal listeners in our own database, unless they're
-			 * already caught up (unlikely, but possible).
-			 */
+			ChannelHashKey key;
+
+			ChannelHashPrepareKey(&key, MyDatabaseId, channel);
+			entry = dshash_find(channelHash, &key, false);
+		}
+
+		if (entry == NULL)
+			continue;
+
+		listeners = (ListenerEntry *) dsa_get_address(channelDSA,
+													  entry->listenersArray);
+
+		for (int j = 0; j < entry->numListeners; j++)
+		{
+			ProcNumber	i;
+			int32		pid;
+			QueuePosition pos;
+
+			if (!listeners[j].listening)
+				continue;
+
+			i = listeners[j].procNo;
+
+			if (QUEUE_BACKEND_WAKEUP_PENDING(i))
+				continue;
+
+			pos = QUEUE_BACKEND_POS(i);
+			pid = QUEUE_BACKEND_PID(i);
+
+			/* Skip if caught up */
 			if (QUEUE_POS_EQUAL(pos, QUEUE_HEAD))
 				continue;
+
+			Assert(pid != InvalidPid);
+
+			QUEUE_BACKEND_WAKEUP_PENDING(i) = true;
+			signalPids[count] = pid;
+			signalProcnos[count] = i;
+			count++;
 		}
-		else
+
+		dshash_release_lock(channelHash, entry);
+	}
+
+	if (pendingNotifies != NULL)
+	{
+		for (ProcNumber i = QUEUE_FIRST_LISTENER;
+			 i != INVALID_PROC_NUMBER;
+			 i = QUEUE_NEXT_LISTENER(i))
 		{
-			/*
-			 * Listeners in other databases should be signaled only if they
-			 * are far behind.
-			 */
-			if (asyncQueuePageDiff(QUEUE_POS_PAGE(QUEUE_HEAD),
-								   QUEUE_POS_PAGE(pos)) < QUEUE_CLEANUP_DELAY)
+			QueuePosition pos;
+			int32		pid;
+
+			if (QUEUE_BACKEND_WAKEUP_PENDING(i))
 				continue;
+
+			pos = QUEUE_BACKEND_POS(i);
+			pid = QUEUE_BACKEND_PID(i);
+
+			if (QUEUE_BACKEND_IS_ADVANCING(i) ?
+				QUEUE_POS_PRECEDES(QUEUE_BACKEND_ADVANCING_POS(i), queueHeadAfterWrite) :
+				QUEUE_POS_PRECEDES(pos, queueHeadBeforeWrite))
+			{
+				Assert(pid != InvalidPid);
+
+				QUEUE_BACKEND_WAKEUP_PENDING(i) = true;
+				signalPids[count] = pid;
+				signalProcnos[count] = i;
+				count++;
+			}
+			else if (!QUEUE_BACKEND_IS_ADVANCING(i) &&
+					 QUEUE_POS_PRECEDES(pos, queueHeadAfterWrite))
+			{
+				Assert(!QUEUE_POS_PRECEDES(pos, queueHeadBeforeWrite));
+
+				QUEUE_BACKEND_POS(i) = queueHeadAfterWrite;
+			}
 		}
-		/* OK, need to signal this one */
-		pids[count] = pid;
-		procnos[count] = i;
-		count++;
 	}
 	LWLockRelease(NotifyQueueLock);
 
 	/* Now send signals */
 	for (int i = 0; i < count; i++)
 	{
-		int32		pid = pids[i];
+		int32		pid = signalPids[i];
 
 		/*
 		 * If we are signaling our own process, no need to involve the kernel;
@@ -1651,12 +2196,9 @@ SignalBackends(void)
 		 * NotifyQueueLock; which is unlikely but certainly possible. So we
 		 * just log a low-level debug message if it happens.
 		 */
-		if (SendProcSignal(pid, PROCSIG_NOTIFY_INTERRUPT, procnos[i]) < 0)
+		if (SendProcSignal(pid, PROCSIG_NOTIFY_INTERRUPT, signalProcnos[i]) < 0)
 			elog(DEBUG3, "could not signal backend with PID %d: %m", pid);
 	}
-
-	pfree(pids);
-	pfree(procnos);
 }
 
 /*
@@ -1664,18 +2206,75 @@ SignalBackends(void)
  *
  *	This is called at transaction abort.
  *
- *	Gets rid of pending actions and outbound notifies that we would have
- *	executed if the transaction got committed.
+ *	Revert any staged listen/unlisten changes and clean up transaction state.
  */
 void
 AtAbort_Notify(void)
 {
 	/*
-	 * If we LISTEN but then roll back the transaction after PreCommit_Notify,
-	 * we have registered as a listener but have not made any entry in
-	 * listenChannels.  In that case, deregister again.
+	 * Revert staged listen/unlisten changes.  For staged LISTENs (entries
+	 * with listening=false), remove from channelHash.  For staged UNLISTENs
+	 * on committed channels (entries with listening=true), nothing to undo
+	 * since we didn't modify channelHash during staging.
 	 */
-	if (amRegisteredListener && listenChannels == NIL)
+	if (pendingListenChannels != NULL && channelHash != NULL)
+	{
+		HASH_SEQ_STATUS seq;
+		struct PendingListenEntry *pending;
+
+		hash_seq_init(&seq, pendingListenChannels);
+		while ((pending = (struct PendingListenEntry *) hash_seq_search(&seq)) != NULL)
+		{
+			ChannelHashKey key;
+			ChannelListeners *entry;
+			ListenerEntry *listeners;
+
+			ChannelHashPrepareKey(&key, MyDatabaseId, pending->channel);
+			entry = dshash_find(channelHash, &key, true);
+			if (entry == NULL)
+				continue;
+
+			listeners = (ListenerEntry *) dsa_get_address(channelDSA, entry->listenersArray);
+
+			for (int i = 0; i < entry->numListeners; i++)
+			{
+				if (listeners[i].procNo == MyProcNumber)
+				{
+					if (!listeners[i].listening)
+					{
+						/* Staged LISTEN (or LISTEN+UNLISTEN) being aborted */
+						/* Remove pre-allocated entries from both hashes */
+						(void) hash_search(listenChannelsHash, pending->channel,
+										   HASH_REMOVE, NULL);
+						entry->numListeners--;
+						if (i < entry->numListeners)
+							memmove(&listeners[i], &listeners[i + 1],
+									sizeof(ListenerEntry) * (entry->numListeners - i));
+
+						if (entry->numListeners == 0)
+						{
+							dsa_free(channelDSA, entry->listenersArray);
+							dshash_delete_entry(channelHash, entry);
+							entry = NULL;
+						}
+					}
+
+					/*
+					 * else: UNLISTEN on committed channel being aborted -
+					 * nothing to undo
+					 */
+					break;
+				}
+			}
+
+			if (entry != NULL)
+				dshash_release_lock(channelHash, entry);
+		}
+	}
+
+	/* If we're no longer listening on anything, unregister */
+	if (amRegisteredListener &&
+		(listenChannelsHash == NULL || hash_get_num_entries(listenChannelsHash) == 0))
 		asyncQueueUnregister();
 
 	/* And clean up */
@@ -1854,20 +2453,29 @@ asyncQueueReadAllNotifications(void)
 	QueuePosition head;
 	Snapshot	snapshot;
 
-	/* Fetch current state */
+	/*
+	 * Fetch current state, indicate to others that we have woken up, and that
+	 * we now will be advancing our position.
+	 */
 	LWLockAcquire(NotifyQueueLock, LW_SHARED);
 	/* Assert checks that we have a valid state entry */
 	Assert(MyProcPid == QUEUE_BACKEND_PID(MyProcNumber));
+	QUEUE_BACKEND_WAKEUP_PENDING(MyProcNumber) = false;
+	head = QUEUE_HEAD;
 	pos = QUEUE_BACKEND_POS(MyProcNumber);
-	head = QUEUE_HEAD;
-	LWLockRelease(NotifyQueueLock);
 
 	if (QUEUE_POS_EQUAL(pos, head))
 	{
 		/* Nothing to do, we have read all notifications already. */
+		LWLockRelease(NotifyQueueLock);
 		return;
 	}
 
+	QUEUE_BACKEND_IS_ADVANCING(MyProcNumber) = true;
+	QUEUE_BACKEND_ADVANCING_POS(MyProcNumber) = head;
+
+	LWLockRelease(NotifyQueueLock);
+
 	/*----------
 	 * Get snapshot we'll use to decide which xacts are still in progress.
 	 * This is trickier than it might seem, because of race conditions.
@@ -1954,6 +2562,8 @@ asyncQueueReadAllNotifications(void)
 
 		/* Update shared state */
 		LWLockAcquire(NotifyQueueLock, LW_SHARED);
+
+		QUEUE_BACKEND_IS_ADVANCING(MyProcNumber) = false;
 		QUEUE_BACKEND_POS(MyProcNumber) = pos;
 		LWLockRelease(NotifyQueueLock);
 
@@ -2051,7 +2661,7 @@ asyncQueueProcessPageEntries(QueuePosition *current,
 			 * over it on the first LISTEN in a session, and not get stuck on
 			 * it indefinitely.
 			 */
-			if (listenChannels == NIL)
+			if (listenChannelsHash == NULL || hash_get_num_entries(listenChannelsHash) == 0)
 				continue;
 
 			if (TransactionIdDidCommit(qe->xid))
@@ -2306,7 +2916,7 @@ ProcessIncomingNotify(bool flush)
 	notifyInterruptPending = false;
 
 	/* Do nothing else if we aren't actively listening */
-	if (listenChannels == NIL)
+	if (listenChannelsHash == NULL || hash_get_num_entries(listenChannelsHash) == 0)
 		return;
 
 	if (Trace_notify)
@@ -2410,13 +3020,15 @@ AddEventToPendingNotifies(Notification *n)
 {
 	Assert(pendingNotifies->events != NIL);
 
-	/* Create the hash table if it's time to */
+	/* Create the hash tables if it's time to */
 	if (list_length(pendingNotifies->events) >= MIN_HASHABLE_NOTIFIES &&
 		pendingNotifies->hashtab == NULL)
 	{
 		HASHCTL		hash_ctl;
 		ListCell   *l;
 
+		Assert(pendingNotifies->channelSet == NULL);
+
 		/* Create the hash table */
 		hash_ctl.keysize = sizeof(Notification *);
 		hash_ctl.entrysize = sizeof(struct NotificationHash);
@@ -2429,10 +3041,22 @@ AddEventToPendingNotifies(Notification *n)
 						&hash_ctl,
 						HASH_ELEM | HASH_FUNCTION | HASH_COMPARE | HASH_CONTEXT);
 
+		/* Create the channel hash table */
+		memset(&hash_ctl, 0, sizeof(hash_ctl));
+		hash_ctl.keysize = NAMEDATALEN;
+		hash_ctl.entrysize = sizeof(struct ChannelName);
+		hash_ctl.hcxt = CurTransactionContext;
+		pendingNotifies->channelSet =
+			hash_create("Pending Notify Channels",
+						64L,
+						&hash_ctl,
+						HASH_ELEM | HASH_STRINGS | HASH_CONTEXT);
+
 		/* Insert all the already-existing events */
 		foreach(l, pendingNotifies->events)
 		{
 			Notification *oldn = (Notification *) lfirst(l);
+			char	   *channel = oldn->data;
 			bool		found;
 
 			(void) hash_search(pendingNotifies->hashtab,
@@ -2440,22 +3064,42 @@ AddEventToPendingNotifies(Notification *n)
 							   HASH_ENTER,
 							   &found);
 			Assert(!found);
+
+			/* Insert channel into channelSet */
+			(void) hash_search(pendingNotifies->channelSet,
+							   channel,
+							   HASH_ENTER,
+							   &found);
+			/* found may be true if multiple events on same channel */
 		}
 	}
 
 	/* Add new event to the list, in order */
 	pendingNotifies->events = lappend(pendingNotifies->events, n);
 
-	/* Add event to the hash table if needed */
+	/* Add event to the hash tables if needed */
 	if (pendingNotifies->hashtab != NULL)
 	{
 		bool		found;
 
+		Assert(pendingNotifies->channelSet != NULL);
+
 		(void) hash_search(pendingNotifies->hashtab,
 						   &n,
 						   HASH_ENTER,
 						   &found);
 		Assert(!found);
+
+		/* Add channel to channelSet */
+		{
+			char	   *channel = n->data;
+
+			(void) hash_search(pendingNotifies->channelSet,
+							   channel,
+							   HASH_ENTER,
+							   &found);
+			/* found may be true if we already have an event on this channel */
+		}
 	}
 }
 
@@ -2493,7 +3137,7 @@ notification_match(const void *key1, const void *key2, Size keysize)
 	return 1;					/* not equal */
 }
 
-/* Clear the pendingActions and pendingNotifies lists. */
+/* Clear the pendingActions, pendingNotifies, and pendingNotifyChannels lists. */
 static void
 ClearPendingActionsAndNotifies(void)
 {
@@ -2505,6 +3149,12 @@ ClearPendingActionsAndNotifies(void)
 	 */
 	pendingActions = NULL;
 	pendingNotifies = NULL;
+	pendingNotifyChannels = NIL;
+	if (pendingListenChannels != NULL)
+	{
+		hash_destroy(pendingListenChannels);
+		pendingListenChannels = NULL;
+	}
 }
 
 /*
@@ -2515,3 +3165,16 @@ check_notify_buffers(int *newval, void **extra, GucSource source)
 {
 	return check_slru_buffers("notify_buffers", newval);
 }
+
+
+/*
+ * ChannelHashPrepareKey
+ *		Prepare a channel key for use as a hash key.
+ */
+static inline void
+ChannelHashPrepareKey(ChannelHashKey *key, Oid dboid, const char *channel)
+{
+	memset(key, 0, sizeof(ChannelHashKey));
+	key->dboid = dboid;
+	strlcpy(key->channel, channel, NAMEDATALEN);
+}
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index dcfadbd5aae..32b0b21f184 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -371,6 +371,7 @@ SubtransBuffer	"Waiting for I/O on a sub-transaction SLRU buffer."
 MultiXactOffsetBuffer	"Waiting for I/O on a multixact offset SLRU buffer."
 MultiXactMemberBuffer	"Waiting for I/O on a multixact member SLRU buffer."
 NotifyBuffer	"Waiting for I/O on a <command>NOTIFY</command> message SLRU buffer."
+NotifyChannelHash	"Waiting to access the <command>NOTIFY</command> channel hash table."
 SerialBuffer	"Waiting for I/O on a serializable transaction conflict SLRU buffer."
 WALInsert	"Waiting to insert WAL data into a memory buffer."
 BufferContent	"Waiting to access a data page in memory."
diff --git a/src/include/storage/lwlocklist.h b/src/include/storage/lwlocklist.h
index 533344509e9..277a78e7954 100644
--- a/src/include/storage/lwlocklist.h
+++ b/src/include/storage/lwlocklist.h
@@ -102,6 +102,7 @@ PG_LWLOCKTRANCHE(SUBTRANS_BUFFER, SubtransBuffer)
 PG_LWLOCKTRANCHE(MULTIXACTOFFSET_BUFFER, MultiXactOffsetBuffer)
 PG_LWLOCKTRANCHE(MULTIXACTMEMBER_BUFFER, MultiXactMemberBuffer)
 PG_LWLOCKTRANCHE(NOTIFY_BUFFER, NotifyBuffer)
+PG_LWLOCKTRANCHE(NOTIFY_CHANNEL_HASH, NotifyChannelHash)
 PG_LWLOCKTRANCHE(SERIAL_BUFFER, SerialBuffer)
 PG_LWLOCKTRANCHE(WAL_INSERT, WALInsert)
 PG_LWLOCKTRANCHE(BUFFER_CONTENT, BufferContent)
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index ceb3fc5d980..b3b3312329e 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -421,6 +421,8 @@ CatalogIdMapEntry
 CatalogIndexState
 ChangeVarNodes_callback
 ChangeVarNodes_context
+ChannelListeners
+ChannelHashKey
 CheckPoint
 CheckPointStmt
 CheckpointStatsData
@@ -1578,6 +1580,7 @@ ListParsedLex
 ListenAction
 ListenActionKind
 ListenStmt
+ListenerEntry
 LoInfo
 LoadStmt
 LocalBufferLookupEnt
-- 
2.50.1

