Hi,

I've attached the v7 patch set.

Changes from v6:
- removed unnecessary global variable BufferActionsStats
- fixed the loop condition in pg_stat_get_buffer_actions()
- updated some comments
- removed buffers_checkpoint and buffers_clean from pg_stat_bgwriter
  view (now pg_stat_bgwriter view is mainly checkpointer statistics,
  which isn't great)
- instead of calling pgstat_send_buffer_actions() in
  pgstat_report_stat(), I renamed pgstat_send_buffer_actions() to
  pgstat_report_buffers() and call it directly from
  pgstat_shutdown_hook() for all types of processes (including processes
  with invalid MyDatabaseId [like auxiliary processes])

I began changing the code to add the stats reset timestamp to the
pg_stat_buffer_actions view, but, I realized that it will be kind of
distracting to have every row for every backend type have a stats reset
timestamp (since it will be the same timestamp over and over). If,
however, you could reset buffer stats for each backend type
individually, then, I could see having it. Otherwise, we could add a
function like pg_stat_get_stats_reset_time(viewname) where viewname
would be pg_stat_buffer_actions in our case. Though, maybe that is
annoying and not very usable--I'm not sure.

I also think it makes sense to rename the pg_stat_buffer_actions view to
pg_stat_buffers and to name the columns using both the buffer action
type and buffer type -- e.g. shared, strategy, local. This leaves open
the possibility of counting buffer actions done on other non-shared
buffers -- like those done while building indexes or those using local
buffers. The third patch in the set does this (I wanted to see if it
made sense before fixing it up into the first patch in the set).

This naming convention (BufferType_BufferActionType) made me think that
it might make sense to have two enumerations: one being the current
BufferActionType (which could also be called BufferAccessType though
that might get confusing with BufferAccessStrategyType and buffer access
strategies in general) and the other being BufferType (which would be
one of shared, local, index, etc).

I attached a patch with the outline of this idea
(buffer_type_enum_addition.patch). It doesn't work because
pg_stat_get_buffer_actions() uses the BufferActionType as an index into
the values array returned. If I wanted to use a combination of the two
enums as an indexing mechanism (BufferActionType and BufferType), we
would end up with a tuple having every combination of the two
enums--some of which aren't valid. It might not make sense to implement
this. I do think it is useful to think of these stats as a combination
of a buffer action and a type of buffer.

- Melanie
From 17bf27ad0a6ae54a6a898e96c630d36867e9d943 Mon Sep 17 00:00:00 2001
From: Melanie Plageman <melanieplage...@gmail.com>
Date: Thu, 2 Sep 2021 11:33:59 -0400
Subject: [PATCH v7 1/3] 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             | 72 +++++++++++++++-
 src/backend/storage/buffer/bufmgr.c         | 27 +++++-
 src/backend/storage/buffer/freelist.c       | 22 ++++-
 src/backend/utils/activity/backend_status.c | 51 ++++++++++-
 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                        | 17 ++++
 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, 426 insertions(+), 10 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..75db7a1995 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,9 +125,12 @@ 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;
@@ -359,6 +363,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 +975,7 @@ pgstat_report_stat(bool disconnect)
 	/* Now, send function statistics */
 	pgstat_send_funcstats();
 
+
 	/* Send WAL statistics */
 	pgstat_send_wal(true);
 
@@ -2903,6 +2909,13 @@ static void
 pgstat_shutdown_hook(int code, Datum arg)
 {
 	Assert(!pgstat_is_shutdown);
+	/*
+	 * Only need to send stats on buffer actions when a process exits, as
+	 * pg_stat_get_buffer_actions() will read from live backends'
+	 * PgBackendStatus and then sum this with totals from exited backends
+	 * persisted by the stats collector.
+	 */
+	pgstat_report_buffers();
 
 	/*
 	 * If we got as far as discovering our own database ID, we can report what
@@ -3085,6 +3098,31 @@ 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_report_buffers(void)
+{
+	PgStat_MsgBufferActions buffer_actions_stats;
+	volatile	PgBackendStatus *beentry = MyBEEntry;
+	if (!beentry)
+		return;
+
+	buffer_actions_stats = (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(&buffer_actions_stats.m_hdr, PGSTAT_MTYPE_BUFFER_ACTIONS);
+	pgstat_send(&buffer_actions_stats, sizeof(buffer_actions_stats));
+}
+
 /* ----------
  * pgstat_send_wal() -
  *
@@ -3427,6 +3465,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 +5448,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..f8f914ac7e 100644
--- a/src/backend/utils/activity/backend_status.c
+++ b/src/backend/utils/activity/backend_status.c
@@ -279,7 +279,7 @@ pgstat_beinit(void)
  * pgstat_bestart() -
  *
  *	Initialize this backend's entry in the PgBackendStatus array.
- *	Called from InitPostgres.
+ *	Called from InitPostgres and AuxiliaryProcessMain
  *
  *	Apart from auxiliary processes, MyBackendId, MyDatabaseId,
  *	session userid, and application_name must be set for a
@@ -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..4ba492c121 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 + NUM_AUXPROCTYPES; 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..21f5f24e8c 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;
 
 /*
@@ -1105,6 +1120,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_report_buffers(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.27.0

From f853ecf1a44984158b2a06705c2f5d01bbace47e Mon Sep 17 00:00:00 2001
From: Melanie Plageman <melanieplage...@gmail.com>
Date: Thu, 2 Sep 2021 11:47:41 -0400
Subject: [PATCH v7 2/3] Remove superfluous bgwriter stats

Remove stats from pg_stat_bgwriter which are now more clearly expressed
in pg_stat_buffer_actions.
---
 doc/src/sgml/monitoring.sgml          | 47 ---------------------------
 src/backend/catalog/system_views.sql  |  5 ---
 src/backend/postmaster/checkpointer.c | 26 ---------------
 src/backend/postmaster/pgstat.c       |  5 ---
 src/backend/storage/buffer/bufmgr.c   |  6 ----
 src/backend/utils/adt/pgstatfuncs.c   | 30 -----------------
 src/include/catalog/pg_proc.dat       | 22 -------------
 src/include/pgstat.h                  | 10 ------
 src/test/regress/expected/rules.out   |  5 ---
 9 files changed, 156 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index edd19368be..96258ada18 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -3416,24 +3416,6 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       </para></entry>
      </row>
 
-     <row>
-      <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_checkpoint</structfield> <type>bigint</type>
-      </para>
-      <para>
-       Number of buffers written during checkpoints
-      </para></entry>
-     </row>
-
-     <row>
-      <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_clean</structfield> <type>bigint</type>
-      </para>
-      <para>
-       Number of buffers written by the background writer
-      </para></entry>
-     </row>
-
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
        <structfield>maxwritten_clean</structfield> <type>bigint</type>
@@ -3444,35 +3426,6 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
       </para></entry>
      </row>
 
-     <row>
-      <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_backend</structfield> <type>bigint</type>
-      </para>
-      <para>
-       Number of buffers written directly by a backend
-      </para></entry>
-     </row>
-
-     <row>
-      <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_backend_fsync</structfield> <type>bigint</type>
-      </para>
-      <para>
-       Number of times a backend had to execute its own
-       <function>fsync</function> call (normally the background writer handles those
-       even when the backend does its own write)
-      </para></entry>
-     </row>
-
-     <row>
-      <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_alloc</structfield> <type>bigint</type>
-      </para>
-      <para>
-       Number of buffers allocated
-      </para></entry>
-     </row>
-
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
        <structfield>stats_reset</structfield> <type>timestamp with time zone</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 7ba54d1119..a5d7972687 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1064,12 +1064,7 @@ CREATE VIEW pg_stat_bgwriter AS
         pg_stat_get_bgwriter_requested_checkpoints() AS checkpoints_req,
         pg_stat_get_checkpoint_write_time() AS checkpoint_write_time,
         pg_stat_get_checkpoint_sync_time() AS checkpoint_sync_time,
-        pg_stat_get_bgwriter_buf_written_checkpoints() AS buffers_checkpoint,
-        pg_stat_get_bgwriter_buf_written_clean() AS buffers_clean,
         pg_stat_get_bgwriter_maxwritten_clean() AS maxwritten_clean,
-        pg_stat_get_buf_written_backend() AS buffers_backend,
-        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;
 
 CREATE VIEW pg_stat_buffer_actions AS
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index 23f2ffccd9..fca78fa4ef 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -90,17 +90,9 @@
  * requesting backends since the last checkpoint start.  The flags are
  * chosen so that OR'ing is the correct way to combine multiple requests.
  *
- * num_backend_writes is used to count the number of buffer writes performed
- * by user backend processes.  This counter should be wide enough that it
- * can't overflow during a single processing cycle.  num_backend_fsync
- * counts the subset of those writes that also had to do their own fsync,
- * because the checkpointer failed to absorb their request.
- *
  * The requests array holds fsync requests sent by backends and not yet
  * absorbed by the checkpointer.
  *
- * Unlike the checkpoint fields, num_backend_writes, num_backend_fsync, and
- * the requests fields are protected by CheckpointerCommLock.
  *----------
  */
 typedef struct
@@ -124,9 +116,6 @@ typedef struct
 	ConditionVariable start_cv; /* signaled when ckpt_started advances */
 	ConditionVariable done_cv;	/* signaled when ckpt_done advances */
 
-	uint32		num_backend_writes; /* counts user backend buffer writes */
-	uint32		num_backend_fsync;	/* counts user backend fsync calls */
-
 	int			num_requests;	/* current # of requests */
 	int			max_requests;	/* allocated array size */
 	CheckpointerRequest requests[FLEXIBLE_ARRAY_MEMBER];
@@ -1085,10 +1074,6 @@ ForwardSyncRequest(const FileTag *ftag, SyncRequestType type)
 
 	LWLockAcquire(CheckpointerCommLock, LW_EXCLUSIVE);
 
-	/* Count all backend writes regardless of if they fit in the queue */
-	if (!AmBackgroundWriterProcess())
-		CheckpointerShmem->num_backend_writes++;
-
 	/*
 	 * If the checkpointer isn't running or the request queue is full, the
 	 * backend will have to perform its own fsync request.  But before forcing
@@ -1102,8 +1087,6 @@ ForwardSyncRequest(const FileTag *ftag, SyncRequestType type)
 		 * Count the subset of writes where backends have to do their own
 		 * fsync
 		 */
-		if (!AmBackgroundWriterProcess())
-			CheckpointerShmem->num_backend_fsync++;
 		pgstat_increment_buffer_action(BA_Fsync);
 		LWLockRelease(CheckpointerCommLock);
 		return false;
@@ -1261,15 +1244,6 @@ AbsorbSyncRequests(void)
 
 	LWLockAcquire(CheckpointerCommLock, LW_EXCLUSIVE);
 
-	/* Transfer stats counts into pending pgstats message */
-	PendingCheckpointerStats.m_buf_written_backend
-		+= CheckpointerShmem->num_backend_writes;
-	PendingCheckpointerStats.m_buf_fsync_backend
-		+= CheckpointerShmem->num_backend_fsync;
-
-	CheckpointerShmem->num_backend_writes = 0;
-	CheckpointerShmem->num_backend_fsync = 0;
-
 	/*
 	 * We try to avoid holding the lock for a long time by copying the request
 	 * array, and processing the requests after releasing the lock.
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 75db7a1995..43e88f488f 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -5425,9 +5425,7 @@ pgstat_recv_archiver(PgStat_MsgArchiver *msg, int len)
 static void
 pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len)
 {
-	globalStats.bgwriter.buf_written_clean += msg->m_buf_written_clean;
 	globalStats.bgwriter.maxwritten_clean += msg->m_maxwritten_clean;
-	globalStats.bgwriter.buf_alloc += msg->m_buf_alloc;
 }
 
 /* ----------
@@ -5443,9 +5441,6 @@ pgstat_recv_checkpointer(PgStat_MsgCheckpointer *msg, int len)
 	globalStats.checkpointer.requested_checkpoints += msg->m_requested_checkpoints;
 	globalStats.checkpointer.checkpoint_write_time += msg->m_checkpoint_write_time;
 	globalStats.checkpointer.checkpoint_sync_time += msg->m_checkpoint_sync_time;
-	globalStats.checkpointer.buf_written_checkpoints += msg->m_buf_written_checkpoints;
-	globalStats.checkpointer.buf_written_backend += msg->m_buf_written_backend;
-	globalStats.checkpointer.buf_fsync_backend += msg->m_buf_fsync_backend;
 }
 
 static void
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ef83c576b0..ff219038e2 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -2158,7 +2158,6 @@ BufferSync(int flags)
 			if (SyncOneBuffer(buf_id, false, &wb_context) & BUF_WRITTEN)
 			{
 				TRACE_POSTGRESQL_BUFFER_SYNC_WRITTEN(buf_id);
-				PendingCheckpointerStats.m_buf_written_checkpoints++;
 				num_written++;
 			}
 		}
@@ -2267,9 +2266,6 @@ BgBufferSync(WritebackContext *wb_context)
 	 */
 	strategy_buf_id = StrategySyncStart(&strategy_passes, &recent_alloc);
 
-	/* Report buffer alloc counts to pgstat */
-	PendingBgWriterStats.m_buf_alloc += recent_alloc;
-
 	/*
 	 * If we're not running the LRU scan, just stop after doing the stats
 	 * stuff.  We mark the saved state invalid so that we can recover sanely
@@ -2466,8 +2462,6 @@ BgBufferSync(WritebackContext *wb_context)
 			reusable_buffers++;
 	}
 
-	PendingBgWriterStats.m_buf_written_clean += num_written;
-
 #ifdef BGW_DEBUG
 	elog(DEBUG1, "bgwriter: recent_alloc=%u smoothed=%.2f delta=%ld ahead=%d density=%.2f reusable_est=%d upcoming_est=%d scanned=%d wrote=%d reusable=%d",
 		 recent_alloc, smoothed_alloc, strategy_delta, bufs_ahead,
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 4ba492c121..e373c20525 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1738,18 +1738,6 @@ pg_stat_get_bgwriter_requested_checkpoints(PG_FUNCTION_ARGS)
 	PG_RETURN_INT64(pgstat_fetch_stat_checkpointer()->requested_checkpoints);
 }
 
-Datum
-pg_stat_get_bgwriter_buf_written_checkpoints(PG_FUNCTION_ARGS)
-{
-	PG_RETURN_INT64(pgstat_fetch_stat_checkpointer()->buf_written_checkpoints);
-}
-
-Datum
-pg_stat_get_bgwriter_buf_written_clean(PG_FUNCTION_ARGS)
-{
-	PG_RETURN_INT64(pgstat_fetch_stat_bgwriter()->buf_written_clean);
-}
-
 Datum
 pg_stat_get_bgwriter_maxwritten_clean(PG_FUNCTION_ARGS)
 {
@@ -1778,24 +1766,6 @@ pg_stat_get_bgwriter_stat_reset_time(PG_FUNCTION_ARGS)
 	PG_RETURN_TIMESTAMPTZ(pgstat_fetch_stat_bgwriter()->stat_reset_timestamp);
 }
 
-Datum
-pg_stat_get_buf_written_backend(PG_FUNCTION_ARGS)
-{
-	PG_RETURN_INT64(pgstat_fetch_stat_checkpointer()->buf_written_backend);
-}
-
-Datum
-pg_stat_get_buf_fsync_backend(PG_FUNCTION_ARGS)
-{
-	PG_RETURN_INT64(pgstat_fetch_stat_checkpointer()->buf_fsync_backend);
-}
-
-Datum
-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)
 {
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index ee3c11db06..afab94ca96 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,16 +5600,6 @@
   proname => 'pg_stat_get_bgwriter_requested_checkpoints', provolatile => 's',
   proparallel => 'r', prorettype => 'int8', proargtypes => '',
   prosrc => 'pg_stat_get_bgwriter_requested_checkpoints' },
-{ oid => '2771',
-  descr => 'statistics: number of buffers written by the bgwriter during checkpoints',
-  proname => 'pg_stat_get_bgwriter_buf_written_checkpoints', provolatile => 's',
-  proparallel => 'r', prorettype => 'int8', proargtypes => '',
-  prosrc => 'pg_stat_get_bgwriter_buf_written_checkpoints' },
-{ oid => '2772',
-  descr => 'statistics: number of buffers written by the bgwriter for cleaning dirty buffers',
-  proname => 'pg_stat_get_bgwriter_buf_written_clean', provolatile => 's',
-  proparallel => 'r', prorettype => 'int8', proargtypes => '',
-  prosrc => 'pg_stat_get_bgwriter_buf_written_clean' },
 { oid => '2773',
   descr => 'statistics: number of times the bgwriter stopped processing when it had written too many buffers while cleaning',
   proname => 'pg_stat_get_bgwriter_maxwritten_clean', provolatile => 's',
@@ -5629,18 +5619,6 @@
   proname => 'pg_stat_get_checkpoint_sync_time', provolatile => 's',
   proparallel => 'r', prorettype => 'float8', proargtypes => '',
   prosrc => 'pg_stat_get_checkpoint_sync_time' },
-{ oid => '2775', descr => 'statistics: number of buffers written by backends',
-  proname => 'pg_stat_get_buf_written_backend', provolatile => 's',
-  proparallel => 'r', prorettype => 'int8', proargtypes => '',
-  prosrc => 'pg_stat_get_buf_written_backend' },
-{ oid => '3063',
-  descr => 'statistics: number of backend buffer writes that did their own fsync',
-  proname => 'pg_stat_get_buf_fsync_backend', provolatile => 's',
-  proparallel => 'r', prorettype => 'int8', proargtypes => '',
-  prosrc => 'pg_stat_get_buf_fsync_backend' },
-{ oid => '2859', descr => 'statistics: number of buffer allocations',
-  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',
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 21f5f24e8c..7c79995a2b 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -452,9 +452,7 @@ typedef struct PgStat_MsgBgWriter
 {
 	PgStat_MsgHdr m_hdr;
 
-	PgStat_Counter m_buf_written_clean;
 	PgStat_Counter m_maxwritten_clean;
-	PgStat_Counter m_buf_alloc;
 } PgStat_MsgBgWriter;
 
 /* ----------
@@ -467,9 +465,6 @@ typedef struct PgStat_MsgCheckpointer
 
 	PgStat_Counter m_timed_checkpoints;
 	PgStat_Counter m_requested_checkpoints;
-	PgStat_Counter m_buf_written_checkpoints;
-	PgStat_Counter m_buf_written_backend;
-	PgStat_Counter m_buf_fsync_backend;
 	PgStat_Counter m_checkpoint_write_time; /* times in milliseconds */
 	PgStat_Counter m_checkpoint_sync_time;
 } PgStat_MsgCheckpointer;
@@ -850,9 +845,7 @@ typedef struct PgStat_ArchiverStats
  */
 typedef struct PgStat_BgWriterStats
 {
-	PgStat_Counter buf_written_clean;
 	PgStat_Counter maxwritten_clean;
-	PgStat_Counter buf_alloc;
 	TimestampTz stat_reset_timestamp;
 } PgStat_BgWriterStats;
 
@@ -866,9 +859,6 @@ typedef struct PgStat_CheckpointerStats
 	PgStat_Counter requested_checkpoints;
 	PgStat_Counter checkpoint_write_time;	/* times in milliseconds */
 	PgStat_Counter checkpoint_sync_time;
-	PgStat_Counter buf_written_checkpoints;
-	PgStat_Counter buf_written_backend;
-	PgStat_Counter buf_fsync_backend;
 } PgStat_CheckpointerStats;
 
 /*
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 5c5445bcd7..f88c060370 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1821,12 +1821,7 @@ pg_stat_bgwriter| SELECT pg_stat_get_bgwriter_timed_checkpoints() AS checkpoints
     pg_stat_get_bgwriter_requested_checkpoints() AS checkpoints_req,
     pg_stat_get_checkpoint_write_time() AS checkpoint_write_time,
     pg_stat_get_checkpoint_sync_time() AS checkpoint_sync_time,
-    pg_stat_get_bgwriter_buf_written_checkpoints() AS buffers_checkpoint,
-    pg_stat_get_bgwriter_buf_written_clean() AS buffers_clean,
     pg_stat_get_bgwriter_maxwritten_clean() AS maxwritten_clean,
-    pg_stat_get_buf_written_backend() AS buffers_backend,
-    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,
-- 
2.27.0

From c60cf6ec100aae062cbaa9cab1639eb077bca6ed Mon Sep 17 00:00:00 2001
From: Melanie Plageman <melanieplage...@gmail.com>
Date: Mon, 13 Sep 2021 16:29:50 -0400
Subject: [PATCH v7 3/3] Rename pg_stat_buffer_actions to pg_stat_buffers

Also, rename all members of view to allow for future expansion of buffer
types covered.
---
 doc/src/sgml/monitoring.sgml          | 32 +++++++++++++--------------
 src/backend/catalog/system_views.sql  | 12 +++++-----
 src/backend/storage/buffer/freelist.c |  4 ++--
 src/include/catalog/pg_proc.dat       |  2 +-
 src/test/regress/expected/rules.out   | 14 ++++++------
 src/test/regress/sql/stats.sql        |  2 +-
 6 files changed, 33 insertions(+), 33 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 96258ada18..1ee46694f0 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -445,11 +445,11 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
      </row>
 
      <row>
-      <entry><structname>pg_stat_buffer_actions</structname><indexterm><primary>pg_stat_buffer_actions</primary></indexterm></entry>
+      <entry><structname>pg_stat_buffers</structname><indexterm><primary>pg_stat_buffers</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.
+       <structname>pg_stat_buffers</structname></link> for details.
      </entry>
      </row>
 
@@ -3441,19 +3441,19 @@ 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>
+  <title><structname>pg_stat_buffers</structname></title>
 
   <indexterm>
-   <primary>pg_stat_buffer_actions</primary>
+   <primary>pg_stat_buffers</primary>
   </indexterm>
 
   <para>
-   The <structname>pg_stat_buffer_actions</structname> view has a row for each
+   The <structname>pg_stat_buffers</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>
+  <table id="pg-stat-buffer-actions-view" xreflabel="pg_stat_buffers">
+   <title><structname>pg_stat_buffers</structname> View</title>
    <tgroup cols="1">
     <thead>
      <row>
@@ -3477,43 +3477,43 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_alloc</structfield> <type>integer</type>
+       <structfield>shared_buffers_alloc</structfield> <type>integer</type>
       </para>
       <para>
-       Number of buffers allocated.
+       Number of shared buffers allocated.
       </para></entry>
      </row>
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_extend</structfield> <type>integer</type>
+       <structfield>shared_buffers_extend</structfield> <type>integer</type>
       </para>
       <para>
-       Number of buffers extended.
+       Number of shared buffers extended.
       </para></entry>
      </row>
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_fsync</structfield> <type>integer</type>
+       <structfield>shared_buffers_fsync</structfield> <type>integer</type>
       </para>
       <para>
-       Number of buffers fsynced. TODO: is this only shared buffers?
+       Number of shared buffers fsynced.
       </para></entry>
      </row>
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_write</structfield> <type>integer</type>
+       <structfield>shared_buffers_write</structfield> <type>integer</type>
       </para>
       <para>
-       Number of buffers written.
+       Number of shared buffers written.
       </para></entry>
      </row>
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>buffers_write_strat</structfield> <type>integer</type>
+       <structfield>strategy_buffers_write</structfield> <type>integer</type>
       </para>
       <para>
        Number of buffers written as part of a buffer access strategy.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index a5d7972687..ff668fb256 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1067,14 +1067,14 @@ CREATE VIEW pg_stat_bgwriter AS
         pg_stat_get_bgwriter_maxwritten_clean() AS maxwritten_clean,
         pg_stat_get_bgwriter_stat_reset_time() AS stats_reset;
 
-CREATE VIEW pg_stat_buffer_actions AS
+CREATE VIEW pg_stat_buffers AS
 SELECT
        b.backend_type,
-       b.buffers_alloc,
-       b.buffers_extend,
-       b.buffers_fsync,
-       b.buffers_write,
-       b.buffers_write_strat
+       b.shared_buffers_alloc,
+       b.shared_buffers_extend,
+       b.shared_buffers_fsync,
+       b.shared_buffers_write,
+       b.strategy_buffers_write
 FROM pg_stat_get_buffer_actions() b;
 
 CREATE VIEW pg_stat_wal AS
diff --git a/src/backend/storage/buffer/freelist.c b/src/backend/storage/buffer/freelist.c
index 17b76e9c2c..c85ec3eec0 100644
--- a/src/backend/storage/buffer/freelist.c
+++ b/src/backend/storage/buffer/freelist.c
@@ -690,7 +690,7 @@ StrategyRejectBuffer(BufferAccessStrategy strategy, BufferDesc *buf, bool *from_
 {
 	/*
 	 * 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.
+	 * ensure that we count it as such in pg_stat_buffers view.
 	 */
 	*from_ring = true;
 
@@ -713,7 +713,7 @@ StrategyRejectBuffer(BufferAccessStrategy strategy, BufferDesc *buf, bool *from_
 	 * 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.
+	 * pg_stat_buffers.
 	 */
 	*from_ring = false;
 
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index afab94ca96..02d161aa5c 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5626,7 +5626,7 @@
   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}',
+  proargnames => '{backend_type,shared_buffers_alloc,shared_buffers_extend,shared_buffers_fsync,shared_buffers_write,strategy_buffers_write}',
   prosrc => 'pg_stat_get_buffer_actions' },
 
 { oid => '1136', descr => 'statistics: information about WAL activity',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index f88c060370..8a61ef93e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1823,13 +1823,13 @@ pg_stat_bgwriter| SELECT pg_stat_get_bgwriter_timed_checkpoints() AS checkpoints
     pg_stat_get_checkpoint_sync_time() AS checkpoint_sync_time,
     pg_stat_get_bgwriter_maxwritten_clean() AS maxwritten_clean,
     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_buffers| SELECT b.backend_type,
+    b.shared_buffers_alloc,
+    b.shared_buffers_extend,
+    b.shared_buffers_fsync,
+    b.shared_buffers_write,
+    b.strategy_buffers_write
+   FROM pg_stat_get_buffer_actions() b(backend_type, shared_buffers_alloc, shared_buffers_extend, shared_buffers_fsync, shared_buffers_write, strategy_buffers_write);
 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 fb4b613d4b..e908ac2591 100644
--- a/src/test/regress/sql/stats.sql
+++ b/src/test/regress/sql/stats.sql
@@ -176,5 +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;
+SELECT * FROM pg_stat_buffers;
 -- End of Stats Test
-- 
2.27.0

diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index e52599dc75..1b201ce829 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -1090,7 +1090,7 @@ ForwardSyncRequest(const FileTag *ftag, SyncRequestType type)
 		 * Count the subset of writes where backends have to do their own
 		 * fsync
 		 */
-		pgstat_increment_buffer_action(BA_Fsync);
+		pgstat_increment_buffer_action(BT_Shared, BA_Fsync);
 		LWLockRelease(CheckpointerCommLock);
 		return false;
 	}
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ff219038e2..0aa1ba8830 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -966,7 +966,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 		/*
 		 * Extends counted here are only those that go through shared buffers
 		 */
-		pgstat_increment_buffer_action(BA_Extend);
+		pgstat_increment_buffer_action(BT_Shared, BA_Extend);
 		/* new buffers are zero-filled */
 		MemSet((char *) bufBlock, 0, BLCKSZ);
 		/* don't set checksum for all-zero page */
@@ -1215,7 +1215,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 			if (LWLockConditionalAcquire(BufferDescriptorGetContentLock(buf),
 										 LW_SHARED))
 			{
-				BufferActionType buffer_action;
+				BufferType buffer_type;
 				/*
 				 * If using a nondefault strategy, and writing the buffer
 				 * would require a WAL flush, let the strategy decide whether
@@ -1253,8 +1253,8 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 				 * only be a "regular" write of a dirty buffer.
 				 */
 
-				buffer_action = from_ring ? BA_Write_Strat : BA_Write;
-				pgstat_increment_buffer_action(buffer_action);
+				buffer_type = from_ring ? BT_Strategy : BT_Shared;
+				pgstat_increment_buffer_action(buffer_type, BA_Write);
 
 
 				/* OK, do the I/O */
@@ -2559,7 +2559,7 @@ SyncOneBuffer(int buf_id, bool skip_recently_used, WritebackContext *wb_context)
 	 * buffer is clean by the time we've locked it.)
 	 */
 
-	pgstat_increment_buffer_action(BA_Write);
+	pgstat_increment_buffer_action(BT_Shared, 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 17b76e9c2c..fd95cba478 100644
--- a/src/backend/storage/buffer/freelist.c
+++ b/src/backend/storage/buffer/freelist.c
@@ -249,7 +249,7 @@ StrategyGetBuffer(BufferAccessStrategy strategy, uint32 *buf_state, bool *from_r
 	 * the rate of buffer consumption.  Note that buffers recycled by a
 	 * strategy object are intentionally not counted here.
 	 */
-	pgstat_increment_buffer_action(BA_Alloc);
+	pgstat_increment_buffer_action(BT_Shared, BA_Alloc);
 	pg_atomic_fetch_add_u32(&StrategyControl->numBufferAllocs, 1);
 
 	/*
diff --git a/src/backend/utils/activity/backend_status.c b/src/backend/utils/activity/backend_status.c
index f8f914ac7e..a0833a733b 100644
--- a/src/backend/utils/activity/backend_status.c
+++ b/src/backend/utils/activity/backend_status.c
@@ -1057,14 +1057,14 @@ pgstat_access_backend_status_array(void)
 }
 
 void
-pgstat_increment_buffer_action(BufferActionType ba_type)
+pgstat_increment_buffer_action(BufferType buffer_type, BufferActionType buffer_action_type)
 {
 	volatile PgBackendStatus *beentry   = MyBEEntry;
 
 	if (!beentry || !pgstat_track_activities)
 		return;
 
-	switch (ba_type)
+	switch (buffer_action_type)
 	{
 		case BA_Alloc:
 			pg_atomic_write_u64(&beentry->buffer_action_stats.allocs,
@@ -1079,18 +1079,18 @@ pgstat_increment_buffer_action(BufferActionType ba_type)
 					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);
+			if (buffer_type == BT_Strategy)
+				pg_atomic_write_u64(&beentry->buffer_action_stats.writes_strat,
+						pg_atomic_read_u64(&beentry->buffer_action_stats.writes_strat) + 1);
+			else if (buffer_type == BT_Shared)
+				pg_atomic_write_u64(&beentry->buffer_action_stats.writes,
+						pg_atomic_read_u64(&beentry->buffer_action_stats.writes) + 1);
 			break;
 		default:
 			ereport(LOG,
 					(errmsg(
 							"Statistics on Buffer Action Type, %d, are not currently collected.",
-							ba_type)));
+							buffer_action_type)));
 	}
 
 }
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 03d5e464a9..b6885a5d66 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -339,6 +339,15 @@ typedef enum BackendType
 	BACKEND_NUM_TYPES,
 } BackendType;
 
+typedef enum BufferType
+{
+	BT_Shared,
+	BT_Strategy,
+	BT_Local,
+	BUFFER_NUM_TYPES,
+} BufferType;
+
+// TODO: should BufferAction be BufferAccess? See BufferAccessStrategyType
 typedef enum BufferActionType
 {
 	BA_Invalid = 0,
@@ -346,7 +355,6 @@ typedef enum BufferActionType
 	BA_Extend,
 	BA_Fsync,
 	BA_Write,
-	BA_Write_Strat,
 	BUFFER_ACTION_NUM_TYPES,
 }	BufferActionType;
 
diff --git a/src/include/utils/backend_status.h b/src/include/utils/backend_status.h
index c23b74b4a6..fb1334401e 100644
--- a/src/include/utils/backend_status.h
+++ b/src/include/utils/backend_status.h
@@ -317,7 +317,7 @@ extern const char *pgstat_get_crashed_backend_activity(int pid, char *buffer,
 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);
+extern void pgstat_increment_buffer_action(BufferType buffer_type, BufferActionType buffer_action_type);
 
 
 /* ----------

Reply via email to