From e30e796f08e6fc16ebd5760ce24a4887e249116a Mon Sep 17 00:00:00 2001
From: Rahila Syed <rahilasyed.90@gmail.com>
Date: Sun, 15 Sep 2024 17:56:06 +0530
Subject: [PATCH] Function to report memory context stats of any backend

This function sends a signal to a backend to publish
statistics of all its memory contexts. Signal handler
sets a flag, which causes the relevant backend to copy its
MemoryContextStats to a DSA, as part
of next CHECK_FOR_INTERRUPTS().
It there are more that 16MB worth of statistics, the
remaining statistics are copied as a cumulative
total of the remaining contexts.
Once its done, it signals the client backend using
a condition variable. The client backend
then wakes up, reads the shared memory and
returns these values in the form of set of records,
one for each memory context, to the user, followed
by a cumulative total of the remaining contexts,
if any.

Each backend and auxiliary process has its own slot
for reporting the stats. There is an array of such
memory slots of size MaxBackends+NumofAuxiliary
processes in fixed shared memory. Each of these slots point
to a DSA, which contains the stats to be shared by the
corresponding process.
Each slot has its own LW lock and condition variable for
synchronization and communication between the
publishing process and the client backend.
---
 doc/src/sgml/func.sgml                        |  26 ++
 src/backend/postmaster/autovacuum.c           |   4 +
 src/backend/postmaster/checkpointer.c         |   4 +
 src/backend/postmaster/interrupt.c            |   4 +
 src/backend/postmaster/pgarch.c               |   4 +
 src/backend/postmaster/startup.c              |   4 +
 src/backend/postmaster/walsummarizer.c        |   4 +
 src/backend/storage/ipc/ipci.c                |   2 +
 src/backend/storage/ipc/procsignal.c          |   3 +
 src/backend/tcop/postgres.c                   |   3 +
 .../utils/activity/wait_event_names.txt       |   1 +
 src/backend/utils/adt/mcxtfuncs.c             | 274 ++++++++++-
 src/backend/utils/init/globals.c              |   1 +
 src/backend/utils/mmgr/mcxt.c                 | 424 +++++++++++++++++-
 src/include/catalog/pg_proc.dat               |  10 +
 src/include/miscadmin.h                       |   1 +
 src/include/storage/procsignal.h              |   1 +
 src/include/utils/memutils.h                  |  51 +++
 src/test/regress/expected/sysviews.out        |  12 +
 src/test/regress/sql/sysviews.sql             |  12 +
 20 files changed, 822 insertions(+), 23 deletions(-)

diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581a..5d0399508e 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -28358,6 +28358,32 @@ acl      | {postgres=arwdDxtm/postgres,foo=r/postgres}
        </para></entry>
       </row>
 
+      <row>
+       <entry role="func_table_entry"><para role="func_signature">
+        <indexterm>
+         <primary>pg_get_process_memory_contexts</primary>
+        </indexterm>
+        <function>pg_get_process_memory_contexts</function> ( <parameter>pid</parameter> <type>integer</type>, <parameter>get_summary</parameter> <type>boolean</type> )
+        <returnvalue>setof record</returnvalue>
+       </para>
+       <para>
+        This function handles requests to display the memory contexts of a
+        PostgreSQL process with the specified process ID (PID). It takes two
+        arguments: PID and a boolean, get_summary. The function can send requests
+        to both backend and auxiliary processes.
+
+        After receiving memory context statistics from the target process, it
+        returns the results as one row per context. The num_agg_contexts column
+        indicates the number of contexts aggregated in the displayed statistics.
+
+        When get_summary is set to true, memory context statistics at levels 1 and 2,
+        are displayed with each level 2 context showing a cumulative total of all
+        its child contexts.
+        When get_summary is set to false, the num_agg_contexts value is 1, indicating
+        that individual statistics are being displayed.
+       </para></entry>
+      </row>
+
       <row>
        <entry role="func_table_entry"><para role="func_signature">
         <indexterm>
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 3f826532b8..eb4c98a17b 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -768,6 +768,10 @@ HandleAutoVacLauncherInterrupts(void)
 
 	/* Process sinval catchup interrupts that happened while sleeping */
 	ProcessCatchupInterrupt();
+
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
 }
 
 /*
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index 9bfd0fd665..ee8360ad6f 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -616,6 +616,10 @@ HandleCheckpointerInterrupts(void)
 	/* Perform logging of memory contexts of this process */
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
+
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
 }
 
 /*
diff --git a/src/backend/postmaster/interrupt.c b/src/backend/postmaster/interrupt.c
index be69e4c713..9481a5cd24 100644
--- a/src/backend/postmaster/interrupt.c
+++ b/src/backend/postmaster/interrupt.c
@@ -48,6 +48,10 @@ HandleMainLoopInterrupts(void)
 	/* Perform logging of memory contexts of this process */
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
+
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
 }
 
 /*
diff --git a/src/backend/postmaster/pgarch.c b/src/backend/postmaster/pgarch.c
index 12ee815a62..cd1ecb6b93 100644
--- a/src/backend/postmaster/pgarch.c
+++ b/src/backend/postmaster/pgarch.c
@@ -865,6 +865,10 @@ HandlePgArchInterrupts(void)
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
 
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
+
 	if (ConfigReloadPending)
 	{
 		char	   *archiveLib = pstrdup(XLogArchiveLibrary);
diff --git a/src/backend/postmaster/startup.c b/src/backend/postmaster/startup.c
index 59d213031b..d670954c4e 100644
--- a/src/backend/postmaster/startup.c
+++ b/src/backend/postmaster/startup.c
@@ -192,6 +192,10 @@ HandleStartupProcInterrupts(void)
 	/* Perform logging of memory contexts of this process */
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
+
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
 }
 
 
diff --git a/src/backend/postmaster/walsummarizer.c b/src/backend/postmaster/walsummarizer.c
index ffbf043935..b1a5e86a85 100644
--- a/src/backend/postmaster/walsummarizer.c
+++ b/src/backend/postmaster/walsummarizer.c
@@ -876,6 +876,10 @@ HandleWalSummarizerInterrupts(void)
 	/* Perform logging of memory contexts of this process */
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
+
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
 }
 
 /*
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index 174eed7036..4a70eabf7f 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -50,6 +50,7 @@
 #include "storage/sinvaladt.h"
 #include "utils/guc.h"
 #include "utils/injection_point.h"
+#include "utils/memutils.h"
 
 /* GUCs */
 int			shared_memory_type = DEFAULT_SHARED_MEMORY_TYPE;
@@ -340,6 +341,7 @@ CreateOrAttachShmemStructs(void)
 	StatsShmemInit();
 	WaitEventCustomShmemInit();
 	InjectionPointShmemInit();
+	MemCtxShmemInit();
 }
 
 /*
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 7401b6e625..e425b9eeb0 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -688,6 +688,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
 	if (CheckProcSignal(PROCSIG_LOG_MEMORY_CONTEXT))
 		HandleLogMemoryContextInterrupt();
 
+	if (CheckProcSignal(PROCSIG_GET_MEMORY_CONTEXT))
+		HandleGetMemoryContextInterrupt();
+
 	if (CheckProcSignal(PROCSIG_PARALLEL_APPLY_MESSAGE))
 		HandleParallelApplyMessageInterrupt();
 
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index c01cff9d65..0eae9be122 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -3497,6 +3497,9 @@ ProcessInterrupts(void)
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
 
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
+
 	if (ParallelApplyMessagePending)
 		HandleParallelApplyMessages();
 }
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index 0b53cba807..68a1769967 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -158,6 +158,7 @@ WAL_RECEIVER_EXIT	"Waiting for the WAL receiver to exit."
 WAL_RECEIVER_WAIT_START	"Waiting for startup process to send initial data for streaming replication."
 WAL_SUMMARY_READY	"Waiting for a new WAL summary to be generated."
 XACT_GROUP_UPDATE	"Waiting for the group leader to update transaction status at transaction end."
+MEM_CTX_PUBLISH	"Waiting for backend to publish memory information."
 
 ABI_compatibility:
 
diff --git a/src/backend/utils/adt/mcxtfuncs.c b/src/backend/utils/adt/mcxtfuncs.c
index 396c2f223b..c067cdf870 100644
--- a/src/backend/utils/adt/mcxtfuncs.c
+++ b/src/backend/utils/adt/mcxtfuncs.c
@@ -17,28 +17,25 @@
 
 #include "funcapi.h"
 #include "mb/pg_wchar.h"
+#include "miscadmin.h"
+#include "access/twophase.h"
+#include "catalog/pg_authid_d.h"
+#include "nodes/pg_list.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
+#include "utils/acl.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
 #include "utils/hsearch.h"
+#include "utils/memutils.h"
+#include "utils/wait_event_types.h"
 
 /* ----------
  * The max bytes for showing identifiers of MemoryContext.
  * ----------
  */
-#define MEMORY_CONTEXT_IDENT_DISPLAY_SIZE	1024
 
-/*
- * MemoryContextId
- *		Used for storage of transient identifiers for
- *		pg_get_backend_memory_contexts.
- */
-typedef struct MemoryContextId
-{
-	MemoryContext context;
-	int			context_id;
-}			MemoryContextId;
+struct MemoryContextState *memCtxState = NULL;
 
 /*
  * int_list_to_array
@@ -71,7 +68,7 @@ PutMemoryContextsStatsTupleStore(Tuplestorestate *tupstore,
 								 TupleDesc tupdesc, MemoryContext context,
 								 HTAB *context_id_lookup)
 {
-#define PG_GET_BACKEND_MEMORY_CONTEXTS_COLS	10
+#define PG_GET_BACKEND_MEMORY_CONTEXTS_COLS	11
 
 	Datum		values[PG_GET_BACKEND_MEMORY_CONTEXTS_COLS];
 	bool		nulls[PG_GET_BACKEND_MEMORY_CONTEXTS_COLS];
@@ -305,3 +302,256 @@ pg_log_backend_memory_contexts(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(true);
 }
+
+/*
+ * pg_get_process_memory_contexts
+ *		Signal a backend or an auxiliary process to send its memory contexts.
+ *
+ * On receipt of this signal, a backend or an auxiliary process sets the flag
+ * in the signal handler, which causes the next CHECK_FOR_INTERRUPTS()
+ * or process-specific interrupt handler to copy the memory context statistics
+ * in a dynamic shared memory space. The statistics for contexts that do not fit in
+ * shared memory area are stored as a cumulative total of those contexts,
+ * at the end in the dynamic shared memory.
+ * Wait for the backend to send signal on the condition variable after
+ * writing statistics to a shared memory.
+ * Once condition variable comes out of sleep, check if the required
+ * backends statistics are available to read and display.
+ */
+Datum
+pg_get_process_memory_contexts(PG_FUNCTION_ARGS)
+{
+	int			pid = PG_GETARG_INT32(0);
+	bool		get_summary = PG_GETARG_BOOL(1);
+	PGPROC	   *proc;
+	ProcNumber	procNumber = INVALID_PROC_NUMBER;
+	int			i;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	dsa_area   *area;
+	dsa_handle	handle;
+	MemoryContextInfo *memctx_info;
+	MemoryContext oldContext;
+	int			num_retries = 0;
+
+	InitMaterializedSRF(fcinfo, 0);
+
+	/*
+	 * Only superusers or users with pg_read_all_stats privileges can view the
+	 * memory context statistics of another process
+	 */
+	if (!has_privs_of_role(GetUserId(), ROLE_PG_READ_ALL_STATS))
+		ereport(ERROR,
+				(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+				 errmsg("memory context statistics privilege error")));
+
+	/*
+	 * See if the process with given pid is a backend or an auxiliary process.
+	 */
+	proc = BackendPidGetProc(pid);
+	if (proc == NULL)
+		proc = AuxiliaryPidGetProc(pid);
+
+	/*
+	 * BackendPidGetProc() and AuxiliaryPidGetProc() return NULL if the pid
+	 * isn't valid; but by the time we reach kill(), a process for which we
+	 * get a valid proc here might have terminated on its own.  There's no way
+	 * to acquire a lock on an arbitrary process to prevent that. But since
+	 * this mechanism is usually used to debug a backend or an auxiliary
+	 * process running and consuming lots of memory, that it might end on its
+	 * own first and its memory contexts are not logged is not a problem.
+	 */
+	if (proc == NULL)
+	{
+		/*
+		 * This is just a warning so a loop-through-resultset will not abort
+		 * if one backend terminated on its own during the run.
+		 */
+		ereport(WARNING,
+				(errmsg("PID %d is not a PostgreSQL server process", pid)));
+		PG_RETURN_NULL();
+	}
+
+	procNumber = GetNumberFromPGProc(proc);
+	if (procNumber == MyProcNumber)
+	{
+		ereport(WARNING,
+				(errmsg("cannot return statistics for local backend"),
+				 errhint("Use pg_backend_memory_contexts view instead")));
+		PG_RETURN_NULL();
+	}
+
+	/*
+	 * Return statistics of top level 1 and 2 contexts, if get_summary is
+	 * true.
+	 */
+	LWLockAcquire(&memCtxState[procNumber].lw_lock, LW_EXCLUSIVE);
+	memCtxState[procNumber].get_summary = get_summary;
+
+	/*
+	 * Create a DSA segment with maximum size of 16MB, send handle to the
+	 * publishing process for storing the stats. If number of contexts exceed
+	 * 16MB, a cumulative total is stored for such contexts.
+	 */
+	if (memCtxState[procNumber].memstats_dsa_handle == DSA_HANDLE_INVALID)
+	{
+		oldContext = MemoryContextSwitchTo(TopMemoryContext);
+		area = dsa_create_ext(memCtxState[procNumber].lw_lock.tranche, DSA_DEFAULT_INIT_SEGMENT_SIZE,
+							  16 * DSA_DEFAULT_INIT_SEGMENT_SIZE);
+		MemoryContextSwitchTo(oldContext);
+		handle = dsa_get_handle(area);
+		memCtxState[procNumber].memstats_dsa_handle = handle;
+		/* Pin the mapping so that it doesn't throw a warning */
+		dsa_pin(area);
+		dsa_pin_mapping(area);
+	}
+	else
+	{
+		area = dsa_attach(memCtxState[procNumber].memstats_dsa_handle);
+		dsa_pin_mapping(area);
+	}
+	LWLockRelease(&memCtxState[procNumber].lw_lock);
+	if (SendProcSignal(pid, PROCSIG_GET_MEMORY_CONTEXT, procNumber) < 0)
+	{
+		ereport(WARNING,
+				(errmsg("could not send signal to process %d: %m", pid)));
+		dsa_detach(area);
+		PG_RETURN_NULL();
+	}
+
+	/*
+	 * Wait for a backend to publish stats, indicated by a valid dsa pointer
+	 * set by the backend.
+	 */
+	ConditionVariablePrepareToSleep(&memCtxState[procNumber].memctx_cv);
+	while (1)
+	{
+		proc = BackendPidGetProc(pid);
+		if (proc == NULL)
+			proc = AuxiliaryPidGetProc(pid);
+		if (proc == NULL)
+		{
+			ereport(WARNING,
+					(errmsg("PID %d is not a PostgreSQL server process", pid)));
+			dsa_detach(area);
+			PG_RETURN_NULL();
+		}
+#define MEMSTATS_WAIT_TIMEOUT 5000
+#define MAX_RETRIES 20
+		if (ConditionVariableTimedSleep(&memCtxState[procNumber].memctx_cv, MEMSTATS_WAIT_TIMEOUT,
+										WAIT_EVENT_MEM_CTX_PUBLISH))
+		{
+			ereport(LOG,
+					(errmsg("Wait for %d process to publish stats timed out, trying again", pid)));
+			if (num_retries > MAX_RETRIES)
+			{
+				dsa_detach(area);
+				PG_RETURN_NULL();
+			}
+			num_retries = num_retries + 1;
+		}
+
+		/*
+		 * We expect to come out of sleep when the requested process has
+		 * finished publishing the statistics, verified using the a valid dsa
+		 * pointer.
+		 *
+		 * Make sure that the information belongs to pid we requested
+		 * information for, Otherwise loop back and wait for the server
+		 * process to finish publishing statistics.
+		 */
+		LWLockAcquire(&memCtxState[procNumber].lw_lock, LW_EXCLUSIVE);
+
+		if (memCtxState[procNumber].proc_id == pid && DsaPointerIsValid(memCtxState[procNumber].memstats_dsa_pointer))
+			break;
+		else
+			LWLockRelease(&memCtxState[procNumber].lw_lock);
+	}
+	if (DsaPointerIsValid(memCtxState[procNumber].memstats_dsa_pointer))
+		memctx_info = (MemoryContextInfo *) dsa_get_address(area, memCtxState[procNumber].memstats_dsa_pointer);
+	/* Backend has finished publishing the stats, read them */
+	for (i = 0; i < memCtxState[procNumber].in_memory_stats; i++)
+	{
+		ArrayType  *path_array;
+		int			path_length;
+		Datum		values[PG_GET_BACKEND_MEMORY_CONTEXTS_COLS];
+		bool		nulls[PG_GET_BACKEND_MEMORY_CONTEXTS_COLS];
+
+		memset(values, 0, sizeof(values));
+		memset(nulls, 0, sizeof(nulls));
+
+		if (strlen(memctx_info[i].name) != 0)
+			values[0] = CStringGetTextDatum(memctx_info[i].name);
+		else
+			nulls[0] = true;
+		if (strlen(memctx_info[i].ident) != 0)
+			values[1] = CStringGetTextDatum(memctx_info[i].ident);
+		else
+			nulls[1] = true;
+
+		values[2] = CStringGetTextDatum(memctx_info[i].type);
+		path_length = memctx_info[i].path_length;
+		path_array = construct_array_builtin(memctx_info[i].path, path_length, INT4OID);
+		values[3] = PointerGetDatum(path_array);
+		values[4] = Int64GetDatum(memctx_info[i].totalspace);
+		values[5] = Int64GetDatum(memctx_info[i].nblocks);
+		values[6] = Int64GetDatum(memctx_info[i].freespace);
+		values[7] = Int64GetDatum(memctx_info[i].freechunks);
+		values[8] = Int64GetDatum(memctx_info[i].totalspace - memctx_info[i].freespace);
+		values[9] = Int32GetDatum(memctx_info[i].num_contexts);
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
+	}
+	/* If there are more contexts, display a cumulative total of those */
+	if (memCtxState[procNumber].total_stats > i)
+	{
+		Datum		values[PG_GET_BACKEND_MEMORY_CONTEXTS_COLS];
+		bool		nulls[PG_GET_BACKEND_MEMORY_CONTEXTS_COLS];
+
+		values[0] = CStringGetTextDatum(memctx_info[i].name);
+		nulls[1] = true;
+		nulls[2] = true;
+		nulls[3] = true;
+		values[4] = Int64GetDatum(memctx_info[i].totalspace);
+		values[5] = Int64GetDatum(memctx_info[i].nblocks);
+		values[6] = Int64GetDatum(memctx_info[i].freespace);
+		values[7] = Int64GetDatum(memctx_info[i].freechunks);
+		values[8] = Int64GetDatum(memctx_info[i].totalspace - memctx_info[i].freespace);
+		values[9] = Int32GetDatum(memctx_info[i].num_contexts);
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
+	}
+	LWLockRelease(&memCtxState[procNumber].lw_lock);
+	ConditionVariableCancelSleep();
+	dsa_detach(area);
+	PG_RETURN_NULL();
+}
+
+static Size
+MemCtxShmemSize(void)
+{
+	Size		size;
+	Size		TotalProcs = add_size(MaxBackends, add_size(NUM_AUXILIARY_PROCS, max_prepared_xacts));
+
+	size = TotalProcs * sizeof(MemoryContextState);
+	return size;
+}
+
+void
+MemCtxShmemInit(void)
+{
+	bool		found;
+	Size		TotalProcs = add_size(MaxBackends, add_size(NUM_AUXILIARY_PROCS, max_prepared_xacts));
+
+	memCtxState = (MemoryContextState *) ShmemInitStruct("MemoryContextState",
+														 MemCtxShmemSize(),
+														 &found);
+	if (!found)
+	{
+		for (int i = 0; i < TotalProcs; i++)
+		{
+			ConditionVariableInit(&memCtxState[i].memctx_cv);
+			LWLockInitialize(&memCtxState[i].lw_lock, LWLockNewTrancheId());
+			LWLockRegisterTranche(memCtxState[i].lw_lock.tranche, "mem_context_stats_reporting");
+			memCtxState[i].memstats_dsa_handle = DSA_HANDLE_INVALID;
+			memCtxState[i].memstats_dsa_pointer = InvalidDsaPointer;
+		}
+	}
+}
diff --git a/src/backend/utils/init/globals.c b/src/backend/utils/init/globals.c
index b844f9fdae..6bc253da5d 100644
--- a/src/backend/utils/init/globals.c
+++ b/src/backend/utils/init/globals.c
@@ -42,6 +42,7 @@ volatile sig_atomic_t IdleStatsUpdateTimeoutPending = false;
 volatile uint32 InterruptHoldoffCount = 0;
 volatile uint32 QueryCancelHoldoffCount = 0;
 volatile uint32 CritSectionCount = 0;
+volatile sig_atomic_t PublishMemoryContextPending = false;
 
 int			MyProcPid;
 pg_time_t	MyStartTime;
diff --git a/src/backend/utils/mmgr/mcxt.c b/src/backend/utils/mmgr/mcxt.c
index aa6da0d035..245aba5987 100644
--- a/src/backend/utils/mmgr/mcxt.c
+++ b/src/backend/utils/mmgr/mcxt.c
@@ -19,16 +19,22 @@
  *-------------------------------------------------------------------------
  */
 
+#include <math.h>
 #include "postgres.h"
 
 #include "mb/pg_wchar.h"
 #include "miscadmin.h"
+#include "nodes/pg_list.h"
+#include "storage/fd.h"
+#include "storage/lwlock.h"
+#include "storage/dsm.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 #include "utils/memdebug.h"
 #include "utils/memutils.h"
 #include "utils/memutils_internal.h"
 #include "utils/memutils_memorychunk.h"
 
-
 static void BogusFree(void *pointer);
 static void *BogusRealloc(void *pointer, Size size, int flags);
 static MemoryContext BogusGetChunkContext(void *pointer);
@@ -135,6 +141,12 @@ static const MemoryContextMethods mcxt_methods[] = {
 };
 
 #undef BOGUS_MCTX
+typedef enum PrintDetails
+{
+	PRINT_STATS_TO_STDERR = 0,
+	PRINT_STATS_TO_LOGS,
+	PRINT_STATS_NONE
+}			PrintDetails;
 
 /*
  * CurrentMemoryContext
@@ -162,10 +174,11 @@ static void MemoryContextCallResetCallbacks(MemoryContext context);
 static void MemoryContextStatsInternal(MemoryContext context, int level,
 									   int max_level, int max_children,
 									   MemoryContextCounters *totals,
-									   bool print_to_stderr);
+									   PrintDetails print_location, int *num_contexts);
 static void MemoryContextStatsPrint(MemoryContext context, void *passthru,
 									const char *stats_string,
 									bool print_to_stderr);
+static void PublishMemoryContext(MemoryContextInfo * memctx_infos, int curr_id, MemoryContext context, List *path, char *clipped_ident, MemoryContextCounters stat, int num_contexts);
 
 /*
  * You should not do memory allocations within a critical section, because
@@ -831,11 +844,19 @@ MemoryContextStatsDetail(MemoryContext context,
 						 bool print_to_stderr)
 {
 	MemoryContextCounters grand_totals;
+	int			num_contexts;
+	PrintDetails print_location;
 
 	memset(&grand_totals, 0, sizeof(grand_totals));
 
+	if (print_to_stderr)
+		print_location = PRINT_STATS_TO_STDERR;
+	else
+		print_location = PRINT_STATS_TO_LOGS;
+
+	/* num_contexts report number of contexts aggregated in the output */
 	MemoryContextStatsInternal(context, 0, max_level, max_children,
-							   &grand_totals, print_to_stderr);
+							   &grand_totals, print_location, &num_contexts);
 
 	if (print_to_stderr)
 		fprintf(stderr,
@@ -876,18 +897,43 @@ static void
 MemoryContextStatsInternal(MemoryContext context, int level,
 						   int max_level, int max_children,
 						   MemoryContextCounters *totals,
-						   bool print_to_stderr)
+						   PrintDetails print_location, int *num_contexts)
 {
 	MemoryContext child;
 	int			ichild;
+	bool		print_to_stderr = true;
 
 	Assert(MemoryContextIsValid(context));
 
-	/* Examine the context itself */
-	context->methods->stats(context,
-							MemoryContextStatsPrint,
-							&level,
-							totals, print_to_stderr);
+	if (print_location == PRINT_STATS_TO_STDERR)
+		print_to_stderr = true;
+	else if (print_location == PRINT_STATS_TO_LOGS)
+		print_to_stderr = false;
+
+	if (print_location != PRINT_STATS_NONE)
+	{
+		/* Examine the context itself */
+		context->methods->stats(context,
+								MemoryContextStatsPrint,
+								&level,
+								totals, print_to_stderr);
+	}
+	/* Do not print the statistics */
+
+	/*
+	 * print_to_stderr is a no-op if no statistics are going to be printed i.e
+	 * print_location == PRINT_STATS_NONE
+	 */
+	else
+	{
+		/* Examine the context itself */
+		context->methods->stats(context,
+								NULL,
+								NULL,
+								totals, print_to_stderr);
+	}
+	/* Increment the context count */
+	*num_contexts = *num_contexts + 1;
 
 	/*
 	 * Examine children.
@@ -907,7 +953,7 @@ MemoryContextStatsInternal(MemoryContext context, int level,
 			MemoryContextStatsInternal(child, level + 1,
 									   max_level, max_children,
 									   totals,
-									   print_to_stderr);
+									   print_location, num_contexts);
 		}
 	}
 
@@ -925,6 +971,7 @@ MemoryContextStatsInternal(MemoryContext context, int level,
 			ichild++;
 			child = MemoryContextTraverseNext(child, context);
 		}
+		*num_contexts = *num_contexts + ichild;
 
 		if (print_to_stderr)
 		{
@@ -939,7 +986,7 @@ MemoryContextStatsInternal(MemoryContext context, int level,
 					local_totals.freechunks,
 					local_totals.totalspace - local_totals.freespace);
 		}
-		else
+		else if (print_location != PRINT_STATS_NONE)
 			ereport(LOG_SERVER_ONLY,
 					(errhidestmt(true),
 					 errhidecontext(true),
@@ -1276,6 +1323,21 @@ HandleLogMemoryContextInterrupt(void)
 	/* latch will be set by procsignal_sigusr1_handler */
 }
 
+/*
+ * HandleGetMemoryContextInterrupt
+ *		Handle receipt of an interrupt indicating publishing of memory
+ *		contexts.
+ *
+ * All the actual work is deferred to ProcessLogMemoryContextInterrupt()
+ */
+void
+HandleGetMemoryContextInterrupt(void)
+{
+	InterruptPending = true;
+	PublishMemoryContextPending = true;
+	/* latch will be set by procsignal_sigusr1_handler */
+}
+
 /*
  * ProcessLogMemoryContextInterrupt
  * 		Perform logging of memory contexts of this backend process.
@@ -1313,6 +1375,346 @@ ProcessLogMemoryContextInterrupt(void)
 	MemoryContextStatsDetail(TopMemoryContext, 100, 100, false);
 }
 
+/*
+ * Run by each backend to publish their memory context
+ * statistics. It performs a breadth first search
+ * on the memory context tree, so that the parents
+ * get a chance to report stats before their children.
+ *
+ * Statistics are shared via dynamic shared memory which
+ * can hold statistics of approx 6700 contexts. Remaining
+ * contexts statistics is captured as a cumulative total.
+ */
+void
+ProcessGetMemoryContextInterrupt(void)
+{
+	/* Store the memory context details in shared memory */
+
+	List	   *contexts;
+
+	HASHCTL		ctl;
+	HTAB	   *context_id_lookup;
+	int			context_id = 0;
+	bool		found;
+	MemoryContext stat_cxt;
+	MemoryContextInfo *meminfo;
+	bool		get_summary = false;
+	dsa_area   *area;
+	int			num_stats;
+	int			idx = MyProcNumber;
+	int			stats_count = 0;
+	MemoryContextCounters stat;
+
+	PublishMemoryContextPending = false;
+
+	/*
+	 * The hash table is used for constructing "path" column of
+	 * pg_get_process_memory_context is view, similar to its local backend
+	 * couterpart.
+	 */
+
+	/*
+	 * Make a new context that will contain the hash table, to ease the
+	 * cleanup
+	 */
+
+	stat_cxt = AllocSetContextCreate(CurrentMemoryContext,
+									 "Memory context statistics",
+									 ALLOCSET_DEFAULT_SIZES);
+
+	ctl.keysize = sizeof(MemoryContext);
+	ctl.entrysize = sizeof(MemoryContextId);
+	ctl.hcxt = stat_cxt;
+
+	context_id_lookup = hash_create("pg_get_remote_backend_memory_contexts",
+									256,
+									&ctl,
+									HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	contexts = list_make1(TopMemoryContext);
+
+	/* Compute the number of stats that can fit in the DSM seg */
+	num_stats = floor(16 * DSA_DEFAULT_INIT_SEGMENT_SIZE / sizeof(MemoryContextInfo));
+
+	/*
+	 * Traverse the memory context tree to find total number of contexts. If
+	 * summary is requested find the total number of contexts at level 1 and
+	 * 2.
+	 */
+	foreach_ptr(MemoryContextData, cur, contexts)
+	{
+		MemoryContextId *entry;
+
+		entry = (MemoryContextId *) hash_search(context_id_lookup, &cur,
+												HASH_ENTER, &found);
+		stats_count = stats_count + 1;
+		/* context id starts with 1 */
+		entry->context_id = stats_count;
+
+		/* Append the children of the current context to the main list */
+		for (MemoryContext c = cur->firstchild; c != NULL; c = c->nextchild)
+		{
+			if (get_summary)
+			{
+				entry = (MemoryContextId *) hash_search(context_id_lookup, &c,
+														HASH_ENTER, &found);
+				stats_count = stats_count + 1;
+				entry->context_id = stats_count;
+			}
+			contexts = lappend(contexts, c);
+		}
+		/* In summary only the first level contexts are displayed */
+		if (get_summary)
+			break;
+	}
+
+	/*
+	 * Allocate memory in this process's dsa for storing statistics of the the
+	 * memory contexts upto num_stats, for contexts that don't fit in the DSA
+	 * segment, a cumulative total is written as the last record in the DSA
+	 * segment.
+	 */
+	stats_count = stats_count > num_stats ? num_stats : stats_count;
+
+	/* Attach to DSA segment */
+	LWLockAcquire(&memCtxState[idx].lw_lock, LW_EXCLUSIVE);
+	area = dsa_attach(memCtxState[idx].memstats_dsa_handle);
+	memCtxState[idx].proc_id = MyProcPid;
+	get_summary = memCtxState[idx].get_summary;
+
+	/* Free the memory allocated previously by the same process */
+	if (DsaPointerIsValid(memCtxState[idx].memstats_dsa_pointer))
+	{
+		dsa_free(area, memCtxState[idx].memstats_dsa_pointer);
+		memCtxState[idx].memstats_dsa_pointer = InvalidDsaPointer;
+	}
+	memCtxState[idx].memstats_dsa_pointer = dsa_allocate0(area, stats_count * sizeof(MemoryContextInfo));
+	meminfo = (MemoryContextInfo *) dsa_get_address(area, memCtxState[idx].memstats_dsa_pointer);
+
+	if (get_summary)
+	{
+		int			ctx_id = 0;
+		List	   *path = NIL;
+
+		/* Copy TopMemoryContext statistics to DSA */
+		path = lcons_int(1, path);
+		memset(&stat, 0, sizeof(stat));
+		(*TopMemoryContext->methods->stats) (TopMemoryContext, NULL, NULL, &stat, true);
+		PublishMemoryContext(meminfo, ctx_id, TopMemoryContext, path, NULL, stat, 1);
+		ctx_id = ctx_id + 1;
+
+		/*
+		 * Copy statistics for each of TopMemoryContexts children(XXX. Make it
+		 * capped at 100). This includes statistics of all of their children
+		 * upto level 100
+		 */
+		for (MemoryContext c = TopMemoryContext->firstchild; c != NULL; c = c->nextchild)
+		{
+			MemoryContextCounters grand_totals;
+			int			num_contexts = 0;
+			char		clipped_ident[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+
+			path = NIL;
+			memset(&grand_totals, 0, sizeof(grand_totals));
+
+			MemoryContextStatsInternal(c, 0, 100, 100, &grand_totals, PRINT_STATS_NONE, &num_contexts);
+
+			/*
+			 * Figure out the transient context_id of this context and each of
+			 * its ancestors.
+			 */
+			for (MemoryContext cur_context = c; cur_context != NULL; cur_context = cur_context->parent)
+			{
+				MemoryContextId *cur_entry;
+
+				cur_entry = hash_search(context_id_lookup, &cur_context, HASH_FIND, &found);
+
+				if (!found)
+				{
+					elog(LOG, "hash table corrupted, can't construct path value");
+					break;
+				}
+				path = lcons_int(cur_entry->context_id, path);
+			}
+			/* Trim and copy the identifier if it is not set to NULL */
+			if (c->ident != NULL)
+			{
+				int			idlen = strlen(c->ident);
+
+				/*
+				 * Some identifiers such as SQL query string can be very long,
+				 * truncate oversize identifiers.
+				 */
+				if (idlen >= MEMORY_CONTEXT_IDENT_DISPLAY_SIZE)
+					idlen = pg_mbcliplen(c->ident, idlen, MEMORY_CONTEXT_IDENT_DISPLAY_SIZE - 1);
+
+				memcpy(clipped_ident, c->ident, idlen);
+				clipped_ident[idlen] = '\0';
+			}
+			PublishMemoryContext(meminfo, ctx_id, c, path, (c->ident != NULL ? clipped_ident : NULL), grand_totals, num_contexts);
+			ctx_id = ctx_id + 1;
+		}
+		/* For summary mode, total_stats and in_memory_stats remain the same */
+		memCtxState[idx].in_memory_stats = ctx_id;
+		memCtxState[idx].total_stats = ctx_id;
+		goto cleanup;
+	}
+
+	foreach_ptr(MemoryContextData, cur, contexts)
+	{
+		List	   *path = NIL;
+		char		clipped_ident[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+
+		/*
+		 * Figure out the transient context_id of this context and each of its
+		 * ancestors.
+		 */
+		for (MemoryContext cur_context = cur; cur_context != NULL; cur_context = cur_context->parent)
+		{
+			MemoryContextId *cur_entry;
+
+			cur_entry = hash_search(context_id_lookup, &cur_context, HASH_FIND, &found);
+
+			if (!found)
+			{
+				elog(LOG, "hash table corrupted, can't construct path value");
+				break;
+			}
+			path = lcons_int(cur_entry->context_id, path);
+		}
+		/* Trim and copy the identifier if it is not set to NULL */
+		if (cur->ident != NULL)
+		{
+			int			idlen = strlen(cur->ident);
+
+			/*
+			 * Some identifiers such as SQL query string can be very long,
+			 * truncate oversize identifiers.
+			 */
+			if (idlen >= MEMORY_CONTEXT_IDENT_DISPLAY_SIZE)
+				idlen = pg_mbcliplen(cur->ident, idlen, MEMORY_CONTEXT_IDENT_DISPLAY_SIZE - 1);
+
+			memcpy(clipped_ident, cur->ident, idlen);
+			clipped_ident[idlen] = '\0';
+		}
+		if (context_id <= (num_stats - 2))
+		{
+			/* Examine the context stats */
+			memset(&stat, 0, sizeof(stat));
+			(*cur->methods->stats) (cur, NULL, NULL, &stat, true);
+			/* Copy statistics to DSM memory */
+			PublishMemoryContext(meminfo, context_id, cur, path, (cur->ident != NULL ? clipped_ident : NULL), stat, 1);
+		}
+		else
+		{
+			/* Examine the context stats */
+			memset(&stat, 0, sizeof(stat));
+			(*cur->methods->stats) (cur, NULL, NULL, &stat, true);
+
+			meminfo[num_stats - 1].totalspace += stat.totalspace;
+			meminfo[num_stats - 1].nblocks += stat.nblocks;
+			meminfo[num_stats - 1].freespace += stat.freespace;
+			meminfo[num_stats - 1].freechunks += stat.freechunks;
+		}
+
+		/*
+		 * DSA max limit is reached, write total of the remaining statistics.
+		 */
+		if (context_id == (num_stats - 2) && context_id < (stats_count - 1))
+		{
+			memCtxState[idx].in_memory_stats = context_id + 1;
+			strncpy(meminfo[num_stats - 1].name, "Remaining Totals", 16);
+		}
+		context_id++;
+	}
+	if (context_id < (num_stats - 2))
+	{
+		memCtxState[idx].in_memory_stats = context_id;
+	}
+	/* Report number of aggregated memory contexts */
+	else
+	{
+		meminfo[num_stats - 1].num_contexts = context_id - memCtxState[idx].in_memory_stats;
+	}
+	memCtxState[idx].total_stats = context_id;
+cleanup:
+
+	/*
+	 * Signal all the waiting client backends after setting the exit condition
+	 * flag
+	 */
+	LWLockRelease(&memCtxState[idx].lw_lock);
+	ConditionVariableBroadcast(&memCtxState[idx].memctx_cv);
+	/* Delete the hash table memory context */
+	MemoryContextDelete(stat_cxt);
+
+	dsa_detach(area);
+}
+
+static void
+PublishMemoryContext(MemoryContextInfo * memctx_info, int curr_id, MemoryContext context, List *path, char *clipped_ident, MemoryContextCounters stat, int num_contexts)
+{
+	char	   *type;
+
+	if (context->name != NULL)
+	{
+		Assert(strlen(context->name) < MEMORY_CONTEXT_IDENT_DISPLAY_SIZE);
+		strncpy(memctx_info[curr_id].name, context->name, strlen(context->name));
+	}
+	else
+		memctx_info[curr_id].name[0] = '\0';
+
+	if (clipped_ident != NULL)
+	{
+		/*
+		 * To be consistent with logging output, we label dynahash contexts
+		 * with just the hash table name as with MemoryContextStatsPrint().
+		 */
+		if (!strncmp(context->name, "dynahash", 8))
+		{
+			strncpy(memctx_info[curr_id].name, clipped_ident, strlen(clipped_ident));
+			memctx_info[curr_id].ident[0] = '\0';
+		}
+		else
+			strncpy(memctx_info[curr_id].ident, clipped_ident, strlen(clipped_ident));
+	}
+	else
+		memctx_info[curr_id].ident[0] = '\0';
+
+	memctx_info[curr_id].path_length = list_length(path);
+	foreach_int(i, path)
+		memctx_info[curr_id].path[foreach_current_index(i)] = Int32GetDatum(i);
+
+	switch (context->type)
+	{
+		case T_AllocSetContext:
+			type = "AllocSet";
+			strncpy(memctx_info[curr_id].type, type, strlen(type));
+			break;
+		case T_GenerationContext:
+			type = "Generation";
+			strncpy(memctx_info[curr_id].type, type, strlen(type));
+			break;
+		case T_SlabContext:
+			type = "Slab";
+			strncpy(memctx_info[curr_id].type, type, strlen(type));
+			break;
+		case T_BumpContext:
+			type = "Bump";
+			strncpy(memctx_info[curr_id].type, type, strlen(type));
+			break;
+		default:
+			type = "???";
+			strncpy(memctx_info[curr_id].type, type, strlen(type));
+			break;
+	}
+	memctx_info[curr_id].totalspace = stat.totalspace;
+	memctx_info[curr_id].nblocks = stat.nblocks;
+	memctx_info[curr_id].freespace = stat.freespace;
+	memctx_info[curr_id].freechunks = stat.freechunks;
+	memctx_info[curr_id].num_contexts = num_contexts;
+}
+
 void *
 palloc(Size size)
 {
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index b37e8a6f88..4d6ae0728a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -8449,6 +8449,16 @@
   prorettype => 'bool', proargtypes => 'int4',
   prosrc => 'pg_log_backend_memory_contexts' },
 
+# publishing memory contexts of the specified postgres process
+{ oid => '2173', descr => 'publish memory contexts of the specified backend',
+  proname => 'pg_get_process_memory_contexts', provolatile => 'v',
+  prorows => '100', proretset => 't', proparallel => 'r',
+  prorettype => 'record', proargtypes => 'int4 bool',
+  proallargtypes => '{int4,bool,text,text,text,_int4,int8,int8,int8,int8,int8,int4}',
+  proargmodes => '{i,i,o,o,o,o,o,o,o,o,o,o}',
+  proargnames => '{oid, summary, name, ident, type, path, total_bytes, total_nblocks, free_bytes, free_chunks, used_bytes, num_agg_contexts}',
+  prosrc => 'pg_get_process_memory_contexts' },
+
 # non-persistent series generator
 { oid => '1066', descr => 'non-persistent series generator',
   proname => 'generate_series', prorows => '1000',
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index d016a9c924..fc75ea143c 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -96,6 +96,7 @@ extern PGDLLIMPORT volatile sig_atomic_t IdleSessionTimeoutPending;
 extern PGDLLIMPORT volatile sig_atomic_t ProcSignalBarrierPending;
 extern PGDLLIMPORT volatile sig_atomic_t LogMemoryContextPending;
 extern PGDLLIMPORT volatile sig_atomic_t IdleStatsUpdateTimeoutPending;
+extern PGDLLIMPORT volatile sig_atomic_t PublishMemoryContextPending;
 
 extern PGDLLIMPORT volatile sig_atomic_t CheckClientConnectionPending;
 extern PGDLLIMPORT volatile sig_atomic_t ClientConnectionLost;
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index 022fd8ed93..477ab99338 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -35,6 +35,7 @@ typedef enum
 	PROCSIG_WALSND_INIT_STOPPING,	/* ask walsenders to prepare for shutdown  */
 	PROCSIG_BARRIER,			/* global barrier interrupt  */
 	PROCSIG_LOG_MEMORY_CONTEXT, /* ask backend to log the memory contexts */
+	PROCSIG_GET_MEMORY_CONTEXT, /* ask backend to log the memory contexts */
 	PROCSIG_PARALLEL_APPLY_MESSAGE, /* Message from parallel apply workers */
 
 	/* Recovery conflict reasons */
diff --git a/src/include/utils/memutils.h b/src/include/utils/memutils.h
index 8abc26abce..9fac394aad 100644
--- a/src/include/utils/memutils.h
+++ b/src/include/utils/memutils.h
@@ -18,6 +18,9 @@
 #define MEMUTILS_H
 
 #include "nodes/memnodes.h"
+#include "storage/condition_variable.h"
+#include "storage/lmgr.h"
+#include "utils/dsa.h"
 
 
 /*
@@ -48,7 +51,11 @@
 
 #define AllocHugeSizeIsValid(size)	((Size) (size) <= MaxAllocHugeSize)
 
+#define MEMORY_CONTEXT_IDENT_DISPLAY_SIZE	1024
 
+#define MEM_CONTEXT_SHMEM_STATS_SIZE	30
+#define MEM_CONTEXT_MAX_LEVEL	64
+#define MAX_TYPE_STRING_LENGTH	64
 /*
  * Standard top-level memory contexts.
  *
@@ -115,6 +122,50 @@ extern MemoryContext AllocSetContextCreateInternal(MemoryContext parent,
 												   Size initBlockSize,
 												   Size maxBlockSize);
 
+/* Dynamic shared memory state for Memory Context Statistics reporting */
+typedef struct MemoryContextInfo
+{
+	char		name[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+	char		ident[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+	Datum		path[MEM_CONTEXT_MAX_LEVEL];
+	char		type[MAX_TYPE_STRING_LENGTH];
+	int			path_length;
+	int64		totalspace;
+	int64		nblocks;
+	int64		freespace;
+	int64		freechunks;
+	int			num_contexts;
+}			MemoryContextInfo;
+
+typedef struct MemoryContextState
+{
+	ConditionVariable memctx_cv;
+	LWLock		lw_lock;
+	int			proc_id;
+	int			in_memory_stats;
+	int			total_stats;
+	bool		get_summary;
+	dsa_handle	memstats_dsa_handle;
+	dsa_pointer memstats_dsa_pointer;
+
+}			MemoryContextState;
+
+/*
+ * MemoryContextId
+ *		Used for storage of transient identifiers for
+ *		pg_get_backend_memory_contexts.
+ */
+typedef struct MemoryContextId
+{
+	MemoryContext context;
+	int			context_id;
+}			MemoryContextId;
+
+extern PGDLLIMPORT MemoryContextState * memCtxState;
+extern void ProcessGetMemoryContextInterrupt(void);
+extern void HandleGetMemoryContextInterrupt(void);
+extern void MemCtxShmemInit(void);
+
 /*
  * This wrapper macro exists to check for non-constant strings used as context
  * names; that's no longer supported.  (Use MemoryContextSetIdentifier if you
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 91089ac215..5e3382132c 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -223,3 +223,15 @@ select count(distinct utc_offset) >= 24 as ok from pg_timezone_abbrevs;
  t
 (1 row)
 
+DO $$
+DECLARE
+    checkpointer_pid int;
+    r RECORD;
+BEGIN
+        SELECT pid from pg_stat_activity where backend_type='checkpointer' INTO checkpointer_pid;
+
+        select type, name, ident
+        from pg_get_process_memory_contexts(checkpointer_pid, false) where path = '{0}' into r;
+	RAISE NOTICE '%', r;
+END $$;
+NOTICE:  (AllocSet,TopMemoryContext,)
diff --git a/src/test/regress/sql/sysviews.sql b/src/test/regress/sql/sysviews.sql
index b2a7923754..f3127fea40 100644
--- a/src/test/regress/sql/sysviews.sql
+++ b/src/test/regress/sql/sysviews.sql
@@ -98,3 +98,15 @@ set timezone_abbreviations = 'Australia';
 select count(distinct utc_offset) >= 24 as ok from pg_timezone_abbrevs;
 set timezone_abbreviations = 'India';
 select count(distinct utc_offset) >= 24 as ok from pg_timezone_abbrevs;
+
+DO $$
+DECLARE
+    checkpointer_pid int;
+    r RECORD;
+BEGIN
+        SELECT pid from pg_stat_activity where backend_type='checkpointer' INTO checkpointer_pid;
+
+        select type, name, ident
+        from pg_get_process_memory_contexts(checkpointer_pid, false) where path = '{0}' into r;
+	RAISE NOTICE '%', r;
+END $$;
-- 
2.34.1

