From 1e174f0dd888d9b89ecefd593cba648db1462086 Mon Sep 17 00:00:00 2001
From: Rahila Syed <rahilasyed.90@gmail.com>
Date: Mon, 3 Feb 2025 15:37:17 +0530
Subject: [PATCH 2/2] Function to report memory context statistics

This function sends a signal to a backend to publish
statistics of all its memory contexts. Signal handler
running in the backend process, sets a flag, which causes
it to copy its MemoryContextStats to a DSA, during the
next call to CHECK_FOR_INTERRUPTS().
If there are more statistics than that fit in 16MB, 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 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.
If get_summary is true return statistics of all children
of TopMemoryContext with aggregated statistics of their
children.
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. Thus 1 DSA area is created per backend
that is publishing the statistics.
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                        |  31 ++
 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             | 425 ++++++++++++++++--
 src/backend/utils/init/globals.c              |   1 +
 src/backend/utils/mmgr/mcxt.c                 | 383 +++++++++++++++-
 src/include/catalog/pg_proc.dat               |  10 +
 src/include/miscadmin.h                       |   1 +
 src/include/storage/procsignal.h              |   1 +
 src/include/utils/memutils.h                  |  59 +++
 src/test/regress/expected/sysviews.out        |  14 +
 src/test/regress/sql/sysviews.sql             |  14 +
 src/tools/pgindent/typedefs.list              |   2 +
 21 files changed, 942 insertions(+), 32 deletions(-)

diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 7efc81936a..9d243df4e1 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -28409,6 +28409,37 @@ 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, statistics for memory contexts at
+        levels 1 and 2 are displayed, with level 1 representing the root node
+        (i.e., TopMemoryContext). Each level 2 context's statistics represent
+        an aggregate of all its child contexts' statistics, with
+        num_agg_contexts indicating the number of these aggregated 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 0ab921a169..0c693cfa48 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -778,6 +778,10 @@ HandleAutoVacLauncherInterrupts(void)
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
 
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
+
 	/* Process sinval catchup interrupts that happened while sleeping */
 	ProcessCatchupInterrupt();
 }
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index b94f9cdff2..33c3c2d9c6 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -661,6 +661,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 5655348a2e..70587771d3 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 e199f07162..3674b5b7b6 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -159,6 +159,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..7c3f9a4f68 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];
@@ -143,24 +140,7 @@ PutMemoryContextsStatsTupleStore(Tuplestorestate *tupstore,
 	else
 		nulls[1] = true;
 
-	switch (context->type)
-	{
-		case T_AllocSetContext:
-			type = "AllocSet";
-			break;
-		case T_GenerationContext:
-			type = "Generation";
-			break;
-		case T_SlabContext:
-			type = "Slab";
-			break;
-		case T_BumpContext:
-			type = "Bump";
-			break;
-		default:
-			type = "???";
-			break;
-	}
+	type = AssignContextType(context->type);
 
 	values[2] = CStringGetTextDatum(type);
 	values[3] = Int32GetDatum(list_length(path));	/* level */
@@ -175,6 +155,32 @@ PutMemoryContextsStatsTupleStore(Tuplestorestate *tupstore,
 	list_free(path);
 }
 
+const char *
+AssignContextType(NodeTag type)
+{
+	const char *context_type;
+
+	switch (type)
+	{
+		case T_AllocSetContext:
+			context_type = "AllocSet";
+			break;
+		case T_GenerationContext:
+			context_type = "Generation";
+			break;
+		case T_SlabContext:
+			context_type = "Slab";
+			break;
+		case T_BumpContext:
+			context_type = "Bump";
+			break;
+		default:
+			context_type = "???";
+			break;
+	}
+	return (context_type);
+}
+
 /*
  * pg_get_backend_memory_contexts
  *		SQL SRF showing backend memory context.
@@ -249,7 +255,7 @@ pg_get_backend_memory_contexts(PG_FUNCTION_ARGS)
 
 /*
  * pg_log_backend_memory_contexts
- *		Signal a backend or an auxiliary process to log its memory contexts.
+ *	Signal a backend or an auxiliary process to log its memory contexts.
  *
  * By default, only superusers are allowed to signal to log the memory
  * contexts because allowing any users to issue this request at an unbounded
@@ -305,3 +311,360 @@ 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,
+ *		wait for the results and display them.
+ *
+ * By default, only superusers are allowed to signal to return the memory
+ * contexts because allowing any users to issue this request at an unbounded
+ * rate would cause lots of requests to be sent and which can lead to denial of
+ * service. Additional roles can be permitted with GRANT.
+ *
+ * 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 details
+ * to a dynamic shared memory space.
+ *
+ * The shared memory buffer has a limited size - it the process has too many
+ * memory contexts, the memory contexts into that do not fit are summarized
+ * and represented as cumulative total at the end of the buffer.
+ *
+ * After sending the signal, wait on a condition variable. The publishing
+ * backend, after copying the data to shared memory, sends signal on that
+ * condition variable.
+ * Once condition variable comes out of sleep, check if the memory context
+ * information is available for read and display.
+ *
+ * If the publishing backend does not respond before the condition variable
+ * times out, which is set to MEMSTATS_WAIT_TIMEOUT, retry for MAX_RETRIES
+ * number of times before giving up and returning without statistics.
+ */
+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;
+	MemoryContextEntry *memctx_info;
+	int			num_retries = 0;
+	TimestampTz curr_timestamp;
+
+	/*
+	 * 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")));
+
+	InitMaterializedSRF(fcinfo, 0);
+
+	/*
+	 * 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();
+	}
+
+	/*
+	 * Check if the another request is not yet addressed by the process as
+	 * that may result in the current request to timeout.
+	 */
+	LWLockAcquire(&memCtxState[procNumber].lw_lock, LW_EXCLUSIVE);
+	if (memCtxState[procNumber].request_pending == true)
+	{
+		LWLockRelease(&memCtxState[procNumber].lw_lock);
+		ereport(WARNING,
+				(errmsg("cannot process the request at the moment"),
+				 errhint("Another request is pending, try again")));
+		PG_RETURN_NULL();
+	}
+	memCtxState[procNumber].request_pending = true;
+	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)
+	{
+
+		LWLockRelease(&memCtxState[procNumber].lw_lock);
+		area = dsa_create_ext(memCtxState[procNumber].lw_lock.tranche,
+							  DSA_DEFAULT_INIT_SEGMENT_SIZE,
+							  MAX_NUM_DEFAULT_SEGMENTS *
+							  DSA_DEFAULT_INIT_SEGMENT_SIZE);
+		handle = dsa_get_handle(area);
+
+		/*
+		 * Pin the dsa area even if the creating backend exits, this is to
+		 * make sure the area remains attachable even if current client exits
+		 */
+		dsa_pin(area);
+		/* Set the handle in shared memory */
+		LWLockAcquire(&memCtxState[procNumber].lw_lock, LW_EXCLUSIVE);
+		memCtxState[procNumber].memstats_dsa_handle = handle;
+		LWLockRelease(&memCtxState[procNumber].lw_lock);
+	}
+	else
+	{
+		LWLockRelease(&memCtxState[procNumber].lw_lock);
+		area = dsa_attach(memCtxState[procNumber].memstats_dsa_handle);
+	}
+
+	curr_timestamp = GetCurrentTimestamp();
+
+	/*
+	 * Send a signal to the auxiliary process, informing it we want it to
+	 * produce information about memory contexts.
+	 */
+	if (SendProcSignal(pid, PROCSIG_GET_MEMORY_CONTEXT, procNumber) < 0)
+	{
+		ereport(WARNING,
+				(errmsg("could not send signal to process %d: %m", pid)));
+
+		goto end;
+	}
+
+	/*
+	 * Wait for a backend to publish stats, indicated by a valid dsa pointer
+	 * set by the backend. A dsa pointer could be valid if statitics have
+	 * previously been published by the backend. In which case, check if
+	 * statistics are not older than curr_timestamp, if they are wait for
+	 * newer statistics.
+	 */
+	while (1)
+	{
+		long		msecs;
+
+		/*
+		 * We expect to come out of sleep when the requested process has
+		 * finished publishing the statistics, verified using the 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);
+		msecs =
+			TimestampDifferenceMilliseconds(curr_timestamp,
+											memCtxState[procNumber].stats_timestamp);
+
+		/*
+		 * Note in procnumber.h file says that a procNumber can be re-used for
+		 * a different backend immediately after a backend exits. In case an
+		 * old process' data was there and not updated by the current process
+		 * in the slot identified by the procNumber, the pid of the requested
+		 * process and the proc_id might not match.
+		 */
+		if (memCtxState[procNumber].proc_id == pid)
+		{
+			/*
+			 * Break if the latest stats have been read, indicated by
+			 * statistics timestamp being newer than the current request
+			 * timestamp.
+			 */
+			if (DsaPointerIsValid(memCtxState[procNumber].memstats_dsa_pointer)
+				&& msecs > 0)
+				break;
+
+		}
+		LWLockRelease(&memCtxState[procNumber].lw_lock);
+
+		/*
+		 * Recheck the state of the backend before sleeping on the condition
+		 * variable
+		 */
+		proc = BackendPidGetProc(pid);
+
+#define MEMSTATS_WAIT_TIMEOUT 5000
+#define MAX_RETRIES 20
+		if (proc == NULL)
+			proc = AuxiliaryPidGetProc(pid);
+		if (proc == NULL)
+		{
+			ereport(WARNING,
+					(errmsg("PID %d is not a PostgreSQL server process",
+							pid)));
+			goto end;
+		}
+		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)
+				goto end;
+			num_retries = num_retries + 1;
+		}
+
+	}
+
+	/* We should land here only with a valid memstats_dsa_pointer */
+	Assert(DsaPointerIsValid(memCtxState[procNumber].memstats_dsa_pointer));
+	memctx_info = (MemoryContextEntry *) dsa_get_address(area,
+														 memCtxState[procNumber].memstats_dsa_pointer);
+
+	/*
+	 * Backend has finished publishing the stats, read them
+	 *
+	 * Read statistics of top level 1 and 2 contexts, if get_summary is true.
+	 */
+	for (i = 0; i < memCtxState[procNumber].num_individual_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_agg_stats);
+		values[10] = TimestampTzGetDatum(memCtxState[procNumber].stats_timestamp);
+
+		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_agg_stats);
+
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
+	}
+
+	LWLockRelease(&memCtxState[procNumber].lw_lock);
+
+	ConditionVariableCancelSleep();
+
+end:
+	dsa_detach(area);
+	PG_RETURN_NULL();
+}
+
+/*
+ * Shared memory sizing for reporting memory context information.
+ */
+static Size
+MemCtxShmemSize(void)
+{
+	Size		TotalProcs =
+		add_size(MaxBackends, add_size(NUM_AUXILIARY_PROCS, max_prepared_xacts));
+
+	return mul_size(TotalProcs, sizeof(MemoryContextState));
+}
+
+/*
+ * Init shared memory for reporting memory context information.
+ */
+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 (!IsUnderPostmaster)
+	{
+		Assert(!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;
+			memCtxState[i].request_pending = false;
+		}
+	}
+	else
+	{
+		Assert(found);
+	}
+}
diff --git a/src/backend/utils/init/globals.c b/src/backend/utils/init/globals.c
index b844f9fdae..13938ccb0f 100644
--- a/src/backend/utils/init/globals.c
+++ b/src/backend/utils/init/globals.c
@@ -38,6 +38,7 @@ volatile sig_atomic_t TransactionTimeoutPending = false;
 volatile sig_atomic_t IdleSessionTimeoutPending = false;
 volatile sig_atomic_t ProcSignalBarrierPending = false;
 volatile sig_atomic_t LogMemoryContextPending = false;
+volatile sig_atomic_t PublishMemoryContextPending = false;
 volatile sig_atomic_t IdleStatsUpdateTimeoutPending = false;
 volatile uint32 InterruptHoldoffCount = 0;
 volatile uint32 QueryCancelHoldoffCount = 0;
diff --git a/src/backend/utils/mmgr/mcxt.c b/src/backend/utils/mmgr/mcxt.c
index 946a3731fd..aa030bddaf 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);
@@ -164,6 +170,7 @@ MemoryContext CacheMemoryContext = NULL;
 MemoryContext MessageContext = NULL;
 MemoryContext TopTransactionContext = NULL;
 MemoryContext CurTransactionContext = NULL;
+static dsa_area *area = NULL;
 
 /* This is a transient link to the active portal's memory context: */
 MemoryContext PortalContext = NULL;
@@ -177,6 +184,16 @@ static void MemoryContextStatsInternal(MemoryContext context, int level,
 static void MemoryContextStatsPrint(MemoryContext context, void *passthru,
 									const char *stats_string,
 									bool print_to_stderr);
+static void PublishMemoryContext(MemoryContextEntry *memctx_infos,
+								 int curr_id, MemoryContext context,
+								 List *path,
+								 MemoryContextCounters stat,
+								 int num_contexts);
+static void compute_num_of_contexts(List *contexts, HTAB *context_id_lookup,
+									int *stats_count,
+									bool get_summary);
+static List *compute_context_path(MemoryContext c, HTAB *context_id_lookup);
+
 
 /*
  * You should not do memory allocations within a critical section, because
@@ -1321,6 +1338,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.
@@ -1358,6 +1390,355 @@ ProcessLogMemoryContextInterrupt(void)
 	MemoryContextStatsDetail(TopMemoryContext, 100, 100, false);
 }
 
+/*
+ * ProcessGetMemoryContextInterrupt
+ *		Generate information about memory contexts used by the process.
+ *
+ * Performs a breadth first search on the memory context tree, so that the
+ * parents get a chance to report stats before their children.
+ *
+ * Statistics for individual contexts are shared via dynamic shared memory.
+ * The statistics for contexts that do not fit in the allocated size of the DSA,
+ * are captured as a cumulative total.
+ *
+ * If get_summary is true, we traversse the memory context tree recursively to
+ * cover all the children of a parent context to be able to display a cumulative
+ * total of memory consumption by a parent.
+ */
+void
+ProcessGetMemoryContextInterrupt(void)
+{
+	List	   *contexts;
+
+	HASHCTL		ctl;
+	HTAB	   *context_id_lookup;
+	int			context_id = 0;
+	MemoryContext stat_cxt;
+	MemoryContextEntry *meminfo;
+	bool		get_summary = false;
+
+	/* dsa_area   *area = NULL; */
+	int			max_stats;
+	int			idx = MyProcNumber;
+	int			stats_count = 0;
+	MemoryContextCounters stat;
+
+	check_stack_depth();
+	PublishMemoryContextPending = false;
+	LWLockAcquire(&memCtxState[idx].lw_lock, LW_EXCLUSIVE);
+	memCtxState[idx].request_pending = false;
+	LWLockRelease(&memCtxState[idx].lw_lock);
+
+	/*
+	 * Make a new context that will contain the hash table, to ease the
+	 * cleanup.
+	 */
+	stat_cxt = AllocSetContextCreate(CurrentMemoryContext,
+									 "Memory context statistics",
+									 ALLOCSET_DEFAULT_SIZES);
+
+	/*
+	 * The hash table used for constructing "path" column of the view, similar
+	 * to its local backend counterpart.
+	 */
+	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);
+
+	/* List of contexts to process in the next round - start at the top. */
+	contexts = list_make1(TopMemoryContext);
+
+	/* Compute the number of stats that can fit in the DSM seg */
+	max_stats = (MAX_NUM_DEFAULT_SEGMENTS * DSA_DEFAULT_INIT_SEGMENT_SIZE)
+		/ sizeof(MemoryContextEntry);
+
+	LWLockAcquire(&memCtxState[idx].lw_lock, LW_EXCLUSIVE);
+	get_summary = memCtxState[idx].get_summary;
+	LWLockRelease(&memCtxState[idx].lw_lock);
+
+	/*
+	 * Traverse the memory context tree to find total number of contexts. If
+	 * summary is requested report the total number of contexts at level 1 and
+	 * 2 from the top.
+	 */
+	compute_num_of_contexts(contexts, context_id_lookup, &stats_count,
+							get_summary);
+
+	/*
+	 * Allocate memory in this process's dsa for storing statistics of the the
+	 * memory contexts upto max_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 > max_stats) ? max_stats : stats_count;
+
+	/* Attach to DSA segment */
+	LWLockAcquire(&memCtxState[idx].lw_lock, LW_EXCLUSIVE);
+	if (area == NULL)
+	{
+		MemoryContext oldcontext = CurrentMemoryContext;
+
+		MemoryContextSwitchTo(TopMemoryContext);
+		area = dsa_attach(memCtxState[idx].memstats_dsa_handle);
+		dsa_pin_mapping(area);
+		MemoryContextSwitchTo(oldcontext);
+	}
+	memCtxState[idx].proc_id = MyProcPid;
+
+	/* 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(MemoryContextEntry));
+	meminfo = (MemoryContextEntry *) dsa_get_address(area,
+													 memCtxState[idx].memstats_dsa_pointer);
+
+	if (get_summary)
+	{
+		int			ctx_id = 0;
+		List	   *path = NIL;
+
+		/* Copy TopMemoryContext statistics to DSA */
+		memset(&stat, 0, sizeof(stat));
+		(*TopMemoryContext->methods->stats) (TopMemoryContext, NULL, NULL,
+											 &stat, true);
+		path = lcons_int(1, path);
+		PublishMemoryContext(meminfo, ctx_id, TopMemoryContext, path, 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;
+
+			path = NIL;
+			memset(&grand_totals, 0, sizeof(grand_totals));
+
+			MemoryContextStatsInternal(c, 0, 100, 100, &grand_totals,
+									   PRINT_STATS_NONE, &num_contexts);
+
+			path = compute_context_path(c, context_id_lookup);
+
+			PublishMemoryContext(meminfo, ctx_id, c, path,
+								 grand_totals, num_contexts);
+			ctx_id = ctx_id + 1;
+		}
+		/* For summary mode, total_stats and in_memory_stats remain the same */
+		memCtxState[idx].num_individual_stats = ctx_id;
+		memCtxState[idx].total_stats = ctx_id;
+		goto cleanup;
+	}
+	foreach_ptr(MemoryContextData, cur, contexts)
+	{
+		List	   *path = NIL;
+
+		/*
+		 * Figure out the transient context_id of this context and each of its
+		 * ancestors.
+		 */
+		path = compute_context_path(cur, context_id_lookup);
+
+		if (context_id <= (max_stats - 2))
+		{
+			/* Examine the context stats */
+			memset(&stat, 0, sizeof(stat));
+			(*cur->methods->stats) (cur, NULL, NULL, &stat, true);
+			/* Copy statistics to DSA memory */
+			PublishMemoryContext(meminfo, context_id, cur, path, stat, 1);
+		}
+		else
+		{
+			/* Examine the context stats */
+			memset(&stat, 0, sizeof(stat));
+			(*cur->methods->stats) (cur, NULL, NULL, &stat, true);
+
+			meminfo[max_stats - 1].totalspace += stat.totalspace;
+			meminfo[max_stats - 1].nblocks += stat.nblocks;
+			meminfo[max_stats - 1].freespace += stat.freespace;
+			meminfo[max_stats - 1].freechunks += stat.freechunks;
+		}
+
+		/*
+		 * DSA max limit is reached, write aggregate of the remaining
+		 * statistics.
+		 */
+		if (context_id == (max_stats - 2) && context_id < (stats_count - 1))
+		{
+			memCtxState[idx].num_individual_stats = context_id + 1;
+			strncpy(meminfo[max_stats - 1].name, "Remaining Totals", 16);
+		}
+		context_id++;
+	}
+	/* No aggregated contexts, individual statistics reported */
+	if (context_id < (max_stats - 2))
+	{
+		memCtxState[idx].num_individual_stats = context_id;
+	}
+	/* Report number of aggregated memory contexts */
+	else
+	{
+		meminfo[max_stats - 1].num_agg_stats = context_id -
+			memCtxState[idx].num_individual_stats;
+	}
+	memCtxState[idx].total_stats = context_id;
+cleanup:
+
+	/*
+	 * Signal all the waiting client backends after setting the exit condition
+	 * flag
+	 */
+	memCtxState[idx].stats_timestamp = GetCurrentTimestamp();
+	LWLockRelease(&memCtxState[idx].lw_lock);
+	ConditionVariableBroadcast(&memCtxState[idx].memctx_cv);
+	/* Delete the hash table memory context */
+	MemoryContextDelete(stat_cxt);
+
+/* 	dsa_detach(area); */
+}
+
+/*
+ * Append the transient context_id of this context and each of
+ * its ancestors to a list, inorder to compute a path.
+ */
+static List *
+compute_context_path(MemoryContext c, HTAB *context_id_lookup)
+{
+	bool		found;
+	List	   *path = NIL;
+
+	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);
+	}
+	return path;
+}
+
+/* Return the number of contexts allocated currently by the backend */
+static void
+compute_num_of_contexts(List *contexts, HTAB *context_id_lookup,
+						int *stats_count, bool get_summary)
+{
+	foreach_ptr(MemoryContextData, cur, contexts)
+	{
+		MemoryContextId *entry;
+		bool		found;
+
+		entry = (MemoryContextId *) hash_search(context_id_lookup, &cur,
+												HASH_ENTER, &found);
+		Assert(!found);
+
+		/* 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);
+				Assert(!found);
+
+				entry->context_id = (++(*stats_count));
+			}
+
+			contexts = lappend(contexts, c);
+		}
+
+		/*
+		 * In summary only the first two level(from top) contexts are
+		 * displayed
+		 */
+		if (get_summary)
+			break;
+	}
+
+}
+
+/* Copy the memory context statistics of a single context to a dsa buffer */
+static void
+PublishMemoryContext(MemoryContextEntry *memctx_info, int curr_id,
+					 MemoryContext context, List *path,
+					 MemoryContextCounters stat, int num_contexts)
+{
+	char		clipped_ident[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+
+	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';
+
+	/* Trim and copy the identifier if it is not set to NULL */
+	if (context->ident != NULL)
+	{
+		int			idlen = strlen(context->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(context->ident, idlen,
+								 MEMORY_CONTEXT_IDENT_DISPLAY_SIZE - 1);
+
+		memcpy(clipped_ident, context->ident, idlen);
+		clipped_ident[idlen] = '\0';
+
+		/*
+		 * 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);
+
+	memctx_info[curr_id].type = AssignContextType(context->type);
+	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_agg_stats = num_contexts;
+}
+
 void *
 palloc(Size size)
 {
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 5b8c2ad2a5..464eb7258d 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -8474,6 +8474,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,timestamptz}',
+  proargmodes => '{i,i,o,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, stats_timestamp}',
+  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 a2b63495ee..3dc3dcfb6c 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..efa40a14af 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,12 @@
 
 #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
+#define MAX_NUM_DEFAULT_SEGMENTS 8
 /*
  * Standard top-level memory contexts.
  *
@@ -319,4 +327,55 @@ pg_memory_is_all_zeros(const void *ptr, size_t len)
 	return true;
 }
 
+/* Dynamic shared memory state for memory context statistics reporting */
+typedef struct MemoryContextEntry
+{
+	/*
+	 * XXX isn't 2 x 1kB for every context a bit too much? Maybe better to
+	 * make it variable-length?
+	 */
+	char		name[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+	char		ident[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+	Datum		path[MEM_CONTEXT_MAX_LEVEL];
+	const char *type;
+	int			path_length;
+	int64		totalspace;
+	int64		nblocks;
+	int64		freespace;
+	int64		freechunks;
+	int			num_agg_stats;
+} MemoryContextEntry;
+
+/* Shared memory state for memory context statistics reporting */
+typedef struct MemoryContextState
+{
+	ConditionVariable memctx_cv;
+	LWLock		lw_lock;
+	int			proc_id;
+	int			num_individual_stats;
+	int			total_stats;
+	bool		get_summary;
+	dsa_handle	memstats_dsa_handle;
+	dsa_pointer memstats_dsa_pointer;
+	TimestampTz stats_timestamp;
+	bool		request_pending;
+} 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 const char *AssignContextType(NodeTag type);
+extern void HandleGetMemoryContextInterrupt(void);
+extern void MemCtxShmemInit(void);
+
 #endif							/* MEMUTILS_H */
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 352abc0bd4..831e0dead1 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -231,3 +231,17 @@ select * from pg_timezone_abbrevs where abbrev = 'LMT';
  LMT    | @ 7 hours 52 mins 58 secs ago | f
 (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 = '{1}' 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 66179f026b..0a4cc3bf4d 100644
--- a/src/test/regress/sql/sysviews.sql
+++ b/src/test/regress/sql/sysviews.sql
@@ -101,3 +101,17 @@ select count(distinct utc_offset) >= 24 as ok from pg_timezone_abbrevs;
 -- One specific case we can check without much fear of breakage
 -- is the historical local-mean-time value used for America/Los_Angeles.
 select * from pg_timezone_abbrevs where abbrev = 'LMT';
+
+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 = '{1}' into r;
+	RAISE NOTICE '%', r;
+END $$;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 9a3bee93de..69089e03e5 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1632,8 +1632,10 @@ MemoryContextCallback
 MemoryContextCallbackFunction
 MemoryContextCounters
 MemoryContextData
+MemoryContextEntry
 MemoryContextMethodID
 MemoryContextMethods
+MemoryContextState
 MemoryStatsPrintFunc
 MergeAction
 MergeActionState
-- 
2.34.1

