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

This function sends a signal to a backend to publish
statistics of all its memory contexts. Signal handler
sets a flag, which causes the relevant backend to copy its
MemoryContextStats to a fixed shared memory area, as part
of next CHECK_FOR_INTERRUPTS().
If there are more statistics to be shared, it creates
a file and copies remaining stats to that file.
Once its done, it signals the
client backend using a condition variable. The client backend
which was waiting on that condition variable, then wakes up,
reads the shared memory and returns these values in the form of
set of records, one for each memory context, to the user.
The client backend tries to read the remaining
statistics from the file if it exists. The client backend
is reponsible for deleting the file when it finishes
reading and also marking the shared memory area as not in
use, in order to allow the other backends to fill it with
new statistics.
---
 src/backend/postmaster/autovacuum.c           |   4 +
 src/backend/postmaster/checkpointer.c         |   4 +
 src/backend/postmaster/interrupt.c            |   4 +
 src/backend/postmaster/pgarch.c               |   4 +
 src/backend/postmaster/startup.c              |   4 +
 src/backend/postmaster/walsummarizer.c        |   4 +
 src/backend/storage/ipc/ipci.c                |   2 +
 src/backend/storage/ipc/procsignal.c          |   3 +
 src/backend/tcop/postgres.c                   |   3 +
 .../utils/activity/wait_event_names.txt       |   1 +
 src/backend/utils/adt/mcxtfuncs.c             | 274 ++++++++++++-
 src/backend/utils/init/globals.c              |   1 +
 src/backend/utils/mmgr/mcxt.c                 | 373 +++++++++++++++++-
 src/include/catalog/pg_proc.dat               |  10 +
 src/include/miscadmin.h                       |   1 +
 src/include/storage/procsignal.h              |   1 +
 src/include/utils/memutils.h                  |  60 +++
 17 files changed, 741 insertions(+), 12 deletions(-)

diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index dc3cf87aba..5d01497ada 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -768,6 +768,10 @@ HandleAutoVacLauncherInterrupts(void)
 
 	/* Process sinval catchup interrupts that happened while sleeping */
 	ProcessCatchupInterrupt();
+
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
 }
 
 /*
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index 9087e3f8db..4551ff2183 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -616,6 +616,10 @@ HandleCheckpointerInterrupts(void)
 	/* Perform logging of memory contexts of this process */
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
+
+	/* Publish memory contexts of this process */
+	if (PublishMemoryContextPending)
+		ProcessGetMemoryContextInterrupt();
 }
 
 /*
diff --git a/src/backend/postmaster/interrupt.c b/src/backend/postmaster/interrupt.c
index eedc0980cf..1107ff6d45 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 02f91431f5..467a253ccd 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 ef6f98ebcd..17beb8737d 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 ee6f1afc9a..b4b56142cd 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 10fc18f252..ff4a607fb3 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -53,6 +53,7 @@
 #include "storage/spin.h"
 #include "utils/guc.h"
 #include "utils/injection_point.h"
+#include "utils/memutils.h"
 
 /* GUCs */
 int			shared_memory_type = DEFAULT_SHARED_MEMORY_TYPE;
@@ -345,6 +346,7 @@ CreateOrAttachShmemStructs(void)
 	WaitEventCustomShmemInit();
 	InjectionPointShmemInit();
 	WaitLSNShmemInit();
+	MemCtxShmemInit();
 }
 
 /*
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 87027f27eb..621726cf03 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 7f5eada9d4..eb0316442c 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -3501,6 +3501,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 8efb4044d6..95b1e36303 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 6a6634e1cd..4c5da91538 100644
--- a/src/backend/utils/adt/mcxtfuncs.c
+++ b/src/backend/utils/adt/mcxtfuncs.c
@@ -17,28 +17,23 @@
 
 #include "funcapi.h"
 #include "mb/pg_wchar.h"
+#include "miscadmin.h"
+#include "nodes/pg_list.h"
 #include "storage/proc.h"
 #include "storage/procarray.h"
 #include "utils/array.h"
 #include "utils/builtins.h"
 #include "utils/hsearch.h"
+#include "utils/memutils.h"
+#include "utils/wait_event_types.h"
+#include "common/file_utils.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
@@ -305,3 +300,260 @@ pg_log_backend_memory_contexts(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(true);
 }
+
+/*
+ * pg_get_remote_backend_memory_contexts
+ *		Signal a backend or an auxiliary process to send its memory contexts.
+ *
+ * On receipt of this signal, a backend or an auxiliary process sets the flag
+ * in the signal handler, which causes the next CHECK_FOR_INTERRUPTS()
+ * or process-specific interrupt handler to copy the memory context statistics
+ * in a shared memory space. The statistics that do not fit in shared
+ * memory area are copied to a file by the backend.
+ *
+ * Wait for the backend to send signal on the condition variable after
+ * writing statistics to a shared memory and if needed to a temp file.
+ * Once condition variable comes out of sleep check if the required
+ * backends statistics are available to read and display.
+ */
+Datum
+pg_get_remote_backend_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;
+	MemoryContextParams *mem_stat = NULL;
+	char		tmpfilename[MAXPGPATH];
+	FILE	   *fp = NULL;
+
+	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_BOOL(false);
+	}
+
+	procNumber = GetNumberFromPGProc(proc);
+
+	/* Only request the statistics that fit in memory, if get_summary is true. */
+	LWLockAcquire(&memCtxState->lw_lock, LW_EXCLUSIVE);
+	memCtxState->get_summary = get_summary;
+	LWLockRelease(&memCtxState->lw_lock);
+	if (SendProcSignal(pid, PROCSIG_GET_MEMORY_CONTEXT, procNumber) < 0)
+	{
+		ereport(WARNING,
+				(errmsg("could not send signal to process %d: %m", pid)));
+		PG_RETURN_BOOL(false);
+	}
+
+	/*
+	 * Wait for a backend to publish stats, indicated when in_use is set true
+	 * by the backend
+	 */
+	while (1)
+	{
+		LWLockAcquire(&memCtxState->lw_lock, LW_EXCLUSIVE);
+
+		/*
+		 * We expect to come out of sleep only when atleast one backend has
+		 * published some memcontext information
+		 *
+		 * Make sure that all the stats has been published and the information
+		 * belongs to pid we requested information for, Otherwise loop back
+		 * and wait for the correct backend to publish the information
+		 */
+		if (memCtxState->in_use == true && memCtxState->proc_id == pid)
+			break;
+		else
+			LWLockRelease(&memCtxState->lw_lock);
+
+		if (ConditionVariableTimedSleep(&memCtxState->memctx_cv, 120000,
+										WAIT_EVENT_MEM_CTX_PUBLISH))
+		{
+			ereport(WARNING,
+					(errmsg("Wait for %d process to publish stats timed out, try again", pid)));
+			return (Datum) 0;
+		}
+	}
+	/* Backend has finished publishing the stats, read them */
+	for (i = 0; i < memCtxState->in_memory_stats; i++)
+	{
+		ArrayType  *path_array;
+		int			path_length;
+		Datum		values[10];
+		bool		nulls[10];
+
+		memset(values, 0, sizeof(values));
+		memset(nulls, 0, sizeof(nulls));
+
+		if (strlen(memCtxState->memctx_infos[i].name) != 0)
+			values[0] = CStringGetTextDatum(memCtxState->memctx_infos[i].name);
+		else
+			nulls[0] = true;
+		if (strlen(memCtxState->memctx_infos[i].ident) != 0)
+			values[1] = CStringGetTextDatum(memCtxState->memctx_infos[i].ident);
+		else
+			nulls[1] = true;
+
+		values[2] = CStringGetTextDatum(memCtxState->memctx_infos[i].type);
+		path_length = memCtxState->memctx_infos[i].path_length;
+		path_array = construct_array_builtin(memCtxState->memctx_infos[i].path, path_length, INT4OID);
+		values[3] = PointerGetDatum(path_array);
+		values[4] = Int64GetDatum(memCtxState->memctx_infos[i].totalspace);
+		values[5] = Int64GetDatum(memCtxState->memctx_infos[i].nblocks);
+		values[6] = Int64GetDatum(memCtxState->memctx_infos[i].freespace);
+		values[7] = Int64GetDatum(memCtxState->memctx_infos[i].freechunks);
+		values[8] = Int64GetDatum(memCtxState->memctx_infos[i].totalspace - memCtxState->memctx_infos[i].freespace);
+		values[9] = Int32GetDatum(memCtxState->proc_id);
+
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
+	}
+	/* No more stats to read return */
+	if (memCtxState->total_stats == i)
+	{
+		/*
+		 * Clear the in_use flag after we have finished reading, the stats, so
+		 * another backend can use the shared space
+		 */
+		memCtxState->in_use = false;
+		memset(&memCtxState->memctx_infos, 0, 30 * sizeof(MemoryContextInfo));
+		LWLockRelease(&memCtxState->lw_lock);
+		return (Datum) 0;
+	}
+	/* Compute name for temp mem stat file */
+	snprintf(tmpfilename, MAXPGPATH, "%s/%s.memstats.%d",
+			 PG_TEMP_FILES_DIR, PG_TEMP_FILE_PREFIX,
+			 memCtxState->proc_id);
+	LWLockRelease(&memCtxState->lw_lock);
+	ConditionVariableCancelSleep();
+
+	/* Open file */
+	fp = AllocateFile(tmpfilename, PG_BINARY_R);
+	if (!fp)
+	{
+		ereport(WARNING,
+				(errcode_for_file_access(),
+				 errmsg("could not read from the file")));
+		LWLockAcquire(&memCtxState->lw_lock, LW_EXCLUSIVE);
+		memCtxState->in_use = false;
+		memset(&memCtxState->memctx_infos, 0, 30 * sizeof(MemoryContextInfo));
+		LWLockRelease(&memCtxState->lw_lock);
+		return (Datum) 0;
+	}
+	mem_stat = palloc0(sizeof(MemoryContextParams));
+	while (!feof(fp))
+	{
+		int			path_length;
+		ArrayType  *path_array;
+		Datum		values[10];
+		bool		nulls[10];
+
+		memset(values, 0, sizeof(values));
+		memset(nulls, 0, sizeof(nulls));
+
+		/* Read stats from file */
+		if (fread(mem_stat, sizeof(MemoryContextParams), 1, fp) != 1)
+		{
+			if (ferror(fp))
+			{
+				elog(WARNING, "File read error");
+				break;
+			}
+			/* EOF reached */
+			break;
+		}
+		path_length = mem_stat->path_length;
+		if (strlen(mem_stat->name) != 0)
+			values[0] = CStringGetTextDatum(mem_stat->name);
+		else
+			nulls[0] = true;
+
+		if (strlen(mem_stat->ident) != 0)
+			values[1] = CStringGetTextDatum(mem_stat->ident);
+		else
+			nulls[1] = true;
+
+		values[2] = CStringGetTextDatum(mem_stat->type);
+
+		path_array = construct_array_builtin(mem_stat->path, path_length, INT4OID);
+		values[3] = PointerGetDatum(path_array);
+		values[4] = Int64GetDatum(mem_stat->totalspace);
+		values[5] = Int64GetDatum(mem_stat->nblocks);
+		values[6] = Int64GetDatum(mem_stat->freespace);
+		values[7] = Int64GetDatum(mem_stat->freechunks);
+		values[8] = Int64GetDatum(mem_stat->totalspace - mem_stat->freespace);
+		LWLockAcquire(&memCtxState->lw_lock, LW_EXCLUSIVE);
+		values[9] = Int32GetDatum(memCtxState->proc_id);
+		LWLockRelease(&memCtxState->lw_lock);
+
+		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
+	}
+
+	/*
+	 * Clear the in_use flag after we have finished reading, the stats, so
+	 * another backend can use the shared space, also reset the contents.
+	 */
+	LWLockAcquire(&memCtxState->lw_lock, LW_EXCLUSIVE);
+	memCtxState->in_use = false;
+	memset(&memCtxState->memctx_infos, 0, 30 * sizeof(MemoryContextInfo));
+	LWLockRelease(&memCtxState->lw_lock);
+	pfree(mem_stat);
+	FreeFile(fp);
+	/* Delete the temp file that stores memory stats */
+	unlink(tmpfilename);
+
+	return (Datum) 0;
+}
+
+static Size
+MemCtxShmemSize(void)
+{
+	Size		size;
+
+	size = offsetof(MemoryContextState, memctx_infos);
+	size = add_size(size, mul_size(30, sizeof(MemoryContextInfo)));
+	return size;
+}
+
+void
+MemCtxShmemInit(void)
+{
+	bool		found;
+
+	memCtxState = (MemoryContextState *) ShmemInitStruct("MemoryContextState",
+														 MemCtxShmemSize(),
+														 &found);
+	if (!found)
+	{
+		ConditionVariableInit(&memCtxState->memctx_cv);
+		memCtxState->in_use = false;
+		memset(&memCtxState->memctx_infos, 0, 30 * sizeof(MemoryContextInfo));
+		LWLockInitialize(&memCtxState->lw_lock, LWLockNewTrancheId());
+		LWLockRegisterTranche(memCtxState->lw_lock.tranche, "mem_context_stats_reporting");
+	}
+}
diff --git a/src/backend/utils/init/globals.c b/src/backend/utils/init/globals.c
index 03a54451ac..7fc600ff7b 100644
--- a/src/backend/utils/init/globals.c
+++ b/src/backend/utils/init/globals.c
@@ -42,6 +42,7 @@ volatile sig_atomic_t IdleStatsUpdateTimeoutPending = false;
 volatile uint32 InterruptHoldoffCount = 0;
 volatile uint32 QueryCancelHoldoffCount = 0;
 volatile uint32 CritSectionCount = 0;
+volatile sig_atomic_t PublishMemoryContextPending = false;
 
 int			MyProcPid;
 pg_time_t	MyStartTime;
diff --git a/src/backend/utils/mmgr/mcxt.c b/src/backend/utils/mmgr/mcxt.c
index bde54326c6..f915d8130c 100644
--- a/src/backend/utils/mmgr/mcxt.c
+++ b/src/backend/utils/mmgr/mcxt.c
@@ -21,18 +21,23 @@
 
 #include "postgres.h"
 
+#include "common/file_utils.h"
 #include "mb/pg_wchar.h"
 #include "miscadmin.h"
+#include "nodes/pg_list.h"
+#include "storage/fd.h"
+#include "storage/lwlock.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);
 static Size BogusGetChunkSpace(void *pointer);
+static int	PublishMemoryContextToFile(MemoryContext context, FILE *fp, List *path, char *clipped_ident);
 
 /*****************************************************************************
  *	  GLOBAL MEMORY															 *
@@ -166,6 +171,7 @@ 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(MemoryContext context, int64 counter, List *path, char *clipped_ident);
 
 /*
  * You should not do memory allocations within a critical section, because
@@ -1276,6 +1282,21 @@ HandleLogMemoryContextInterrupt(void)
 	/* latch will be set by procsignal_sigusr1_handler */
 }
 
+/*
+ * HandleGetMemoryContextInterrupt
+ *		Handle receipt of an interrupt indicating publishing of memory
+ *		contexts.
+ *
+ * All the actual work is deferred to ProcessLogMemoryContextInterrupt()
+ */
+void
+HandleGetMemoryContextInterrupt(void)
+{
+	InterruptPending = true;
+	PublishMemoryContextPending = true;
+	/* latch will be set by procsignal_sigusr1_handler */
+}
+
 /*
  * ProcessLogMemoryContextInterrupt
  * 		Perform logging of memory contexts of this backend process.
@@ -1313,6 +1334,356 @@ ProcessLogMemoryContextInterrupt(void)
 	MemoryContextStatsDetail(TopMemoryContext, 100, 100, false);
 }
 
+/*
+ * Run by each backend to publish their memory context
+ * statistics. It performs a breadth first search
+ * on the memory context tree, so that the parents
+ * get a chance to report stats before their children.
+ *
+ * Statistics are shared via fixed shared memory which
+ * can hold statistics for 29 contexts. The rest of the
+ * statistics are stored in a file. This file is created
+ * in PG_TEMP_FILES_DIR and deleted by the client after
+ * reading the stats.
+ */
+void
+ProcessGetMemoryContextInterrupt(void)
+{
+	/* Store the memory context details in shared memory */
+
+	List	   *contexts;
+	FILE	   *fp = NULL;
+	char		tmpfilename[MAXPGPATH];
+
+	HASHCTL		ctl;
+	HTAB	   *context_id_lookup;
+	int			context_id = 0;
+	bool		found;
+	MemoryContext stat_cxt;
+	bool		get_summary = false;
+
+	PublishMemoryContextPending = false;
+
+	/*
+	 * Shared memory is not available to be written, return. The waiting
+	 * client backend will timeout with a warning.
+	 */
+	LWLockAcquire(&memCtxState->lw_lock, LW_EXCLUSIVE);
+	if (memCtxState->in_use)
+	{
+		LWLockRelease(&memCtxState->lw_lock);
+		return;
+	}
+	LWLockRelease(&memCtxState->lw_lock);
+
+	/*
+	 * The hash table is used for constructing "path" column of
+	 * pg_get_remote_backend_memory_contextis view, similar to its local
+	 * backend couterpart.
+	 */
+
+	/*
+	 * Make a new context that will contain the hash table, to ease the
+	 * cleanup
+	 */
+
+	stat_cxt = AllocSetContextCreate(CurrentMemoryContext,
+									 "Memory context statistics",
+									 ALLOCSET_DEFAULT_SIZES);
+
+	ctl.keysize = sizeof(MemoryContext);
+	ctl.entrysize = sizeof(MemoryContextId);
+	ctl.hcxt = stat_cxt;
+
+	context_id_lookup = hash_create("pg_get_remote_backend_memory_contexts",
+									256,
+									&ctl,
+									HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	contexts = list_make1(TopMemoryContext);
+
+	/*
+	 * As in OpenTemporaryFileInTablespace, try to make the temp-file
+	 * directory, ignoring errors.
+	 */
+	(void) MakePGDirectory(PG_TEMP_FILES_DIR);
+
+	LWLockAcquire(&memCtxState->lw_lock, LW_EXCLUSIVE);
+	memCtxState->proc_id = MyProcPid;
+
+	foreach_ptr(MemoryContextData, cur, contexts)
+	{
+		MemoryContextId *entry;
+		List	   *path = NIL;
+		char		clipped_ident[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+
+		entry = (MemoryContextId *) hash_search(context_id_lookup, &cur,
+												HASH_ENTER, &found);
+		entry->context_id = context_id;
+
+		/*
+		 * Figure out the transient context_id of this context and each of its
+		 * ancestors.
+		 */
+		for (MemoryContext cur_context = cur; cur_context != NULL; cur_context = cur_context->parent)
+		{
+			MemoryContextId *cur_entry;
+
+			cur_entry = hash_search(context_id_lookup, &cur_context, HASH_FIND, &found);
+
+			if (!found)
+			{
+				elog(LOG, "hash table corrupted, can't construct path value");
+				break;
+			}
+			path = lcons_int(cur_entry->context_id, path);
+		}
+		/* Trim and copy the identifier if it is not set to NULL */
+		if (cur->ident != NULL)
+		{
+			int			idlen = strlen(cur->ident);
+
+			/*
+			 * Some identifiers such as SQL query string can be very long,
+			 * truncate oversize identifiers.
+			 */
+			if (idlen >= MEMORY_CONTEXT_IDENT_DISPLAY_SIZE)
+				idlen = pg_mbcliplen(cur->ident, idlen, MEMORY_CONTEXT_IDENT_DISPLAY_SIZE - 1);
+
+			memcpy(clipped_ident, cur->ident, idlen);
+			clipped_ident[idlen] = '\0';
+		}
+		if (context_id <= 28)
+		{
+			/* Copy statistics to shared memory */
+			PublishMemoryContext(cur, context_id, path, (cur->ident != NULL ? clipped_ident : NULL));
+		}
+		else
+		{
+			if (PublishMemoryContextToFile(cur, fp, path, (cur->ident != NULL ? clipped_ident : NULL)) == -1)
+				break;
+		}
+		/* Append the children of the current context to the main list */
+		for (MemoryContext c = cur->firstchild; c != NULL; c = c->nextchild)
+			contexts = lappend(contexts, c);
+
+		/*
+		 * Shared memory is full, release lock and write to file from next
+		 * iteration
+		 */
+		context_id++;
+		if (context_id == 29)
+		{
+			memCtxState->in_memory_stats = context_id;
+			get_summary = memCtxState->get_summary;
+			LWLockRelease(&memCtxState->lw_lock);
+			/* Construct name for temp file */
+			snprintf(tmpfilename, MAXPGPATH, "%s/%s.memstats.%d",
+					 PG_TEMP_FILES_DIR, PG_TEMP_FILE_PREFIX,
+					 MyProcPid);
+			/* Open file to copy rest of the stats in the file */
+			fp = AllocateFile(tmpfilename, PG_BINARY_A);
+
+			/*
+			 * Only in-memory stats(summary) are requested, so do not write to
+			 * file
+			 */
+			if (fp == NULL || get_summary)
+				break;
+		}
+	}
+	if (context_id < 29)
+	{
+		memCtxState->in_memory_stats = context_id;
+		LWLockRelease(&memCtxState->lw_lock);
+	}
+
+	/* Delete the hash table memory context */
+	MemoryContextDelete(stat_cxt);
+
+	/*
+	 * Signal the waiting client backend after setting the exit condition flag
+	 */
+	LWLockAcquire(&memCtxState->lw_lock, LW_EXCLUSIVE);
+	memCtxState->in_use = true;
+	memCtxState->total_stats = context_id;
+	LWLockRelease(&memCtxState->lw_lock);
+	ConditionVariableBroadcast(&memCtxState->memctx_cv);
+
+	/* Release file */
+	if (fp && FreeFile(fp))
+	{
+		ereport(LOG,
+				(errcode_for_file_access(),
+				 errmsg("could not free file \"%s\": %m", tmpfilename)));
+	}
+}
+
+static void
+PublishMemoryContext(MemoryContext context, int64 counter, List *path, char *clipped_ident)
+{
+	MemoryContextCounters stat;
+	char	   *type;
+
+	if (context->name != NULL)
+	{
+		Assert(strlen(context->name) < MEMORY_CONTEXT_IDENT_DISPLAY_SIZE);
+		strncpy(memCtxState->memctx_infos[counter].name, context->name, strlen(context->name));
+	}
+	else
+		memCtxState->memctx_infos[counter].name[0] = '\0';
+
+	if (clipped_ident != NULL)
+	{
+		/*
+		 * To be consistent with logging output, we label dynahash contexts
+		 * with just the hash table name as with MemoryContextStatsPrint().
+		 */
+		if (!strncmp(context->name, "dynahash", 8))
+		{
+			strncpy(memCtxState->memctx_infos[counter].name, clipped_ident, strlen(clipped_ident));
+			memCtxState->memctx_infos[counter].ident[0] = '\0';
+		}
+		else
+			strncpy(memCtxState->memctx_infos[counter].ident, clipped_ident, strlen(clipped_ident));
+	}
+	else
+		memCtxState->memctx_infos[counter].ident[0] = '\0';
+
+	memCtxState->memctx_infos[counter].path_length = list_length(path);
+	foreach_int(i, path)
+		memCtxState->memctx_infos[counter].path[foreach_current_index(i)] = Int32GetDatum(i);
+
+	/* Examine the context stats */
+	memset(&stat, 0, sizeof(stat));
+	(*context->methods->stats) (context, NULL, NULL, &stat, true);
+
+	switch (context->type)
+	{
+		case T_AllocSetContext:
+			type = "AllocSet";
+			strncpy(memCtxState->memctx_infos[counter].type, type, strlen(type));
+			break;
+		case T_GenerationContext:
+			type = "Generation";
+			strncpy(memCtxState->memctx_infos[counter].type, type, strlen(type));
+			break;
+		case T_SlabContext:
+			type = "Slab";
+			strncpy(memCtxState->memctx_infos[counter].type, type, strlen(type));
+			break;
+		case T_BumpContext:
+			type = "Bump";
+			strncpy(memCtxState->memctx_infos[counter].type, type, strlen(type));
+			break;
+		default:
+			type = "???";
+			strncpy(memCtxState->memctx_infos[counter].type, type, strlen(type));
+			break;
+	}
+	memCtxState->memctx_infos[counter].totalspace = stat.totalspace;
+	memCtxState->memctx_infos[counter].nblocks = stat.nblocks;
+	memCtxState->memctx_infos[counter].freespace = stat.freespace;
+	memCtxState->memctx_infos[counter].freechunks = stat.freechunks;
+}
+
+static int
+PublishMemoryContextToFile(MemoryContext context, FILE *fp, List *path, char *clipped_ident)
+{
+	MemoryContextCounters stat;
+	MemoryContextParams *mem_stat;
+	char	   *type;
+
+	mem_stat = palloc0(sizeof(MemoryContextParams));
+
+	/*
+	 * Assuming the context name will not exceed context identifier display
+	 * size XXX Reduce the limit for name length to correctly reflect
+	 * practical examples XXX  Add handling similar to clipped_ident of name
+	 * exceeds the size limit
+	 */
+	if (context->name != NULL)
+	{
+		Assert(strlen(context->name) < MEMORY_CONTEXT_IDENT_DISPLAY_SIZE);
+		strncpy(mem_stat->name, context->name, strlen(context->name));
+	}
+	else
+		mem_stat->name[0] = '\0';
+
+	if (clipped_ident != NULL)
+	{
+		/*
+		 * To be consistent with logging output, we label dynahash contexts
+		 * with just the hash table name as with MemoryContextStatsPrint().
+		 */
+		if (!strncmp(context->name, "dynahash", 8))
+		{
+			strncpy(mem_stat->name, clipped_ident, strlen(clipped_ident));
+			mem_stat->ident[0] = '\0';
+		}
+		else
+			strncpy(mem_stat->ident, clipped_ident, strlen(clipped_ident));
+	}
+	else
+		mem_stat->ident[0] = '\0';
+
+	mem_stat->path_length = list_length(path);
+	foreach_int(i, path)
+		mem_stat->path[foreach_current_index(i)] = Int32GetDatum(i);
+
+	/* Examine the context itself */
+	memset(&stat, 0, sizeof(stat));
+	(*context->methods->stats) (context, NULL, NULL, &stat, true);
+
+	switch (context->type)
+	{
+		case T_AllocSetContext:
+			type = "AllocSet";
+			strncpy(mem_stat->type, type, strlen(type));
+			break;
+		case T_GenerationContext:
+			type = "Generation";
+			strncpy(mem_stat->type, type, strlen(type));
+			break;
+		case T_SlabContext:
+			type = "Slab";
+			strncpy(mem_stat->type, type, strlen(type));
+			break;
+		case T_BumpContext:
+			type = "Bump";
+			strncpy(mem_stat->type, type, strlen(type));
+			break;
+		default:
+			type = "???";
+			strncpy(mem_stat->type, type, strlen(type));
+			break;
+	}
+	mem_stat->totalspace = stat.totalspace;
+	mem_stat->nblocks = stat.nblocks;
+	mem_stat->freespace = stat.freespace;
+	mem_stat->freechunks = stat.freechunks;
+
+	if (!fp)
+	{
+		ereport(LOG,
+				(errcode_for_file_access(),
+				 errmsg("could not create file")));
+		pfree(mem_stat);
+		return -1;
+	}
+	if (fwrite(mem_stat, sizeof(MemoryContextParams), 1, fp) != 1)
+	{
+		ereport(LOG,
+				(errcode_for_file_access(),
+				 errmsg("could not write to file")));
+		pfree(mem_stat);
+		return -1;
+	}
+	pfree(mem_stat);
+
+	return 0;
+}
+
 void *
 palloc(Size size)
 {
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 7c0b74fe05..5d7d0bcbf5 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -8429,6 +8429,16 @@
   prorettype => 'bool', proargtypes => 'int4',
   prosrc => 'pg_log_backend_memory_contexts' },
 
+# publishing memory contexts of the specified backend
+{ oid => '2173', descr => 'publish memory contexts of the specified backend',
+  proname => 'pg_get_remote_backend_memory_contexts', provolatile => 'v',
+  prorows => '100', proretset => 't', proparallel => 'r',
+  prorettype => 'record', proargtypes => 'int4 bool',
+  proallargtypes => '{int4,bool,text,text,text,_int4,int4,int4,int4,int4,int4,int4}',
+  proargmodes => '{i,i,o,o,o,o,o,o,o,o,o,o}',
+  proargnames => '{oid, summary, name, ident, type, path, total_bytes, total_nblocks, free_bytes, free_chunks, used_bytes, pid}',
+  prosrc => 'pg_get_remote_backend_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 e26d108a47..da07f99d7d 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 221073def3..8cbf6e201c 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 cd9596ff21..6d05465253 100644
--- a/src/include/utils/memutils.h
+++ b/src/include/utils/memutils.h
@@ -18,6 +18,8 @@
 #define MEMUTILS_H
 
 #include "nodes/memnodes.h"
+#include "storage/condition_variable.h"
+#include "storage/lmgr.h"
 
 
 /*
@@ -48,6 +50,8 @@
 
 #define AllocHugeSizeIsValid(size)	((Size) (size) <= MaxAllocHugeSize)
 
+#define MEMORY_CONTEXT_IDENT_DISPLAY_SIZE	1024
+
 
 /*
  * Standard top-level memory contexts.
@@ -115,6 +119,62 @@ extern MemoryContext AllocSetContextCreateInternal(MemoryContext parent,
 												   Size initBlockSize,
 												   Size maxBlockSize);
 
+/* Shared memory state for Memory Context Statistics reporting */
+typedef struct MemoryContextInfo
+{
+	char		name[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+	char		ident[MEMORY_CONTEXT_IDENT_DISPLAY_SIZE];
+	Datum		path[128];
+	char		type[128];
+	int			path_length;
+	int64		totalspace;
+	int64		nblocks;
+	int64		freespace;
+	int64		freechunks;
+}			MemoryContextInfo;
+
+typedef struct MemoryContextState
+{
+	ConditionVariable memctx_cv;
+	LWLock		lw_lock;
+	bool		in_use;
+	int			proc_id;
+	int			in_memory_stats;
+	int			total_stats;
+	bool		get_summary;
+	MemoryContextInfo memctx_infos[30];
+}			MemoryContextState;
+
+/* Backend local struct used to write statistics to a file */
+typedef struct MemoryContextParams
+{
+	char		name[1024];
+	char		ident[1024];
+	char		type[128];
+	Datum		path[128];
+	int			path_length;
+	int64		totalspace;
+	int64		nblocks;
+	int64		freespace;
+	int64		freechunks;
+}			MemoryContextParams;
+
+/*
+ * MemoryContextId
+ *		Used for storage of transient identifiers for
+ *		pg_get_backend_memory_contexts.
+ */
+typedef struct MemoryContextId
+{
+	MemoryContext context;
+	int			context_id;
+}			MemoryContextId;
+
+extern PGDLLIMPORT MemoryContextState * memCtxState;
+extern void ProcessGetMemoryContextInterrupt(void);
+extern void HandleGetMemoryContextInterrupt(void);
+extern void MemCtxShmemInit(void);
+
 /*
  * This wrapper macro exists to check for non-constant strings used as context
  * names; that's no longer supported.  (Use MemoryContextSetIdentifier if you
-- 
2.34.1

