From 3c7ba68d18c43bc0d0c6d0873ab477bec263c1dd Mon Sep 17 00:00:00 2001
From: Melanie Plageman <melanieplageman@gmail.com>
Date: Thu, 2 Sep 2021 11:33:59 -0400
Subject: [PATCH v6 1/2] Add system view tracking shared buffer actions

Add a system view which tracks
- number of shared buffers the checkpointer and bgwriter write out
- number of shared buffers a regular backend is forced to flush
- number of extends done by a regular backend through shared buffers
- number of buffers flushed by a backend or autovacuum using a
  BufferAccessStrategy which, were they not to use this strategy, could
  perhaps have been avoided if a clean shared buffer was available
- number of fsyncs done by a backend which could have been done by
  checkpointer if sync queue had not been full
- number of buffers allocated by a regular backend or autovacuum worker
  for either a new block or an existing block of a relation which is not
  currently in a buffer

All backends increment a counter in their PgBackendStatus when
performing one of these buffer actions. On exit, backends send these
stats to the stats collector to be persisted.

When pg_stat_buffer_actions view is queried, add all live backend's
statuses to the saved stats kept by the stats collector (since the last
stats reset) and return that as the total.

Each row of the view is for a particular backend type and each column is
the number of a particular kind of buffer action taken by the various
backends.

TODO:
- Some kind of test to protect against regressions in counting these
  (and remove unstable pg_stats test)
- stats reset refactor
- when finished, catalog bump
- pgindent
---
 doc/src/sgml/monitoring.sgml                | 94 +++++++++++++++++++++
 src/backend/catalog/system_views.sql        | 10 +++
 src/backend/postmaster/checkpointer.c       |  1 +
 src/backend/postmaster/pgstat.c             | 71 +++++++++++++++-
 src/backend/storage/buffer/bufmgr.c         | 27 +++++-
 src/backend/storage/buffer/freelist.c       | 22 ++++-
 src/backend/utils/activity/backend_status.c | 49 +++++++++++
 src/backend/utils/adt/pgstatfuncs.c         | 93 ++++++++++++++++++++
 src/backend/utils/init/miscinit.c           |  2 +
 src/include/catalog/pg_proc.dat             |  9 ++
 src/include/miscadmin.h                     | 13 +++
 src/include/pgstat.h                        | 18 ++++
 src/include/storage/buf_internals.h         |  4 +-
 src/include/utils/backend_status.h          | 13 +++
 src/test/regress/expected/rules.out         |  7 ++
 src/test/regress/sql/stats.sql              |  1 +
 16 files changed, 425 insertions(+), 9 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 2281ba120f..edd19368be 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -444,6 +444,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
      </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_buffer_actions</structname><indexterm><primary>pg_stat_buffer_actions</primary></indexterm></entry>
+      <entry>One row for each backend type showing statistics about
+      backend buffer activity. See
+       <link linkend="monitoring-pg-stat-buffer-actions-view">
+       <structname>pg_stat_buffer_actions</structname></link> for details.
+     </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_wal</structname><indexterm><primary>pg_stat_wal</primary></indexterm></entry>
       <entry>One row only, showing statistics about WAL activity. See
@@ -3478,6 +3487,91 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
 
  </sect2>
 
+ <sect2 id="monitoring-pg-stat-buffer-actions-view">
+  <title><structname>pg_stat_buffer_actions</structname></title>
+
+  <indexterm>
+   <primary>pg_stat_buffer_actions</primary>
+  </indexterm>
+
+  <para>
+   The <structname>pg_stat_buffer_actions</structname> view has a row for each
+   backend type, containing global data for the cluster for that backend type.
+  </para>
+
+  <table id="pg-stat-buffer-actions-view" xreflabel="pg_stat_buffer_actions">
+   <title><structname>pg_stat_buffer_actions</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>backend_type</structfield> <type>text</type>
+      </para>
+      <para>
+       Type of backend (e.g. background worker, autovacuum worker).
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>buffers_alloc</structfield> <type>integer</type>
+      </para>
+      <para>
+       Number of buffers allocated.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>buffers_extend</structfield> <type>integer</type>
+      </para>
+      <para>
+       Number of buffers extended.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>buffers_fsync</structfield> <type>integer</type>
+      </para>
+      <para>
+       Number of buffers fsynced. TODO: is this only shared buffers?
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>buffers_write</structfield> <type>integer</type>
+      </para>
+      <para>
+       Number of buffers written.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>buffers_write_strat</structfield> <type>integer</type>
+      </para>
+      <para>
+       Number of buffers written as part of a buffer access strategy.
+      </para></entry>
+     </row>
+    </tbody>
+   </tgroup>
+  </table>
+
+ </sect2>
+
  <sect2 id="monitoring-pg-stat-wal-view">
    <title><structname>pg_stat_wal</structname></title>
 
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 55f6e3711d..7ba54d1119 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1072,6 +1072,16 @@ CREATE VIEW pg_stat_bgwriter AS
         pg_stat_get_buf_alloc() AS buffers_alloc,
         pg_stat_get_bgwriter_stat_reset_time() AS stats_reset;
 
+CREATE VIEW pg_stat_buffer_actions AS
+SELECT
+       b.backend_type,
+       b.buffers_alloc,
+       b.buffers_extend,
+       b.buffers_fsync,
+       b.buffers_write,
+       b.buffers_write_strat
+FROM pg_stat_get_buffer_actions() b;
+
 CREATE VIEW pg_stat_wal AS
     SELECT
         w.wal_records,
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index be7366379d..23f2ffccd9 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -1104,6 +1104,7 @@ ForwardSyncRequest(const FileTag *ftag, SyncRequestType type)
 		 */
 		if (!AmBackgroundWriterProcess())
 			CheckpointerShmem->num_backend_fsync++;
+		pgstat_increment_buffer_action(BA_Fsync);
 		LWLockRelease(CheckpointerCommLock);
 		return false;
 	}
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 3450a10129..244e5a7e44 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -63,6 +63,7 @@
 #include "storage/pg_shmem.h"
 #include "storage/proc.h"
 #include "storage/procsignal.h"
+#include "utils/backend_status.h"
 #include "utils/builtins.h"
 #include "utils/guc.h"
 #include "utils/memutils.h"
@@ -124,12 +125,16 @@ char	   *pgstat_stat_filename = NULL;
 char	   *pgstat_stat_tmpname = NULL;
 
 /*
- * BgWriter and WAL global statistics counters.
- * Stored directly in a stats message structure so they can be sent
- * without needing to copy things around.  We assume these init to zeroes.
+ * BgWriter, Checkpointer, WAL, and I/O global statistics counters. I/O global
+ * statistics on various buffer actions are tracked in PgBackendStatus while a
+ * backend is alive and then sent to stats collector before a backend exits in
+ * a PgStat_MsgBufferActions.
+ * All others are stored directly in a stats message structure so they can be
+ * sent without needing to copy things around.  We assume these init to zeroes.
  */
 PgStat_MsgBgWriter PendingBgWriterStats;
 PgStat_MsgCheckpointer PendingCheckpointerStats;
+PgStat_MsgBufferActions BufferActionsStats;
 PgStat_MsgWal WalStats;
 
 /*
@@ -359,6 +364,7 @@ static void pgstat_recv_analyze(PgStat_MsgAnalyze *msg, int len);
 static void pgstat_recv_archiver(PgStat_MsgArchiver *msg, int len);
 static void pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len);
 static void pgstat_recv_checkpointer(PgStat_MsgCheckpointer *msg, int len);
+static void pgstat_recv_buffer_actions(PgStat_MsgBufferActions *msg, int len);
 static void pgstat_recv_wal(PgStat_MsgWal *msg, int len);
 static void pgstat_recv_slru(PgStat_MsgSLRU *msg, int len);
 static void pgstat_recv_funcstat(PgStat_MsgFuncstat *msg, int len);
@@ -970,6 +976,8 @@ pgstat_report_stat(bool disconnect)
 	/* Now, send function statistics */
 	pgstat_send_funcstats();
 
+	pgstat_send_buffer_actions();
+
 	/* Send WAL statistics */
 	pgstat_send_wal(true);
 
@@ -3085,6 +3093,35 @@ pgstat_send_checkpointer(void)
 	MemSet(&PendingCheckpointerStats, 0, sizeof(PendingCheckpointerStats));
 }
 
+/*
+ * Called for a single backend at the time of death to send its I/O stats to
+ * the stats collector so that they may be persisted.
+ */
+void
+pgstat_send_buffer_actions(void)
+{
+	volatile	PgBackendStatus *beentry = MyBEEntry;
+	if (!beentry)
+		return;
+
+	BufferActionsStats = (PgStat_MsgBufferActions) {
+		.backend_type = beentry->st_backendType,
+		.allocs = pg_atomic_read_u64(&beentry->buffer_action_stats.allocs),
+		.extends = pg_atomic_read_u64(&beentry->buffer_action_stats.extends),
+		.fsyncs = pg_atomic_read_u64(&beentry->buffer_action_stats.fsyncs),
+		.writes = pg_atomic_read_u64(&beentry->buffer_action_stats.writes),
+		.writes_strat = pg_atomic_read_u64(&beentry->buffer_action_stats.writes_strat)
+	};
+
+	pgstat_setheader(&BufferActionsStats.m_hdr, PGSTAT_MTYPE_BUFFER_ACTIONS);
+	pgstat_send(&BufferActionsStats, sizeof(BufferActionsStats));
+
+	/*
+	 * pgstat_send_buffer_actions() is only called before a backend exits, so
+	 * BufferActionsStats should not be reused.
+	 */
+}
+
 /* ----------
  * pgstat_send_wal() -
  *
@@ -3427,6 +3464,10 @@ PgstatCollectorMain(int argc, char *argv[])
 					pgstat_recv_checkpointer(&msg.msg_checkpointer, len);
 					break;
 
+				case PGSTAT_MTYPE_BUFFER_ACTIONS:
+					pgstat_recv_buffer_actions(&msg.msg_buffer_actions, len);
+					break;
+
 				case PGSTAT_MTYPE_WAL:
 					pgstat_recv_wal(&msg.msg_wal, len);
 					break;
@@ -5406,6 +5447,30 @@ pgstat_recv_checkpointer(PgStat_MsgCheckpointer *msg, int len)
 	globalStats.checkpointer.buf_fsync_backend += msg->m_buf_fsync_backend;
 }
 
+static void
+pgstat_recv_buffer_actions(PgStat_MsgBufferActions *msg, int len)
+{
+	/*
+	 * No users will likely need PgStat_MsgBufferActions->backend_type when
+	 * accessing it from globalStats since its place in the
+	 * globalStats.buffer_actions array indicates backend_type. However,
+	 * leaving it undefined seemed like an invitation for unnecessary future
+	 * bugs.
+	 */
+	globalStats.buffer_actions[msg->backend_type].backend_type = msg->backend_type;
+	globalStats.buffer_actions[msg->backend_type].allocs += msg->allocs;
+	globalStats.buffer_actions[msg->backend_type].extends += msg->extends;
+	globalStats.buffer_actions[msg->backend_type].fsyncs += msg->fsyncs;
+	globalStats.buffer_actions[msg->backend_type].writes += msg->writes;
+	globalStats.buffer_actions[msg->backend_type].writes_strat += msg->writes_strat;
+}
+
+const PgStat_MsgBufferActions *
+pgstat_get_buffer_action_stats(BackendType backend_type)
+{
+	return &globalStats.buffer_actions[backend_type];
+}
+
 /* ----------
  * pgstat_recv_wal() -
  *
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index bc1753ae91..ef83c576b0 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -963,6 +963,10 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (isExtend)
 	{
+		/*
+		 * Extends counted here are only those that go through shared buffers
+		 */
+		pgstat_increment_buffer_action(BA_Extend);
 		/* new buffers are zero-filled */
 		MemSet((char *) bufBlock, 0, BLCKSZ);
 		/* don't set checksum for all-zero page */
@@ -1163,6 +1167,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 	/* Loop here in case we have to try another victim buffer */
 	for (;;)
 	{
+		bool from_ring;
 		/*
 		 * Ensure, while the spinlock's not yet held, that there's a free
 		 * refcount entry.
@@ -1173,7 +1178,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 		 * Select a victim buffer.  The buffer is returned with its header
 		 * spinlock still held!
 		 */
-		buf = StrategyGetBuffer(strategy, &buf_state);
+		buf = StrategyGetBuffer(strategy, &buf_state, &from_ring);
 
 		Assert(BUF_STATE_GET_REFCOUNT(buf_state) == 0);
 
@@ -1210,6 +1215,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 			if (LWLockConditionalAcquire(BufferDescriptorGetContentLock(buf),
 										 LW_SHARED))
 			{
+				BufferActionType buffer_action;
 				/*
 				 * If using a nondefault strategy, and writing the buffer
 				 * would require a WAL flush, let the strategy decide whether
@@ -1227,7 +1233,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 					UnlockBufHdr(buf, buf_state);
 
 					if (XLogNeedsFlush(lsn) &&
-						StrategyRejectBuffer(strategy, buf))
+						StrategyRejectBuffer(strategy, buf, &from_ring))
 					{
 						/* Drop lock/pin and loop around for another buffer */
 						LWLockRelease(BufferDescriptorGetContentLock(buf));
@@ -1236,6 +1242,21 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 					}
 				}
 
+				/*
+				 * When a strategy is in use, if the dirty buffer was selected
+				 * from the strategy ring and we did not bother checking the
+				 * freelist or doing a clock sweep to look for a clean shared
+				 * buffer to use, the write will be counted as a strategy
+				 * write. However, if the dirty buffer was obtained from the
+				 * freelist or a clock sweep, it is counted as a regular write.
+				 * When a strategy is not in use, at this point, the write can
+				 * only be a "regular" write of a dirty buffer.
+				 */
+
+				buffer_action = from_ring ? BA_Write_Strat : BA_Write;
+				pgstat_increment_buffer_action(buffer_action);
+
+
 				/* OK, do the I/O */
 				TRACE_POSTGRESQL_BUFFER_WRITE_DIRTY_START(forkNum, blockNum,
 														  smgr->smgr_rnode.node.spcNode,
@@ -2543,6 +2564,8 @@ SyncOneBuffer(int buf_id, bool skip_recently_used, WritebackContext *wb_context)
 	 * Pin it, share-lock it, write it.  (FlushBuffer will do nothing if the
 	 * buffer is clean by the time we've locked it.)
 	 */
+
+	pgstat_increment_buffer_action(BA_Write);
 	PinBuffer_Locked(bufHdr);
 	LWLockAcquire(BufferDescriptorGetContentLock(bufHdr), LW_SHARED);
 
diff --git a/src/backend/storage/buffer/freelist.c b/src/backend/storage/buffer/freelist.c
index 6be80476db..17b76e9c2c 100644
--- a/src/backend/storage/buffer/freelist.c
+++ b/src/backend/storage/buffer/freelist.c
@@ -19,6 +19,7 @@
 #include "storage/buf_internals.h"
 #include "storage/bufmgr.h"
 #include "storage/proc.h"
+#include "utils/backend_status.h"
 
 #define INT_ACCESS_ONCE(var)	((int)(*((volatile int *)&(var))))
 
@@ -198,7 +199,7 @@ have_free_buffer(void)
  *	return the buffer with the buffer header spinlock still held.
  */
 BufferDesc *
-StrategyGetBuffer(BufferAccessStrategy strategy, uint32 *buf_state)
+StrategyGetBuffer(BufferAccessStrategy strategy, uint32 *buf_state, bool *from_ring)
 {
 	BufferDesc *buf;
 	int			bgwprocno;
@@ -212,6 +213,7 @@ StrategyGetBuffer(BufferAccessStrategy strategy, uint32 *buf_state)
 	if (strategy != NULL)
 	{
 		buf = GetBufferFromRing(strategy, buf_state);
+		*from_ring = buf == NULL ? false : true;
 		if (buf != NULL)
 			return buf;
 	}
@@ -247,6 +249,7 @@ StrategyGetBuffer(BufferAccessStrategy strategy, uint32 *buf_state)
 	 * the rate of buffer consumption.  Note that buffers recycled by a
 	 * strategy object are intentionally not counted here.
 	 */
+	pgstat_increment_buffer_action(BA_Alloc);
 	pg_atomic_fetch_add_u32(&StrategyControl->numBufferAllocs, 1);
 
 	/*
@@ -683,8 +686,14 @@ AddBufferToRing(BufferAccessStrategy strategy, BufferDesc *buf)
  * if this buffer should be written and re-used.
  */
 bool
-StrategyRejectBuffer(BufferAccessStrategy strategy, BufferDesc *buf)
+StrategyRejectBuffer(BufferAccessStrategy strategy, BufferDesc *buf, bool *from_ring)
 {
+	/*
+	 * If we decide to use the dirty buffer selected by StrategyGetBuffer, then
+	 * ensure that we count it as such in pg_stat_buffer_actions view.
+	 */
+	*from_ring = true;
+
 	/* We only do this in bulkread mode */
 	if (strategy->btype != BAS_BULKREAD)
 		return false;
@@ -700,5 +709,14 @@ StrategyRejectBuffer(BufferAccessStrategy strategy, BufferDesc *buf)
 	 */
 	strategy->buffers[strategy->current] = InvalidBuffer;
 
+	/*
+	 * Since we will not be writing out a dirty buffer from the ring, set
+	 * from_ring to false so that the caller does not count this write as a
+	 * "strategy write" and can do proper bookkeeping for
+	 * pg_stat_buffer_actions.
+	 */
+	*from_ring = false;
+
+
 	return true;
 }
diff --git a/src/backend/utils/activity/backend_status.c b/src/backend/utils/activity/backend_status.c
index e19c4506ef..6f1e1c30d2 100644
--- a/src/backend/utils/activity/backend_status.c
+++ b/src/backend/utils/activity/backend_status.c
@@ -399,6 +399,11 @@ pgstat_bestart(void)
 	lbeentry.st_progress_command = PROGRESS_COMMAND_INVALID;
 	lbeentry.st_progress_command_target = InvalidOid;
 	lbeentry.st_query_id = UINT64CONST(0);
+	pg_atomic_init_u64(&lbeentry.buffer_action_stats.allocs, 0);
+	pg_atomic_init_u64(&lbeentry.buffer_action_stats.extends, 0);
+	pg_atomic_init_u64(&lbeentry.buffer_action_stats.fsyncs, 0);
+	pg_atomic_init_u64(&lbeentry.buffer_action_stats.writes, 0);
+	pg_atomic_init_u64(&lbeentry.buffer_action_stats.writes_strat, 0);
 
 	/*
 	 * we don't zero st_progress_param here to save cycles; nobody should
@@ -1045,6 +1050,50 @@ pgstat_get_my_query_id(void)
 	return MyBEEntry->st_query_id;
 }
 
+volatile PgBackendStatus *
+pgstat_access_backend_status_array(void)
+{
+	return BackendStatusArray;
+}
+
+void
+pgstat_increment_buffer_action(BufferActionType ba_type)
+{
+	volatile PgBackendStatus *beentry   = MyBEEntry;
+
+	if (!beentry || !pgstat_track_activities)
+		return;
+
+	switch (ba_type)
+	{
+		case BA_Alloc:
+			pg_atomic_write_u64(&beentry->buffer_action_stats.allocs,
+					pg_atomic_read_u64(&beentry->buffer_action_stats.allocs) + 1);
+			break;
+		case BA_Extend:
+			pg_atomic_write_u64(&beentry->buffer_action_stats.extends,
+					pg_atomic_read_u64(&beentry->buffer_action_stats.extends) + 1);
+			break;
+		case BA_Fsync:
+			pg_atomic_write_u64(&beentry->buffer_action_stats.fsyncs,
+					pg_atomic_read_u64(&beentry->buffer_action_stats.fsyncs) + 1);
+			break;
+		case BA_Write:
+			pg_atomic_write_u64(&beentry->buffer_action_stats.writes,
+					pg_atomic_read_u64(&beentry->buffer_action_stats.writes) + 1);
+			break;
+		case BA_Write_Strat:
+			pg_atomic_write_u64(&beentry->buffer_action_stats.writes_strat,
+					pg_atomic_read_u64(&beentry->buffer_action_stats.writes_strat) + 1);
+			break;
+		default:
+			ereport(LOG,
+					(errmsg(
+							"Statistics on Buffer Action Type, %d, are not currently collected.",
+							ba_type)));
+	}
+
+}
 
 /* ----------
  * pgstat_fetch_stat_beentry() -
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index ff5aedc99c..278257de80 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1796,6 +1796,99 @@ pg_stat_get_buf_alloc(PG_FUNCTION_ARGS)
 	PG_RETURN_INT64(pgstat_fetch_stat_bgwriter()->buf_alloc);
 }
 
+Datum
+pg_stat_get_buffer_actions(PG_FUNCTION_ARGS)
+{
+	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	TupleDesc	tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext per_query_ctx;
+	MemoryContext oldcontext;
+	const PgStat_MsgBufferActions *buffer_actions;
+	int i;
+	volatile PgBackendStatus *beentry;
+	Datum all_values[BACKEND_NUM_TYPES][BUFFER_ACTION_NUM_TYPES];
+	bool all_nulls[BACKEND_NUM_TYPES][BUFFER_ACTION_NUM_TYPES];
+
+	/* check to see if caller supports us returning a tuplestore */
+	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("set-valued function called in context that cannot accept a set")));
+	if (!(rsinfo->allowedModes & SFRM_Materialize))
+		ereport(ERROR,
+				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+				 errmsg("materialize mode required, but it is not allowed in this context")));
+
+	/* Build a tuple descriptor for our result type */
+	if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
+		elog(ERROR, "return type must be a row type");
+
+	per_query_ctx = rsinfo->econtext->ecxt_per_query_memory;
+	oldcontext = MemoryContextSwitchTo(per_query_ctx);
+
+	tupstore = tuplestore_begin_heap(true, false, work_mem);
+	rsinfo->returnMode = SFRM_Materialize;
+	rsinfo->setResult = tupstore;
+	rsinfo->setDesc = tupdesc;
+
+	MemoryContextSwitchTo(oldcontext);
+
+	MemSet(all_values, 0, sizeof(Datum[BACKEND_NUM_TYPES][BUFFER_ACTION_NUM_TYPES]));
+	MemSet(all_nulls, 0, sizeof(bool[BACKEND_NUM_TYPES][BUFFER_ACTION_NUM_TYPES]));
+
+	/* Add stats from all exited backends */
+	pgstat_fetch_global();
+	/* 0 is not a valid BackendType */
+	for (i = 1; i < BACKEND_NUM_TYPES; i++)
+	{
+		Datum *values = all_values[i];
+
+		values[0] = CStringGetTextDatum(GetBackendTypeDesc(i));
+		buffer_actions = pgstat_get_buffer_action_stats(i);
+
+		values[BA_Alloc] += buffer_actions->allocs;
+		values[BA_Extend] += buffer_actions->extends;
+		values[BA_Fsync] += buffer_actions->fsyncs;
+		values[BA_Write] += buffer_actions->writes;
+		values[BA_Write_Strat] += buffer_actions->writes_strat;
+	}
+
+	/*
+	 * Loop through all live backends and count their buffer actions
+	 */
+	beentry = pgstat_access_backend_status_array();
+	for (i = 0; i <= MaxBackends; i++)
+	{
+		Datum *values;
+		beentry++;
+		/* Don't count dead backends. They should already be counted */
+		if (beentry->st_procpid == 0)
+			continue;
+		values = all_values[beentry->st_backendType];
+
+
+		values[BA_Alloc] += pg_atomic_read_u64(&beentry->buffer_action_stats.allocs);
+		values[BA_Extend] += pg_atomic_read_u64(&beentry->buffer_action_stats.extends);
+		values[BA_Fsync] += pg_atomic_read_u64(&beentry->buffer_action_stats.fsyncs);
+		values[BA_Write] += pg_atomic_read_u64(&beentry->buffer_action_stats.writes);
+		values[BA_Write_Strat] += pg_atomic_read_u64(&beentry->buffer_action_stats.writes_strat);
+
+	}
+
+	for (i = 1; i < BACKEND_NUM_TYPES; i++)
+	{
+		Datum *values = all_values[i];
+		bool *nulls = all_nulls[i];
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	/* clean up and return the tuplestore */
+	tuplestore_donestoring(tupstore);
+
+	return (Datum) 0;
+}
+
 /*
  * Returns statistics of WAL activity
  */
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index 88801374b5..cbeaa9ab94 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -294,6 +294,8 @@ GetBackendTypeDesc(BackendType backendType)
 		case B_LOGGER:
 			backendDesc = "logger";
 			break;
+		case BACKEND_NUM_TYPES:
+			break;
 	}
 
 	return backendDesc;
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index d068d6532e..ee3c11db06 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5642,6 +5642,15 @@
   proname => 'pg_stat_get_buf_alloc', provolatile => 's', proparallel => 'r',
   prorettype => 'int8', proargtypes => '', prosrc => 'pg_stat_get_buf_alloc' },
 
+{ oid => '8459', descr => 'statistics: counts of buffer actions taken by each backend type',
+  proname => 'pg_stat_get_buffer_actions', provolatile => 's', proisstrict => 'f',
+  prorows => '13', proretset => 't',
+  proparallel => 'r', prorettype => 'record', proargtypes => '',
+  proallargtypes => '{text,int8,int8,int8,int8,int8}',
+  proargmodes => '{o,o,o,o,o,o}',
+  proargnames => '{backend_type,buffers_alloc,buffers_extend,buffers_fsync,buffers_write,buffers_write_strat}',
+  prosrc => 'pg_stat_get_buffer_actions' },
+
 { oid => '1136', descr => 'statistics: information about WAL activity',
   proname => 'pg_stat_get_wal', proisstrict => 'f', provolatile => 's',
   proparallel => 'r', prorettype => 'record', proargtypes => '',
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 2e2e9a364a..03d5e464a9 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -336,8 +336,21 @@ typedef enum BackendType
 	B_ARCHIVER,
 	B_STATS_COLLECTOR,
 	B_LOGGER,
+	BACKEND_NUM_TYPES,
 } BackendType;
 
+typedef enum BufferActionType
+{
+	BA_Invalid = 0,
+	BA_Alloc,
+	BA_Extend,
+	BA_Fsync,
+	BA_Write,
+	BA_Write_Strat,
+	BUFFER_ACTION_NUM_TYPES,
+}	BufferActionType;
+
+
 extern BackendType MyBackendType;
 
 extern const char *GetBackendTypeDesc(BackendType backendType);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 509849c7ff..57c642aeca 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -72,6 +72,7 @@ typedef enum StatMsgType
 	PGSTAT_MTYPE_ARCHIVER,
 	PGSTAT_MTYPE_BGWRITER,
 	PGSTAT_MTYPE_CHECKPOINTER,
+	PGSTAT_MTYPE_BUFFER_ACTIONS,
 	PGSTAT_MTYPE_WAL,
 	PGSTAT_MTYPE_SLRU,
 	PGSTAT_MTYPE_FUNCSTAT,
@@ -473,6 +474,18 @@ typedef struct PgStat_MsgCheckpointer
 	PgStat_Counter m_checkpoint_sync_time;
 } PgStat_MsgCheckpointer;
 
+typedef struct PgStat_MsgBufferActions
+{
+	PgStat_MsgHdr m_hdr;
+
+	BackendType backend_type;
+	uint64 allocs;
+	uint64 extends;
+	uint64 fsyncs;
+	uint64 writes;
+	uint64 writes_strat;
+} PgStat_MsgBufferActions;
+
 /* ----------
  * PgStat_MsgWal			Sent by backends and background processes to update WAL statistics.
  * ----------
@@ -691,6 +704,7 @@ typedef union PgStat_Msg
 	PgStat_MsgArchiver msg_archiver;
 	PgStat_MsgBgWriter msg_bgwriter;
 	PgStat_MsgCheckpointer msg_checkpointer;
+	PgStat_MsgBufferActions msg_buffer_actions;
 	PgStat_MsgWal msg_wal;
 	PgStat_MsgSLRU msg_slru;
 	PgStat_MsgFuncstat msg_funcstat;
@@ -866,6 +880,7 @@ typedef struct PgStat_GlobalStats
 
 	PgStat_CheckpointerStats checkpointer;
 	PgStat_BgWriterStats bgwriter;
+	PgStat_MsgBufferActions buffer_actions[BACKEND_NUM_TYPES];
 } PgStat_GlobalStats;
 
 /*
@@ -955,6 +970,7 @@ extern PgStat_MsgBgWriter PendingBgWriterStats;
  */
 extern PgStat_MsgCheckpointer PendingCheckpointerStats;
 
+extern PgStat_MsgBufferActions BufferActionsStats;
 /*
  * WAL statistics counter is updated by backends and background processes
  */
@@ -1105,6 +1121,8 @@ extern void pgstat_twophase_postabort(TransactionId xid, uint16 info,
 extern void pgstat_send_archiver(const char *xlog, bool failed);
 extern void pgstat_send_bgwriter(void);
 extern void pgstat_send_checkpointer(void);
+extern void pgstat_send_buffer_actions(void);
+extern const PgStat_MsgBufferActions * pgstat_get_buffer_action_stats(BackendType backend_type);
 extern void pgstat_send_wal(bool force);
 
 /* ----------
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index 33fcaf5c9a..7e385135db 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -310,10 +310,10 @@ extern void ScheduleBufferTagForWriteback(WritebackContext *context, BufferTag *
 
 /* freelist.c */
 extern BufferDesc *StrategyGetBuffer(BufferAccessStrategy strategy,
-									 uint32 *buf_state);
+									 uint32 *buf_state, bool *from_ring);
 extern void StrategyFreeBuffer(BufferDesc *buf);
 extern bool StrategyRejectBuffer(BufferAccessStrategy strategy,
-								 BufferDesc *buf);
+								 BufferDesc *buf, bool *from_ring);
 
 extern int	StrategySyncStart(uint32 *complete_passes, uint32 *num_buf_alloc);
 extern void StrategyNotifyBgWriter(int bgwprocno);
diff --git a/src/include/utils/backend_status.h b/src/include/utils/backend_status.h
index 8042b817df..c23b74b4a6 100644
--- a/src/include/utils/backend_status.h
+++ b/src/include/utils/backend_status.h
@@ -13,6 +13,7 @@
 #include "datatype/timestamp.h"
 #include "libpq/pqcomm.h"
 #include "miscadmin.h"			/* for BackendType */
+#include "port/atomics.h"
 #include "utils/backend_progress.h"
 
 
@@ -79,6 +80,14 @@ typedef struct PgBackendGSSStatus
 
 } PgBackendGSSStatus;
 
+typedef struct PgBackendBufferActionStats
+{
+	pg_atomic_uint64 allocs;
+	pg_atomic_uint64 extends;
+	pg_atomic_uint64 fsyncs;
+	pg_atomic_uint64 writes;
+	pg_atomic_uint64 writes_strat;
+} PgBackendBufferActionStats;
 
 /* ----------
  * PgBackendStatus
@@ -168,6 +177,7 @@ typedef struct PgBackendStatus
 
 	/* query identifier, optionally computed using post_parse_analyze_hook */
 	uint64		st_query_id;
+	PgBackendBufferActionStats buffer_action_stats;
 } PgBackendStatus;
 
 
@@ -306,6 +316,9 @@ extern const char *pgstat_get_crashed_backend_activity(int pid, char *buffer,
 													   int buflen);
 extern uint64 pgstat_get_my_query_id(void);
 
+extern volatile PgBackendStatus *pgstat_access_backend_status_array(void);
+extern void pgstat_increment_buffer_action(BufferActionType ba_type);
+
 
 /* ----------
  * Support functions for the SQL-callable functions to
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 2fa00a3c29..5c5445bcd7 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1828,6 +1828,13 @@ pg_stat_bgwriter| SELECT pg_stat_get_bgwriter_timed_checkpoints() AS checkpoints
     pg_stat_get_buf_fsync_backend() AS buffers_backend_fsync,
     pg_stat_get_buf_alloc() AS buffers_alloc,
     pg_stat_get_bgwriter_stat_reset_time() AS stats_reset;
+pg_stat_buffer_actions| SELECT b.backend_type,
+    b.buffers_alloc,
+    b.buffers_extend,
+    b.buffers_fsync,
+    b.buffers_write,
+    b.buffers_write_strat
+   FROM pg_stat_get_buffer_actions() b(backend_type, buffers_alloc, buffers_extend, buffers_fsync, buffers_write, buffers_write_strat);
 pg_stat_database| SELECT d.oid AS datid,
     d.datname,
         CASE
diff --git a/src/test/regress/sql/stats.sql b/src/test/regress/sql/stats.sql
index feaaee6326..fb4b613d4b 100644
--- a/src/test/regress/sql/stats.sql
+++ b/src/test/regress/sql/stats.sql
@@ -176,4 +176,5 @@ FROM prevstats AS pr;
 
 DROP TABLE trunc_stats_test, trunc_stats_test1, trunc_stats_test2, trunc_stats_test3, trunc_stats_test4;
 DROP TABLE prevstats;
+SELECT * FROM pg_stat_buffer_actions;
 -- End of Stats Test
-- 
2.32.0

