From ddaf0ff1fdc7f0b68743bb8676d04ca7c2c63432 Mon Sep 17 00:00:00 2001
From: Rahila Syed <rahilasyed.90@gmail.com>
Date: Wed, 26 Mar 2025 15:08:16 +0530
Subject: [PATCH v21 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.
User can pass num_of_tries which determines the total
number of wait cycles in a client backend for latest
statistics.
Each cycle wait timeout is set to 1 seconds. Post this
the client displays previously published statistics or
returns without results.
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 smaller dsa allocations within a single 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                        |  94 ++++
 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                |   3 +
 src/backend/storage/ipc/procsignal.c          |   3 +
 src/backend/storage/lmgr/proc.c               |  15 +
 src/backend/tcop/postgres.c                   |   3 +
 .../utils/activity/wait_event_names.txt       |   1 +
 src/backend/utils/adt/mcxtfuncs.c             | 432 +++++++++++++-
 src/backend/utils/init/globals.c              |   1 +
 src/backend/utils/mmgr/mcxt.c                 | 531 +++++++++++++++++-
 src/include/catalog/pg_proc.dat               |  10 +
 src/include/miscadmin.h                       |   1 +
 src/include/storage/procsignal.h              |   1 +
 src/include/utils/memutils.h                  |  69 +++
 src/test/regress/expected/sysviews.out        |  14 +
 src/test/regress/sql/sysviews.sql             |  14 +
 src/tools/pgindent/typedefs.list              |   4 +
 22 files changed, 1190 insertions(+), 30 deletions(-)

diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 2488e9ba998..b163da128c9 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -28645,6 +28645,67 @@ 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>summary</parameter> <type>boolean</type>, <parameter>retries</parameter> <type>integer</type> )
+        <returnvalue>setof record</returnvalue>
+        ( <parameter>name</parameter> <type>text</type>,
+        <parameter>ident</parameter> <type>text</type>,
+        <parameter>type</parameter> <type>text</type>,
+        <parameter>path</parameter> <type>integer[]</type>,
+        <parameter>level</parameter> <type>integer</type>,
+        <parameter>total_bytes</parameter> <type>bigint</type>,
+        <parameter>total_nblocks</parameter> <type>bigint</type>,
+        <parameter>free_bytes</parameter> <type>bigint</type>,
+        <parameter>free_chunks</parameter> <type>bigint</type>,
+        <parameter>used_bytes</parameter> <type>bigint</type>,
+        <parameter>num_agg_contexts</parameter> <type>integer</type>,
+        <parameter>stats_timestamp</parameter> <type>timestamptz</type> )
+       </para>
+       <para>
+        This function handles requests to display the memory contexts of a
+        <productname>PostgreSQL</productname> process with the specified
+        process ID.  The function can be used to send requests to backends as
+        well as <glossterm linkend="glossary-auxiliary-proc">auxiliary processes</glossterm>.
+       </para>
+       <para>
+        Memory contexts are arranged in a tree-like hierarchy.  When
+        <parameter>summary</parameter> is <literal>true</literal>, statistics
+        for memory contexts at levels 1 and 2 are displayed, with level 1
+        representing the root node (i.e., <literal>TopMemoryContext</literal>).
+        Statistics for contexts on level 2 and below are aggregates of all
+        child contexts' statistics, where <literal>num_agg_contexts</literal>
+        indicate the number aggregated child contexts.  When
+        <parameter>summary</parameter> is <literal>false</literal>,
+        <literal>the num_agg_contexts</literal> value is <literal>1</literal>,
+        indicating that individual statistics are being displayed.
+       </para>
+       <para>
+        Busy processes can delay reporting memory context statistics,
+        <parameter>retries</parameter> sets the number of times the client will
+        retry to get updated statistics.  The sleep per try is
+        <literal>0.5</literal> seconds.  This parameter can be increased if the
+        user anticipates a delay in the response from the reporting process.
+        Conversely, if users are frequently and periodically querying the
+        process for statistics, or if there are concurrent requests for
+        statistics of the same process, lowering the parameter might help
+        achieve a faster response.
+       </para>
+       <para>
+        After receiving memory context statistics from the target process, it
+        returns the results as one row per context.  If all the contexts don't
+        fit within the pre-determined size limit, the remaining context
+        statistics are aggregated and a cumulative total is displayed.  The
+        <literal>num_agg_contexts</literal> column indicates the number of
+        contexts aggregated in the displayed statistics.  When
+        <literal>num_agg_contexts</literal> is <literal>1</literal> is means
+        that the context statistics are displayed separately.
+       </para></entry>
+      </row>
+
       <row>
        <entry role="func_table_entry"><para role="func_signature">
         <indexterm>
@@ -28784,6 +28845,39 @@ LOG:  Grand total: 1651920 bytes in 201 blocks; 622360 free (88 chunks); 1029560
     because it may generate a large number of log messages.
    </para>
 
+   <para>
+    <function>pg_get_process_memory_contexts</function> can be used
+    to request the memory contexts statistics of any postgres process. For example:
+<programlisting>
+postgres=# SELECT * FROM pg_get_process_memory_contexts(
+  (SELECT pid FROM pg_stat_activity
+    WHERE backend_type = 'checkpointer')
+  , false, 5) LIMIT 1;
+-[ RECORD 1 ]----+------------------------------
+name             | TopMemoryContext
+ident            |
+type             | AllocSet
+path             | {1}
+level            | 1
+total_bytes      | 90304
+total_nblocks    | 3
+free_bytes       | 2880
+free_chunks      | 1
+used_bytes       | 87424
+num_agg_contexts | 1
+stats_timestamp  | 2025-03-24 13:55:47.796698+01
+</programlisting>
+    <note>
+     <para>
+      While <function>pg_get_process_memory_contexts</function> can be used to
+      query memory contexts of the local backend,
+      <structname>pg_backend_memory_contexts</structname>
+      (see <xref linkend="view-pg-backend-memory-contexts"/> for more details)
+      will be less resource intensive when only the local backend is of interest.
+     </para>
+    </note>
+   </para>
+
   </sect2>
 
   <sect2 id="functions-admin-backup">
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 2513a8ef8a6..16756152b71 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -781,6 +781,10 @@ ProcessAutoVacLauncherInterrupts(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 fda91ffd1ce..d3cb3f1891c 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -663,6 +663,10 @@ ProcessCheckpointerInterrupts(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 0ae9bf906ec..f24f574e748 100644
--- a/src/backend/postmaster/interrupt.c
+++ b/src/backend/postmaster/interrupt.c
@@ -48,6 +48,10 @@ ProcessMainLoopInterrupts(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 7e622ae4bd2..cb7408acf4c 100644
--- a/src/backend/postmaster/pgarch.c
+++ b/src/backend/postmaster/pgarch.c
@@ -867,6 +867,10 @@ ProcessPgArchInterrupts(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 27e86cf393f..7149a67fcbc 100644
--- a/src/backend/postmaster/startup.c
+++ b/src/backend/postmaster/startup.c
@@ -192,6 +192,10 @@ ProcessStartupProcInterrupts(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 0fec4f1f871..c7a76711cc5 100644
--- a/src/backend/postmaster/walsummarizer.c
+++ b/src/backend/postmaster/walsummarizer.c
@@ -879,6 +879,10 @@ ProcessWalSummarizerInterrupts(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 2fa045e6b0f..362a6dc9528 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -51,6 +51,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;
@@ -343,6 +344,8 @@ CreateOrAttachShmemStructs(void)
 	WaitEventCustomShmemInit();
 	InjectionPointShmemInit();
 	AioShmemInit();
+	MemCtxShmemInit();
+	MemCtxBackendShmemInit();
 }
 
 /*
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 7d201965503..b59034fdc38 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -690,6 +690,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/storage/lmgr/proc.c b/src/backend/storage/lmgr/proc.c
index 066319afe2b..026c4bc992f 100644
--- a/src/backend/storage/lmgr/proc.c
+++ b/src/backend/storage/lmgr/proc.c
@@ -50,6 +50,7 @@
 #include "storage/procsignal.h"
 #include "storage/spin.h"
 #include "storage/standby.h"
+#include "utils/memutils.h"
 #include "utils/timeout.h"
 #include "utils/timestamp.h"
 
@@ -497,6 +498,13 @@ InitProcess(void)
 	 */
 	PGSemaphoreReset(MyProc->sem);
 
+	/*
+	 * The before shmem exit callback frees the DSA memory occupied by the
+	 * latest memory context statistics that could be published by this
+	 * backend if requested.
+	 */
+	before_shmem_exit(AtProcExit_memstats_dsa_free, 0);
+
 	/*
 	 * Arrange to clean up at backend exit.
 	 */
@@ -671,6 +679,13 @@ InitAuxiliaryProcess(void)
 	 */
 	PGSemaphoreReset(MyProc->sem);
 
+	/*
+	 * The before shmem exit callback frees the DSA memory occupied by the
+	 * latest memory context statistics that could be published by this
+	 * process if requested.
+	 */
+	before_shmem_exit(AtProcExit_memstats_dsa_free, 0);
+
 	/*
 	 * Arrange to clean up at process exit.
 	 */
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index aec65007bb6..ad020adee93 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -3531,6 +3531,9 @@ ProcessInterrupts(void)
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
 
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
+
 	if (ParallelApplyMessagePending)
 		ProcessParallelApplyMessages();
 }
diff --git a/src/backend/utils/activity/wait_event_names.txt b/src/backend/utils/activity/wait_event_names.txt
index 8bce14c38fd..ee0e42535b6 100644
--- a/src/backend/utils/activity/wait_event_names.txt
+++ b/src/backend/utils/activity/wait_event_names.txt
@@ -161,6 +161,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 a process to publish memory information."
 
 ABI_compatibility:
 
diff --git a/src/backend/utils/adt/mcxtfuncs.c b/src/backend/utils/adt/mcxtfuncs.c
index 396c2f223b4..375509f2551 100644
--- a/src/backend/utils/adt/mcxtfuncs.c
+++ b/src/backend/utils/adt/mcxtfuncs.c
@@ -17,28 +17,26 @@
 
 #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 MemoryContextBackendState *memCtxState = NULL;
+struct MemoryContextState *memCtxArea = NULL;
 
 /*
  * int_list_to_array
@@ -143,24 +141,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 = ContextTypeToString(context->type, true);
 
 	values[2] = CStringGetTextDatum(type);
 	values[3] = Int32GetDatum(list_length(path));	/* level */
@@ -175,6 +156,44 @@ PutMemoryContextsStatsTupleStore(Tuplestorestate *tupstore,
 	list_free(path);
 }
 
+/*
+ * ContextTypeToString
+ *		Returns a textual representation of a context type
+ *
+ * This should cover the same types as MemoryContextIsValid. If missing_ok is
+ * false then execution will error out on invalid context types.
+ */
+const char *
+ContextTypeToString(NodeTag type, bool missing_ok)
+{
+	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:
+			if (missing_ok)
+				context_type = "???";
+			else
+				ereport(ERROR,
+						errcode(ERRCODE_UNDEFINED_OBJECT),
+						errmsg("invalid memory context type specified"));
+			break;
+	}
+	return (context_type);
+}
+
 /*
  * pg_get_backend_memory_contexts
  *		SQL SRF showing backend memory context.
@@ -305,3 +324,358 @@ 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 or users with PG_READ_ALL_STATS are allowed to
+ * signal a process to return the memory contexts. This is because allowing
+ * any users to issue this request at an unbounded rate would cause lots of
+ * requests to be sent, 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.
+ *
+ * We have defined a limit on DSA memory that could be allocated per process -
+ * if the process has more memory contexts than what can fit in the allocated
+ * size, the excess contexts 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. There is one condition variable per publishing backend.
+ * Once the condition variable is signalled, check if the latest memory context
+ * information is available 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, which is defined by user, before giving up and
+ * returning previously published statistics, if any. If no previous statistics
+ * exist, return NULL.
+ */
+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;
+	bool		proc_is_aux = false;
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	dsa_area   *area;
+	MemoryContextEntry *memctx_info;
+	int			num_retries = 0;
+	TimestampTz curr_timestamp;
+	int			max_retries = PG_GETARG_INT32(2);
+
+	/*
+	 * 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("permission denied to extract memory context statistics"));
+
+	/*
+	 * See if the process with given pid is a backend or an auxiliary process
+	 * and remember the type for when we requery the process later.
+	 */
+	proc = BackendPidGetProc(pid);
+	if (proc == NULL)
+	{
+		proc = AuxiliaryPidGetProc(pid);
+		proc_is_aux = true;
+	}
+
+	/*
+	 * BackendPidGetProc() and AuxiliaryPidGetProc() return NULL if the pid
+	 * isn't valid; this is however not a problem and leave with a WARNING.
+	 * See comment in pg_log_backend_memory_contexts for a discussion on this.
+	 */
+	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();
+	}
+
+	InitMaterializedSRF(fcinfo, 0);
+
+	procNumber = GetNumberFromPGProc(proc);
+
+	LWLockAcquire(&memCtxState[procNumber].lw_lock, LW_EXCLUSIVE);
+	memCtxState[procNumber].get_summary = get_summary;
+	LWLockRelease(&memCtxState[procNumber].lw_lock);
+
+	curr_timestamp = GetCurrentTimestamp();
+
+	/*
+	 * Send a signal to a PostgreSQL process, informing it we want it to
+	 * produce information about its memory contexts.
+	 */
+	if (SendProcSignal(pid, PROCSIG_GET_MEMORY_CONTEXT, procNumber) < 0)
+	{
+		ereport(WARNING,
+				errmsg("could not send signal to process %d: %m", pid));
+		PG_RETURN_NULL();
+	}
+
+	/*
+	 * A valid DSA pointer isn't proof that statistics are available, it can
+	 * be valid due to previously published stats. Check if the stats are
+	 * updated by comparing the timestamp, if the stats are newer than our
+	 * previously recorded timestamp from before sending the procsignal, they
+	 * must by definition be updated. Wait for max_retries *
+	 * MEMSTATS_WAIT_TIMEOUT, following which display old statistics if
+	 * available or return NULL.
+	 */
+	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);
+
+		/*
+		 * 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.
+			 */
+			msecs = TimestampDifferenceMilliseconds(curr_timestamp,
+													memCtxState[procNumber].stats_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 to ensure the process is still alive.  Only check the
+		 * relevant process type based on the earlier PID check.
+		 */
+		if (proc_is_aux)
+			proc = AuxiliaryPidGetProc(pid);
+		else
+			proc = BackendPidGetProc(pid);
+
+		/*
+		 * The process ending during memory context processing is not an
+		 * error.
+		 */
+		if (proc == NULL)
+		{
+			ereport(WARNING,
+					errmsg("PID %d is no longer a PostgreSQL server process",
+						   pid));
+			PG_RETURN_NULL();
+		}
+
+#define MEMSTATS_WAIT_TIMEOUT 500
+
+		if (ConditionVariableTimedSleep(&memCtxState[procNumber].memctx_cv,
+										MEMSTATS_WAIT_TIMEOUT,
+										WAIT_EVENT_MEM_CTX_PUBLISH))
+		{
+			/*
+			 * Wait for max_retries, as defined by the user. If no updated
+			 * statistics are available within the wait time defined by
+			 * max_retries then display previously published statistics if
+			 * there are any. If no previous statistics are available then
+			 * return NULL.
+			 */
+			if (num_retries > max_retries)
+			{
+				LWLockAcquire(&memCtxState[procNumber].lw_lock, LW_EXCLUSIVE);
+				/* Displaying previously published statistics if available */
+				if (DsaPointerIsValid(memCtxState[procNumber].memstats_dsa_pointer))
+					break;
+				else
+				{
+					LWLockRelease(&memCtxState[procNumber].lw_lock);
+					PG_RETURN_NULL();
+				}
+			}
+			ereport(DEBUG1,
+					errmsg("timed out waiting for process with PID %d to publish stats, retrying",
+						   pid));
+			num_retries = num_retries + 1;
+		}
+	}
+
+	/*
+	 * We should only reach here with a valid DSA handle, either containing
+	 * updated statistics or previously published statistics (identified by
+	 * the timestamp.
+	 */
+	Assert(memCtxArea->memstats_dsa_handle != DSA_HANDLE_INVALID);
+	area = dsa_attach(memCtxArea->memstats_dsa_handle);
+
+	/*
+	 * Backend has finished publishing the stats, project them.
+	 */
+	memctx_info = (MemoryContextEntry *)
+		dsa_get_address(area, memCtxState[procNumber].memstats_dsa_pointer);
+
+#define PG_GET_PROCESS_MEMORY_CONTEXTS_COLS	12
+	for (int i = 0; i < memCtxState[procNumber].total_stats; i++)
+	{
+		ArrayType  *path_array;
+		int			path_length;
+		Datum		values[PG_GET_PROCESS_MEMORY_CONTEXTS_COLS];
+		bool		nulls[PG_GET_PROCESS_MEMORY_CONTEXTS_COLS];
+		char	   *name;
+		char	   *ident;
+		Datum	   *path_datum = NULL;
+		int		   *path_int = NULL;
+
+		memset(values, 0, sizeof(values));
+		memset(nulls, 0, sizeof(nulls));
+
+		if (DsaPointerIsValid(memctx_info[i].name))
+		{
+			name = (char *) dsa_get_address(area, memctx_info[i].name);
+			values[0] = CStringGetTextDatum(name);
+		}
+		else
+			nulls[0] = true;
+
+		if (DsaPointerIsValid(memctx_info[i].ident))
+		{
+			ident = (char *) dsa_get_address(area, memctx_info[i].ident);
+			values[1] = CStringGetTextDatum(ident);
+		}
+		else
+			nulls[1] = true;
+
+		if (memctx_info[i].type != NULL)
+			values[2] = CStringGetTextDatum(memctx_info[i].type);
+		else
+			nulls[2] = true;
+
+		path_length = memctx_info[i].path_length;
+		path_datum = (Datum *) palloc(path_length * sizeof(Datum));
+		if (DsaPointerIsValid(memctx_info[i].path))
+		{
+			path_int = (int *) dsa_get_address(area, memctx_info[i].path);
+			for (int j = 0; j < path_length; j++)
+				path_datum[j] = Int32GetDatum(path_int[j]);
+			path_array = construct_array_builtin(path_datum, path_length, INT4OID);
+			values[3] = PointerGetDatum(path_array);
+		}
+		else
+			nulls[3] = true;
+
+		values[4] = Int32GetDatum(path_length); /* level */
+		values[5] = Int64GetDatum(memctx_info[i].totalspace);
+		values[6] = Int64GetDatum(memctx_info[i].nblocks);
+		values[7] = Int64GetDatum(memctx_info[i].freespace);
+		values[8] = Int64GetDatum(memctx_info[i].freechunks);
+		values[9] = Int64GetDatum(memctx_info[i].totalspace -
+								  memctx_info[i].freespace);
+		values[10] = Int32GetDatum(memctx_info[i].num_agg_stats);
+		values[11] = TimestampTzGetDatum(memCtxState[procNumber].stats_timestamp);
+
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
+							 values, nulls);
+	}
+	LWLockRelease(&memCtxState[procNumber].lw_lock);
+
+	ConditionVariableCancelSleep();
+	dsa_detach(area);
+
+	PG_RETURN_NULL();
+}
+
+/*
+ * Init shared memory for reporting memory context information.
+ */
+void
+MemCtxBackendShmemInit(void)
+{
+	bool		found;
+	Size		TotalProcs;
+
+	TotalProcs = add_size(MaxBackends, NUM_AUXILIARY_PROCS);
+	TotalProcs = add_size(TotalProcs, max_prepared_xacts);
+
+	memCtxState = (MemoryContextBackendState *)
+		ShmemInitStruct("MemoryContextBackendState",
+						mul_size(TotalProcs, sizeof(MemoryContextBackendState)),
+						&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_backend_stats_reporting");
+
+			memCtxState[i].memstats_dsa_pointer = InvalidDsaPointer;
+		}
+	}
+	else
+	{
+		Assert(found);
+	}
+}
+
+/*
+ * Initialize shared memory for displaying memory context statistics
+ */
+void
+MemCtxShmemInit(void)
+{
+	bool		found;
+
+	memCtxArea = (MemoryContextState *)
+		ShmemInitStruct("MemoryContextState", sizeof(MemoryContextState), &found);
+
+	if (!IsUnderPostmaster)
+	{
+		Assert(!found);
+
+		LWLockInitialize(&memCtxArea->lw_lock,
+						 LWLockNewTrancheId());
+		LWLockRegisterTranche(memCtxArea->lw_lock.tranche,
+							  "mem_context_stats_reporting");
+		memCtxArea->memstats_dsa_handle = DSA_HANDLE_INVALID;
+	}
+	else
+	{
+		Assert(found);
+	}
+}
diff --git a/src/backend/utils/init/globals.c b/src/backend/utils/init/globals.c
index b844f9fdaef..13938ccb0f5 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 2cbde8f39c3..bfcd65a69a5 100644
--- a/src/backend/utils/mmgr/mcxt.c
+++ b/src/backend/utils/mmgr/mcxt.c
@@ -23,6 +23,13 @@
 
 #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 "storage/ipc.h"
+#include "utils/dsa.h"
+#include "utils/hsearch.h"
 #include "utils/memdebug.h"
 #include "utils/memutils.h"
 #include "utils/memutils_internal.h"
@@ -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, dsa_area *area);
+static void compute_contexts_count_and_ids(List *contexts, HTAB *context_id_lookup,
+										   int *stats_count,
+										   bool get_summary);
+static List *compute_context_path(MemoryContext c, HTAB *context_id_lookup);
+static void dsa_free_previous_stats(dsa_area *area, int total_stats, dsa_pointer prev_dsa_pointer);
 
 /*
  * You should not do memory allocations within a critical section, because
@@ -927,7 +944,7 @@ MemoryContextStatsInternal(MemoryContext context, int level,
 			 * PRINT_STATS_NONE, only compute totals. This is used in
 			 * reporting of memory context statistics via a sql function. Last
 			 * parameter is not relevant.
-+			 */
+			 */
 			context->methods->stats(context,
 									NULL,
 									NULL,
@@ -1331,6 +1348,22 @@ HandleLogMemoryContextInterrupt(void)
 	/* latch will be set by procsignal_sigusr1_handler */
 }
 
+/*
+ * HandleGetMemoryContextInterrupt
+ *		Handle receipt of an interrupt indicating a request to publish memory
+ *		contexts statistics.
+ *
+ * All the actual work is deferred to ProcessGetMemoryContextInterrupt() as
+ * this cannot be performed in a signal handler.
+ */
+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.
@@ -1368,6 +1401,502 @@ 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, thus parents
+ * statistics are reported before their children in the monitoring function
+ * output.
+ *
+ * Statistics for all the processes are shared via the same dynamic shared
+ * area.  Statistics written by each process are tracked independently in
+ * per-process DSA pointers. These pointers are stored in static shared memory.
+ *
+ * We calculate maximum number of context's statistics that can be displayed
+ * using a pre-determined limit for memory available per process for this
+ * utility maximum size of statistics for each context.  The remaining context
+ * statistics if any are captured as a cumulative total at the end of
+ * individual context's statistics.
+ *
+ * If get_summary is true, we capture the level 1 and level 2 contexts
+ * statistics.  For that we traverse the memory context tree recursively in
+ * depth first search manner to cover all the children of a parent context, to
+ * be able to display a cumulative total of memory consumption by a parent at
+ * level 2 and all its children.
+ */
+void
+ProcessGetMemoryContextInterrupt(void)
+{
+	List	   *contexts;
+	HASHCTL		ctl;
+	HTAB	   *context_id_lookup;
+	int			context_id = 0;
+	MemoryContextEntry *meminfo;
+	bool		get_summary = false;
+	dsa_area   *area = NULL;
+	int			max_stats;
+	int			idx = MyProcNumber;
+	int			stats_count = 0;
+	MemoryContextCounters stat;
+	int			num_individual_stats = 0;
+
+	PublishMemoryContextPending = false;
+
+	/*
+	 * The hash table is used for constructing "path" column of the view,
+	 * similar to its local backend counterpart.
+	 */
+	ctl.keysize = sizeof(MemoryContext);
+	ctl.entrysize = sizeof(MemoryContextId);
+	ctl.hcxt = CurrentMemoryContext;
+
+	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 defined limit */
+	max_stats = (MAX_SEGMENTS_PER_BACKEND * DSA_DEFAULT_INIT_SEGMENT_SIZE)
+		/ (MAX_MEMORY_CONTEXT_STATS_SIZE);
+	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. Also, populate the hash table of context ids.
+	 */
+	compute_contexts_count_and_ids(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 within a
+	 * limit, a cumulative total is written as the last record in the DSA
+	 * segment.
+	 */
+	stats_count = (stats_count > max_stats) ? max_stats : stats_count;
+
+	LWLockAcquire(&memCtxArea->lw_lock, LW_EXCLUSIVE);
+
+	/*
+	 * Create a DSA and send handle to the the client process after storing
+	 * the context statistics. If number of contexts exceed a predefined
+	 * limit(8MB), a cumulative total is stored for such contexts.
+	 */
+	if (memCtxArea->memstats_dsa_handle == DSA_HANDLE_INVALID)
+	{
+		MemoryContext oldcontext = CurrentMemoryContext;
+		dsa_handle	handle;
+
+		MemoryContextSwitchTo(TopMemoryContext);
+
+		area = dsa_create(memCtxArea->lw_lock.tranche);
+
+		handle = dsa_get_handle(area);
+		MemoryContextSwitchTo(oldcontext);
+
+		dsa_pin_mapping(area);
+
+		/*
+		 * Pin the DSA area, this is to make sure the area remains attachable
+		 * even if current backend exits. This is done so that the statistics
+		 * are published even if the process exits while a client is waiting.
+		 */
+		dsa_pin(area);
+
+		/* Set the handle in shared memory */
+		memCtxArea->memstats_dsa_handle = handle;
+	}
+
+	/*
+	 * If DSA exists, created by another process publishing statistics, or by
+	 * the previous execution of this function by this process, attach to it.
+	 */
+	else if (area == NULL)
+	{
+		MemoryContext oldcontext = CurrentMemoryContext;
+
+		MemoryContextSwitchTo(TopMemoryContext);
+		area = dsa_attach(memCtxArea->memstats_dsa_handle);
+		MemoryContextSwitchTo(oldcontext);
+		dsa_pin_mapping(area);
+	}
+	LWLockRelease(&memCtxArea->lw_lock);
+
+	/*
+	 * Hold the process lock to protect writes to process specific memory. Two
+	 * processes publishing statistics do not block each other.
+	 */
+	LWLockAcquire(&memCtxState[idx].lw_lock, LW_EXCLUSIVE);
+	memCtxState[idx].proc_id = MyProcPid;
+
+	if (DsaPointerIsValid(memCtxState[idx].memstats_dsa_pointer))
+	{
+		/*
+		 * Free any previous allocations, free the name, ident and path
+		 * pointers before freeing the pointer that contains them.
+		 */
+		dsa_free_previous_stats(area, memCtxState[idx].total_stats,
+								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, area);
+		ctx_id = ctx_id + 1;
+
+		/*
+		 * Copy statistics for each of TopMemoryContexts children.	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, area);
+			ctx_id = ctx_id + 1;
+		}
+		memCtxState[idx].total_stats = ctx_id;
+		goto cleanup;
+	}
+
+	foreach_ptr(MemoryContextData, cur, contexts)
+	{
+		List	   *path = NIL;
+		char	   *name;
+
+		/*
+		 * Figure out the transient context_id of this context and each of its
+		 * ancestors, to compute a path for this context.
+		 */
+		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, area);
+		}
+		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 per process is reached, write aggregate of the
+		 * remaining statistics.
+		 */
+		if (context_id == (max_stats - 2) && context_id < (stats_count - 1))
+		{
+			num_individual_stats = context_id + 1;
+			meminfo[max_stats - 1].name = dsa_allocate0(area, 17);
+			name = dsa_get_address(area, meminfo[max_stats - 1].name);
+			strncpy(name, "Remaining Totals", 16);
+			meminfo[max_stats - 1].ident = InvalidDsaPointer;
+			meminfo[max_stats - 1].path = InvalidDsaPointer;
+			meminfo[max_stats - 1].type = NULL;
+		}
+		context_id++;
+	}
+	/* Statistics are not aggregated, i.e individual statistics reported */
+	if (context_id < (max_stats - 2))
+	{
+		memCtxState[idx].total_stats = context_id;
+	}
+	/* Report number of aggregated memory contexts */
+	else
+	{
+		meminfo[max_stats - 1].num_agg_stats = context_id -
+			num_individual_stats;
+
+		/*
+		 * Total stats equals num_individual_stats + 1 record for cumulative
+		 * statistics.
+		 */
+		memCtxState[idx].total_stats = num_individual_stats + 1;
+	}
+
+cleanup:
+
+	/*
+	 * Signal all the waiting client backends after copying all the statistics
+	 */
+	memCtxState[idx].stats_timestamp = GetCurrentTimestamp();
+	LWLockRelease(&memCtxState[idx].lw_lock);
+	ConditionVariableBroadcast(&memCtxState[idx].memctx_cv);
+	hash_destroy(context_id_lookup);
+	dsa_detach(area);
+}
+
+/*
+ * compute_context_path
+ *
+ * Append the transient context_id of this context and each of its ancestors
+ * to a list, in order 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(ERROR, "hash table corrupted, can't construct path value");
+
+		path = lcons_int(cur_entry->context_id, path);
+	}
+	return path;
+}
+
+/*
+ * Return the number of contexts allocated currently by the backend
+ * Assign context ids to each of the contexts.
+ */
+static void
+compute_contexts_count_and_ids(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;
+	}
+}
+
+/*
+ * PublishMemoryContext
+ *
+ * Copy the memory context statistics of a single context to a DSA memory
+ */
+static void
+PublishMemoryContext(MemoryContextEntry *memctx_info, int curr_id,
+					 MemoryContext context, List *path,
+					 MemoryContextCounters stat, int num_contexts,
+					 dsa_area *area)
+{
+	char		clipped_ident[MEMORY_CONTEXT_IDENT_SHMEM_SIZE];
+	char	   *name;
+	char	   *ident;
+	int		   *path_list;
+
+	Assert(MemoryContextIsValid(context));
+
+	if (context->name != NULL)
+	{
+		Assert(strlen(context->name) < MEMORY_CONTEXT_IDENT_SHMEM_SIZE);
+		memctx_info[curr_id].name = dsa_allocate0(area, strlen(context->name) + 1);
+		name = (char *) dsa_get_address(area, memctx_info[curr_id].name);
+		strlcpy(name, context->name, strlen(context->name) + 1);
+	}
+	else
+		memctx_info[curr_id].name = InvalidDsaPointer;
+
+	/* 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_SHMEM_SIZE)
+			idlen = pg_mbcliplen(context->ident, idlen,
+								 MEMORY_CONTEXT_IDENT_SHMEM_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) == 0)
+		{
+			dsa_free(area, memctx_info[curr_id].name);
+			memctx_info[curr_id].name = dsa_allocate0(area, idlen + 1);
+			name = (char *) dsa_get_address(area, memctx_info[curr_id].name);
+			strlcpy(name, clipped_ident, idlen + 1);
+			memctx_info[curr_id].ident = InvalidDsaPointer;
+		}
+		else
+		{
+			memctx_info[curr_id].ident = dsa_allocate0(area, idlen + 1);
+			ident = (char *) dsa_get_address(area, memctx_info[curr_id].ident);
+			strlcpy(ident, clipped_ident, idlen + 1);
+		}
+	}
+	else
+		memctx_info[curr_id].ident = InvalidDsaPointer;
+
+	/* Allocate DSA memory for storing path information */
+	if (path == NIL)
+		memctx_info[curr_id].path = InvalidDsaPointer;
+	else
+	{
+		memctx_info[curr_id].path_length = list_length(path);
+		memctx_info[curr_id].path = dsa_allocate0(area,
+												  memctx_info[curr_id].path_length * sizeof(int));
+		path_list = (int *) dsa_get_address(area, memctx_info[curr_id].path);
+		foreach_int(i, path)
+			path_list[foreach_current_index(i)] = i;
+	}
+	memctx_info[curr_id].type = ContextTypeToString(context->type, true);
+	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;
+}
+
+/*
+ * dsa_free_previous_stats
+ *
+ * Worker for freeing resources from a MemoryContextEntry.  Callers are
+ * responsible for ensuring that the DSA pointer is valid.
+ */
+static void
+dsa_free_previous_stats(dsa_area *area, int total_stats,
+						dsa_pointer prev_dsa_pointer)
+{
+	MemoryContextEntry *meminfo;
+
+	meminfo = (MemoryContextEntry *) dsa_get_address(area, prev_dsa_pointer);
+	Assert(meminfo != NULL);
+	for (int i = 0; i < total_stats; i++)
+	{
+		if (DsaPointerIsValid(meminfo[i].name))
+			dsa_free(area, meminfo[i].name);
+
+		if (DsaPointerIsValid(meminfo[i].ident))
+			dsa_free(area, meminfo[i].ident);
+
+		if (DsaPointerIsValid(meminfo[i].path))
+			dsa_free(area, meminfo[i].path);
+	}
+}
+
+/*
+ * Free the memory context statistics stored by this process
+ * in DSA area.
+ */
+void
+AtProcExit_memstats_dsa_free(int code, Datum arg)
+{
+	int			idx = MyProcNumber;
+	dsm_segment *dsm_seg = NULL;
+	dsa_area   *area = NULL;
+
+	if (memCtxArea->memstats_dsa_handle == DSA_HANDLE_INVALID)
+		return;
+
+	dsm_seg = dsm_find_mapping(memCtxArea->memstats_dsa_handle);
+
+	LWLockAcquire(&memCtxState[idx].lw_lock, LW_EXCLUSIVE);
+
+	if (!DsaPointerIsValid(memCtxState[idx].memstats_dsa_pointer))
+	{
+		LWLockRelease(&memCtxState[idx].lw_lock);
+		return;
+	}
+
+	/* If the dsm mapping could not be found, attach to the area */
+	if (dsm_seg != NULL)
+		return;
+	area = dsa_attach(memCtxArea->memstats_dsa_handle);
+
+	/*
+	 * Free the memory context statistics, free the name, ident and path
+	 * pointers before freeing the pointer that contains these pointers and
+	 * integer statistics.
+	 */
+	dsa_free_previous_stats(area, memCtxState[idx].total_stats,
+							memCtxState[idx].memstats_dsa_pointer);
+	dsa_free(area, memCtxState[idx].memstats_dsa_pointer);
+	memCtxState[idx].memstats_dsa_pointer = InvalidDsaPointer;
+
+	dsa_detach(area);
+	LWLockRelease(&memCtxState[idx].lw_lock);
+}
+
 void *
 palloc(Size size)
 {
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 6b57b7e18d9..ff25f729d94 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -8535,6 +8535,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 int4',
+  proallargtypes => '{int4,bool,int4,text,text,text,_int4,int4,int8,int8,int8,int8,int8,int4,timestamptz}',
+  proargmodes => '{i,i,i,o,o,o,o,o,o,o,o,o,o,o,o}',
+  proargnames => '{pid, summary, retries, name, ident, type, path, level, 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 603d0424354..d3c44df6e13 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 022fd8ed933..5d4b2fbfc9c 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 send 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 8abc26abce2..da6b633cbda 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_SHMEM_SIZE 128
 
+#define MEM_CONTEXT_MAX_LEVEL 64
+#define MAX_SEGMENTS_PER_BACKEND 1
+#define MEM_CONTEXT_PATH_SIZE  MEM_CONTEXT_MAX_LEVEL * sizeof(int)
+#define MAX_MEMORY_CONTEXT_STATS_SIZE sizeof(MemoryContextEntry) + MEM_CONTEXT_PATH_SIZE + 2 * MEMORY_CONTEXT_IDENT_SHMEM_SIZE
 /*
  * Standard top-level memory contexts.
  *
@@ -319,4 +327,65 @@ pg_memory_is_all_zeros(const void *ptr, size_t len)
 	return true;
 }
 
+/* Dynamic shared memory state for statistics per context */
+typedef struct MemoryContextEntry
+{
+	dsa_pointer name;
+	dsa_pointer ident;
+	dsa_pointer path;
+	const char *type;
+	int			path_length;
+	int64		totalspace;
+	int64		nblocks;
+	int64		freespace;
+	int64		freechunks;
+	int			num_agg_stats;
+} MemoryContextEntry;
+
+/*
+ * Static shared memory state representing the DSA area created for memory
+ * context statistics reporting.  A single DSA area is created and used by all
+ * the processes, each having its specific DSA allocations for sharing memory
+ * statistics, tracked by per backend static shared memory state.
+ */
+typedef struct MemoryContextState
+{
+	dsa_handle	memstats_dsa_handle;
+	LWLock		lw_lock;
+} MemoryContextState;
+
+/*
+ * Per backend static shared memory state for memory context statistics
+ * reporting.
+ */
+typedef struct MemoryContextBackendState
+{
+	ConditionVariable memctx_cv;
+	LWLock		lw_lock;
+	int			proc_id;
+	int			total_stats;
+	bool		get_summary;
+	dsa_pointer memstats_dsa_pointer;
+	TimestampTz stats_timestamp;
+} MemoryContextBackendState;
+
+
+/*
+ * Used for storage of transient identifiers for pg_get_backend_memory_contexts
+ */
+typedef struct MemoryContextId
+{
+	MemoryContext context;
+	int			context_id;
+} MemoryContextId;
+
+extern PGDLLIMPORT MemoryContextBackendState *memCtxState;
+extern PGDLLIMPORT MemoryContextState *memCtxArea;
+extern void ProcessGetMemoryContextInterrupt(void);
+extern const char *ContextTypeToString(NodeTag type, bool missing_ok);
+extern void HandleGetMemoryContextInterrupt(void);
+extern void MemCtxShmemInit(void);
+extern void MemCtxBackendShmemInit(void);
+extern void AtProcExit_memstats_dsa_free(int code, Datum arg);
+
 #endif							/* MEMUTILS_H */
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 83228cfca29..dca20ae1a26 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -232,3 +232,17 @@ select * from pg_timezone_abbrevs where abbrev = 'LMT';
  LMT    | @ 7 hours 52 mins 58 secs ago | f
 (1 row)
 
+DO $$
+DECLARE
+    launcher_pid int;
+    r RECORD;
+BEGIN
+        SELECT pid from pg_stat_activity where backend_type='autovacuum launcher'
+	 INTO launcher_pid;
+
+        select type, name, ident
+        from pg_get_process_memory_contexts(launcher_pid, false, 20)
+	 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 66179f026b3..4767351d4e2 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
+    launcher_pid int;
+    r RECORD;
+BEGIN
+        SELECT pid from pg_stat_activity where backend_type='autovacuum launcher'
+	 INTO launcher_pid;
+
+        select type, name, ident
+        from pg_get_process_memory_contexts(launcher_pid, false, 20)
+	 where path = '{1}' into r;
+	RAISE NOTICE '%', r;
+END $$;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 449bafc123c..f083690c5ce 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1654,12 +1654,16 @@ MemoizeState
 MemoizeTuple
 MemoryChunk
 MemoryContext
+MemoryContextBackendState
 MemoryContextCallback
 MemoryContextCallbackFunction
 MemoryContextCounters
 MemoryContextData
+MemoryContextEntry
+MemoryContextId
 MemoryContextMethodID
 MemoryContextMethods
+MemoryContextState
 MemoryStatsPrintFunc
 MergeAction
 MergeActionState
-- 
2.39.3 (Apple Git-146)

