From 6a701ba5e069f03289f01ebf1bd41c73a0196aea Mon Sep 17 00:00:00 2001
From: Melanie Plageman <melanieplageman@gmail.com>
Date: Wed, 29 Jun 2022 18:37:42 -0400
Subject: [PATCH v22 2/3] Track IO operation statistics

Introduce IOOp, an IO operation done by a backend, and IOPath, the
location or type of IO done by a backend. For example, the checkpointer
may write a shared buffer out. This would be counted as an IOOp write on
an IOPath IOPATH_SHARED.

Stats on IOOps for all IOPaths for a backend are initially accumulated
locally.

Later they are flushed to shared memory and accumulated with those from
all other backends, exited and live.

Some BackendTypes will not execute pgstat_report_stat() and thus must
explicitly call pgstat_report_io_ops() in order to flush their backend
local IO operation statistics to shared memory.

Author: Melanie Plageman <melanieplageman@gmail.com>
Reviewed-by: Justin Pryzby <pryzby@telsasoft.com>
Discussion: https://www.postgresql.org/message-id/flat/20200124195226.lth52iydq2n2uilq%40alap3.anarazel.de
---
 doc/src/sgml/monitoring.sgml               |   2 +
 src/backend/bootstrap/bootstrap.c          |   3 +
 src/backend/postmaster/autovacuum.c        |   3 +
 src/backend/postmaster/bgwriter.c          |   1 +
 src/backend/postmaster/checkpointer.c      |   4 +
 src/backend/postmaster/startup.c           |   2 +
 src/backend/postmaster/walwriter.c         |   2 +
 src/backend/storage/buffer/bufmgr.c        |  44 ++-
 src/backend/storage/buffer/freelist.c      |  23 +-
 src/backend/storage/buffer/localbuf.c      |   3 +
 src/backend/storage/sync/sync.c            |   2 +
 src/backend/utils/activity/Makefile        |   1 +
 src/backend/utils/activity/pgstat.c        |  24 ++
 src/backend/utils/activity/pgstat_io_ops.c | 296 +++++++++++++++++++++
 src/backend/utils/adt/pgstatfuncs.c        |   4 +-
 src/include/miscadmin.h                    |   2 +
 src/include/pgstat.h                       |  51 ++++
 src/include/storage/buf_internals.h        |   4 +-
 src/include/utils/backend_status.h         |  34 +++
 src/include/utils/pgstat_internal.h        |  20 ++
 20 files changed, 509 insertions(+), 16 deletions(-)
 create mode 100644 src/backend/utils/activity/pgstat_io_ops.c

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4549c2560e..602f0a827a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -5355,6 +5355,8 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
         the <structname>pg_stat_bgwriter</structname>
         view, <literal>archiver</literal> to reset all the counters shown in
         the <structname>pg_stat_archiver</structname> view,
+        <literal>buffers</literal> to reset all the counters shown in the
+        <structname>pg_stat_buffers</structname> view,
         <literal>wal</literal> to reset all the counters shown in the
         <structname>pg_stat_wal</structname> view or
         <literal>recovery_prefetch</literal> to reset all the counters shown
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index 088556ab54..87298ede77 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -33,6 +33,7 @@
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "pg_getopt.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/bufpage.h"
 #include "storage/condition_variable.h"
@@ -375,6 +376,8 @@ BootstrapModeMain(int argc, char *argv[], bool check_only)
 	 * out the initial relation mapping files.
 	 */
 	RelationMapFinishBootstrap();
+	// TODO: should this be done for bootstrap?
+	pgstat_report_io_ops();
 
 	/* Clean up and exit */
 	cleanup();
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 2e146aac93..e6dbb1c4bb 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -1712,6 +1712,9 @@ AutoVacWorkerMain(int argc, char *argv[])
 		recentXid = ReadNextTransactionId();
 		recentMulti = ReadNextMultiXactId();
 		do_autovacuum();
+
+		// TODO: should this be done more often somewhere in do_autovacuum()?
+		pgstat_report_io_ops();
 	}
 
 	/*
diff --git a/src/backend/postmaster/bgwriter.c b/src/backend/postmaster/bgwriter.c
index 91e6f6ea18..87e4b9e9bd 100644
--- a/src/backend/postmaster/bgwriter.c
+++ b/src/backend/postmaster/bgwriter.c
@@ -242,6 +242,7 @@ BackgroundWriterMain(void)
 
 		/* Report pending statistics to the cumulative stats system */
 		pgstat_report_bgwriter();
+		pgstat_report_io_ops();
 
 		if (FirstCallSinceLastCheckpoint())
 		{
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index c937c39f50..1f72789739 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -504,6 +504,7 @@ CheckpointerMain(void)
 
 		/* Report pending statistics to the cumulative stats system */
 		pgstat_report_checkpointer();
+		pgstat_report_io_ops();
 		pgstat_report_wal(true);
 
 		/*
@@ -582,6 +583,7 @@ HandleCheckpointerInterrupts(void)
 		PendingCheckpointerStats.requested_checkpoints++;
 		ShutdownXLOG(0, 0);
 		pgstat_report_checkpointer();
+		pgstat_report_io_ops();
 		pgstat_report_wal(true);
 
 		/* Normal exit from the checkpointer is here */
@@ -726,6 +728,7 @@ CheckpointWriteDelay(int flags, double progress)
 
 		/* Report interim statistics to the cumulative stats system */
 		pgstat_report_checkpointer();
+		pgstat_report_io_ops();
 
 		/*
 		 * This sleep used to be connected to bgwriter_delay, typically 200ms.
@@ -1116,6 +1119,7 @@ ForwardSyncRequest(const FileTag *ftag, SyncRequestType type)
 		if (!AmBackgroundWriterProcess())
 			CheckpointerShmem->num_backend_fsync++;
 		LWLockRelease(CheckpointerCommLock);
+		pgstat_count_io_op(IOOP_FSYNC, IOPATH_SHARED);
 		return false;
 	}
 
diff --git a/src/backend/postmaster/startup.c b/src/backend/postmaster/startup.c
index f99186eab7..678663798d 100644
--- a/src/backend/postmaster/startup.c
+++ b/src/backend/postmaster/startup.c
@@ -266,6 +266,8 @@ StartupProcessMain(void)
 	 */
 	StartupXLOG();
 
+	pgstat_report_io_ops();
+
 	/*
 	 * Exit normally. Exit code 0 tells postmaster that we completed recovery
 	 * successfully.
diff --git a/src/backend/postmaster/walwriter.c b/src/backend/postmaster/walwriter.c
index e926f8c27c..f649282443 100644
--- a/src/backend/postmaster/walwriter.c
+++ b/src/backend/postmaster/walwriter.c
@@ -259,6 +259,7 @@ WalWriterMain(void)
 
 		/* report pending statistics to the cumulative stats system */
 		pgstat_report_wal(false);
+		pgstat_report_io_ops();
 
 		/*
 		 * Sleep until we are signaled or WalWriterDelay has elapsed.  If we
@@ -302,6 +303,7 @@ HandleWalWriterInterrupts(void)
 		 * exist unreported stats counters for the WAL writer.
 		 */
 		pgstat_report_wal(true);
+		pgstat_report_io_ops();
 
 		proc_exit(0);
 	}
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index ae13011d27..f1ddd696b1 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -482,7 +482,7 @@ static BufferDesc *BufferAlloc(SMgrRelation smgr,
 							   BlockNumber blockNum,
 							   BufferAccessStrategy strategy,
 							   bool *foundPtr);
-static void FlushBuffer(BufferDesc *buf, SMgrRelation reln);
+static void FlushBuffer(BufferDesc *buf, SMgrRelation reln, IOPath iopath);
 static void FindAndDropRelFileNodeBuffers(RelFileNode rnode,
 										  ForkNumber forkNum,
 										  BlockNumber nForkBlock,
@@ -980,6 +980,7 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 
 	if (isExtend)
 	{
+		pgstat_count_io_op(IOOP_EXTEND, IOPATH_SHARED);
 		/* new buffers are zero-filled */
 		MemSet((char *) bufBlock, 0, BLCKSZ);
 		/* don't set checksum for all-zero page */
@@ -1180,6 +1181,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.
@@ -1190,7 +1192,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);
 
@@ -1227,6 +1229,7 @@ BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 			if (LWLockConditionalAcquire(BufferDescriptorGetContentLock(buf),
 										 LW_SHARED))
 			{
+				IOPath iopath;
 				/*
 				 * If using a nondefault strategy, and writing the buffer
 				 * would require a WAL flush, let the strategy decide whether
@@ -1244,7 +1247,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));
@@ -1253,13 +1256,27 @@ 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.
+				 */
+
+				iopath = from_ring ? IOPATH_STRATEGY : IOPATH_SHARED;
+
 				/* OK, do the I/O */
 				TRACE_POSTGRESQL_BUFFER_WRITE_DIRTY_START(forkNum, blockNum,
 														  smgr->smgr_rnode.node.spcNode,
 														  smgr->smgr_rnode.node.dbNode,
 														  smgr->smgr_rnode.node.relNode);
 
-				FlushBuffer(buf, NULL);
+				FlushBuffer(buf, NULL, iopath);
 				LWLockRelease(BufferDescriptorGetContentLock(buf));
 
 				ScheduleBufferTagForWriteback(&BackendWritebackContext,
@@ -2563,7 +2580,7 @@ SyncOneBuffer(int buf_id, bool skip_recently_used, WritebackContext *wb_context)
 	PinBuffer_Locked(bufHdr);
 	LWLockAcquire(BufferDescriptorGetContentLock(bufHdr), LW_SHARED);
 
-	FlushBuffer(bufHdr, NULL);
+	FlushBuffer(bufHdr, NULL, IOPATH_SHARED);
 
 	LWLockRelease(BufferDescriptorGetContentLock(bufHdr));
 
@@ -2810,9 +2827,12 @@ BufferGetTag(Buffer buffer, RelFileNode *rnode, ForkNumber *forknum,
  *
  * If the caller has an smgr reference for the buffer's relation, pass it
  * as the second parameter.  If not, pass NULL.
+ *
+ * IOPath will always be IOPATH_SHARED except when a buffer access strategy is
+ * used and the buffer being flushed is a buffer from the strategy ring.
  */
 static void
-FlushBuffer(BufferDesc *buf, SMgrRelation reln)
+FlushBuffer(BufferDesc *buf, SMgrRelation reln, IOPath iopath)
 {
 	XLogRecPtr	recptr;
 	ErrorContextCallback errcallback;
@@ -2892,6 +2912,8 @@ FlushBuffer(BufferDesc *buf, SMgrRelation reln)
 	 */
 	bufToWrite = PageSetChecksumCopy((Page) bufBlock, buf->tag.blockNum);
 
+	pgstat_count_io_op(IOOP_WRITE, iopath);
+
 	if (track_io_timing)
 		INSTR_TIME_SET_CURRENT(io_start);
 
@@ -3540,6 +3562,8 @@ FlushRelationBuffers(Relation rel)
 						  localpage,
 						  false);
 
+				pgstat_count_io_op(IOOP_WRITE, IOPATH_LOCAL);
+
 				buf_state &= ~(BM_DIRTY | BM_JUST_DIRTIED);
 				pg_atomic_unlocked_write_u32(&bufHdr->state, buf_state);
 
@@ -3575,7 +3599,7 @@ FlushRelationBuffers(Relation rel)
 		{
 			PinBuffer_Locked(bufHdr);
 			LWLockAcquire(BufferDescriptorGetContentLock(bufHdr), LW_SHARED);
-			FlushBuffer(bufHdr, RelationGetSmgr(rel));
+			FlushBuffer(bufHdr, RelationGetSmgr(rel), IOPATH_SHARED);
 			LWLockRelease(BufferDescriptorGetContentLock(bufHdr));
 			UnpinBuffer(bufHdr, true);
 		}
@@ -3670,7 +3694,7 @@ FlushRelationsAllBuffers(SMgrRelation *smgrs, int nrels)
 		{
 			PinBuffer_Locked(bufHdr);
 			LWLockAcquire(BufferDescriptorGetContentLock(bufHdr), LW_SHARED);
-			FlushBuffer(bufHdr, srelent->srel);
+			FlushBuffer(bufHdr, srelent->srel, IOPATH_SHARED);
 			LWLockRelease(BufferDescriptorGetContentLock(bufHdr));
 			UnpinBuffer(bufHdr, true);
 		}
@@ -3878,7 +3902,7 @@ FlushDatabaseBuffers(Oid dbid)
 		{
 			PinBuffer_Locked(bufHdr);
 			LWLockAcquire(BufferDescriptorGetContentLock(bufHdr), LW_SHARED);
-			FlushBuffer(bufHdr, NULL);
+			FlushBuffer(bufHdr, NULL, IOPATH_SHARED);
 			LWLockRelease(BufferDescriptorGetContentLock(bufHdr));
 			UnpinBuffer(bufHdr, true);
 		}
@@ -3905,7 +3929,7 @@ FlushOneBuffer(Buffer buffer)
 
 	Assert(LWLockHeldByMe(BufferDescriptorGetContentLock(bufHdr)));
 
-	FlushBuffer(bufHdr, NULL);
+	FlushBuffer(bufHdr, NULL, IOPATH_SHARED);
 }
 
 /*
diff --git a/src/backend/storage/buffer/freelist.c b/src/backend/storage/buffer/freelist.c
index 990e081aae..afc22a4203 100644
--- a/src/backend/storage/buffer/freelist.c
+++ b/src/backend/storage/buffer/freelist.c
@@ -15,6 +15,7 @@
  */
 #include "postgres.h"
 
+#include "pgstat.h"
 #include "port/atomics.h"
 #include "storage/buf_internals.h"
 #include "storage/bufmgr.h"
@@ -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,7 +213,8 @@ StrategyGetBuffer(BufferAccessStrategy strategy, uint32 *buf_state)
 	if (strategy != NULL)
 	{
 		buf = GetBufferFromRing(strategy, buf_state);
-		if (buf != NULL)
+		*from_ring = buf != NULL;
+		if (*from_ring)
 			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_count_io_op(IOOP_ALLOC, IOPATH_SHARED);
 	pg_atomic_fetch_add_u32(&StrategyControl->numBufferAllocs, 1);
 
 	/*
@@ -682,8 +685,15 @@ 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)
 {
+
+	/*
+	 * Start by assuming that we will use the dirty buffer selected by
+	 * StrategyGetBuffer().
+	 */
+	*from_ring = true;
+
 	/* We only do this in bulkread mode */
 	if (strategy->btype != BAS_BULKREAD)
 		return false;
@@ -699,5 +709,12 @@ 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.
+	 */
+	*from_ring = false;
+
 	return true;
 }
diff --git a/src/backend/storage/buffer/localbuf.c b/src/backend/storage/buffer/localbuf.c
index e71f95ac1f..cb7b1720a4 100644
--- a/src/backend/storage/buffer/localbuf.c
+++ b/src/backend/storage/buffer/localbuf.c
@@ -15,6 +15,7 @@
  */
 #include "postgres.h"
 
+#include "pgstat.h"
 #include "access/parallel.h"
 #include "catalog/catalog.h"
 #include "executor/instrument.h"
@@ -226,6 +227,8 @@ LocalBufferAlloc(SMgrRelation smgr, ForkNumber forkNum, BlockNumber blockNum,
 				  localpage,
 				  false);
 
+		pgstat_count_io_op(IOOP_WRITE, IOPATH_LOCAL);
+
 		/* Mark not-dirty now in case we error out below */
 		buf_state &= ~BM_DIRTY;
 		pg_atomic_unlocked_write_u32(&bufHdr->state, buf_state);
diff --git a/src/backend/storage/sync/sync.c b/src/backend/storage/sync/sync.c
index e1fb631003..20e259edef 100644
--- a/src/backend/storage/sync/sync.c
+++ b/src/backend/storage/sync/sync.c
@@ -432,6 +432,8 @@ ProcessSyncRequests(void)
 					total_elapsed += elapsed;
 					processed++;
 
+					pgstat_count_io_op(IOOP_FSYNC, IOPATH_SHARED);
+
 					if (log_checkpoints)
 						elog(DEBUG1, "checkpoint sync: number=%d file=%s time=%.3f ms",
 							 processed,
diff --git a/src/backend/utils/activity/Makefile b/src/backend/utils/activity/Makefile
index a2e8507fd6..0098785089 100644
--- a/src/backend/utils/activity/Makefile
+++ b/src/backend/utils/activity/Makefile
@@ -22,6 +22,7 @@ OBJS = \
 	pgstat_checkpointer.o \
 	pgstat_database.o \
 	pgstat_function.o \
+	pgstat_io_ops.o \
 	pgstat_relation.o \
 	pgstat_replslot.o \
 	pgstat_shmem.o \
diff --git a/src/backend/utils/activity/pgstat.c b/src/backend/utils/activity/pgstat.c
index 0d9d09c492..27507dfbb6 100644
--- a/src/backend/utils/activity/pgstat.c
+++ b/src/backend/utils/activity/pgstat.c
@@ -359,6 +359,15 @@ static const PgStat_KindInfo pgstat_kind_infos[PGSTAT_NUM_KINDS] = {
 		.snapshot_cb = pgstat_checkpointer_snapshot_cb,
 	},
 
+	[PGSTAT_KIND_IOOPS] = {
+		.name = "io_ops",
+
+		.fixed_amount = true,
+
+		.reset_all_cb = pgstat_io_ops_reset_all_cb,
+		.snapshot_cb = pgstat_io_ops_snapshot_cb,
+	},
+
 	[PGSTAT_KIND_SLRU] = {
 		.name = "slru",
 
@@ -628,6 +637,9 @@ pgstat_report_stat(bool force)
 	/* flush database / relation / function / ... stats */
 	partial_flush |= pgstat_flush_pending_entries(nowait);
 
+	/* flush IO Operations stats */
+	partial_flush |= pgstat_flush_io_ops(nowait);
+
 	/* flush wal stats */
 	partial_flush |= pgstat_flush_wal(nowait);
 
@@ -1312,6 +1324,12 @@ pgstat_write_statsfile(void)
 	pgstat_build_snapshot_fixed(PGSTAT_KIND_CHECKPOINTER);
 	write_chunk_s(fpout, &pgStatLocal.snapshot.checkpointer);
 
+	/*
+	 * Write IO Operations stats struct
+	 */
+	pgstat_build_snapshot_fixed(PGSTAT_KIND_IOOPS);
+	write_chunk_s(fpout, &pgStatLocal.snapshot.io_path_ops);
+
 	/*
 	 * Write SLRU stats struct
 	 */
@@ -1486,6 +1504,12 @@ pgstat_read_statsfile(void)
 	if (!read_chunk_s(fpin, &shmem->checkpointer.stats))
 		goto error;
 
+	/*
+	 * Read IO Operations stats struct
+	 */
+	if (!read_chunk_s(fpin, &shmem->io_ops.stats))
+		goto error;
+
 	/*
 	 * Read SLRU stats struct
 	 */
diff --git a/src/backend/utils/activity/pgstat_io_ops.c b/src/backend/utils/activity/pgstat_io_ops.c
new file mode 100644
index 0000000000..e184b0b484
--- /dev/null
+++ b/src/backend/utils/activity/pgstat_io_ops.c
@@ -0,0 +1,296 @@
+/* -------------------------------------------------------------------------
+ *
+ * pgstat_io_ops.c
+ *	  Implementation of IO operation statistics.
+ *
+ * This file contains the implementation of IO operation statistics. It is kept
+ * separate from pgstat.c to enforce the line between the statistics access /
+ * storage implementation and the details about individual types of
+ * statistics.
+ *
+ * Copyright (c) 2001-2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/backend/utils/activity/pgstat_io_ops.c
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "utils/pgstat_internal.h"
+
+static PgStat_IOPathOps pending_IOOpStats;
+static PgStat_IOPathOps cumulative_IOOpStats;
+bool have_ioopstats = false;
+
+
+/*
+ * Flush out locally pending IO Operation statistics entries
+ *
+ * If nowait is true, this function returns false on lock failure. Otherwise
+ * this function always returns true. Writer processes are mutually excluded
+ * using LWLock, but readers are expected to use change-count protocol to avoid
+ * interference with writers.
+ *
+ * If nowait is true, this function returns true if the lock could not be
+ * acquired. Otherwise return false.
+ *
+ */
+bool
+pgstat_flush_io_ops(bool nowait)
+{
+	PgStat_IOPathOps *dest_io_path_ops;
+	PgStatShared_BackendIOPathOps *stats_shmem;
+
+	PgBackendStatus *beentry = MyBEEntry;
+
+	if (!have_ioopstats)
+		return false;
+
+	if (!beentry || beentry->st_backendType == B_INVALID)
+		return false;
+
+	stats_shmem = &pgStatLocal.shmem->io_ops;
+
+	if (!nowait)
+		LWLockAcquire(&stats_shmem->lock, LW_EXCLUSIVE);
+	else if (!LWLockConditionalAcquire(&stats_shmem->lock, LW_EXCLUSIVE))
+		return true;
+
+	dest_io_path_ops =
+		&stats_shmem->stats[backend_type_get_idx(beentry->st_backendType)];
+
+	for (int i = 0; i < IOPATH_NUM_TYPES; i++)
+	{
+		PgStat_IOOpCounters *sharedent = &dest_io_path_ops->data[i];
+		PgStat_IOOpCounters *pendingent = &pending_IOOpStats.data[i];
+
+#define IO_OP_ACC(fld) sharedent->fld += pendingent->fld
+		IO_OP_ACC(allocs);
+		IO_OP_ACC(extends);
+		IO_OP_ACC(fsyncs);
+		IO_OP_ACC(writes);
+#undef IO_OP_ACC
+	}
+
+	LWLockRelease(&stats_shmem->lock);
+
+	MemSet(&pending_IOOpStats, 0, sizeof(pending_IOOpStats));
+
+	have_ioopstats = false;
+
+	return false;
+}
+
+void
+pgstat_io_ops_snapshot_cb(void)
+{
+	PgStatShared_BackendIOPathOps *stats_shmem = &pgStatLocal.shmem->io_ops;
+	PgStat_IOPathOps *snapshot_ops = pgStatLocal.snapshot.io_path_ops;
+	PgStat_IOPathOps *reset_ops;
+
+	PgStat_IOPathOps *reset_offset = stats_shmem->reset_offset;
+	PgStat_IOPathOps reset[BACKEND_NUM_TYPES];
+
+	pgstat_copy_changecounted_stats(snapshot_ops,
+			&stats_shmem->stats, sizeof(stats_shmem->stats),
+			&stats_shmem->changecount);
+
+	LWLockAcquire(&stats_shmem->lock, LW_SHARED);
+	memcpy(&reset, reset_offset, sizeof(stats_shmem->stats));
+	LWLockRelease(&stats_shmem->lock);
+
+	reset_ops = reset;
+	for (int i = 0; i < BACKEND_NUM_TYPES; i++)
+	{
+		PgStat_IOOpCounters *counters = snapshot_ops->data;
+		PgStat_IOOpCounters *reset_counters = reset_ops->data;
+
+		for (int j = 0; j < IOPATH_NUM_TYPES; j++)
+		{
+			counters->allocs -= reset_counters->allocs;
+			counters->extends -= reset_counters->extends;
+			counters->fsyncs -= reset_counters->fsyncs;
+			counters->writes -= reset_counters->writes;
+
+			counters++;
+			reset_counters++;
+		}
+		snapshot_ops++;
+		reset_ops++;
+	}
+}
+
+void
+pgstat_io_ops_reset_all_cb(TimestampTz ts)
+{
+	PgStatShared_BackendIOPathOps *stats_shmem = &pgStatLocal.shmem->io_ops;
+
+	LWLockAcquire(&stats_shmem->lock, LW_EXCLUSIVE);
+	pgstat_copy_changecounted_stats(&stats_shmem->reset_offset,
+			&stats_shmem->stats, sizeof(stats_shmem->stats),
+			&stats_shmem->changecount);
+
+	for (int i = 0; i < BACKEND_NUM_TYPES; i++)
+		stats_shmem->stats[i].stat_reset_timestamp = ts;
+	LWLockRelease(&stats_shmem->lock);
+}
+
+void
+pgstat_count_io_op(IOOp io_op, IOPath io_path)
+{
+	PgStat_IOOpCounters *pending_counters = &pending_IOOpStats.data[io_path];
+	PgStat_IOOpCounters *cumulative_counters =
+			&cumulative_IOOpStats.data[io_path];
+
+	switch (io_op)
+	{
+		case IOOP_ALLOC:
+			pending_counters->allocs++;
+			cumulative_counters->allocs++;
+			break;
+		case IOOP_EXTEND:
+			pending_counters->extends++;
+			cumulative_counters->extends++;
+			break;
+		case IOOP_FSYNC:
+			pending_counters->fsyncs++;
+			cumulative_counters->fsyncs++;
+			break;
+		case IOOP_WRITE:
+			pending_counters->writes++;
+			cumulative_counters->writes++;
+			break;
+	}
+
+	have_ioopstats = true;
+}
+
+/*
+ * Report IO operation statistics
+ *
+ * This works in much the same way as pgstat_flush_io_ops() but is meant for
+ * BackendTypes like bgwriter for whom pgstat_report_stat() will not be called
+ * frequently enough to keep shared memory stats fresh.
+ * Backends not typically calling pgstat_report_stat() can invoke
+ * pgstat_report_io_ops() explicitly.
+ */
+void
+pgstat_report_io_ops(void)
+{
+	PgStat_IOPathOps *dest_io_path_ops;
+	PgStatShared_BackendIOPathOps *stats_shmem;
+
+	PgBackendStatus *beentry = MyBEEntry;
+
+	Assert(!pgStatLocal.shmem->is_shutdown);
+	pgstat_assert_is_up();
+
+	if (!have_ioopstats)
+		return;
+
+	if (!beentry || beentry->st_backendType == B_INVALID)
+		return;
+
+	stats_shmem = &pgStatLocal.shmem->io_ops;
+
+	dest_io_path_ops =
+		&stats_shmem->stats[backend_type_get_idx(beentry->st_backendType)];
+
+	pgstat_begin_changecount_write(&stats_shmem->changecount);
+
+	for (int i = 0; i < IOPATH_NUM_TYPES; i++)
+	{
+		PgStat_IOOpCounters *sharedent = &dest_io_path_ops->data[i];
+		PgStat_IOOpCounters *pendingent = &pending_IOOpStats.data[i];
+
+#define IO_OP_ACC(fld) sharedent->fld += pendingent->fld
+		IO_OP_ACC(allocs);
+		IO_OP_ACC(extends);
+		IO_OP_ACC(fsyncs);
+		IO_OP_ACC(writes);
+#undef IO_OP_ACC
+	}
+
+	pgstat_end_changecount_write(&stats_shmem->changecount);
+
+	MemSet(&pending_IOOpStats, 0, sizeof(pending_IOOpStats));
+
+	have_ioopstats = false;
+}
+
+PgStat_IOPathOps *
+pgstat_fetch_backend_io_path_ops(void)
+{
+	pgstat_snapshot_fixed(PGSTAT_KIND_IOOPS);
+	return pgStatLocal.snapshot.io_path_ops;
+}
+
+PgStat_Counter
+pgstat_fetch_cumulative_io_ops(IOPath io_path, IOOp io_op)
+{
+	PgStat_IOOpCounters *counters = &cumulative_IOOpStats.data[io_path];
+
+	switch (io_op)
+	{
+		case IOOP_ALLOC:
+			return counters->allocs;
+		case IOOP_EXTEND:
+			return counters->extends;
+		case IOOP_FSYNC:
+			return counters->fsyncs;
+		case IOOP_WRITE:
+			return counters->writes;
+		default:
+			elog(ERROR, "IO Operation %s for IO Path %s is undefined.",
+					pgstat_io_op_desc(io_op), pgstat_io_path_desc(io_path));
+	}
+}
+
+const char *
+pgstat_io_path_desc(IOPath io_path)
+{
+	const char *io_path_desc = "Unknown IO Path";
+
+	switch (io_path)
+	{
+		case IOPATH_DIRECT:
+			io_path_desc = "Direct";
+			break;
+		case IOPATH_LOCAL:
+			io_path_desc = "Local";
+			break;
+		case IOPATH_SHARED:
+			io_path_desc = "Shared";
+			break;
+		case IOPATH_STRATEGY:
+			io_path_desc = "Strategy";
+			break;
+	}
+
+	return io_path_desc;
+}
+
+const char *
+pgstat_io_op_desc(IOOp io_op)
+{
+	const char *io_op_desc = "Unknown IO Op";
+
+	switch (io_op)
+	{
+		case IOOP_ALLOC:
+			io_op_desc = "Alloc";
+			break;
+		case IOOP_EXTEND:
+			io_op_desc = "Extend";
+			break;
+		case IOOP_FSYNC:
+			io_op_desc = "Fsync";
+			break;
+		case IOOP_WRITE:
+			io_op_desc = "Write";
+			break;
+	}
+
+	return io_op_desc;
+}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 893690dad5..47ceba78e6 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2104,6 +2104,8 @@ pg_stat_reset_shared(PG_FUNCTION_ARGS)
 		pgstat_reset_of_kind(PGSTAT_KIND_BGWRITER);
 		pgstat_reset_of_kind(PGSTAT_KIND_CHECKPOINTER);
 	}
+	else if (strcmp(target, "buffers") == 0)
+		pgstat_reset_of_kind(PGSTAT_KIND_IOOPS);
 	else if (strcmp(target, "recovery_prefetch") == 0)
 		XLogPrefetchResetStats();
 	else if (strcmp(target, "wal") == 0)
@@ -2112,7 +2114,7 @@ pg_stat_reset_shared(PG_FUNCTION_ARGS)
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
 				 errmsg("unrecognized reset target: \"%s\"", target),
-				 errhint("Target must be \"archiver\", \"bgwriter\", \"recovery_prefetch\", or \"wal\".")));
+				 errhint("Target must be \"archiver\", \"buffers\", \"bgwriter\", \"recovery_prefetch\", or \"wal\".")));
 
 	PG_RETURN_VOID();
 }
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 78097e714b..70e1f66370 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -339,6 +339,8 @@ typedef enum BackendType
 	B_WAL_WRITER,
 } BackendType;
 
+#define BACKEND_NUM_TYPES B_WAL_WRITER
+
 extern PGDLLIMPORT BackendType MyBackendType;
 
 extern const char *GetBackendTypeDesc(BackendType backendType);
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index ac28f813b4..71f75806f6 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -48,6 +48,7 @@ typedef enum PgStat_Kind
 	PGSTAT_KIND_ARCHIVER,
 	PGSTAT_KIND_BGWRITER,
 	PGSTAT_KIND_CHECKPOINTER,
+	PGSTAT_KIND_IOOPS,
 	PGSTAT_KIND_SLRU,
 	PGSTAT_KIND_WAL,
 } PgStat_Kind;
@@ -276,6 +277,44 @@ typedef struct PgStat_CheckpointerStats
 	PgStat_Counter buf_fsync_backend;
 } PgStat_CheckpointerStats;
 
+/*
+ * Types related to counting IO Operations for various IO Paths
+ */
+
+typedef enum IOOp
+{
+	IOOP_ALLOC,
+	IOOP_EXTEND,
+	IOOP_FSYNC,
+	IOOP_WRITE,
+} IOOp;
+
+#define IOOP_NUM_TYPES (IOOP_WRITE + 1)
+
+typedef enum IOPath
+{
+	IOPATH_DIRECT,
+	IOPATH_LOCAL,
+	IOPATH_SHARED,
+	IOPATH_STRATEGY,
+} IOPath;
+
+#define IOPATH_NUM_TYPES (IOPATH_STRATEGY + 1)
+
+typedef struct PgStat_IOOpCounters
+{
+	PgStat_Counter allocs;
+	PgStat_Counter extends;
+	PgStat_Counter fsyncs;
+	PgStat_Counter writes;
+} PgStat_IOOpCounters;
+
+typedef struct PgStat_IOPathOps
+{
+	PgStat_IOOpCounters data[IOPATH_NUM_TYPES];
+	TimestampTz stat_reset_timestamp;
+} PgStat_IOPathOps;
+
 typedef struct PgStat_StatDBEntry
 {
 	PgStat_Counter n_xact_commit;
@@ -453,6 +492,18 @@ extern void pgstat_report_checkpointer(void);
 extern PgStat_CheckpointerStats *pgstat_fetch_stat_checkpointer(void);
 
 
+/*
+ * Functions in pgstat_io_ops.c
+ */
+
+extern void pgstat_count_io_op(IOOp io_op, IOPath io_path);
+extern void pgstat_report_io_ops(void);
+extern PgStat_IOPathOps *pgstat_fetch_backend_io_path_ops(void);
+extern PgStat_Counter pgstat_fetch_cumulative_io_ops(IOPath io_path, IOOp io_op);
+extern const char *pgstat_io_op_desc(IOOp io_op);
+extern const char *pgstat_io_path_desc(IOPath io_path);
+
+
 /*
  * Functions in pgstat_database.c
  */
diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index a17e7b28a5..c8546be22b 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 7403bca25e..49d062b1af 100644
--- a/src/include/utils/backend_status.h
+++ b/src/include/utils/backend_status.h
@@ -306,6 +306,40 @@ extern const char *pgstat_get_crashed_backend_activity(int pid, char *buffer,
 													   int buflen);
 extern uint64 pgstat_get_my_query_id(void);
 
+/* Utility functions */
+
+/*
+ * When maintaining an array of information about all valid BackendTypes, in
+ * order to avoid wasting the 0th spot, use this helper to convert a valid
+ * BackendType to a valid location in the array (given that no spot is
+ * maintained for B_INVALID BackendType).
+ */
+static inline int backend_type_get_idx(BackendType backend_type)
+{
+	/*
+	 * backend_type must be one of the valid backend types. If caller is
+	 * maintaining backend information in an array that includes B_INVALID,
+	 * this function is unnecessary.
+	 */
+	Assert(backend_type > B_INVALID && backend_type <= BACKEND_NUM_TYPES);
+	return backend_type - 1;
+}
+
+/*
+ * When using a value from an array of information about all valid
+ * BackendTypes, add 1 to the index before using it as a BackendType to adjust
+ * for not maintaining a spot for B_INVALID BackendType.
+ */
+static inline BackendType idx_get_backend_type(int idx)
+{
+	int backend_type = idx + 1;
+	/*
+	 * If the array includes a spot for B_INVALID BackendType this function is
+	 * not required.
+	 */
+	Assert(backend_type > B_INVALID && backend_type <= BACKEND_NUM_TYPES);
+	return backend_type;
+}
 
 /* ----------
  * Support functions for the SQL-callable functions to
diff --git a/src/include/utils/pgstat_internal.h b/src/include/utils/pgstat_internal.h
index 9303d05427..2dfa750df9 100644
--- a/src/include/utils/pgstat_internal.h
+++ b/src/include/utils/pgstat_internal.h
@@ -329,6 +329,14 @@ typedef struct PgStatShared_Checkpointer
 	PgStat_CheckpointerStats reset_offset;
 } PgStatShared_Checkpointer;
 
+typedef struct PgStatShared_BackendIOPathOps
+{
+	LWLock		lock;
+	uint32		changecount;
+	PgStat_IOPathOps stats[BACKEND_NUM_TYPES];
+	PgStat_IOPathOps reset_offset[BACKEND_NUM_TYPES];
+} PgStatShared_BackendIOPathOps;
+
 typedef struct PgStatShared_SLRU
 {
 	/* lock protects ->stats */
@@ -419,6 +427,7 @@ typedef struct PgStat_ShmemControl
 	PgStatShared_Archiver archiver;
 	PgStatShared_BgWriter bgwriter;
 	PgStatShared_Checkpointer checkpointer;
+	PgStatShared_BackendIOPathOps io_ops;
 	PgStatShared_SLRU slru;
 	PgStatShared_Wal wal;
 } PgStat_ShmemControl;
@@ -442,6 +451,8 @@ typedef struct PgStat_Snapshot
 
 	PgStat_CheckpointerStats checkpointer;
 
+	PgStat_IOPathOps io_path_ops[BACKEND_NUM_TYPES];
+
 	PgStat_SLRUStats slru[SLRU_NUM_ELEMENTS];
 
 	PgStat_WalStats wal;
@@ -549,6 +560,15 @@ extern void pgstat_database_reset_timestamp_cb(PgStatShared_Common *header, Time
 extern bool pgstat_function_flush_cb(PgStat_EntryRef *entry_ref, bool nowait);
 
 
+/*
+ * Functions in pgstat_io_ops.c
+ */
+
+extern bool pgstat_flush_io_ops(bool nowait);
+extern void pgstat_io_ops_snapshot_cb(void);
+extern void pgstat_io_ops_reset_all_cb(TimestampTz ts);
+
+
 /*
  * Functions in pgstat_relation.c
  */
-- 
2.37.0

