SLRU statistics

Started by Tomas Vondraalmost 6 years ago49 messages
#1Tomas Vondra
tomas.vondra@2ndquadrant.com
1 attachment(s)

Hi,

One of the stats I occasionally wanted to know are stats for the SLRU
stats (we have couple of those - clog, subtrans, ...). So here is a WIP
version of a patch adding that.

The implementation is fairly simple - the slru code updates counters in
local memory, and then sends them to the collector at the end of the
transaction (similarly to table/func stats). The collector stores it
similarly to global stats. And the collected stats are accessible
through pg_stat_slru.

The main issue is that we have multiple SLRU caches, and it seems handy
to have separate stats for each. OTOH the number of SLRU stats is not
fixed, so e.g. extensions might define their own SLRU caches. But
handing dynamic number of SLRU caches seems a bit hard (we'd need to
assign some sort of unique IDs etc.) so what I did was define a fixed
number of SLRU types

typedef enum SlruType
{
SLRU_CLOG,
SLRU_COMMIT_TS,
SLRU_MULTIXACT_OFFSET,
SLRU_MULTIXACT_MEMBER,
SLRU_SUBTRANS,
SLRU_ASYNC,
SLRU_OLDSERXID,
SLRU_OTHER
} SlruType;

with one group of counters for each group. The last type (SLRU_OTHER) is
used to store stats for all SLRUs that are not predefined. It wouldn't
be that difficult to store dynamic number of SLRUs, but I'm not sure how
to solve issues with identifying SLRUs etc. And there are probably very
few extensions adding custom SLRU anyway.

The one thing missing from the patch is a way to reset the SLRU stats,
similarly to how we can reset bgwriter stats.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

stats-slru-v1.patchtext/plain; charset=us-asciiDownload
diff --git a/src/backend/access/transam/clog.c b/src/backend/access/transam/clog.c
index f8e7670f8d..3f45db7ea9 100644
--- a/src/backend/access/transam/clog.c
+++ b/src/backend/access/transam/clog.c
@@ -692,7 +692,8 @@ CLOGShmemInit(void)
 {
 	ClogCtl->PagePrecedes = CLOGPagePrecedes;
 	SimpleLruInit(ClogCtl, "clog", CLOGShmemBuffers(), CLOG_LSNS_PER_PAGE,
-				  CLogControlLock, "pg_xact", LWTRANCHE_CLOG_BUFFERS);
+				  CLogControlLock, "pg_xact", LWTRANCHE_CLOG_BUFFERS,
+				  SLRU_CLOG);
 }
 
 /*
diff --git a/src/backend/access/transam/commit_ts.c b/src/backend/access/transam/commit_ts.c
index 630df672cc..44d7ca4483 100644
--- a/src/backend/access/transam/commit_ts.c
+++ b/src/backend/access/transam/commit_ts.c
@@ -494,7 +494,7 @@ CommitTsShmemInit(void)
 	CommitTsCtl->PagePrecedes = CommitTsPagePrecedes;
 	SimpleLruInit(CommitTsCtl, "commit_timestamp", CommitTsShmemBuffers(), 0,
 				  CommitTsControlLock, "pg_commit_ts",
-				  LWTRANCHE_COMMITTS_BUFFERS);
+				  LWTRANCHE_COMMITTS_BUFFERS, SLRU_COMMIT_TS);
 
 	commitTsShared = ShmemInitStruct("CommitTs shared",
 									 sizeof(CommitTimestampShared),
diff --git a/src/backend/access/transam/multixact.c b/src/backend/access/transam/multixact.c
index 50e98caaeb..37a5854284 100644
--- a/src/backend/access/transam/multixact.c
+++ b/src/backend/access/transam/multixact.c
@@ -1831,11 +1831,11 @@ MultiXactShmemInit(void)
 	SimpleLruInit(MultiXactOffsetCtl,
 				  "multixact_offset", NUM_MXACTOFFSET_BUFFERS, 0,
 				  MultiXactOffsetControlLock, "pg_multixact/offsets",
-				  LWTRANCHE_MXACTOFFSET_BUFFERS);
+				  LWTRANCHE_MXACTOFFSET_BUFFERS, SLRU_MULTIXACT_OFFSET);
 	SimpleLruInit(MultiXactMemberCtl,
 				  "multixact_member", NUM_MXACTMEMBER_BUFFERS, 0,
 				  MultiXactMemberControlLock, "pg_multixact/members",
-				  LWTRANCHE_MXACTMEMBER_BUFFERS);
+				  LWTRANCHE_MXACTMEMBER_BUFFERS, SLRU_MULTIXACT_MEMBER);
 
 	/* Initialize our shared state struct */
 	MultiXactState = ShmemInitStruct("Shared MultiXact State",
diff --git a/src/backend/access/transam/slru.c b/src/backend/access/transam/slru.c
index d5b7a08f73..dab15e6ab4 100644
--- a/src/backend/access/transam/slru.c
+++ b/src/backend/access/transam/slru.c
@@ -162,7 +162,8 @@ SimpleLruShmemSize(int nslots, int nlsns)
 
 void
 SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
-			  LWLock *ctllock, const char *subdir, int tranche_id)
+			  LWLock *ctllock, const char *subdir, int tranche_id,
+			  SlruType type)
 {
 	SlruShared	shared;
 	bool		found;
@@ -247,6 +248,7 @@ SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
 	 */
 	ctl->shared = shared;
 	ctl->do_fsync = true;		/* default behavior */
+	ctl->type = type;
 	StrNCpy(ctl->Dir, subdir, sizeof(ctl->Dir));
 }
 
@@ -286,6 +288,9 @@ SimpleLruZeroPage(SlruCtl ctl, int pageno)
 	/* Assume this page is now the latest active page */
 	shared->latest_page_number = pageno;
 
+	/* update the stats counter of zeroed pages */
+	pgstat_count_slru_zero_page(ctl);
+
 	return slotno;
 }
 
@@ -403,9 +408,16 @@ SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 			}
 			/* Otherwise, it's ready to use */
 			SlruRecentlyUsed(shared, slotno);
+
+			/* update the stats counter of pages found in shared buffers */
+			pgstat_count_slru_page_hit(ctl);
+
 			return slotno;
 		}
 
+		/* update the stats counter of pages not found */
+		pgstat_count_slru_page_miss(ctl);
+
 		/* We found no match; assert we selected a freeable slot */
 		Assert(shared->page_status[slotno] == SLRU_PAGE_EMPTY ||
 			   (shared->page_status[slotno] == SLRU_PAGE_VALID &&
@@ -596,6 +608,9 @@ SimpleLruDoesPhysicalPageExist(SlruCtl ctl, int pageno)
 	bool		result;
 	off_t		endpos;
 
+	/* update the stats counter of checked pages */
+	pgstat_count_slru_page_exists(ctl);
+
 	SlruFileName(ctl, path, segno);
 
 	fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
@@ -650,6 +665,9 @@ SlruPhysicalReadPage(SlruCtl ctl, int pageno, int slotno)
 	char		path[MAXPGPATH];
 	int			fd;
 
+	/* update the stats counter of read pages */
+	pgstat_count_slru_page_read(ctl);
+
 	SlruFileName(ctl, path, segno);
 
 	/*
@@ -730,6 +748,9 @@ SlruPhysicalWritePage(SlruCtl ctl, int pageno, int slotno, SlruFlush fdata)
 	char		path[MAXPGPATH];
 	int			fd = -1;
 
+	/* update the stats counter of written pages */
+	pgstat_count_slru_page_write(ctl);
+
 	/*
 	 * Honor the write-WAL-before-data rule, if appropriate, so that we do not
 	 * write out data before associated WAL records.  This is the same action
@@ -901,6 +922,9 @@ SlruReportIOError(SlruCtl ctl, int pageno, TransactionId xid)
 	int			offset = rpageno * BLCKSZ;
 	char		path[MAXPGPATH];
 
+	/* update the stats counter of errors */
+	pgstat_count_slru_io_error(ctl);
+
 	SlruFileName(ctl, path, segno);
 	errno = slru_errno;
 	switch (slru_errcause)
@@ -1125,6 +1149,9 @@ SimpleLruFlush(SlruCtl ctl, bool allow_redirtied)
 	int			i;
 	bool		ok;
 
+	/* update the stats counter of flushes */
+	pgstat_count_slru_flush(ctl);
+
 	/*
 	 * Find and write dirty pages
 	 */
@@ -1186,6 +1213,9 @@ SimpleLruTruncate(SlruCtl ctl, int cutoffPage)
 	SlruShared	shared = ctl->shared;
 	int			slotno;
 
+	/* update the stats counter of truncates */
+	pgstat_count_slru_truncate(ctl);
+
 	/*
 	 * The cutoff point is the start of the segment containing cutoffPage.
 	 */
diff --git a/src/backend/access/transam/subtrans.c b/src/backend/access/transam/subtrans.c
index 25d7d739cf..3316accb50 100644
--- a/src/backend/access/transam/subtrans.c
+++ b/src/backend/access/transam/subtrans.c
@@ -193,7 +193,7 @@ SUBTRANSShmemInit(void)
 	SubTransCtl->PagePrecedes = SubTransPagePrecedes;
 	SimpleLruInit(SubTransCtl, "subtrans", NUM_SUBTRANS_BUFFERS, 0,
 				  SubtransControlLock, "pg_subtrans",
-				  LWTRANCHE_SUBTRANS_BUFFERS);
+				  LWTRANCHE_SUBTRANS_BUFFERS, SLRU_SUBTRANS);
 	/* Override default assumption that writes should be fsync'd */
 	SubTransCtl->do_fsync = false;
 }
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c9e75f4370..1844f56859 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -790,6 +790,20 @@ CREATE VIEW pg_stat_replication AS
         JOIN pg_stat_get_wal_senders() AS W ON (S.pid = W.pid)
         LEFT JOIN pg_authid AS U ON (S.usesysid = U.oid);
 
+CREATE VIEW pg_stat_slru AS
+    SELECT
+            s.name,
+            s.pages_zero,
+            s.pages_hit,
+            s.pages_miss,
+            s.pages_exists,
+            s.pages_read,
+            s.pages_write,
+            s.io_error,
+            s.flushes,
+            s.truncates
+    FROM pg_stat_get_slru() s;
+
 CREATE VIEW pg_stat_wal_receiver AS
     SELECT
             s.pid,
diff --git a/src/backend/commands/async.c b/src/backend/commands/async.c
index 9aa2b61600..02222cc92a 100644
--- a/src/backend/commands/async.c
+++ b/src/backend/commands/async.c
@@ -552,7 +552,8 @@ AsyncShmemInit(void)
 	 */
 	AsyncCtl->PagePrecedes = asyncQueuePagePrecedes;
 	SimpleLruInit(AsyncCtl, "async", NUM_ASYNC_BUFFERS, 0,
-				  AsyncCtlLock, "pg_notify", LWTRANCHE_ASYNC_BUFFERS);
+				  AsyncCtlLock, "pg_notify", LWTRANCHE_ASYNC_BUFFERS,
+				  SLRU_ASYNC);
 	/* Override default assumption that writes should be fsync'd */
 	AsyncCtl->do_fsync = false;
 
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 51c486bebd..aa825029b1 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -141,6 +141,9 @@ char	   *pgstat_stat_tmpname = NULL;
  */
 PgStat_MsgBgWriter BgWriterStats;
 
+/* TODO */
+PgStat_MsgSlru SlruStats[SLRU_OTHER + 1];
+
 /* ----------
  * Local data
  * ----------
@@ -255,6 +258,7 @@ static int	localNumBackends = 0;
  */
 static PgStat_ArchiverStats archiverStats;
 static PgStat_GlobalStats globalStats;
+static PgStat_SlruStats slruStats[SLRU_OTHER + 1];
 
 /*
  * List of OIDs of databases we need to write out.  If an entry is InvalidOid,
@@ -297,6 +301,7 @@ static bool pgstat_db_requested(Oid databaseid);
 
 static void pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg);
 static void pgstat_send_funcstats(void);
+static void pgstat_send_slru(void);
 static HTAB *pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid);
 
 static PgStat_TableStatus *get_tabstat_entry(Oid rel_id, bool isshared);
@@ -324,6 +329,7 @@ static void pgstat_recv_vacuum(PgStat_MsgVacuum *msg, int len);
 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_slru(PgStat_MsgSlru *msg, int len);
 static void pgstat_recv_funcstat(PgStat_MsgFuncstat *msg, int len);
 static void pgstat_recv_funcpurge(PgStat_MsgFuncpurge *msg, int len);
 static void pgstat_recv_recoveryconflict(PgStat_MsgRecoveryConflict *msg, int len);
@@ -904,6 +910,9 @@ pgstat_report_stat(bool force)
 
 	/* Now, send function statistics */
 	pgstat_send_funcstats();
+
+	/* Finally send SLRU statistics */
+	pgstat_send_slru();
 }
 
 /*
@@ -2619,6 +2628,23 @@ pgstat_fetch_global(void)
 }
 
 
+/*
+ * ---------
+ * pgstat_fetch_slru() -
+ *
+ *	Support function for the SQL-callable pgstat* functions. Returns
+ *	a pointer to the slru statistics struct.
+ * ---------
+ */
+PgStat_SlruStats *
+pgstat_fetch_slru(void)
+{
+	backend_read_statsfile();
+
+	return slruStats;
+}
+
+
 /* ------------------------------------------------------------
  * Functions for management of the shared-memory PgBackendStatus array
  * ------------------------------------------------------------
@@ -4410,6 +4436,46 @@ pgstat_send_bgwriter(void)
 	MemSet(&BgWriterStats, 0, sizeof(BgWriterStats));
 }
 
+/* ----------
+ * pgstat_send_slru() -
+ *
+ *		Send slru statistics to the collector
+ * ----------
+ */
+static void
+pgstat_send_slru(void)
+{
+	int		i;
+
+	/* We assume this initializes to zeroes */
+	static const PgStat_MsgSlru all_zeroes;
+
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		/*
+		 * This function can be called even if nothing at all has happened. In
+		 * this case, avoid sending a completely empty message to the stats
+		 * collector.
+		 */
+		if (memcmp(&SlruStats[i], &all_zeroes, sizeof(PgStat_MsgSlru)) == 0)
+			continue;
+
+		/* set the SLRU type before each send */
+		SlruStats[i].m_type = i;
+
+		/*
+		 * Prepare and send the message
+		 */
+		pgstat_setheader(&SlruStats[i].m_hdr, PGSTAT_MTYPE_SLRU);
+		pgstat_send(&SlruStats[i], sizeof(PgStat_MsgSlru));
+
+		/*
+		 * Clear out the statistics buffer, so it can be re-used.
+		 */
+		MemSet(&SlruStats[i], 0, sizeof(PgStat_MsgSlru));
+	}
+}
+
 
 /* ----------
  * PgstatCollectorMain() -
@@ -4602,6 +4668,10 @@ PgstatCollectorMain(int argc, char *argv[])
 					pgstat_recv_bgwriter(&msg.msg_bgwriter, len);
 					break;
 
+				case PGSTAT_MTYPE_SLRU:
+					pgstat_recv_slru(&msg.msg_slru, len);
+					break;
+
 				case PGSTAT_MTYPE_FUNCSTAT:
 					pgstat_recv_funcstat(&msg.msg_funcstat, len);
 					break;
@@ -4832,6 +4902,7 @@ pgstat_write_statsfiles(bool permanent, bool allDbs)
 	const char *tmpfile = permanent ? PGSTAT_STAT_PERMANENT_TMPFILE : pgstat_stat_tmpname;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
 	int			rc;
+	int			i;
 
 	elog(DEBUG2, "writing stats file \"%s\"", statfile);
 
@@ -4872,6 +4943,15 @@ pgstat_write_statsfiles(bool permanent, bool allDbs)
 	rc = fwrite(&archiverStats, sizeof(archiverStats), 1, fpout);
 	(void) rc;					/* we'll check for error with ferror */
 
+	/*
+	 * Write SLRU stats struct
+	 */
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		rc = fwrite(&slruStats[i], sizeof(PgStat_SlruStats), 1, fpout);
+		(void) rc;					/* we'll check for error with ferror */
+	}
+
 	/*
 	 * Walk through the database table.
 	 */
@@ -5107,6 +5187,7 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	int32		format_id;
 	bool		found;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	int			i;
 
 	/*
 	 * The tables will live in pgStatLocalContext.
@@ -5129,6 +5210,7 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	 */
 	memset(&globalStats, 0, sizeof(globalStats));
 	memset(&archiverStats, 0, sizeof(archiverStats));
+	// memset(&slruStats, 0, sizeof(slruStats));
 
 	/*
 	 * Set the current timestamp (will be kept only in case we can't load an
@@ -5199,6 +5281,20 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 		goto done;
 	}
 
+	/*
+	 * Read SLRU stats struct
+	 */
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		if (fread(&slruStats[i], 1, sizeof(PgStat_SlruStats), fpin) != sizeof(PgStat_SlruStats))
+		{
+			ereport(pgStatRunningInCollector ? LOG : WARNING,
+					(errmsg("corrupted statistics file \"%s\"", statfile)));
+			memset(&slruStats[i], 0, sizeof(PgStat_SlruStats));
+			goto done;
+		}
+	}
+
 	/*
 	 * We found an existing collector stats file. Read it and put all the
 	 * hashtable entries into place.
@@ -5497,9 +5593,11 @@ pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
 	PgStat_StatDBEntry dbentry;
 	PgStat_GlobalStats myGlobalStats;
 	PgStat_ArchiverStats myArchiverStats;
+	PgStat_SlruStats mySlruStats;
 	FILE	   *fpin;
 	int32		format_id;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	int			i;
 
 	/*
 	 * Try to open the stats file.  As above, anything but ENOENT is worthy of
@@ -5551,6 +5649,21 @@ pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
 		return false;
 	}
 
+	/*
+	 * Read SLRU stats struct
+	 */
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		if (fread(&mySlruStats, 1, sizeof(PgStat_SlruStats),
+				  fpin) != sizeof(PgStat_SlruStats))
+		{
+			ereport(pgStatRunningInCollector ? LOG : WARNING,
+					(errmsg("corrupted statistics file \"%s\"", statfile)));
+			FreeFile(fpin);
+			return false;
+		}
+	}
+
 	/* By default, we're going to return the timestamp of the global file. */
 	*ts = myGlobalStats.stats_timestamp;
 
@@ -6292,6 +6405,26 @@ pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len)
 	globalStats.buf_alloc += msg->m_buf_alloc;
 }
 
+/* ----------
+ * pgstat_recv_slru() -
+ *
+ *	Process a SLRU message.
+ * ----------
+ */
+static void
+pgstat_recv_slru(PgStat_MsgSlru *msg, int len)
+{
+	slruStats[msg->m_type].pages_zero += msg->m_pages_zero;
+	slruStats[msg->m_type].pages_hit += msg->m_pages_hit;
+	slruStats[msg->m_type].pages_miss += msg->m_pages_miss;
+	slruStats[msg->m_type].pages_exists += msg->m_pages_exists;
+	slruStats[msg->m_type].pages_read += msg->m_pages_read;
+	slruStats[msg->m_type].pages_write += msg->m_pages_write;
+	slruStats[msg->m_type].io_error += msg->m_io_error;
+	slruStats[msg->m_type].flush += msg->m_flush;
+	slruStats[msg->m_type].truncate += msg->m_truncate;
+}
+
 /* ----------
  * pgstat_recv_recoveryconflict() -
  *
@@ -6546,3 +6679,57 @@ pgstat_clip_activity(const char *raw_activity)
 
 	return activity;
 }
+
+void
+pgstat_count_slru_zero_page(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_zero += 1;
+}
+
+void
+pgstat_count_slru_page_hit(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_hit += 1;
+}
+
+void
+pgstat_count_slru_page_miss(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_miss += 1;
+}
+
+void
+pgstat_count_slru_page_exists(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_exists += 1;
+}
+
+void
+pgstat_count_slru_page_read(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_read += 1;
+}
+
+void
+pgstat_count_slru_page_write(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_write += 1;
+}
+
+void
+pgstat_count_slru_io_error(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_io_error += 1;
+}
+
+void
+pgstat_count_slru_flush(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_flush += 1;
+}
+
+void
+pgstat_count_slru_truncate(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_truncate += 1;
+}
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index de46b841cb..82e5d7ec57 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -824,7 +824,7 @@ OldSerXidInit(void)
 	OldSerXidSlruCtl->PagePrecedes = OldSerXidPagePrecedesLogically;
 	SimpleLruInit(OldSerXidSlruCtl, "oldserxid",
 				  NUM_OLDSERXID_BUFFERS, 0, OldSerXidLock, "pg_serial",
-				  LWTRANCHE_OLDSERXID_BUFFERS);
+				  LWTRANCHE_OLDSERXID_BUFFERS, SLRU_OLDSERXID);
 	/* Override default assumption that writes should be fsync'd */
 	OldSerXidSlruCtl->do_fsync = false;
 
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 74f899f24d..8233568800 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1665,6 +1665,96 @@ pg_stat_get_buf_alloc(PG_FUNCTION_ARGS)
 	PG_RETURN_INT64(pgstat_fetch_global()->buf_alloc);
 }
 
+/*
+ * Returns statistics of SLRU caches.
+ */
+Datum
+pg_stat_get_slru(PG_FUNCTION_ARGS)
+{
+#define PG_STAT_GET_SLRU_COLS	10
+	ReturnSetInfo  *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	TupleDesc		tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext 	per_query_ctx;
+	MemoryContext 	oldcontext;
+	int				i;
+	PgStat_SlruStats *stats;
+
+	/* 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);
+
+	/* request SLRU stats from the stat collector */
+	stats = pgstat_fetch_slru();
+
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		/* for each row */
+		Datum		values[PG_STAT_GET_SLRU_COLS];
+		bool		nulls[PG_STAT_GET_SLRU_COLS];
+		PgStat_SlruStats	stat = stats[i];
+		text	   *name;
+
+		MemSet(values, 0, sizeof(values));
+		MemSet(nulls, 0, sizeof(nulls));
+
+		if (i == SLRU_CLOG)
+			name = cstring_to_text("clog");
+		else if (i == SLRU_COMMIT_TS)
+			name = cstring_to_text("commit_timestamp");
+		else if (i == SLRU_MULTIXACT_OFFSET)
+			name = cstring_to_text("multixact_offset");
+		else if (i == SLRU_MULTIXACT_MEMBER)
+			name = cstring_to_text("multixact_member");
+		else if (i == SLRU_SUBTRANS)
+			name = cstring_to_text("subtrans");
+		else if (i == SLRU_ASYNC)
+			name = cstring_to_text("async");
+		else if (i == SLRU_OLDSERXID)
+			name = cstring_to_text("oldserxid");
+		else if (i == SLRU_OTHER)
+			name = cstring_to_text("other");
+
+		values[0] = PointerGetDatum(name);
+		values[1] = Int64GetDatum(stat.pages_zero);
+		values[2] = Int64GetDatum(stat.pages_hit);
+		values[3] = Int64GetDatum(stat.pages_miss);
+		values[4] = Int64GetDatum(stat.pages_exists);
+		values[5] = Int64GetDatum(stat.pages_read);
+		values[6] = Int64GetDatum(stat.pages_write);
+		values[7] = Int64GetDatum(stat.io_error);
+		values[8] = Int64GetDatum(stat.flush);
+		values[9] = Int64GetDatum(stat.truncate);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	/* clean up and return the tuplestore */
+	tuplestore_donestoring(tupstore);
+
+	return (Datum) 0;
+}
+
 Datum
 pg_stat_get_xact_numscans(PG_FUNCTION_ARGS)
 {
diff --git a/src/include/access/slru.h b/src/include/access/slru.h
index 00dbd803e1..eb79ab346f 100644
--- a/src/include/access/slru.h
+++ b/src/include/access/slru.h
@@ -106,6 +106,18 @@ typedef struct SlruSharedData
 
 typedef SlruSharedData *SlruShared;
 
+typedef enum SlruType
+{
+	SLRU_CLOG,
+	SLRU_COMMIT_TS,
+	SLRU_MULTIXACT_OFFSET,
+	SLRU_MULTIXACT_MEMBER,
+	SLRU_SUBTRANS,
+	SLRU_ASYNC,
+	SLRU_OLDSERXID,
+	SLRU_OTHER
+} SlruType;
+
 /*
  * SlruCtlData is an unshared structure that points to the active information
  * in shared memory.
@@ -114,6 +126,9 @@ typedef struct SlruCtlData
 {
 	SlruShared	shared;
 
+	/* type of the SLRU */
+	SlruType	type;
+
 	/*
 	 * This flag tells whether to fsync writes (true for pg_xact and multixact
 	 * stuff, false for pg_subtrans and pg_notify).
@@ -139,7 +154,8 @@ typedef SlruCtlData *SlruCtl;
 
 extern Size SimpleLruShmemSize(int nslots, int nlsns);
 extern void SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
-						  LWLock *ctllock, const char *subdir, int tranche_id);
+						  LWLock *ctllock, const char *subdir, int tranche_id,
+						  SlruType type);
 extern int	SimpleLruZeroPage(SlruCtl ctl, int pageno);
 extern int	SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 							  TransactionId xid);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index fcf2a1214c..eadd68e409 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5408,6 +5408,16 @@
   proname => 'pg_stat_get_buf_alloc', provolatile => 's', proparallel => 'r',
   prorettype => 'int8', proargtypes => '', prosrc => 'pg_stat_get_buf_alloc' },
 
+{ oid => '8614',
+  descr => 'statistics: information about SLRU caches',
+  proname => 'pg_stat_get_slru', prorows => '100', proisstrict => 'f',
+  proretset => 't', provolatile => 's', proparallel => 'r',
+  prorettype => 'record', proargtypes => '',
+  proallargtypes => '{text,int8,int8,int8,int8,int8,int8,int8,int8,int8}',
+  proargmodes => '{o,o,o,o,o,o,o,o,o,o}',
+  proargnames => '{name,pages_zero,pages_hit,pages_miss,pages_exists,pages_read,pages_write,io_error,flushes,truncates}',
+  prosrc => 'pg_stat_get_slru' },
+
 { oid => '2978', descr => 'statistics: number of function calls',
   proname => 'pg_stat_get_function_calls', provolatile => 's',
   proparallel => 'r', prorettype => 'int8', proargtypes => 'oid',
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 36b530bc27..677ce2a87a 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -11,6 +11,7 @@
 #ifndef PGSTAT_H
 #define PGSTAT_H
 
+#include "access/slru.h"
 #include "datatype/timestamp.h"
 #include "libpq/pqcomm.h"
 #include "port/atomics.h"
@@ -59,6 +60,7 @@ typedef enum StatMsgType
 	PGSTAT_MTYPE_ANALYZE,
 	PGSTAT_MTYPE_ARCHIVER,
 	PGSTAT_MTYPE_BGWRITER,
+	PGSTAT_MTYPE_SLRU,
 	PGSTAT_MTYPE_FUNCSTAT,
 	PGSTAT_MTYPE_FUNCPURGE,
 	PGSTAT_MTYPE_RECOVERYCONFLICT,
@@ -422,6 +424,25 @@ typedef struct PgStat_MsgBgWriter
 	PgStat_Counter m_checkpoint_sync_time;
 } PgStat_MsgBgWriter;
 
+/* ----------
+ * PgStat_MsgSlru			Sent by the slru to update statistics.
+ * ----------
+ */
+typedef struct PgStat_MsgSlru
+{
+	PgStat_MsgHdr m_hdr;
+	PgStat_Counter m_type;
+	PgStat_Counter m_pages_zero;
+	PgStat_Counter m_pages_hit;
+	PgStat_Counter m_pages_miss;
+	PgStat_Counter m_pages_exists;
+	PgStat_Counter m_pages_read;
+	PgStat_Counter m_pages_write;
+	PgStat_Counter m_io_error;
+	PgStat_Counter m_flush;
+	PgStat_Counter m_truncate;
+} PgStat_MsgSlru;
+
 /* ----------
  * PgStat_MsgRecoveryConflict	Sent by the backend upon recovery conflict
  * ----------
@@ -564,6 +585,7 @@ typedef union PgStat_Msg
 	PgStat_MsgAnalyze msg_analyze;
 	PgStat_MsgArchiver msg_archiver;
 	PgStat_MsgBgWriter msg_bgwriter;
+	PgStat_MsgSlru msg_slru;
 	PgStat_MsgFuncstat msg_funcstat;
 	PgStat_MsgFuncpurge msg_funcpurge;
 	PgStat_MsgRecoveryConflict msg_recoveryconflict;
@@ -711,6 +733,22 @@ typedef struct PgStat_GlobalStats
 	TimestampTz stat_reset_timestamp;
 } PgStat_GlobalStats;
 
+/*
+ * Slru statistics kept in the stats collector
+ */
+typedef struct PgStat_SlruStats
+{
+	PgStat_Counter pages_zero;
+	PgStat_Counter pages_hit;
+	PgStat_Counter pages_miss;
+	PgStat_Counter pages_exists;
+	PgStat_Counter pages_read;
+	PgStat_Counter pages_write;
+	PgStat_Counter io_error;
+	PgStat_Counter flush;
+	PgStat_Counter truncate;
+} PgStat_SlruStats;
+
 
 /* ----------
  * Backend types
@@ -1223,6 +1261,11 @@ extern char *pgstat_stat_filename;
  */
 extern PgStat_MsgBgWriter BgWriterStats;
 
+/*
+ * SLRU statistics counters are updated directly by slru.
+ */
+extern PgStat_MsgSlru SlruStats[SLRU_OTHER + 1];
+
 /*
  * Updated by pgstat_count_buffer_*_time macros
  */
@@ -1436,5 +1479,16 @@ extern PgStat_StatFuncEntry *pgstat_fetch_stat_funcentry(Oid funcid);
 extern int	pgstat_fetch_stat_numbackends(void);
 extern PgStat_ArchiverStats *pgstat_fetch_stat_archiver(void);
 extern PgStat_GlobalStats *pgstat_fetch_global(void);
+extern PgStat_SlruStats *pgstat_fetch_slru(void);
+
+extern void pgstat_count_slru_zero_page(SlruCtl ctl);
+extern void pgstat_count_slru_page_hit(SlruCtl ctl);
+extern void pgstat_count_slru_page_miss(SlruCtl ctl);
+extern void pgstat_count_slru_page_exists(SlruCtl ctl);
+extern void pgstat_count_slru_page_read(SlruCtl ctl);
+extern void pgstat_count_slru_page_write(SlruCtl ctl);
+extern void pgstat_count_slru_io_error(SlruCtl ctl);
+extern void pgstat_count_slru_flush(SlruCtl ctl);
+extern void pgstat_count_slru_truncate(SlruCtl ctl);
 
 #endif							/* PGSTAT_H */
#2tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Tomas Vondra (#1)
RE: SLRU statistics

From: Tomas Vondra <tomas.vondra@2ndquadrant.com>

One of the stats I occasionally wanted to know are stats for the SLRU
stats (we have couple of those - clog, subtrans, ...). So here is a WIP
version of a patch adding that.

How can users take advantage of this information? I think we also need the ability to set the size of SLRU buffers. (I want to be freed from the concern about the buffer shortage by setting the buffer size to its maximum. For example, CLOG would be only 1 GB.)

Regards
Takayuki Tsunakawa

#3Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: tsunakawa.takay@fujitsu.com (#2)
Re: SLRU statistics

On Mon, Jan 20, 2020 at 01:04:33AM +0000, tsunakawa.takay@fujitsu.com wrote:

From: Tomas Vondra <tomas.vondra@2ndquadrant.com>

One of the stats I occasionally wanted to know are stats for the SLRU
stats (we have couple of those - clog, subtrans, ...). So here is a WIP
version of a patch adding that.

How can users take advantage of this information? I think we also need
the ability to set the size of SLRU buffers. (I want to be freed from
the concern about the buffer shortage by setting the buffer size to its
maximum. For example, CLOG would be only 1 GB.)

You're right the users can't really take advantage of this - my primary
motivation was providing a feedback for devs, benchmarking etc. That
might have been done with DEBUG messages or something, but this seems
more convenient.

I think it's unclear how desirable / necessary it is to allow users to
tweak those caches. I don't think we should have a GUC for everything,
but maybe there's some sort of heuristics to determine the size. The
assumption is we actually find practical workloads where the size of
these SLRUs is a performance issue.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#4Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Tomas Vondra (#3)
Re: SLRU statistics

On 2020-Jan-20, Tomas Vondra wrote:

On Mon, Jan 20, 2020 at 01:04:33AM +0000, tsunakawa.takay@fujitsu.com wrote:

From: Tomas Vondra <tomas.vondra@2ndquadrant.com>

One of the stats I occasionally wanted to know are stats for the SLRU
stats (we have couple of those - clog, subtrans, ...). So here is a WIP
version of a patch adding that.

How can users take advantage of this information? I think we also need
the ability to set the size of SLRU buffers. (I want to be freed from
the concern about the buffer shortage by setting the buffer size to its
maximum. For example, CLOG would be only 1 GB.)

You're right the users can't really take advantage of this - my primary
motivation was providing a feedback for devs, benchmarking etc. That
might have been done with DEBUG messages or something, but this seems
more convenient.

I think the stats are definitely needed if we keep the current code.
I've researched some specific problems in this code, such as the need
for more subtrans SLRU buffers; IIRC it was pretty painful to figure out
what the problem was without counters, and it'd have been trivial with
them.

I think it's unclear how desirable / necessary it is to allow users to
tweak those caches. I don't think we should have a GUC for everything,
but maybe there's some sort of heuristics to determine the size. The
assumption is we actually find practical workloads where the size of
these SLRUs is a performance issue.

I expect we'll eventually realize the need for changes in this area.
Either configurability in the buffer pool sizes, or moving them to be
part of shared_buffers (IIRC Thomas Munro had a patch for this.)
Example: SLRUs like pg_commit and pg_subtrans have higher buffer
consumption as the range of open transactions increases; for many users
this is not a concern and they can live with the default values.

(I think when pg_commit (n�e pg_clog) buffers were increased, we should
have increased pg_subtrans buffers to match.)

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#5Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Alvaro Herrera (#4)
Re: SLRU statistics

On Mon, Jan 20, 2020 at 03:01:36PM -0300, Alvaro Herrera wrote:

On 2020-Jan-20, Tomas Vondra wrote:

On Mon, Jan 20, 2020 at 01:04:33AM +0000, tsunakawa.takay@fujitsu.com wrote:

From: Tomas Vondra <tomas.vondra@2ndquadrant.com>

One of the stats I occasionally wanted to know are stats for the SLRU
stats (we have couple of those - clog, subtrans, ...). So here is a WIP
version of a patch adding that.

How can users take advantage of this information? I think we also need
the ability to set the size of SLRU buffers. (I want to be freed from
the concern about the buffer shortage by setting the buffer size to its
maximum. For example, CLOG would be only 1 GB.)

You're right the users can't really take advantage of this - my primary
motivation was providing a feedback for devs, benchmarking etc. That
might have been done with DEBUG messages or something, but this seems
more convenient.

I think the stats are definitely needed if we keep the current code.
I've researched some specific problems in this code, such as the need
for more subtrans SLRU buffers; IIRC it was pretty painful to figure out
what the problem was without counters, and it'd have been trivial with
them.

Right. Improving our ability to monitor/measure things is the goal of
this patch.

I think it's unclear how desirable / necessary it is to allow users to
tweak those caches. I don't think we should have a GUC for everything,
but maybe there's some sort of heuristics to determine the size. The
assumption is we actually find practical workloads where the size of
these SLRUs is a performance issue.

I expect we'll eventually realize the need for changes in this area.
Either configurability in the buffer pool sizes, or moving them to be
part of shared_buffers (IIRC Thomas Munro had a patch for this.)
Example: SLRUs like pg_commit and pg_subtrans have higher buffer
consumption as the range of open transactions increases; for many users
this is not a concern and they can live with the default values.

(I think when pg_commit (n�e pg_clog) buffers were increased, we should
have increased pg_subtrans buffers to match.)

Quite possibly, yes. All I'm saying is that it's not something I intend
to address with this patch. It's quite possible the solutions will be
different for each SLRU, and that will require more research.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#6tsunakawa.takay@fujitsu.com
tsunakawa.takay@fujitsu.com
In reply to: Tomas Vondra (#3)
RE: SLRU statistics

From: Tomas Vondra <tomas.vondra@2ndquadrant.com>

You're right the users can't really take advantage of this - my primary
motivation was providing a feedback for devs, benchmarking etc. That
might have been done with DEBUG messages or something, but this seems
more convenient.

Understood. I'm in favor of adding performance information even if it doesn't make sense for users (like other DBMSs sometimes do.) One concern is that all the PostgreSQL performance statistics have been useful so far for tuning in some way, and this may become the first exception. Do we describe the SLRU stats view in the manual, or hide it only for PG devs and support staff?

I think it's unclear how desirable / necessary it is to allow users to
tweak those caches. I don't think we should have a GUC for everything,
but maybe there's some sort of heuristics to determine the size. The
assumption is we actually find practical workloads where the size of
these SLRUs is a performance issue.

I understood that the new performance statistics are expected to reveal what SLRUs need to be tunable and/or implemented with a different mechanism like shared buffers.

Regards
Takayuki Tsunakawa

#7Masahiko Sawada
masahiko.sawada@2ndquadrant.com
In reply to: Tomas Vondra (#3)
Re: SLRU statistics

On Tue, 21 Jan 2020 at 01:38, Tomas Vondra <tomas.vondra@2ndquadrant.com> wrote:

On Mon, Jan 20, 2020 at 01:04:33AM +0000, tsunakawa.takay@fujitsu.com wrote:

From: Tomas Vondra <tomas.vondra@2ndquadrant.com>

One of the stats I occasionally wanted to know are stats for the SLRU
stats (we have couple of those - clog, subtrans, ...). So here is a WIP
version of a patch adding that.

+1

How can users take advantage of this information? I think we also need
the ability to set the size of SLRU buffers. (I want to be freed from
the concern about the buffer shortage by setting the buffer size to its
maximum. For example, CLOG would be only 1 GB.)

You're right the users can't really take advantage of this - my primary
motivation was providing a feedback for devs, benchmarking etc. That
might have been done with DEBUG messages or something, but this seems
more convenient.

I've not tested the performance impact but perhaps we might want to
disable these counter by default and controlled by a GUC. And similar
to buffer statistics it might be better to inline
pgstat_count_slru_page_xxx function for better performance.

Regards,

--
Masahiko Sawada http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#8Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Masahiko Sawada (#7)
Re: SLRU statistics

On Tue, Jan 21, 2020 at 05:09:33PM +0900, Masahiko Sawada wrote:

On Tue, 21 Jan 2020 at 01:38, Tomas Vondra <tomas.vondra@2ndquadrant.com> wrote:

On Mon, Jan 20, 2020 at 01:04:33AM +0000, tsunakawa.takay@fujitsu.com wrote:

From: Tomas Vondra <tomas.vondra@2ndquadrant.com>

One of the stats I occasionally wanted to know are stats for the SLRU
stats (we have couple of those - clog, subtrans, ...). So here is a WIP
version of a patch adding that.

+1

How can users take advantage of this information? I think we also need
the ability to set the size of SLRU buffers. (I want to be freed from
the concern about the buffer shortage by setting the buffer size to its
maximum. For example, CLOG would be only 1 GB.)

You're right the users can't really take advantage of this - my primary
motivation was providing a feedback for devs, benchmarking etc. That
might have been done with DEBUG messages or something, but this seems
more convenient.

I've not tested the performance impact but perhaps we might want to
disable these counter by default and controlled by a GUC. And similar
to buffer statistics it might be better to inline
pgstat_count_slru_page_xxx function for better performance.

Hmmm, yeah. Inlining seems like a good idea, and maybe we should have
something like track_slru GUC.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#9Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: tsunakawa.takay@fujitsu.com (#6)
Re: SLRU statistics

On Tue, Jan 21, 2020 at 06:24:29AM +0000, tsunakawa.takay@fujitsu.com
wrote:

From: Tomas Vondra <tomas.vondra@2ndquadrant.com>

You're right the users can't really take advantage of this - my
primary motivation was providing a feedback for devs, benchmarking
etc. That might have been done with DEBUG messages or something, but
this seems more convenient.

Understood. I'm in favor of adding performance information even if it
doesn't make sense for users (like other DBMSs sometimes do.) One
concern is that all the PostgreSQL performance statistics have been
useful so far for tuning in some way, and this may become the first
exception. Do we describe the SLRU stats view in the manual, or hide
it only for PG devs and support staff?

Yes, the pg_stat_slru view should be described in a manual. That's
missing from the patch.

I think it's unclear how desirable / necessary it is to allow users
to tweak those caches. I don't think we should have a GUC for
everything, but maybe there's some sort of heuristics to determine
the size. The assumption is we actually find practical workloads
where the size of these SLRUs is a performance issue.

I understood that the new performance statistics are expected to reveal
what SLRUs need to be tunable and/or implemented with a different
mechanism like shared buffers.

Right. It's certainly meant to provide information for further tuning.
I'm just saying it's targeted more at developers, at least initially.
Maybe we'll end up with GUCs, maybe we'll choose other approaches for
some SLRUs. I don't have an opinion on that yet.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#10Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Tomas Vondra (#8)
Re: SLRU statistics

On 2020-Jan-21, Tomas Vondra wrote:

On Tue, Jan 21, 2020 at 05:09:33PM +0900, Masahiko Sawada wrote:

I've not tested the performance impact but perhaps we might want to
disable these counter by default and controlled by a GUC. And similar
to buffer statistics it might be better to inline
pgstat_count_slru_page_xxx function for better performance.

Hmmm, yeah. Inlining seems like a good idea, and maybe we should have
something like track_slru GUC.

I disagree with adding a GUC. If a performance impact can be measured
let's turn the functions to static inline, as already proposed. My
guess is that pgstat_count_slru_page_hit() is the only candidate for
that; all the other paths involve I/O or lock acquisition or even WAL
generation, so the impact won't be measurable anyhow. We removed
track-enabling GUCs years ago.

BTW, this comment:
/* update the stats counter of pages found in shared buffers */

is not strictly true, because we don't use what we normally call "shared
buffers" for SLRUs.

Patch applies cleanly. I suggest to move the page_miss() call until
after SlruRecentlyUsed(), for consistency with the other case.

I find SlruType pretty odd, and the accompanying "if" list in
pg_stat_get_slru() correspondingly so. Would it be possible to have
each SLRU enumerate itself somehow? Maybe add the name in SlruCtlData
and query that, somehow. (I don't think we have an array of SlruCtlData
anywhere though, so this might be a useless idea.)

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#11Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Alvaro Herrera (#10)
Re: SLRU statistics

On Fri, Feb 28, 2020 at 08:19:18PM -0300, Alvaro Herrera wrote:

On 2020-Jan-21, Tomas Vondra wrote:

On Tue, Jan 21, 2020 at 05:09:33PM +0900, Masahiko Sawada wrote:

I've not tested the performance impact but perhaps we might want to
disable these counter by default and controlled by a GUC. And similar
to buffer statistics it might be better to inline
pgstat_count_slru_page_xxx function for better performance.

Hmmm, yeah. Inlining seems like a good idea, and maybe we should have
something like track_slru GUC.

I disagree with adding a GUC. If a performance impact can be measured
let's turn the functions to static inline, as already proposed. My
guess is that pgstat_count_slru_page_hit() is the only candidate for
that; all the other paths involve I/O or lock acquisition or even WAL
generation, so the impact won't be measurable anyhow. We removed
track-enabling GUCs years ago.

Did we actually remove track-enabling GUCs? I think we still have

- track_activities
- track_counts
- track_io_timing
- track_functions

But maybe I'm missing something?

That being said, I'm not sure we need to add a GUC. I'll do some
measurements and we'll see. Maybe the statis inline will me enough.

BTW, this comment:
/* update the stats counter of pages found in shared buffers */

is not strictly true, because we don't use what we normally call "shared
buffers" for SLRUs.

Oh, right. Will fix.

Patch applies cleanly. I suggest to move the page_miss() call until
after SlruRecentlyUsed(), for consistency with the other case.

OK.

I find SlruType pretty odd, and the accompanying "if" list in
pg_stat_get_slru() correspondingly so. Would it be possible to have
each SLRU enumerate itself somehow? Maybe add the name in SlruCtlData
and query that, somehow. (I don't think we have an array of SlruCtlData
anywhere though, so this might be a useless idea.)

Well, maybe. We could have a system to register SLRUs dynamically, but
the trick here is that by having a fixed predefined number of SLRUs
simplifies serialization in pgstat.c and so on. I don't think the "if"
branches in pg_stat_get_slru() are particularly ugly, but maybe we could
replace the enum with a registry of structs, something like rmgrlist.h.
It seems like an overkill to me, though.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#12Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Tomas Vondra (#11)
Re: SLRU statistics

On 2020-Feb-29, Tomas Vondra wrote:

Did we actually remove track-enabling GUCs? I think we still have

- track_activities
- track_counts
- track_io_timing
- track_functions

But maybe I'm missing something?

Hm I remembered we removed the one for row-level stats
(track_row_stats), but what we really did is merge it with block-level
stats (track_block_stats) into track_counts -- commit 48f7e6439568.
Funnily enough, if you disable that autovacuum won't work, so I'm not
sure it's a very useful tunable. And it definitely has more overhead
than what this new GUC would have.

I find SlruType pretty odd, and the accompanying "if" list in
pg_stat_get_slru() correspondingly so. Would it be possible to have
each SLRU enumerate itself somehow? Maybe add the name in SlruCtlData
and query that, somehow. (I don't think we have an array of SlruCtlData
anywhere though, so this might be a useless idea.)

Well, maybe. We could have a system to register SLRUs dynamically, but
the trick here is that by having a fixed predefined number of SLRUs
simplifies serialization in pgstat.c and so on. I don't think the "if"
branches in pg_stat_get_slru() are particularly ugly, but maybe we could
replace the enum with a registry of structs, something like rmgrlist.h.
It seems like an overkill to me, though.

Yeah, maybe we don't have to fix that now.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#13Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Alvaro Herrera (#12)
1 attachment(s)
Re: SLRU statistics

On Sat, Feb 29, 2020 at 11:44:26AM -0300, Alvaro Herrera wrote:

On 2020-Feb-29, Tomas Vondra wrote:

Did we actually remove track-enabling GUCs? I think we still have

- track_activities
- track_counts
- track_io_timing
- track_functions

But maybe I'm missing something?

Hm I remembered we removed the one for row-level stats
(track_row_stats), but what we really did is merge it with block-level
stats (track_block_stats) into track_counts -- commit 48f7e6439568.
Funnily enough, if you disable that autovacuum won't work, so I'm not
sure it's a very useful tunable. And it definitely has more overhead
than what this new GUC would have.

OK

I find SlruType pretty odd, and the accompanying "if" list in
pg_stat_get_slru() correspondingly so. Would it be possible to have
each SLRU enumerate itself somehow? Maybe add the name in SlruCtlData
and query that, somehow. (I don't think we have an array of SlruCtlData
anywhere though, so this might be a useless idea.)

Well, maybe. We could have a system to register SLRUs dynamically, but
the trick here is that by having a fixed predefined number of SLRUs
simplifies serialization in pgstat.c and so on. I don't think the "if"
branches in pg_stat_get_slru() are particularly ugly, but maybe we could
replace the enum with a registry of structs, something like rmgrlist.h.
It seems like an overkill to me, though.

Yeah, maybe we don't have to fix that now.

IMO the current solution is sufficient for the purpose. I guess we could
just stick a name into the SlruCtlData (and remove SlruType entirely),
and use that to identify the stats entries. That might be enough, and in
fact we already have that - SimpleLruInit gets a name parameter and
copies that to the lwlock_tranche_name.

One of the main reasons why I opted to use the enum is that it makes
tracking, lookup and serialization pretty trivial - it's just an index
lookup, etc. But maybe it wouldn't be much more complex with the name,
considering the name length is limited by SLRU_MAX_NAME_LENGTH. And we
probably don't expect many entries, so we could keep them in a simple
list, or maybe a simplehash.

I'm not sure what to do with data for SLRUs that might have disappeared
after a restart (e.g. because someone removed an extension). Until now
those would be in the all in the "other" entry.

The attached v2 fixes the issues in your first message:

- I moved the page_miss() call after SlruRecentlyUsed(), but then I
realized it's entirely duplicate with the page_read() update done in
SlruPhysicalReadPage(). I removed the call from SlruPhysicalReadPage()
and renamed page_miss to page_read - that's more consistent with
shared buffers stats, which also have buffers_hit and buffer_read.

- I've also implemented the reset. I ended up adding a new option to
pg_stat_reset_shared, which always resets all SLRU entries. We track
the reset timestamp for each SLRU entry, but the value is always the
same. I admit this is a bit weird - I did it like this because (a) I'm
not sure how to identify the individual entries and (b) the SLRU is
shared, so pg_stat_reset_shared seems kinda natural.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

stats-slru-v2.patchtext/plain; charset=us-asciiDownload
diff --git a/src/backend/access/transam/clog.c b/src/backend/access/transam/clog.c
index f8e7670f8d..3f45db7ea9 100644
--- a/src/backend/access/transam/clog.c
+++ b/src/backend/access/transam/clog.c
@@ -692,7 +692,8 @@ CLOGShmemInit(void)
 {
 	ClogCtl->PagePrecedes = CLOGPagePrecedes;
 	SimpleLruInit(ClogCtl, "clog", CLOGShmemBuffers(), CLOG_LSNS_PER_PAGE,
-				  CLogControlLock, "pg_xact", LWTRANCHE_CLOG_BUFFERS);
+				  CLogControlLock, "pg_xact", LWTRANCHE_CLOG_BUFFERS,
+				  SLRU_CLOG);
 }
 
 /*
diff --git a/src/backend/access/transam/commit_ts.c b/src/backend/access/transam/commit_ts.c
index 630df672cc..44d7ca4483 100644
--- a/src/backend/access/transam/commit_ts.c
+++ b/src/backend/access/transam/commit_ts.c
@@ -494,7 +494,7 @@ CommitTsShmemInit(void)
 	CommitTsCtl->PagePrecedes = CommitTsPagePrecedes;
 	SimpleLruInit(CommitTsCtl, "commit_timestamp", CommitTsShmemBuffers(), 0,
 				  CommitTsControlLock, "pg_commit_ts",
-				  LWTRANCHE_COMMITTS_BUFFERS);
+				  LWTRANCHE_COMMITTS_BUFFERS, SLRU_COMMIT_TS);
 
 	commitTsShared = ShmemInitStruct("CommitTs shared",
 									 sizeof(CommitTimestampShared),
diff --git a/src/backend/access/transam/multixact.c b/src/backend/access/transam/multixact.c
index 50e98caaeb..37a5854284 100644
--- a/src/backend/access/transam/multixact.c
+++ b/src/backend/access/transam/multixact.c
@@ -1831,11 +1831,11 @@ MultiXactShmemInit(void)
 	SimpleLruInit(MultiXactOffsetCtl,
 				  "multixact_offset", NUM_MXACTOFFSET_BUFFERS, 0,
 				  MultiXactOffsetControlLock, "pg_multixact/offsets",
-				  LWTRANCHE_MXACTOFFSET_BUFFERS);
+				  LWTRANCHE_MXACTOFFSET_BUFFERS, SLRU_MULTIXACT_OFFSET);
 	SimpleLruInit(MultiXactMemberCtl,
 				  "multixact_member", NUM_MXACTMEMBER_BUFFERS, 0,
 				  MultiXactMemberControlLock, "pg_multixact/members",
-				  LWTRANCHE_MXACTMEMBER_BUFFERS);
+				  LWTRANCHE_MXACTMEMBER_BUFFERS, SLRU_MULTIXACT_MEMBER);
 
 	/* Initialize our shared state struct */
 	MultiXactState = ShmemInitStruct("Shared MultiXact State",
diff --git a/src/backend/access/transam/slru.c b/src/backend/access/transam/slru.c
index d5b7a08f73..0c3f8b9251 100644
--- a/src/backend/access/transam/slru.c
+++ b/src/backend/access/transam/slru.c
@@ -162,7 +162,8 @@ SimpleLruShmemSize(int nslots, int nlsns)
 
 void
 SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
-			  LWLock *ctllock, const char *subdir, int tranche_id)
+			  LWLock *ctllock, const char *subdir, int tranche_id,
+			  SlruType type)
 {
 	SlruShared	shared;
 	bool		found;
@@ -247,6 +248,7 @@ SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
 	 */
 	ctl->shared = shared;
 	ctl->do_fsync = true;		/* default behavior */
+	ctl->type = type;
 	StrNCpy(ctl->Dir, subdir, sizeof(ctl->Dir));
 }
 
@@ -286,6 +288,9 @@ SimpleLruZeroPage(SlruCtl ctl, int pageno)
 	/* Assume this page is now the latest active page */
 	shared->latest_page_number = pageno;
 
+	/* update the stats counter of zeroed pages */
+	pgstat_count_slru_zero_page(ctl);
+
 	return slotno;
 }
 
@@ -403,6 +408,10 @@ SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 			}
 			/* Otherwise, it's ready to use */
 			SlruRecentlyUsed(shared, slotno);
+
+			/* update the stats counter of pages found in the SLRU */
+			pgstat_count_slru_page_hit(ctl);
+
 			return slotno;
 		}
 
@@ -444,6 +453,10 @@ SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 			SlruReportIOError(ctl, pageno, xid);
 
 		SlruRecentlyUsed(shared, slotno);
+
+		/* update the stats counter of pages not found in SLRU */
+		pgstat_count_slru_page_read(ctl);
+
 		return slotno;
 	}
 }
@@ -596,6 +609,9 @@ SimpleLruDoesPhysicalPageExist(SlruCtl ctl, int pageno)
 	bool		result;
 	off_t		endpos;
 
+	/* update the stats counter of checked pages */
+	pgstat_count_slru_page_exists(ctl);
+
 	SlruFileName(ctl, path, segno);
 
 	fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
@@ -730,6 +746,9 @@ SlruPhysicalWritePage(SlruCtl ctl, int pageno, int slotno, SlruFlush fdata)
 	char		path[MAXPGPATH];
 	int			fd = -1;
 
+	/* update the stats counter of written pages */
+	pgstat_count_slru_page_write(ctl);
+
 	/*
 	 * Honor the write-WAL-before-data rule, if appropriate, so that we do not
 	 * write out data before associated WAL records.  This is the same action
@@ -901,6 +920,9 @@ SlruReportIOError(SlruCtl ctl, int pageno, TransactionId xid)
 	int			offset = rpageno * BLCKSZ;
 	char		path[MAXPGPATH];
 
+	/* update the stats counter of errors */
+	pgstat_count_slru_io_error(ctl);
+
 	SlruFileName(ctl, path, segno);
 	errno = slru_errno;
 	switch (slru_errcause)
@@ -1125,6 +1147,9 @@ SimpleLruFlush(SlruCtl ctl, bool allow_redirtied)
 	int			i;
 	bool		ok;
 
+	/* update the stats counter of flushes */
+	pgstat_count_slru_flush(ctl);
+
 	/*
 	 * Find and write dirty pages
 	 */
@@ -1186,6 +1211,9 @@ SimpleLruTruncate(SlruCtl ctl, int cutoffPage)
 	SlruShared	shared = ctl->shared;
 	int			slotno;
 
+	/* update the stats counter of truncates */
+	pgstat_count_slru_truncate(ctl);
+
 	/*
 	 * The cutoff point is the start of the segment containing cutoffPage.
 	 */
diff --git a/src/backend/access/transam/subtrans.c b/src/backend/access/transam/subtrans.c
index 25d7d739cf..3316accb50 100644
--- a/src/backend/access/transam/subtrans.c
+++ b/src/backend/access/transam/subtrans.c
@@ -193,7 +193,7 @@ SUBTRANSShmemInit(void)
 	SubTransCtl->PagePrecedes = SubTransPagePrecedes;
 	SimpleLruInit(SubTransCtl, "subtrans", NUM_SUBTRANS_BUFFERS, 0,
 				  SubtransControlLock, "pg_subtrans",
-				  LWTRANCHE_SUBTRANS_BUFFERS);
+				  LWTRANCHE_SUBTRANS_BUFFERS, SLRU_SUBTRANS);
 	/* Override default assumption that writes should be fsync'd */
 	SubTransCtl->do_fsync = false;
 }
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f681aafcf9..10677ff778 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -792,6 +792,20 @@ CREATE VIEW pg_stat_replication AS
         JOIN pg_stat_get_wal_senders() AS W ON (S.pid = W.pid)
         LEFT JOIN pg_authid AS U ON (S.usesysid = U.oid);
 
+CREATE VIEW pg_stat_slru AS
+    SELECT
+            s.name,
+            s.pages_zero,
+            s.pages_hit,
+            s.pages_read,
+            s.pages_write,
+            s.pages_exists,
+            s.io_error,
+            s.flushes,
+            s.truncates,
+            s.stat_reset
+    FROM pg_stat_get_slru() s;
+
 CREATE VIEW pg_stat_wal_receiver AS
     SELECT
             s.pid,
diff --git a/src/backend/commands/async.c b/src/backend/commands/async.c
index dae939a4ab..f442125ead 100644
--- a/src/backend/commands/async.c
+++ b/src/backend/commands/async.c
@@ -552,7 +552,8 @@ AsyncShmemInit(void)
 	 */
 	AsyncCtl->PagePrecedes = asyncQueuePagePrecedes;
 	SimpleLruInit(AsyncCtl, "async", NUM_ASYNC_BUFFERS, 0,
-				  AsyncCtlLock, "pg_notify", LWTRANCHE_ASYNC_BUFFERS);
+				  AsyncCtlLock, "pg_notify", LWTRANCHE_ASYNC_BUFFERS,
+				  SLRU_ASYNC);
 	/* Override default assumption that writes should be fsync'd */
 	AsyncCtl->do_fsync = false;
 
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 462b4d7e06..37e1312fa2 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -141,6 +141,9 @@ char	   *pgstat_stat_tmpname = NULL;
  */
 PgStat_MsgBgWriter BgWriterStats;
 
+/* TODO */
+PgStat_MsgSlru SlruStats[SLRU_OTHER + 1];
+
 /* ----------
  * Local data
  * ----------
@@ -255,6 +258,7 @@ static int	localNumBackends = 0;
  */
 static PgStat_ArchiverStats archiverStats;
 static PgStat_GlobalStats globalStats;
+static PgStat_SlruStats slruStats[SLRU_OTHER + 1];
 
 /*
  * List of OIDs of databases we need to write out.  If an entry is InvalidOid,
@@ -297,6 +301,7 @@ static bool pgstat_db_requested(Oid databaseid);
 
 static void pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg);
 static void pgstat_send_funcstats(void);
+static void pgstat_send_slru(void);
 static HTAB *pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid);
 
 static PgStat_TableStatus *get_tabstat_entry(Oid rel_id, bool isshared);
@@ -324,6 +329,7 @@ static void pgstat_recv_vacuum(PgStat_MsgVacuum *msg, int len);
 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_slru(PgStat_MsgSlru *msg, int len);
 static void pgstat_recv_funcstat(PgStat_MsgFuncstat *msg, int len);
 static void pgstat_recv_funcpurge(PgStat_MsgFuncpurge *msg, int len);
 static void pgstat_recv_recoveryconflict(PgStat_MsgRecoveryConflict *msg, int len);
@@ -907,6 +913,9 @@ pgstat_report_stat(bool force)
 
 	/* Now, send function statistics */
 	pgstat_send_funcstats();
+
+	/* Finally send SLRU statistics */
+	pgstat_send_slru();
 }
 
 /*
@@ -1337,6 +1346,8 @@ pgstat_reset_shared_counters(const char *target)
 		msg.m_resettarget = RESET_ARCHIVER;
 	else if (strcmp(target, "bgwriter") == 0)
 		msg.m_resettarget = RESET_BGWRITER;
+	else if (strcmp(target, "slru") == 0)
+		msg.m_resettarget = RESET_SLRU;
 	else
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -2622,6 +2633,23 @@ pgstat_fetch_global(void)
 }
 
 
+/*
+ * ---------
+ * pgstat_fetch_slru() -
+ *
+ *	Support function for the SQL-callable pgstat* functions. Returns
+ *	a pointer to the slru statistics struct.
+ * ---------
+ */
+PgStat_SlruStats *
+pgstat_fetch_slru(void)
+{
+	backend_read_statsfile();
+
+	return slruStats;
+}
+
+
 /* ------------------------------------------------------------
  * Functions for management of the shared-memory PgBackendStatus array
  * ------------------------------------------------------------
@@ -4413,6 +4441,46 @@ pgstat_send_bgwriter(void)
 	MemSet(&BgWriterStats, 0, sizeof(BgWriterStats));
 }
 
+/* ----------
+ * pgstat_send_slru() -
+ *
+ *		Send slru statistics to the collector
+ * ----------
+ */
+static void
+pgstat_send_slru(void)
+{
+	int		i;
+
+	/* We assume this initializes to zeroes */
+	static const PgStat_MsgSlru all_zeroes;
+
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		/*
+		 * This function can be called even if nothing at all has happened. In
+		 * this case, avoid sending a completely empty message to the stats
+		 * collector.
+		 */
+		if (memcmp(&SlruStats[i], &all_zeroes, sizeof(PgStat_MsgSlru)) == 0)
+			continue;
+
+		/* set the SLRU type before each send */
+		SlruStats[i].m_type = i;
+
+		/*
+		 * Prepare and send the message
+		 */
+		pgstat_setheader(&SlruStats[i].m_hdr, PGSTAT_MTYPE_SLRU);
+		pgstat_send(&SlruStats[i], sizeof(PgStat_MsgSlru));
+
+		/*
+		 * Clear out the statistics buffer, so it can be re-used.
+		 */
+		MemSet(&SlruStats[i], 0, sizeof(PgStat_MsgSlru));
+	}
+}
+
 
 /* ----------
  * PgstatCollectorMain() -
@@ -4603,6 +4671,10 @@ PgstatCollectorMain(int argc, char *argv[])
 					pgstat_recv_bgwriter(&msg.msg_bgwriter, len);
 					break;
 
+				case PGSTAT_MTYPE_SLRU:
+					pgstat_recv_slru(&msg.msg_slru, len);
+					break;
+
 				case PGSTAT_MTYPE_FUNCSTAT:
 					pgstat_recv_funcstat(&msg.msg_funcstat, len);
 					break;
@@ -4831,6 +4903,7 @@ pgstat_write_statsfiles(bool permanent, bool allDbs)
 	const char *tmpfile = permanent ? PGSTAT_STAT_PERMANENT_TMPFILE : pgstat_stat_tmpname;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
 	int			rc;
+	int			i;
 
 	elog(DEBUG2, "writing stats file \"%s\"", statfile);
 
@@ -4871,6 +4944,15 @@ pgstat_write_statsfiles(bool permanent, bool allDbs)
 	rc = fwrite(&archiverStats, sizeof(archiverStats), 1, fpout);
 	(void) rc;					/* we'll check for error with ferror */
 
+	/*
+	 * Write SLRU stats struct
+	 */
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		rc = fwrite(&slruStats[i], sizeof(PgStat_SlruStats), 1, fpout);
+		(void) rc;					/* we'll check for error with ferror */
+	}
+
 	/*
 	 * Walk through the database table.
 	 */
@@ -5106,6 +5188,7 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	int32		format_id;
 	bool		found;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	int			i;
 
 	/*
 	 * The tables will live in pgStatLocalContext.
@@ -5128,6 +5211,7 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	 */
 	memset(&globalStats, 0, sizeof(globalStats));
 	memset(&archiverStats, 0, sizeof(archiverStats));
+	memset(&slruStats, 0, sizeof(slruStats));
 
 	/*
 	 * Set the current timestamp (will be kept only in case we can't load an
@@ -5136,6 +5220,12 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	globalStats.stat_reset_timestamp = GetCurrentTimestamp();
 	archiverStats.stat_reset_timestamp = globalStats.stat_reset_timestamp;
 
+	/*
+	 * Set the same reset timestamp for all SLRU items.
+	 */
+	for (i = 0; i <= SLRU_OTHER; i++)
+		slruStats[i].stat_reset_timestamp = globalStats.stat_reset_timestamp;
+
 	/*
 	 * Try to open the stats file. If it doesn't exist, the backends simply
 	 * return zero for anything and the collector simply starts from scratch
@@ -5198,6 +5288,20 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 		goto done;
 	}
 
+	/*
+	 * Read SLRU stats struct
+	 */
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		if (fread(&slruStats[i], 1, sizeof(PgStat_SlruStats), fpin) != sizeof(PgStat_SlruStats))
+		{
+			ereport(pgStatRunningInCollector ? LOG : WARNING,
+					(errmsg("corrupted statistics file \"%s\"", statfile)));
+			memset(&slruStats[i], 0, sizeof(PgStat_SlruStats));
+			goto done;
+		}
+	}
+
 	/*
 	 * We found an existing collector stats file. Read it and put all the
 	 * hashtable entries into place.
@@ -5496,9 +5600,11 @@ pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
 	PgStat_StatDBEntry dbentry;
 	PgStat_GlobalStats myGlobalStats;
 	PgStat_ArchiverStats myArchiverStats;
+	PgStat_SlruStats mySlruStats;
 	FILE	   *fpin;
 	int32		format_id;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	int			i;
 
 	/*
 	 * Try to open the stats file.  As above, anything but ENOENT is worthy of
@@ -5550,6 +5656,21 @@ pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
 		return false;
 	}
 
+	/*
+	 * Read SLRU stats struct
+	 */
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		if (fread(&mySlruStats, 1, sizeof(PgStat_SlruStats),
+				  fpin) != sizeof(PgStat_SlruStats))
+		{
+			ereport(pgStatRunningInCollector ? LOG : WARNING,
+					(errmsg("corrupted statistics file \"%s\"", statfile)));
+			FreeFile(fpin);
+			return false;
+		}
+	}
+
 	/* By default, we're going to return the timestamp of the global file. */
 	*ts = myGlobalStats.stats_timestamp;
 
@@ -6112,6 +6233,17 @@ pgstat_recv_resetsharedcounter(PgStat_MsgResetsharedcounter *msg, int len)
 		memset(&archiverStats, 0, sizeof(archiverStats));
 		archiverStats.stat_reset_timestamp = GetCurrentTimestamp();
 	}
+	else if (msg->m_resettarget == RESET_SLRU)
+	{
+		int			i;
+		TimestampTz	ts = GetCurrentTimestamp();
+
+		/* Reset the SLRU statistics for the cluster. */
+		memset(&slruStats, 0, sizeof(slruStats));
+
+		for (i = 0; i <= SLRU_OTHER; i++)
+			slruStats[i].stat_reset_timestamp = ts;
+	}
 
 	/*
 	 * Presumably the sender of this message validated the target, don't
@@ -6291,6 +6423,25 @@ pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len)
 	globalStats.buf_alloc += msg->m_buf_alloc;
 }
 
+/* ----------
+ * pgstat_recv_slru() -
+ *
+ *	Process a SLRU message.
+ * ----------
+ */
+static void
+pgstat_recv_slru(PgStat_MsgSlru *msg, int len)
+{
+	slruStats[msg->m_type].pages_zero += msg->m_pages_zero;
+	slruStats[msg->m_type].pages_hit += msg->m_pages_hit;
+	slruStats[msg->m_type].pages_read += msg->m_pages_read;
+	slruStats[msg->m_type].pages_write += msg->m_pages_write;
+	slruStats[msg->m_type].pages_exists += msg->m_pages_exists;
+	slruStats[msg->m_type].io_error += msg->m_io_error;
+	slruStats[msg->m_type].flush += msg->m_flush;
+	slruStats[msg->m_type].truncate += msg->m_truncate;
+}
+
 /* ----------
  * pgstat_recv_recoveryconflict() -
  *
@@ -6545,3 +6696,51 @@ pgstat_clip_activity(const char *raw_activity)
 
 	return activity;
 }
+
+void
+pgstat_count_slru_zero_page(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_zero += 1;
+}
+
+void
+pgstat_count_slru_page_hit(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_hit += 1;
+}
+
+void
+pgstat_count_slru_page_exists(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_exists += 1;
+}
+
+void
+pgstat_count_slru_page_read(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_read += 1;
+}
+
+void
+pgstat_count_slru_page_write(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_pages_write += 1;
+}
+
+void
+pgstat_count_slru_io_error(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_io_error += 1;
+}
+
+void
+pgstat_count_slru_flush(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_flush += 1;
+}
+
+void
+pgstat_count_slru_truncate(SlruCtl ctl)
+{
+	SlruStats[ctl->type].m_truncate += 1;
+}
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index 654584b77a..53164bdff3 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -821,7 +821,7 @@ OldSerXidInit(void)
 	OldSerXidSlruCtl->PagePrecedes = OldSerXidPagePrecedesLogically;
 	SimpleLruInit(OldSerXidSlruCtl, "oldserxid",
 				  NUM_OLDSERXID_BUFFERS, 0, OldSerXidLock, "pg_serial",
-				  LWTRANCHE_OLDSERXID_BUFFERS);
+				  LWTRANCHE_OLDSERXID_BUFFERS, SLRU_OLDSERXID);
 	/* Override default assumption that writes should be fsync'd */
 	OldSerXidSlruCtl->do_fsync = false;
 
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 7e6a3c1774..40b56e0cd0 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1672,6 +1672,96 @@ pg_stat_get_buf_alloc(PG_FUNCTION_ARGS)
 	PG_RETURN_INT64(pgstat_fetch_global()->buf_alloc);
 }
 
+/*
+ * Returns statistics of SLRU caches.
+ */
+Datum
+pg_stat_get_slru(PG_FUNCTION_ARGS)
+{
+#define PG_STAT_GET_SLRU_COLS	10
+	ReturnSetInfo  *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	TupleDesc		tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext 	per_query_ctx;
+	MemoryContext 	oldcontext;
+	int				i;
+	PgStat_SlruStats *stats;
+
+	/* 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);
+
+	/* request SLRU stats from the stat collector */
+	stats = pgstat_fetch_slru();
+
+	for (i = 0; i <= SLRU_OTHER; i++)
+	{
+		/* for each row */
+		Datum		values[PG_STAT_GET_SLRU_COLS];
+		bool		nulls[PG_STAT_GET_SLRU_COLS];
+		PgStat_SlruStats	stat = stats[i];
+		text	   *name;
+
+		MemSet(values, 0, sizeof(values));
+		MemSet(nulls, 0, sizeof(nulls));
+
+		if (i == SLRU_CLOG)
+			name = cstring_to_text("clog");
+		else if (i == SLRU_COMMIT_TS)
+			name = cstring_to_text("commit_timestamp");
+		else if (i == SLRU_MULTIXACT_OFFSET)
+			name = cstring_to_text("multixact_offset");
+		else if (i == SLRU_MULTIXACT_MEMBER)
+			name = cstring_to_text("multixact_member");
+		else if (i == SLRU_SUBTRANS)
+			name = cstring_to_text("subtrans");
+		else if (i == SLRU_ASYNC)
+			name = cstring_to_text("async");
+		else if (i == SLRU_OLDSERXID)
+			name = cstring_to_text("oldserxid");
+		else if (i == SLRU_OTHER)
+			name = cstring_to_text("other");
+
+		values[0] = PointerGetDatum(name);
+		values[1] = Int64GetDatum(stat.pages_zero);
+		values[2] = Int64GetDatum(stat.pages_hit);
+		values[3] = Int64GetDatum(stat.pages_read);
+		values[4] = Int64GetDatum(stat.pages_write);
+		values[5] = Int64GetDatum(stat.pages_exists);
+		values[6] = Int64GetDatum(stat.io_error);
+		values[7] = Int64GetDatum(stat.flush);
+		values[8] = Int64GetDatum(stat.truncate);
+		values[9] = Int64GetDatum(stat.stat_reset_timestamp);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	/* clean up and return the tuplestore */
+	tuplestore_donestoring(tupstore);
+
+	return (Datum) 0;
+}
+
 Datum
 pg_stat_get_xact_numscans(PG_FUNCTION_ARGS)
 {
diff --git a/src/include/access/slru.h b/src/include/access/slru.h
index 00dbd803e1..eb79ab346f 100644
--- a/src/include/access/slru.h
+++ b/src/include/access/slru.h
@@ -106,6 +106,18 @@ typedef struct SlruSharedData
 
 typedef SlruSharedData *SlruShared;
 
+typedef enum SlruType
+{
+	SLRU_CLOG,
+	SLRU_COMMIT_TS,
+	SLRU_MULTIXACT_OFFSET,
+	SLRU_MULTIXACT_MEMBER,
+	SLRU_SUBTRANS,
+	SLRU_ASYNC,
+	SLRU_OLDSERXID,
+	SLRU_OTHER
+} SlruType;
+
 /*
  * SlruCtlData is an unshared structure that points to the active information
  * in shared memory.
@@ -114,6 +126,9 @@ typedef struct SlruCtlData
 {
 	SlruShared	shared;
 
+	/* type of the SLRU */
+	SlruType	type;
+
 	/*
 	 * This flag tells whether to fsync writes (true for pg_xact and multixact
 	 * stuff, false for pg_subtrans and pg_notify).
@@ -139,7 +154,8 @@ typedef SlruCtlData *SlruCtl;
 
 extern Size SimpleLruShmemSize(int nslots, int nlsns);
 extern void SimpleLruInit(SlruCtl ctl, const char *name, int nslots, int nlsns,
-						  LWLock *ctllock, const char *subdir, int tranche_id);
+						  LWLock *ctllock, const char *subdir, int tranche_id,
+						  SlruType type);
 extern int	SimpleLruZeroPage(SlruCtl ctl, int pageno);
 extern int	SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 							  TransactionId xid);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 07a86c7b7b..0d75eda8f2 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5431,6 +5431,16 @@
   proname => 'pg_stat_get_buf_alloc', provolatile => 's', proparallel => 'r',
   prorettype => 'int8', proargtypes => '', prosrc => 'pg_stat_get_buf_alloc' },
 
+{ oid => '8614',
+  descr => 'statistics: information about SLRU caches',
+  proname => 'pg_stat_get_slru', prorows => '100', proisstrict => 'f',
+  proretset => 't', provolatile => 's', proparallel => 'r',
+  prorettype => 'record', proargtypes => '',
+  proallargtypes => '{text,int8,int8,int8,int8,int8,int8,int8,int8,timestamptz}',
+  proargmodes => '{o,o,o,o,o,o,o,o,o,o}',
+  proargnames => '{name,pages_zero,pages_hit,pages_read,pages_write,pages_exists,io_error,flushes,truncates,stat_reset}',
+  prosrc => 'pg_stat_get_slru' },
+
 { oid => '2978', descr => 'statistics: number of function calls',
   proname => 'pg_stat_get_function_calls', provolatile => 's',
   proparallel => 'r', prorettype => 'int8', proargtypes => 'oid',
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 3a65a51696..18a40e5c2d 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -11,6 +11,7 @@
 #ifndef PGSTAT_H
 #define PGSTAT_H
 
+#include "access/slru.h"
 #include "datatype/timestamp.h"
 #include "libpq/pqcomm.h"
 #include "port/atomics.h"
@@ -59,6 +60,7 @@ typedef enum StatMsgType
 	PGSTAT_MTYPE_ANALYZE,
 	PGSTAT_MTYPE_ARCHIVER,
 	PGSTAT_MTYPE_BGWRITER,
+	PGSTAT_MTYPE_SLRU,
 	PGSTAT_MTYPE_FUNCSTAT,
 	PGSTAT_MTYPE_FUNCPURGE,
 	PGSTAT_MTYPE_RECOVERYCONFLICT,
@@ -119,7 +121,8 @@ typedef struct PgStat_TableCounts
 typedef enum PgStat_Shared_Reset_Target
 {
 	RESET_ARCHIVER,
-	RESET_BGWRITER
+	RESET_BGWRITER,
+	RESET_SLRU
 } PgStat_Shared_Reset_Target;
 
 /* Possible object types for resetting single counters */
@@ -422,6 +425,24 @@ typedef struct PgStat_MsgBgWriter
 	PgStat_Counter m_checkpoint_sync_time;
 } PgStat_MsgBgWriter;
 
+/* ----------
+ * PgStat_MsgSlru			Sent by the slru to update statistics.
+ * ----------
+ */
+typedef struct PgStat_MsgSlru
+{
+	PgStat_MsgHdr m_hdr;
+	PgStat_Counter m_type;
+	PgStat_Counter m_pages_zero;
+	PgStat_Counter m_pages_hit;
+	PgStat_Counter m_pages_read;
+	PgStat_Counter m_pages_write;
+	PgStat_Counter m_pages_exists;
+	PgStat_Counter m_io_error;
+	PgStat_Counter m_flush;
+	PgStat_Counter m_truncate;
+} PgStat_MsgSlru;
+
 /* ----------
  * PgStat_MsgRecoveryConflict	Sent by the backend upon recovery conflict
  * ----------
@@ -564,6 +585,7 @@ typedef union PgStat_Msg
 	PgStat_MsgAnalyze msg_analyze;
 	PgStat_MsgArchiver msg_archiver;
 	PgStat_MsgBgWriter msg_bgwriter;
+	PgStat_MsgSlru msg_slru;
 	PgStat_MsgFuncstat msg_funcstat;
 	PgStat_MsgFuncpurge msg_funcpurge;
 	PgStat_MsgRecoveryConflict msg_recoveryconflict;
@@ -711,6 +733,22 @@ typedef struct PgStat_GlobalStats
 	TimestampTz stat_reset_timestamp;
 } PgStat_GlobalStats;
 
+/*
+ * Slru statistics kept in the stats collector
+ */
+typedef struct PgStat_SlruStats
+{
+	PgStat_Counter pages_zero;
+	PgStat_Counter pages_hit;
+	PgStat_Counter pages_read;
+	PgStat_Counter pages_write;
+	PgStat_Counter pages_exists;
+	PgStat_Counter io_error;
+	PgStat_Counter flush;
+	PgStat_Counter truncate;
+	TimestampTz stat_reset_timestamp;
+} PgStat_SlruStats;
+
 
 /* ----------
  * Backend types
@@ -1223,6 +1261,11 @@ extern char *pgstat_stat_filename;
  */
 extern PgStat_MsgBgWriter BgWriterStats;
 
+/*
+ * SLRU statistics counters are updated directly by slru.
+ */
+extern PgStat_MsgSlru SlruStats[SLRU_OTHER + 1];
+
 /*
  * Updated by pgstat_count_buffer_*_time macros
  */
@@ -1436,5 +1479,16 @@ extern PgStat_StatFuncEntry *pgstat_fetch_stat_funcentry(Oid funcid);
 extern int	pgstat_fetch_stat_numbackends(void);
 extern PgStat_ArchiverStats *pgstat_fetch_stat_archiver(void);
 extern PgStat_GlobalStats *pgstat_fetch_global(void);
+extern PgStat_SlruStats *pgstat_fetch_slru(void);
+
+extern void pgstat_count_slru_zero_page(SlruCtl ctl);
+extern void pgstat_count_slru_page_hit(SlruCtl ctl);
+extern void pgstat_count_slru_page_miss(SlruCtl ctl);
+extern void pgstat_count_slru_page_exists(SlruCtl ctl);
+extern void pgstat_count_slru_page_read(SlruCtl ctl);
+extern void pgstat_count_slru_page_write(SlruCtl ctl);
+extern void pgstat_count_slru_io_error(SlruCtl ctl);
+extern void pgstat_count_slru_flush(SlruCtl ctl);
+extern void pgstat_count_slru_truncate(SlruCtl ctl);
 
 #endif							/* PGSTAT_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 634f8256f7..714bce17d1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1988,6 +1988,17 @@ pg_stat_replication| SELECT s.pid,
    FROM ((pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, backend_type, ssl, sslversion, sslcipher, sslbits, sslcompression, ssl_client_dn, ssl_client_serial, ssl_issuer_dn, gss_auth, gss_princ, gss_enc, leader_pid)
      JOIN pg_stat_get_wal_senders() w(pid, state, sent_lsn, write_lsn, flush_lsn, replay_lsn, write_lag, flush_lag, replay_lag, sync_priority, sync_state, reply_time, spill_txns, spill_count, spill_bytes) ON ((s.pid = w.pid)))
      LEFT JOIN pg_authid u ON ((s.usesysid = u.oid)));
+pg_stat_slru| SELECT s.name,
+    s.pages_zero,
+    s.pages_hit,
+    s.pages_read,
+    s.pages_write,
+    s.pages_exists,
+    s.io_error,
+    s.flushes,
+    s.truncates,
+    s.stat_reset
+   FROM pg_stat_get_slru() s(name, pages_zero, pages_hit, pages_read, pages_write, pages_exists, io_error, flushes, truncates, stat_reset);
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
     s.sslversion AS version,
#14Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Tomas Vondra (#13)
1 attachment(s)
Re: SLRU statistics

Hi,

Attached is v3 of the patch with one big change and various small ones.

The main change is that it gets rid of the SlruType enum and the new
field in SlruCtlData. Instead, the patch now uses the name passed to
SimpleLruInit (which is then stored as LWLock tranche name).

The counters are still stored in a fixed-sized array, and there's a
simple name/index mapping. We don't have any registry of stable SLRU
IDs, so I can't think of anything better, and I think this is good
enough for now.

The other change is that I got rid of the io_error counter. We don't
have that for shared buffers etc. either, anyway.

I've also renamed the colunms from "pages" to "blks" to make it
consistent with other similar stats (blks_hit, blks_read). I've
renamed the fields to "blks_written" and "blks_zeroed".

And finally, I've added the view to monitoring.sgml.

Barring objections, I'll get this committed in the next few days, after
reviewing the comments a bit.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

0001-SLRU-stats-v3.patchtext/plain; charset=us-asciiDownload
From e1a187b9b331798c87900f94aa77999f9198f556 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Thu, 26 Mar 2020 20:52:26 +0100
Subject: [PATCH] SLRU stats

---
 doc/src/sgml/monitoring.sgml         |  77 +++++++++
 src/backend/access/transam/slru.c    |  23 +++
 src/backend/catalog/system_views.sql |  13 ++
 src/backend/postmaster/pgstat.c      | 238 +++++++++++++++++++++++++++
 src/backend/utils/adt/pgstatfuncs.c  |  77 +++++++++
 src/include/catalog/pg_proc.dat      |  10 ++
 src/include/pgstat.h                 |  53 +++++-
 src/test/regress/expected/rules.out  |  10 ++
 8 files changed, 500 insertions(+), 1 deletion(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 270178d57e..b58ac5acb8 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -575,6 +575,13 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       yet included in <structname>pg_stat_user_functions</structname>).</entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_slru</structname><indexterm><primary>pg_stat_slru</primary></indexterm></entry>
+      <entry>One row per SLRU, showing statistics of operations. See
+       <xref linkend="pg-stat-slru-view"/> for details.
+      </entry>
+     </row>
+
     </tbody>
    </tgroup>
   </table>
@@ -3254,6 +3261,76 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
    </tgroup>
   </table>
 
+  <para>
+   The <structname>pg_stat_slru</structname> view will contain
+   one row for each tracked SLRU cache, showing statistics about access
+   to cached pages.
+  </para>
+
+  <table id="pg-stat-slru-view" xreflabel="pg_stat_slru">
+   <title><structname>pg_stat_slru</structname> View</title>
+   <tgroup cols="3">
+    <thead>
+    <row>
+      <entry>Column</entry>
+      <entry>Type</entry>
+      <entry>Description</entry>
+     </row>
+    </thead>
+
+   <tbody>
+    <row>
+     <entry><structfield>name</structfield></entry>
+     <entry><type>name</type></entry>
+     <entry>name of the SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>blks_zeroed</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of blocks zeroed during initializations</entry>
+    </row>
+    <row>
+     <entry><structfield>blks_hit</structfield></entry>
+     <entry><type>biging</type></entry>
+     <entry>Number of times disk blocks were found already in the SLRU,
+      so that a read was not necessary (this only includes hits in the
+      SLRU, not the operating system's file system cache)
+     </entry>
+    </row>
+    <row>
+     <entry><structfield>blks_read</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of disk blocks read for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>blks_written</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of disk blocks written for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>blks_exists</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of blocks checked for existence for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>flushes</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of flushes of dirty data for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>truncates</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of truncates for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>stats_reset</structfield></entry>
+     <entry><type>timestamp with time zone</type></entry>
+     <entry>Time at which these statistics were last reset</entry>
+    </row>
+   </tbody>
+   </tgroup>
+  </table>
+
   <para>
    The <structname>pg_stat_user_functions</structname> view will contain
    one row for each tracked function, showing statistics about executions of
diff --git a/src/backend/access/transam/slru.c b/src/backend/access/transam/slru.c
index d5b7a08f73..f7160dd574 100644
--- a/src/backend/access/transam/slru.c
+++ b/src/backend/access/transam/slru.c
@@ -286,6 +286,9 @@ SimpleLruZeroPage(SlruCtl ctl, int pageno)
 	/* Assume this page is now the latest active page */
 	shared->latest_page_number = pageno;
 
+	/* update the stats counter of zeroed pages */
+	pgstat_slru_count_page_zeroed(ctl);
+
 	return slotno;
 }
 
@@ -403,6 +406,10 @@ SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 			}
 			/* Otherwise, it's ready to use */
 			SlruRecentlyUsed(shared, slotno);
+
+			/* update the stats counter of pages found in the SLRU */
+			pgstat_slru_count_page_hit(ctl);
+
 			return slotno;
 		}
 
@@ -444,6 +451,10 @@ SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 			SlruReportIOError(ctl, pageno, xid);
 
 		SlruRecentlyUsed(shared, slotno);
+
+		/* update the stats counter of pages not found in SLRU */
+		pgstat_slru_count_page_read(ctl);
+
 		return slotno;
 	}
 }
@@ -596,6 +607,9 @@ SimpleLruDoesPhysicalPageExist(SlruCtl ctl, int pageno)
 	bool		result;
 	off_t		endpos;
 
+	/* update the stats counter of checked pages */
+	pgstat_slru_count_page_exists(ctl);
+
 	SlruFileName(ctl, path, segno);
 
 	fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
@@ -730,6 +744,9 @@ SlruPhysicalWritePage(SlruCtl ctl, int pageno, int slotno, SlruFlush fdata)
 	char		path[MAXPGPATH];
 	int			fd = -1;
 
+	/* update the stats counter of written pages */
+	pgstat_slru_count_page_written(ctl);
+
 	/*
 	 * Honor the write-WAL-before-data rule, if appropriate, so that we do not
 	 * write out data before associated WAL records.  This is the same action
@@ -1125,6 +1142,9 @@ SimpleLruFlush(SlruCtl ctl, bool allow_redirtied)
 	int			i;
 	bool		ok;
 
+	/* update the stats counter of flushes */
+	pgstat_slru_count_flush(ctl);
+
 	/*
 	 * Find and write dirty pages
 	 */
@@ -1186,6 +1206,9 @@ SimpleLruTruncate(SlruCtl ctl, int cutoffPage)
 	SlruShared	shared = ctl->shared;
 	int			slotno;
 
+	/* update the stats counter of truncates */
+	pgstat_slru_count_truncate(ctl);
+
 	/*
 	 * The cutoff point is the start of the segment containing cutoffPage.
 	 */
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5a6dc61630..7dba85dd07 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -792,6 +792,19 @@ CREATE VIEW pg_stat_replication AS
         JOIN pg_stat_get_wal_senders() AS W ON (S.pid = W.pid)
         LEFT JOIN pg_authid AS U ON (S.usesysid = U.oid);
 
+CREATE VIEW pg_stat_slru AS
+    SELECT
+            s.name,
+            s.blks_zeroed,
+            s.blks_hit,
+            s.blks_read,
+            s.blks_written,
+            s.blks_exists,
+            s.flushes,
+            s.truncates,
+            s.stats_reset
+    FROM pg_stat_get_slru() s;
+
 CREATE VIEW pg_stat_wal_receiver AS
     SELECT
             s.pid,
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 4763c24be9..895efb4cd2 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -141,6 +141,25 @@ char	   *pgstat_stat_tmpname = NULL;
  */
 PgStat_MsgBgWriter BgWriterStats;
 
+/*
+ * SLRU statistics counters (unused in other processes) stored directly in
+ * stats structure so it can be sent without needing to copy things around.
+ * We assume this inits to zeroes.
+ *
+ * There's a separte entry for each SLRU we have. The "other" entry is used
+ * for all SLRUs without an explicit entry (e.g. SLRUs in extensions).
+ */
+static char *slru_names[] = {"async", "clog", "commit_timestamp",
+							  "multixact_offset", "multixact_member",
+							  "oldserxid", "pg_xact", "subtrans",
+							  "other" /* has to be last */};
+
+#define SLRU_NUM_ELEMENTS	(sizeof(slru_names) / sizeof(char *))
+static int		slru_index(char *name);
+
+/* entries in the same order as slru_names */
+PgStat_MsgSLRU SLRUStats[SLRU_NUM_ELEMENTS];
+
 /* ----------
  * Local data
  * ----------
@@ -255,6 +274,7 @@ static int	localNumBackends = 0;
  */
 static PgStat_ArchiverStats archiverStats;
 static PgStat_GlobalStats globalStats;
+static PgStat_SLRUStats slruStats[SLRU_NUM_ELEMENTS];
 
 /*
  * List of OIDs of databases we need to write out.  If an entry is InvalidOid,
@@ -297,6 +317,7 @@ static bool pgstat_db_requested(Oid databaseid);
 
 static void pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg);
 static void pgstat_send_funcstats(void);
+static void pgstat_send_slru(void);
 static HTAB *pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid);
 
 static PgStat_TableStatus *get_tabstat_entry(Oid rel_id, bool isshared);
@@ -324,6 +345,7 @@ static void pgstat_recv_vacuum(PgStat_MsgVacuum *msg, int len);
 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_slru(PgStat_MsgSLRU *msg, int len);
 static void pgstat_recv_funcstat(PgStat_MsgFuncstat *msg, int len);
 static void pgstat_recv_funcpurge(PgStat_MsgFuncpurge *msg, int len);
 static void pgstat_recv_recoveryconflict(PgStat_MsgRecoveryConflict *msg, int len);
@@ -907,6 +929,9 @@ pgstat_report_stat(bool force)
 
 	/* Now, send function statistics */
 	pgstat_send_funcstats();
+
+	/* Finally send SLRU statistics */
+	pgstat_send_slru();
 }
 
 /*
@@ -1337,6 +1362,8 @@ pgstat_reset_shared_counters(const char *target)
 		msg.m_resettarget = RESET_ARCHIVER;
 	else if (strcmp(target, "bgwriter") == 0)
 		msg.m_resettarget = RESET_BGWRITER;
+	else if (strcmp(target, "slru") == 0)
+		msg.m_resettarget = RESET_SLRU;
 	else
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@@ -2622,6 +2649,23 @@ pgstat_fetch_global(void)
 }
 
 
+/*
+ * ---------
+ * pgstat_fetch_slru() -
+ *
+ *	Support function for the SQL-callable pgstat* functions. Returns
+ *	a pointer to the slru statistics struct.
+ * ---------
+ */
+PgStat_SLRUStats *
+pgstat_fetch_slru(void)
+{
+	backend_read_statsfile();
+
+	return slruStats;
+}
+
+
 /* ------------------------------------------------------------
  * Functions for management of the shared-memory PgBackendStatus array
  * ------------------------------------------------------------
@@ -4325,6 +4369,46 @@ pgstat_send_bgwriter(void)
 	MemSet(&BgWriterStats, 0, sizeof(BgWriterStats));
 }
 
+/* ----------
+ * pgstat_send_slru() -
+ *
+ *		Send SLRU statistics to the collector
+ * ----------
+ */
+static void
+pgstat_send_slru(void)
+{
+	int		i;
+
+	/* We assume this initializes to zeroes */
+	static const PgStat_MsgSLRU all_zeroes;
+
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	{
+		/*
+		 * This function can be called even if nothing at all has happened. In
+		 * this case, avoid sending a completely empty message to the stats
+		 * collector.
+		 */
+		if (memcmp(&SLRUStats[i], &all_zeroes, sizeof(PgStat_MsgSLRU)) == 0)
+			continue;
+
+		/* set the SLRU type before each send */
+		SLRUStats[i].m_index = i;
+
+		/*
+		 * Prepare and send the message
+		 */
+		pgstat_setheader(&SLRUStats[i].m_hdr, PGSTAT_MTYPE_SLRU);
+		pgstat_send(&SLRUStats[i], sizeof(PgStat_MsgSLRU));
+
+		/*
+		 * Clear out the statistics buffer, so it can be re-used.
+		 */
+		MemSet(&SLRUStats[i], 0, sizeof(PgStat_MsgSLRU));
+	}
+}
+
 
 /* ----------
  * PgstatCollectorMain() -
@@ -4513,6 +4597,10 @@ PgstatCollectorMain(int argc, char *argv[])
 					pgstat_recv_bgwriter(&msg.msg_bgwriter, len);
 					break;
 
+				case PGSTAT_MTYPE_SLRU:
+					pgstat_recv_slru(&msg.msg_slru, len);
+					break;
+
 				case PGSTAT_MTYPE_FUNCSTAT:
 					pgstat_recv_funcstat(&msg.msg_funcstat, len);
 					break;
@@ -4781,6 +4869,12 @@ pgstat_write_statsfiles(bool permanent, bool allDbs)
 	rc = fwrite(&archiverStats, sizeof(archiverStats), 1, fpout);
 	(void) rc;					/* we'll check for error with ferror */
 
+	/*
+	 * Write SLRU stats struct
+	 */
+	rc = fwrite(slruStats, sizeof(slruStats), 1, fpout);
+	(void) rc;					/* we'll check for error with ferror */
+
 	/*
 	 * Walk through the database table.
 	 */
@@ -5016,6 +5110,7 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	int32		format_id;
 	bool		found;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	int			i;
 
 	/*
 	 * The tables will live in pgStatLocalContext.
@@ -5038,6 +5133,7 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	 */
 	memset(&globalStats, 0, sizeof(globalStats));
 	memset(&archiverStats, 0, sizeof(archiverStats));
+	memset(&slruStats, 0, sizeof(slruStats));
 
 	/*
 	 * Set the current timestamp (will be kept only in case we can't load an
@@ -5046,6 +5142,13 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	globalStats.stat_reset_timestamp = GetCurrentTimestamp();
 	archiverStats.stat_reset_timestamp = globalStats.stat_reset_timestamp;
 
+	/*
+	 * Set the same reset timestamp for all SLRU items (one
+	 * day we might allow resetting individual SLRUs).
+	 */
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+		slruStats[i].stat_reset_timestamp = globalStats.stat_reset_timestamp;
+
 	/*
 	 * Try to open the stats file. If it doesn't exist, the backends simply
 	 * return zero for anything and the collector simply starts from scratch
@@ -5108,6 +5211,17 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 		goto done;
 	}
 
+	/*
+	 * Read SLRU stats struct
+	 */
+	if (fread(slruStats, 1, sizeof(slruStats), fpin) != sizeof(slruStats))
+	{
+		ereport(pgStatRunningInCollector ? LOG : WARNING,
+				(errmsg("corrupted statistics file \"%s\"", statfile)));
+		memset(&slruStats, 0, sizeof(slruStats));
+		goto done;
+	}
+
 	/*
 	 * We found an existing collector stats file. Read it and put all the
 	 * hashtable entries into place.
@@ -5406,9 +5520,11 @@ pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
 	PgStat_StatDBEntry dbentry;
 	PgStat_GlobalStats myGlobalStats;
 	PgStat_ArchiverStats myArchiverStats;
+	PgStat_SLRUStats mySLRUStats;
 	FILE	   *fpin;
 	int32		format_id;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	int			i;
 
 	/*
 	 * Try to open the stats file.  As above, anything but ENOENT is worthy of
@@ -5460,6 +5576,21 @@ pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
 		return false;
 	}
 
+	/*
+	 * Read SLRU stats struct
+	 */
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	{
+		if (fread(&mySLRUStats, 1, sizeof(PgStat_SLRUStats),
+				  fpin) != sizeof(PgStat_SLRUStats))
+		{
+			ereport(pgStatRunningInCollector ? LOG : WARNING,
+					(errmsg("corrupted statistics file \"%s\"", statfile)));
+			FreeFile(fpin);
+			return false;
+		}
+	}
+
 	/* By default, we're going to return the timestamp of the global file. */
 	*ts = myGlobalStats.stats_timestamp;
 
@@ -6022,6 +6153,17 @@ pgstat_recv_resetsharedcounter(PgStat_MsgResetsharedcounter *msg, int len)
 		memset(&archiverStats, 0, sizeof(archiverStats));
 		archiverStats.stat_reset_timestamp = GetCurrentTimestamp();
 	}
+	else if (msg->m_resettarget == RESET_SLRU)
+	{
+		int			i;
+		TimestampTz	ts = GetCurrentTimestamp();
+
+		/* Reset the SLRU statistics for the cluster. */
+		memset(&slruStats, 0, sizeof(slruStats));
+
+		for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+			slruStats[i].stat_reset_timestamp = ts;
+	}
 
 	/*
 	 * Presumably the sender of this message validated the target, don't
@@ -6201,6 +6343,24 @@ pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len)
 	globalStats.buf_alloc += msg->m_buf_alloc;
 }
 
+/* ----------
+ * pgstat_recv_slru() -
+ *
+ *	Process a SLRU message.
+ * ----------
+ */
+static void
+pgstat_recv_slru(PgStat_MsgSLRU *msg, int len)
+{
+	slruStats[msg->m_index].blocks_zeroed += msg->m_blocks_zeroed;
+	slruStats[msg->m_index].blocks_hit += msg->m_blocks_hit;
+	slruStats[msg->m_index].blocks_read += msg->m_blocks_read;
+	slruStats[msg->m_index].blocks_written += msg->m_blocks_written;
+	slruStats[msg->m_index].blocks_exists += msg->m_blocks_exists;
+	slruStats[msg->m_index].flush += msg->m_flush;
+	slruStats[msg->m_index].truncate += msg->m_truncate;
+}
+
 /* ----------
  * pgstat_recv_recoveryconflict() -
  *
@@ -6455,3 +6615,81 @@ pgstat_clip_activity(const char *raw_activity)
 
 	return activity;
 }
+
+static int
+slru_index(char *name)
+{
+	int	i;
+
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	{
+		if (strcmp(slru_names[i], name) == 0)
+			return i;
+	}
+
+	/* return index of the last entry (which is the "other" one) */
+	return (SLRU_NUM_ELEMENTS - 1);
+}
+
+char *
+pgstat_slru_name(int idx)
+{
+	Assert(idx >= 0);
+
+	if (idx >= SLRU_NUM_ELEMENTS)
+		return NULL;
+
+	return slru_names[idx];
+}
+
+static PgStat_MsgSLRU *
+slru_entry(SlruCtl ctl)
+{
+	int		idx = slru_index(ctl->shared->lwlock_tranche_name);
+
+	Assert((idx >= 0) && (idx < SLRU_NUM_ELEMENTS));
+
+	return &SLRUStats[idx];
+}
+
+void
+pgstat_slru_count_page_zeroed(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_zeroed += 1;
+}
+
+void
+pgstat_slru_count_page_hit(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_hit += 1;
+}
+
+void
+pgstat_slru_count_page_exists(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_exists += 1;
+}
+
+void
+pgstat_slru_count_page_read(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_read += 1;
+}
+
+void
+pgstat_slru_count_page_written(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_written += 1;
+}
+
+void
+pgstat_slru_count_flush(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_flush += 1;
+}
+
+void
+pgstat_slru_count_truncate(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_truncate += 1;
+}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index cea01534a5..bc87c093d6 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1674,6 +1674,83 @@ pg_stat_get_buf_alloc(PG_FUNCTION_ARGS)
 	PG_RETURN_INT64(pgstat_fetch_global()->buf_alloc);
 }
 
+/*
+ * Returns statistics of SLRU caches.
+ */
+Datum
+pg_stat_get_slru(PG_FUNCTION_ARGS)
+{
+#define PG_STAT_GET_SLRU_COLS	9
+	ReturnSetInfo  *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	TupleDesc		tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext 	per_query_ctx;
+	MemoryContext 	oldcontext;
+	int				i;
+	PgStat_SLRUStats *stats;
+
+	/* 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);
+
+	/* request SLRU stats from the stat collector */
+	stats = pgstat_fetch_slru();
+
+	for (i = 0; ; i++)
+	{
+		/* for each row */
+		Datum		values[PG_STAT_GET_SLRU_COLS];
+		bool		nulls[PG_STAT_GET_SLRU_COLS];
+		PgStat_SLRUStats	stat = stats[i];
+		char	   *name;
+
+		name = pgstat_slru_name(i);
+
+		if (!name)
+			break;
+
+		MemSet(values, 0, sizeof(values));
+		MemSet(nulls, 0, sizeof(nulls));
+
+		values[0] = PointerGetDatum(cstring_to_text(name));
+		values[1] = Int64GetDatum(stat.blocks_zeroed);
+		values[2] = Int64GetDatum(stat.blocks_hit);
+		values[3] = Int64GetDatum(stat.blocks_read);
+		values[4] = Int64GetDatum(stat.blocks_written);
+		values[5] = Int64GetDatum(stat.blocks_exists);
+		values[6] = Int64GetDatum(stat.flush);
+		values[7] = Int64GetDatum(stat.truncate);
+		values[8] = Int64GetDatum(stat.stat_reset_timestamp);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	/* clean up and return the tuplestore */
+	tuplestore_donestoring(tupstore);
+
+	return (Datum) 0;
+}
+
 Datum
 pg_stat_get_xact_numscans(PG_FUNCTION_ARGS)
 {
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 87d25d4a4b..7e4184c7dc 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5431,6 +5431,16 @@
   proname => 'pg_stat_get_buf_alloc', provolatile => 's', proparallel => 'r',
   prorettype => 'int8', proargtypes => '', prosrc => 'pg_stat_get_buf_alloc' },
 
+{ oid => '8614',
+  descr => 'statistics: information about SLRU caches',
+  proname => 'pg_stat_get_slru', prorows => '100', proisstrict => 'f',
+  proretset => 't', provolatile => 's', proparallel => 'r',
+  prorettype => 'record', proargtypes => '',
+  proallargtypes => '{text,int8,int8,int8,int8,int8,int8,int8,timestamptz}',
+  proargmodes => '{o,o,o,o,o,o,o,o,o}',
+  proargnames => '{name,blks_zeroed,blks_hit,blks_read,blks_written,blks_exists,flushes,truncates,stats_reset}',
+  prosrc => 'pg_stat_get_slru' },
+
 { oid => '2978', descr => 'statistics: number of function calls',
   proname => 'pg_stat_get_function_calls', provolatile => 's',
   proparallel => 'r', prorettype => 'int8', proargtypes => 'oid',
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index a07012bf4b..5bf68a0a2a 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -11,6 +11,7 @@
 #ifndef PGSTAT_H
 #define PGSTAT_H
 
+#include "access/slru.h"
 #include "datatype/timestamp.h"
 #include "libpq/pqcomm.h"
 #include "miscadmin.h"
@@ -60,6 +61,7 @@ typedef enum StatMsgType
 	PGSTAT_MTYPE_ANALYZE,
 	PGSTAT_MTYPE_ARCHIVER,
 	PGSTAT_MTYPE_BGWRITER,
+	PGSTAT_MTYPE_SLRU,
 	PGSTAT_MTYPE_FUNCSTAT,
 	PGSTAT_MTYPE_FUNCPURGE,
 	PGSTAT_MTYPE_RECOVERYCONFLICT,
@@ -120,7 +122,8 @@ typedef struct PgStat_TableCounts
 typedef enum PgStat_Shared_Reset_Target
 {
 	RESET_ARCHIVER,
-	RESET_BGWRITER
+	RESET_BGWRITER,
+	RESET_SLRU
 } PgStat_Shared_Reset_Target;
 
 /* Possible object types for resetting single counters */
@@ -423,6 +426,23 @@ typedef struct PgStat_MsgBgWriter
 	PgStat_Counter m_checkpoint_sync_time;
 } PgStat_MsgBgWriter;
 
+/* ----------
+ * PgStat_MsgSLRU			Sent by the SLRU to update statistics.
+ * ----------
+ */
+typedef struct PgStat_MsgSLRU
+{
+	PgStat_MsgHdr m_hdr;
+	PgStat_Counter m_index;
+	PgStat_Counter m_blocks_zeroed;
+	PgStat_Counter m_blocks_hit;
+	PgStat_Counter m_blocks_read;
+	PgStat_Counter m_blocks_written;
+	PgStat_Counter m_blocks_exists;
+	PgStat_Counter m_flush;
+	PgStat_Counter m_truncate;
+} PgStat_MsgSLRU;
+
 /* ----------
  * PgStat_MsgRecoveryConflict	Sent by the backend upon recovery conflict
  * ----------
@@ -565,6 +585,7 @@ typedef union PgStat_Msg
 	PgStat_MsgAnalyze msg_analyze;
 	PgStat_MsgArchiver msg_archiver;
 	PgStat_MsgBgWriter msg_bgwriter;
+	PgStat_MsgSLRU msg_slru;
 	PgStat_MsgFuncstat msg_funcstat;
 	PgStat_MsgFuncpurge msg_funcpurge;
 	PgStat_MsgRecoveryConflict msg_recoveryconflict;
@@ -712,6 +733,21 @@ typedef struct PgStat_GlobalStats
 	TimestampTz stat_reset_timestamp;
 } PgStat_GlobalStats;
 
+/*
+ * SLRU statistics kept in the stats collector
+ */
+typedef struct PgStat_SLRUStats
+{
+	PgStat_Counter blocks_zeroed;
+	PgStat_Counter blocks_hit;
+	PgStat_Counter blocks_read;
+	PgStat_Counter blocks_written;
+	PgStat_Counter blocks_exists;
+	PgStat_Counter flush;
+	PgStat_Counter truncate;
+	TimestampTz stat_reset_timestamp;
+} PgStat_SLRUStats;
+
 
 /* ----------
  * Backend states
@@ -1209,6 +1245,11 @@ extern char *pgstat_stat_filename;
  */
 extern PgStat_MsgBgWriter BgWriterStats;
 
+/*
+ * SLRU statistics counters are updated directly by slru.
+ */
+extern PgStat_MsgSLRU SlruStats[];
+
 /*
  * Updated by pgstat_count_buffer_*_time macros
  */
@@ -1421,5 +1462,15 @@ extern PgStat_StatFuncEntry *pgstat_fetch_stat_funcentry(Oid funcid);
 extern int	pgstat_fetch_stat_numbackends(void);
 extern PgStat_ArchiverStats *pgstat_fetch_stat_archiver(void);
 extern PgStat_GlobalStats *pgstat_fetch_global(void);
+extern PgStat_SLRUStats *pgstat_fetch_slru(void);
+
+extern void pgstat_slru_count_page_zeroed(SlruCtl ctl);
+extern void pgstat_slru_count_page_hit(SlruCtl ctl);
+extern void pgstat_slru_count_page_read(SlruCtl ctl);
+extern void pgstat_slru_count_page_written(SlruCtl ctl);
+extern void pgstat_slru_count_page_exists(SlruCtl ctl);
+extern void pgstat_slru_count_flush(SlruCtl ctl);
+extern void pgstat_slru_count_truncate(SlruCtl ctl);
+extern char *pgstat_slru_name(int idx);
 
 #endif							/* PGSTAT_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index a2077bbad4..798364230e 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2006,6 +2006,16 @@ pg_stat_replication| SELECT s.pid,
    FROM ((pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, backend_type, ssl, sslversion, sslcipher, sslbits, sslcompression, ssl_client_dn, ssl_client_serial, ssl_issuer_dn, gss_auth, gss_princ, gss_enc, leader_pid)
      JOIN pg_stat_get_wal_senders() w(pid, state, sent_lsn, write_lsn, flush_lsn, replay_lsn, write_lag, flush_lag, replay_lag, sync_priority, sync_state, reply_time, spill_txns, spill_count, spill_bytes) ON ((s.pid = w.pid)))
      LEFT JOIN pg_authid u ON ((s.usesysid = u.oid)));
+pg_stat_slru| SELECT s.name,
+    s.blks_zeroed,
+    s.blks_hit,
+    s.blks_read,
+    s.blks_written,
+    s.blks_exists,
+    s.flushes,
+    s.truncates,
+    s.stats_reset
+   FROM pg_stat_get_slru() s(name, blks_zeroed, blks_hit, blks_read, blks_written, blks_exists, flushes, truncates, stats_reset);
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
     s.sslversion AS version,
-- 
2.21.1

#15Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Tomas Vondra (#14)
1 attachment(s)
Re: SLRU statistics

Hi,

here is a bit improved version of the patch - I've been annoyed by how
the resetting works (per-entry timestamp, but resetting all entries) so
I've added a new function pg_stat_reset_slru() that allows resetting
either all entries or just one entry (identified by name). So

SELECT pg_stat_reset_slru('clog');

resets just "clog" SLRU counters, while

SELECT pg_stat_reset_slru(NULL);

resets all entries.

I've also done a bit of benchmarking, to see if this has measurable
impact (in which case it might deserve a new GUC), and I think it's not
measurable. I've used a tiny unlogged table (single row).

CREATE UNLOGGED TABLE t (a int);
INSERT INTO t VALUES (1);

and then short pgbench runs with a single client, updatint the row. I've
been unable to measure any regression, it's all well within 1% so noise.
But perhaps there's some other benchmark that I should do?

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

0001-Collect-SLRU-statistics.patchtext/plain; charset=us-asciiDownload
From 1a065c21a9a791909cd1ca752db5aaf1f814fe37 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Thu, 26 Mar 2020 20:52:26 +0100
Subject: [PATCH] Collect SLRU statistics

Adds a new system view pg_stats_slru with stats about SLRU caches, and
a function pg_stat_reset_slru() to reset either all counters or just
counters for a single SLRU.

There is no SLRU registry this patch could use, so it simply uses the
SLRU name (which is also used for LWLock tranche name) as an identifier,
and a predefined list of SLRU names, and an extra "others" entry for
SLRUs without a dedicated entry. Presumably, the number of extensions
defining their own SLRU is very small.

Author: Tomas Vondra
Reviewed-by: Alvaro Herrera
Discussion: https://www.postgresql.org/message-id/flat/20200119143707.gyinppnigokesjok@development
---
 doc/src/sgml/monitoring.sgml         |  97 +++++++++
 src/backend/access/transam/slru.c    |  23 ++
 src/backend/catalog/system_views.sql |  14 ++
 src/backend/postmaster/pgstat.c      | 300 +++++++++++++++++++++++++++
 src/backend/utils/adt/pgstatfuncs.c  |  91 ++++++++
 src/include/catalog/pg_proc.dat      |  15 ++
 src/include/pgstat.h                 |  65 ++++++
 src/test/regress/expected/rules.out  |  10 +
 8 files changed, 615 insertions(+)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 270178d57e..7ba0dbee6a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -575,6 +575,13 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       yet included in <structname>pg_stat_user_functions</structname>).</entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_slru</structname><indexterm><primary>pg_stat_slru</primary></indexterm></entry>
+      <entry>One row per SLRU, showing statistics of operations. See
+       <xref linkend="pg-stat-slru-view"/> for details.
+      </entry>
+     </row>
+
     </tbody>
    </tgroup>
   </table>
@@ -3254,6 +3261,76 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
    </tgroup>
   </table>
 
+  <para>
+   The <structname>pg_stat_slru</structname> view will contain
+   one row for each tracked SLRU cache, showing statistics about access
+   to cached pages.
+  </para>
+
+  <table id="pg-stat-slru-view" xreflabel="pg_stat_slru">
+   <title><structname>pg_stat_slru</structname> View</title>
+   <tgroup cols="3">
+    <thead>
+    <row>
+      <entry>Column</entry>
+      <entry>Type</entry>
+      <entry>Description</entry>
+     </row>
+    </thead>
+
+   <tbody>
+    <row>
+     <entry><structfield>name</structfield></entry>
+     <entry><type>name</type></entry>
+     <entry>name of the SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>blks_zeroed</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of blocks zeroed during initializations</entry>
+    </row>
+    <row>
+     <entry><structfield>blks_hit</structfield></entry>
+     <entry><type>biging</type></entry>
+     <entry>Number of times disk blocks were found already in the SLRU,
+      so that a read was not necessary (this only includes hits in the
+      SLRU, not the operating system's file system cache)
+     </entry>
+    </row>
+    <row>
+     <entry><structfield>blks_read</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of disk blocks read for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>blks_written</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of disk blocks written for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>blks_exists</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of blocks checked for existence for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>flushes</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of flushes of dirty data for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>truncates</structfield></entry>
+     <entry><type>bigint</type></entry>
+     <entry>Number of truncates for this SLRU</entry>
+    </row>
+    <row>
+     <entry><structfield>stats_reset</structfield></entry>
+     <entry><type>timestamp with time zone</type></entry>
+     <entry>Time at which these statistics were last reset</entry>
+    </row>
+   </tbody>
+   </tgroup>
+  </table>
+
   <para>
    The <structname>pg_stat_user_functions</structname> view will contain
    one row for each tracked function, showing statistics about executions of
@@ -3378,6 +3455,26 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
        function can be granted to others)
       </entry>
      </row>
+
+     <row>
+      <entry><literal><function>pg_stat_reset_slru</function>(text)</literal><indexterm><primary>pg_stat_reset_slru</primary></indexterm></entry>
+      <entry><type>void</type></entry>
+      <entry>
+       Reset statistics either for a single SLRU or all SLRUs in the cluster
+       to zero (requires superuser privileges by default, but EXECUTE for this
+       function can be granted to others).
+       Calling <literal>pg_stat_reset_slru(NULL)</literal> will zero all the
+       counters shown in the <structname>pg_stat_slru</structname> view for
+       all SLRU caches.
+       Calling <literal>pg_stat_reset_slru(name)</literal> with names from a
+       predefined list (<literal>async</literal>, <literal>clog</literal>,
+       <literal>commit_timestamp</literal>, <literal>multixact_offset</literal>,
+       <literal>multixact_member</literal>, <literal>oldserxid</literal>,
+       <literal>pg_xact</literal>, <literal>subtrans</literal> and
+       <literal>other</literal>) resets counters for only that entry.
+       Names not included in this list are treated as <literal>other</literal>.
+      </entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/transam/slru.c b/src/backend/access/transam/slru.c
index d5b7a08f73..f7160dd574 100644
--- a/src/backend/access/transam/slru.c
+++ b/src/backend/access/transam/slru.c
@@ -286,6 +286,9 @@ SimpleLruZeroPage(SlruCtl ctl, int pageno)
 	/* Assume this page is now the latest active page */
 	shared->latest_page_number = pageno;
 
+	/* update the stats counter of zeroed pages */
+	pgstat_slru_count_page_zeroed(ctl);
+
 	return slotno;
 }
 
@@ -403,6 +406,10 @@ SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 			}
 			/* Otherwise, it's ready to use */
 			SlruRecentlyUsed(shared, slotno);
+
+			/* update the stats counter of pages found in the SLRU */
+			pgstat_slru_count_page_hit(ctl);
+
 			return slotno;
 		}
 
@@ -444,6 +451,10 @@ SimpleLruReadPage(SlruCtl ctl, int pageno, bool write_ok,
 			SlruReportIOError(ctl, pageno, xid);
 
 		SlruRecentlyUsed(shared, slotno);
+
+		/* update the stats counter of pages not found in SLRU */
+		pgstat_slru_count_page_read(ctl);
+
 		return slotno;
 	}
 }
@@ -596,6 +607,9 @@ SimpleLruDoesPhysicalPageExist(SlruCtl ctl, int pageno)
 	bool		result;
 	off_t		endpos;
 
+	/* update the stats counter of checked pages */
+	pgstat_slru_count_page_exists(ctl);
+
 	SlruFileName(ctl, path, segno);
 
 	fd = OpenTransientFile(path, O_RDONLY | PG_BINARY);
@@ -730,6 +744,9 @@ SlruPhysicalWritePage(SlruCtl ctl, int pageno, int slotno, SlruFlush fdata)
 	char		path[MAXPGPATH];
 	int			fd = -1;
 
+	/* update the stats counter of written pages */
+	pgstat_slru_count_page_written(ctl);
+
 	/*
 	 * Honor the write-WAL-before-data rule, if appropriate, so that we do not
 	 * write out data before associated WAL records.  This is the same action
@@ -1125,6 +1142,9 @@ SimpleLruFlush(SlruCtl ctl, bool allow_redirtied)
 	int			i;
 	bool		ok;
 
+	/* update the stats counter of flushes */
+	pgstat_slru_count_flush(ctl);
+
 	/*
 	 * Find and write dirty pages
 	 */
@@ -1186,6 +1206,9 @@ SimpleLruTruncate(SlruCtl ctl, int cutoffPage)
 	SlruShared	shared = ctl->shared;
 	int			slotno;
 
+	/* update the stats counter of truncates */
+	pgstat_slru_count_truncate(ctl);
+
 	/*
 	 * The cutoff point is the start of the segment containing cutoffPage.
 	 */
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5a6dc61630..09e226f34d 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -792,6 +792,19 @@ CREATE VIEW pg_stat_replication AS
         JOIN pg_stat_get_wal_senders() AS W ON (S.pid = W.pid)
         LEFT JOIN pg_authid AS U ON (S.usesysid = U.oid);
 
+CREATE VIEW pg_stat_slru AS
+    SELECT
+            s.name,
+            s.blks_zeroed,
+            s.blks_hit,
+            s.blks_read,
+            s.blks_written,
+            s.blks_exists,
+            s.flushes,
+            s.truncates,
+            s.stats_reset
+    FROM pg_stat_get_slru() s;
+
 CREATE VIEW pg_stat_wal_receiver AS
     SELECT
             s.pid,
@@ -1409,6 +1422,7 @@ REVOKE EXECUTE ON FUNCTION pg_promote(boolean, integer) FROM public;
 
 REVOKE EXECUTE ON FUNCTION pg_stat_reset() FROM public;
 REVOKE EXECUTE ON FUNCTION pg_stat_reset_shared(text) FROM public;
+REVOKE EXECUTE ON FUNCTION pg_stat_reset_slru(text) FROM public;
 REVOKE EXECUTE ON FUNCTION pg_stat_reset_single_table_counters(oid) FROM public;
 REVOKE EXECUTE ON FUNCTION pg_stat_reset_single_function_counters(oid) FROM public;
 
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 4763c24be9..ea0fdade58 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -141,6 +141,24 @@ char	   *pgstat_stat_tmpname = NULL;
  */
 PgStat_MsgBgWriter BgWriterStats;
 
+/*
+ * SLRU statistics counters (unused in other processes) stored directly in
+ * stats structure so it can be sent without needing to copy things around.
+ * We assume this inits to zeroes.
+ *
+ * There's a separte entry for each SLRU we have. The "other" entry is used
+ * for all SLRUs without an explicit entry (e.g. SLRUs in extensions).
+ */
+static char *slru_names[] = {"async", "clog", "commit_timestamp",
+							  "multixact_offset", "multixact_member",
+							  "oldserxid", "pg_xact", "subtrans",
+							  "other" /* has to be last */};
+
+#define SLRU_NUM_ELEMENTS	(sizeof(slru_names) / sizeof(char *))
+
+/* entries in the same order as slru_names */
+PgStat_MsgSLRU SLRUStats[SLRU_NUM_ELEMENTS];
+
 /* ----------
  * Local data
  * ----------
@@ -255,6 +273,7 @@ static int	localNumBackends = 0;
  */
 static PgStat_ArchiverStats archiverStats;
 static PgStat_GlobalStats globalStats;
+static PgStat_SLRUStats slruStats[SLRU_NUM_ELEMENTS];
 
 /*
  * List of OIDs of databases we need to write out.  If an entry is InvalidOid,
@@ -297,6 +316,7 @@ static bool pgstat_db_requested(Oid databaseid);
 
 static void pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg);
 static void pgstat_send_funcstats(void);
+static void pgstat_send_slru(void);
 static HTAB *pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid);
 
 static PgStat_TableStatus *get_tabstat_entry(Oid rel_id, bool isshared);
@@ -319,11 +339,13 @@ static void pgstat_recv_dropdb(PgStat_MsgDropdb *msg, int len);
 static void pgstat_recv_resetcounter(PgStat_MsgResetcounter *msg, int len);
 static void pgstat_recv_resetsharedcounter(PgStat_MsgResetsharedcounter *msg, int len);
 static void pgstat_recv_resetsinglecounter(PgStat_MsgResetsinglecounter *msg, int len);
+static void pgstat_recv_resetslrucounter(PgStat_MsgResetslrucounter *msg, int len);
 static void pgstat_recv_autovac(PgStat_MsgAutovacStart *msg, int len);
 static void pgstat_recv_vacuum(PgStat_MsgVacuum *msg, int len);
 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_slru(PgStat_MsgSLRU *msg, int len);
 static void pgstat_recv_funcstat(PgStat_MsgFuncstat *msg, int len);
 static void pgstat_recv_funcpurge(PgStat_MsgFuncpurge *msg, int len);
 static void pgstat_recv_recoveryconflict(PgStat_MsgRecoveryConflict *msg, int len);
@@ -907,6 +929,9 @@ pgstat_report_stat(bool force)
 
 	/* Now, send function statistics */
 	pgstat_send_funcstats();
+
+	/* Finally send SLRU statistics */
+	pgstat_send_slru();
 }
 
 /*
@@ -1372,6 +1397,30 @@ pgstat_reset_single_counter(Oid objoid, PgStat_Single_Reset_Type type)
 	pgstat_send(&msg, sizeof(msg));
 }
 
+/* ----------
+ * pgstat_reset_slru_counter() -
+ *
+ *	Tell the statistics collector to reset a single SLRU counter, or all
+ *	SLRU counters (when name is null).
+ *
+ *	Permission checking for this function is managed through the normal
+ *	GRANT system.
+ * ----------
+ */
+void
+pgstat_reset_slru_counter(const char *name)
+{
+	PgStat_MsgResetslrucounter msg;
+
+	if (pgStatSock == PGINVALID_SOCKET)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETSLRUCOUNTER);
+	msg.m_index = (name) ? pgstat_slru_index(name) : -1;
+
+	pgstat_send(&msg, sizeof(msg));
+}
+
 /* ----------
  * pgstat_report_autovac() -
  *
@@ -2622,6 +2671,23 @@ pgstat_fetch_global(void)
 }
 
 
+/*
+ * ---------
+ * pgstat_fetch_slru() -
+ *
+ *	Support function for the SQL-callable pgstat* functions. Returns
+ *	a pointer to the slru statistics struct.
+ * ---------
+ */
+PgStat_SLRUStats *
+pgstat_fetch_slru(void)
+{
+	backend_read_statsfile();
+
+	return slruStats;
+}
+
+
 /* ------------------------------------------------------------
  * Functions for management of the shared-memory PgBackendStatus array
  * ------------------------------------------------------------
@@ -4325,6 +4391,46 @@ pgstat_send_bgwriter(void)
 	MemSet(&BgWriterStats, 0, sizeof(BgWriterStats));
 }
 
+/* ----------
+ * pgstat_send_slru() -
+ *
+ *		Send SLRU statistics to the collector
+ * ----------
+ */
+static void
+pgstat_send_slru(void)
+{
+	int		i;
+
+	/* We assume this initializes to zeroes */
+	static const PgStat_MsgSLRU all_zeroes;
+
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	{
+		/*
+		 * This function can be called even if nothing at all has happened. In
+		 * this case, avoid sending a completely empty message to the stats
+		 * collector.
+		 */
+		if (memcmp(&SLRUStats[i], &all_zeroes, sizeof(PgStat_MsgSLRU)) == 0)
+			continue;
+
+		/* set the SLRU type before each send */
+		SLRUStats[i].m_index = i;
+
+		/*
+		 * Prepare and send the message
+		 */
+		pgstat_setheader(&SLRUStats[i].m_hdr, PGSTAT_MTYPE_SLRU);
+		pgstat_send(&SLRUStats[i], sizeof(PgStat_MsgSLRU));
+
+		/*
+		 * Clear out the statistics buffer, so it can be re-used.
+		 */
+		MemSet(&SLRUStats[i], 0, sizeof(PgStat_MsgSLRU));
+	}
+}
+
 
 /* ----------
  * PgstatCollectorMain() -
@@ -4493,6 +4599,11 @@ PgstatCollectorMain(int argc, char *argv[])
 												   len);
 					break;
 
+				case PGSTAT_MTYPE_RESETSLRUCOUNTER:
+					pgstat_recv_resetslrucounter(&msg.msg_resetslrucounter,
+												 len);
+					break;
+
 				case PGSTAT_MTYPE_AUTOVAC_START:
 					pgstat_recv_autovac(&msg.msg_autovacuum_start, len);
 					break;
@@ -4513,6 +4624,10 @@ PgstatCollectorMain(int argc, char *argv[])
 					pgstat_recv_bgwriter(&msg.msg_bgwriter, len);
 					break;
 
+				case PGSTAT_MTYPE_SLRU:
+					pgstat_recv_slru(&msg.msg_slru, len);
+					break;
+
 				case PGSTAT_MTYPE_FUNCSTAT:
 					pgstat_recv_funcstat(&msg.msg_funcstat, len);
 					break;
@@ -4781,6 +4896,12 @@ pgstat_write_statsfiles(bool permanent, bool allDbs)
 	rc = fwrite(&archiverStats, sizeof(archiverStats), 1, fpout);
 	(void) rc;					/* we'll check for error with ferror */
 
+	/*
+	 * Write SLRU stats struct
+	 */
+	rc = fwrite(slruStats, sizeof(slruStats), 1, fpout);
+	(void) rc;					/* we'll check for error with ferror */
+
 	/*
 	 * Walk through the database table.
 	 */
@@ -5016,6 +5137,7 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	int32		format_id;
 	bool		found;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	int			i;
 
 	/*
 	 * The tables will live in pgStatLocalContext.
@@ -5038,6 +5160,7 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	 */
 	memset(&globalStats, 0, sizeof(globalStats));
 	memset(&archiverStats, 0, sizeof(archiverStats));
+	memset(&slruStats, 0, sizeof(slruStats));
 
 	/*
 	 * Set the current timestamp (will be kept only in case we can't load an
@@ -5046,6 +5169,13 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 	globalStats.stat_reset_timestamp = GetCurrentTimestamp();
 	archiverStats.stat_reset_timestamp = globalStats.stat_reset_timestamp;
 
+	/*
+	 * Set the same reset timestamp for all SLRU items (one
+	 * day we might allow resetting individual SLRUs).
+	 */
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+		slruStats[i].stat_reset_timestamp = globalStats.stat_reset_timestamp;
+
 	/*
 	 * Try to open the stats file. If it doesn't exist, the backends simply
 	 * return zero for anything and the collector simply starts from scratch
@@ -5108,6 +5238,17 @@ pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
 		goto done;
 	}
 
+	/*
+	 * Read SLRU stats struct
+	 */
+	if (fread(slruStats, 1, sizeof(slruStats), fpin) != sizeof(slruStats))
+	{
+		ereport(pgStatRunningInCollector ? LOG : WARNING,
+				(errmsg("corrupted statistics file \"%s\"", statfile)));
+		memset(&slruStats, 0, sizeof(slruStats));
+		goto done;
+	}
+
 	/*
 	 * We found an existing collector stats file. Read it and put all the
 	 * hashtable entries into place.
@@ -5406,9 +5547,11 @@ pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
 	PgStat_StatDBEntry dbentry;
 	PgStat_GlobalStats myGlobalStats;
 	PgStat_ArchiverStats myArchiverStats;
+	PgStat_SLRUStats mySLRUStats;
 	FILE	   *fpin;
 	int32		format_id;
 	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	int			i;
 
 	/*
 	 * Try to open the stats file.  As above, anything but ENOENT is worthy of
@@ -5460,6 +5603,21 @@ pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
 		return false;
 	}
 
+	/*
+	 * Read SLRU stats struct
+	 */
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	{
+		if (fread(&mySLRUStats, 1, sizeof(PgStat_SLRUStats),
+				  fpin) != sizeof(PgStat_SLRUStats))
+		{
+			ereport(pgStatRunningInCollector ? LOG : WARNING,
+					(errmsg("corrupted statistics file \"%s\"", statfile)));
+			FreeFile(fpin);
+			return false;
+		}
+	}
+
 	/* By default, we're going to return the timestamp of the global file. */
 	*ts = myGlobalStats.stats_timestamp;
 
@@ -6057,6 +6215,32 @@ pgstat_recv_resetsinglecounter(PgStat_MsgResetsinglecounter *msg, int len)
 						   HASH_REMOVE, NULL);
 }
 
+/* ----------
+ * pgstat_recv_resetslrucounter() -
+ *
+ *	Reset some SLRU statistics of the cluster.
+ * ----------
+ */
+static void
+pgstat_recv_resetslrucounter(PgStat_MsgResetslrucounter *msg, int len)
+{
+	int			i;
+	TimestampTz	ts = GetCurrentTimestamp();
+
+	memset(&slruStats, 0, sizeof(slruStats));
+
+	elog(LOG, "msg->m_index = %d", msg->m_index);
+
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	{
+		if ((msg->m_index == -1) || (msg->m_index == i))
+		{
+			memset(&slruStats[i], 0, sizeof(slruStats[i]));
+			slruStats[i].stat_reset_timestamp = ts;
+		}
+	}
+}
+
 /* ----------
  * pgstat_recv_autovac() -
  *
@@ -6201,6 +6385,24 @@ pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len)
 	globalStats.buf_alloc += msg->m_buf_alloc;
 }
 
+/* ----------
+ * pgstat_recv_slru() -
+ *
+ *	Process a SLRU message.
+ * ----------
+ */
+static void
+pgstat_recv_slru(PgStat_MsgSLRU *msg, int len)
+{
+	slruStats[msg->m_index].blocks_zeroed += msg->m_blocks_zeroed;
+	slruStats[msg->m_index].blocks_hit += msg->m_blocks_hit;
+	slruStats[msg->m_index].blocks_read += msg->m_blocks_read;
+	slruStats[msg->m_index].blocks_written += msg->m_blocks_written;
+	slruStats[msg->m_index].blocks_exists += msg->m_blocks_exists;
+	slruStats[msg->m_index].flush += msg->m_flush;
+	slruStats[msg->m_index].truncate += msg->m_truncate;
+}
+
 /* ----------
  * pgstat_recv_recoveryconflict() -
  *
@@ -6455,3 +6657,101 @@ pgstat_clip_activity(const char *raw_activity)
 
 	return activity;
 }
+
+/*
+ * pgstat_slru_index
+ *
+ * Determine index of entry for a SLRU with a given name. If there's no exact
+ * match, returns index of the last "other" entry used for SLRUs defined in
+ * external proejcts.
+ */
+int
+pgstat_slru_index(const char *name)
+{
+	int	i;
+
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	{
+		if (strcmp(slru_names[i], name) == 0)
+			return i;
+	}
+
+	/* return index of the last entry (which is the "other" one) */
+	return (SLRU_NUM_ELEMENTS - 1);
+}
+
+/*
+ * pgstat_slru_name
+ *
+ * Returns SLRU name for an index. The index may be above SLRU_NUM_ELEMENTS,
+ * in which case this returns NULL. This allows writing code that does not
+ * know the number of entries in advance.
+ */
+char *
+pgstat_slru_name(int idx)
+{
+	Assert(idx >= 0);
+
+	if (idx >= SLRU_NUM_ELEMENTS)
+		return NULL;
+
+	return slru_names[idx];
+}
+
+/*
+ * slru_entry
+ *
+ * Returns pointer to entry with counters for given SLRU (based on the name
+ * stored in SlruCtl as lwlock tranche name).
+ */
+static PgStat_MsgSLRU *
+slru_entry(SlruCtl ctl)
+{
+	int		idx = pgstat_slru_index(ctl->shared->lwlock_tranche_name);
+
+	Assert((idx >= 0) && (idx < SLRU_NUM_ELEMENTS));
+
+	return &SLRUStats[idx];
+}
+
+void
+pgstat_slru_count_page_zeroed(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_zeroed += 1;
+}
+
+void
+pgstat_slru_count_page_hit(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_hit += 1;
+}
+
+void
+pgstat_slru_count_page_exists(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_exists += 1;
+}
+
+void
+pgstat_slru_count_page_read(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_read += 1;
+}
+
+void
+pgstat_slru_count_page_written(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_blocks_written += 1;
+}
+
+void
+pgstat_slru_count_flush(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_flush += 1;
+}
+
+void
+pgstat_slru_count_truncate(SlruCtl ctl)
+{
+	slru_entry(ctl)->m_truncate += 1;
+}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index cea01534a5..99b20de773 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1674,6 +1674,83 @@ pg_stat_get_buf_alloc(PG_FUNCTION_ARGS)
 	PG_RETURN_INT64(pgstat_fetch_global()->buf_alloc);
 }
 
+/*
+ * Returns statistics of SLRU caches.
+ */
+Datum
+pg_stat_get_slru(PG_FUNCTION_ARGS)
+{
+#define PG_STAT_GET_SLRU_COLS	9
+	ReturnSetInfo  *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
+	TupleDesc		tupdesc;
+	Tuplestorestate *tupstore;
+	MemoryContext 	per_query_ctx;
+	MemoryContext 	oldcontext;
+	int				i;
+	PgStat_SLRUStats *stats;
+
+	/* 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);
+
+	/* request SLRU stats from the stat collector */
+	stats = pgstat_fetch_slru();
+
+	for (i = 0; ; i++)
+	{
+		/* for each row */
+		Datum		values[PG_STAT_GET_SLRU_COLS];
+		bool		nulls[PG_STAT_GET_SLRU_COLS];
+		PgStat_SLRUStats	stat = stats[i];
+		char	   *name;
+
+		name = pgstat_slru_name(i);
+
+		if (!name)
+			break;
+
+		MemSet(values, 0, sizeof(values));
+		MemSet(nulls, 0, sizeof(nulls));
+
+		values[0] = PointerGetDatum(cstring_to_text(name));
+		values[1] = Int64GetDatum(stat.blocks_zeroed);
+		values[2] = Int64GetDatum(stat.blocks_hit);
+		values[3] = Int64GetDatum(stat.blocks_read);
+		values[4] = Int64GetDatum(stat.blocks_written);
+		values[5] = Int64GetDatum(stat.blocks_exists);
+		values[6] = Int64GetDatum(stat.flush);
+		values[7] = Int64GetDatum(stat.truncate);
+		values[8] = Int64GetDatum(stat.stat_reset_timestamp);
+
+		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
+	}
+
+	/* clean up and return the tuplestore */
+	tuplestore_donestoring(tupstore);
+
+	return (Datum) 0;
+}
+
 Datum
 pg_stat_get_xact_numscans(PG_FUNCTION_ARGS)
 {
@@ -1919,6 +1996,20 @@ pg_stat_reset_single_function_counters(PG_FUNCTION_ARGS)
 	PG_RETURN_VOID();
 }
 
+/* Reset SLRU counters (a specific one or all of them). */
+Datum
+pg_stat_reset_slru(PG_FUNCTION_ARGS)
+{
+	char	   *target = NULL;
+
+	if (!PG_ARGISNULL(0))
+		target = text_to_cstring(PG_GETARG_TEXT_PP(0));
+
+	pgstat_reset_slru_counter(target);
+
+	PG_RETURN_VOID();
+}
+
 Datum
 pg_stat_get_archiver(PG_FUNCTION_ARGS)
 {
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 87d25d4a4b..96a93d8570 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5431,6 +5431,16 @@
   proname => 'pg_stat_get_buf_alloc', provolatile => 's', proparallel => 'r',
   prorettype => 'int8', proargtypes => '', prosrc => 'pg_stat_get_buf_alloc' },
 
+{ oid => '8614',
+  descr => 'statistics: information about SLRU caches',
+  proname => 'pg_stat_get_slru', prorows => '100', proisstrict => 'f',
+  proretset => 't', provolatile => 's', proparallel => 'r',
+  prorettype => 'record', proargtypes => '',
+  proallargtypes => '{text,int8,int8,int8,int8,int8,int8,int8,timestamptz}',
+  proargmodes => '{o,o,o,o,o,o,o,o,o}',
+  proargnames => '{name,blks_zeroed,blks_hit,blks_read,blks_written,blks_exists,flushes,truncates,stats_reset}',
+  prosrc => 'pg_stat_get_slru' },
+
 { oid => '2978', descr => 'statistics: number of function calls',
   proname => 'pg_stat_get_function_calls', provolatile => 's',
   proparallel => 'r', prorettype => 'int8', proargtypes => 'oid',
@@ -5535,6 +5545,11 @@
   proname => 'pg_stat_reset_single_function_counters', provolatile => 'v',
   prorettype => 'void', proargtypes => 'oid',
   prosrc => 'pg_stat_reset_single_function_counters' },
+{ oid => '4179',
+  descr => 'statistics: reset collected statistics for a single SLRU',
+  proname => 'pg_stat_reset_slru', provolatile => 'v', proisstrict => 'f',
+  prorettype => 'void', proargtypes => 'text',
+  prosrc => 'pg_stat_reset_slru' },
 
 { oid => '3163', descr => 'current trigger depth',
   proname => 'pg_trigger_depth', provolatile => 's', proparallel => 'r',
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index a07012bf4b..0fac192ea1 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -11,6 +11,7 @@
 #ifndef PGSTAT_H
 #define PGSTAT_H
 
+#include "access/slru.h"
 #include "datatype/timestamp.h"
 #include "libpq/pqcomm.h"
 #include "miscadmin.h"
@@ -55,11 +56,13 @@ typedef enum StatMsgType
 	PGSTAT_MTYPE_RESETCOUNTER,
 	PGSTAT_MTYPE_RESETSHAREDCOUNTER,
 	PGSTAT_MTYPE_RESETSINGLECOUNTER,
+	PGSTAT_MTYPE_RESETSLRUCOUNTER,
 	PGSTAT_MTYPE_AUTOVAC_START,
 	PGSTAT_MTYPE_VACUUM,
 	PGSTAT_MTYPE_ANALYZE,
 	PGSTAT_MTYPE_ARCHIVER,
 	PGSTAT_MTYPE_BGWRITER,
+	PGSTAT_MTYPE_SLRU,
 	PGSTAT_MTYPE_FUNCSTAT,
 	PGSTAT_MTYPE_FUNCPURGE,
 	PGSTAT_MTYPE_RECOVERYCONFLICT,
@@ -343,6 +346,17 @@ typedef struct PgStat_MsgResetsinglecounter
 	Oid			m_objectid;
 } PgStat_MsgResetsinglecounter;
 
+/* ----------
+ * PgStat_MsgResetslrucounter Sent by the backend to tell the collector
+ *								to reset a SLRU counter
+ * ----------
+ */
+typedef struct PgStat_MsgResetslrucounter
+{
+	PgStat_MsgHdr m_hdr;
+	int			m_index;
+} PgStat_MsgResetslrucounter;
+
 /* ----------
  * PgStat_MsgAutovacStart		Sent by the autovacuum daemon to signal
  *								that a database is going to be processed
@@ -423,6 +437,23 @@ typedef struct PgStat_MsgBgWriter
 	PgStat_Counter m_checkpoint_sync_time;
 } PgStat_MsgBgWriter;
 
+/* ----------
+ * PgStat_MsgSLRU			Sent by the SLRU to update statistics.
+ * ----------
+ */
+typedef struct PgStat_MsgSLRU
+{
+	PgStat_MsgHdr m_hdr;
+	PgStat_Counter m_index;
+	PgStat_Counter m_blocks_zeroed;
+	PgStat_Counter m_blocks_hit;
+	PgStat_Counter m_blocks_read;
+	PgStat_Counter m_blocks_written;
+	PgStat_Counter m_blocks_exists;
+	PgStat_Counter m_flush;
+	PgStat_Counter m_truncate;
+} PgStat_MsgSLRU;
+
 /* ----------
  * PgStat_MsgRecoveryConflict	Sent by the backend upon recovery conflict
  * ----------
@@ -560,11 +591,13 @@ typedef union PgStat_Msg
 	PgStat_MsgResetcounter msg_resetcounter;
 	PgStat_MsgResetsharedcounter msg_resetsharedcounter;
 	PgStat_MsgResetsinglecounter msg_resetsinglecounter;
+	PgStat_MsgResetslrucounter msg_resetslrucounter;
 	PgStat_MsgAutovacStart msg_autovacuum_start;
 	PgStat_MsgVacuum msg_vacuum;
 	PgStat_MsgAnalyze msg_analyze;
 	PgStat_MsgArchiver msg_archiver;
 	PgStat_MsgBgWriter msg_bgwriter;
+	PgStat_MsgSLRU msg_slru;
 	PgStat_MsgFuncstat msg_funcstat;
 	PgStat_MsgFuncpurge msg_funcpurge;
 	PgStat_MsgRecoveryConflict msg_recoveryconflict;
@@ -712,6 +745,21 @@ typedef struct PgStat_GlobalStats
 	TimestampTz stat_reset_timestamp;
 } PgStat_GlobalStats;
 
+/*
+ * SLRU statistics kept in the stats collector
+ */
+typedef struct PgStat_SLRUStats
+{
+	PgStat_Counter blocks_zeroed;
+	PgStat_Counter blocks_hit;
+	PgStat_Counter blocks_read;
+	PgStat_Counter blocks_written;
+	PgStat_Counter blocks_exists;
+	PgStat_Counter flush;
+	PgStat_Counter truncate;
+	TimestampTz stat_reset_timestamp;
+} PgStat_SLRUStats;
+
 
 /* ----------
  * Backend states
@@ -1209,6 +1257,11 @@ extern char *pgstat_stat_filename;
  */
 extern PgStat_MsgBgWriter BgWriterStats;
 
+/*
+ * SLRU statistics counters are updated directly by slru.
+ */
+extern PgStat_MsgSLRU SlruStats[];
+
 /*
  * Updated by pgstat_count_buffer_*_time macros
  */
@@ -1246,6 +1299,7 @@ extern void pgstat_clear_snapshot(void);
 extern void pgstat_reset_counters(void);
 extern void pgstat_reset_shared_counters(const char *);
 extern void pgstat_reset_single_counter(Oid objectid, PgStat_Single_Reset_Type type);
+extern void pgstat_reset_slru_counter(const char *);
 
 extern void pgstat_report_autovac(Oid dboid);
 extern void pgstat_report_vacuum(Oid tableoid, bool shared,
@@ -1421,5 +1475,16 @@ extern PgStat_StatFuncEntry *pgstat_fetch_stat_funcentry(Oid funcid);
 extern int	pgstat_fetch_stat_numbackends(void);
 extern PgStat_ArchiverStats *pgstat_fetch_stat_archiver(void);
 extern PgStat_GlobalStats *pgstat_fetch_global(void);
+extern PgStat_SLRUStats *pgstat_fetch_slru(void);
+
+extern void pgstat_slru_count_page_zeroed(SlruCtl ctl);
+extern void pgstat_slru_count_page_hit(SlruCtl ctl);
+extern void pgstat_slru_count_page_read(SlruCtl ctl);
+extern void pgstat_slru_count_page_written(SlruCtl ctl);
+extern void pgstat_slru_count_page_exists(SlruCtl ctl);
+extern void pgstat_slru_count_flush(SlruCtl ctl);
+extern void pgstat_slru_count_truncate(SlruCtl ctl);
+extern char *pgstat_slru_name(int idx);
+extern int pgstat_slru_index(const char *name);
 
 #endif							/* PGSTAT_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index a2077bbad4..798364230e 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2006,6 +2006,16 @@ pg_stat_replication| SELECT s.pid,
    FROM ((pg_stat_get_activity(NULL::integer) s(datid, pid, usesysid, application_name, state, query, wait_event_type, wait_event, xact_start, query_start, backend_start, state_change, client_addr, client_hostname, client_port, backend_xid, backend_xmin, backend_type, ssl, sslversion, sslcipher, sslbits, sslcompression, ssl_client_dn, ssl_client_serial, ssl_issuer_dn, gss_auth, gss_princ, gss_enc, leader_pid)
      JOIN pg_stat_get_wal_senders() w(pid, state, sent_lsn, write_lsn, flush_lsn, replay_lsn, write_lag, flush_lag, replay_lag, sync_priority, sync_state, reply_time, spill_txns, spill_count, spill_bytes) ON ((s.pid = w.pid)))
      LEFT JOIN pg_authid u ON ((s.usesysid = u.oid)));
+pg_stat_slru| SELECT s.name,
+    s.blks_zeroed,
+    s.blks_hit,
+    s.blks_read,
+    s.blks_written,
+    s.blks_exists,
+    s.flushes,
+    s.truncates,
+    s.stats_reset
+   FROM pg_stat_get_slru() s(name, blks_zeroed, blks_hit, blks_read, blks_written, blks_exists, flushes, truncates, stats_reset);
 pg_stat_ssl| SELECT s.pid,
     s.ssl,
     s.sslversion AS version,
-- 
2.21.1

#16Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Tomas Vondra (#15)
Re: SLRU statistics

Hi,

I've pushed this after some minor cleanup and improvements.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

In reply to: Tomas Vondra (#16)
1 attachment(s)
RE: SLRU statistics

Hi,

Thank you for developing great features.
The attached patch is a small fix to the committed documentation for the data type name of blks_hit column.

Best regards,
Noriyoshi Shinoda

-----Original Message-----
From: Tomas Vondra [mailto:tomas.vondra@2ndquadrant.com]
Sent: Thursday, April 2, 2020 9:42 AM
To: Alvaro Herrera <alvherre@2ndquadrant.com>
Cc: Masahiko Sawada <masahiko.sawada@2ndquadrant.com>; tsunakawa.takay@fujitsu.com; pgsql-hackers@postgresql.org
Subject: Re: SLRU statistics

Hi,

I've pushed this after some minor cleanup and improvements.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

monitoring_pg_stat_slru.diffapplication/octet-stream; name=monitoring_pg_stat_slru.diffDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 28ceb04..0ebadf0 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -3296,7 +3296,7 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
     </row>
     <row>
      <entry><structfield>blks_hit</structfield></entry>
-     <entry><type>biging</type></entry>
+     <entry><type>bigint</type></entry>
      <entry>Number of times disk blocks were found already in the SLRU,
       so that a read was not necessary (this only includes hits in the
       SLRU, not the operating system's file system cache)
#18Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Shinoda, Noriyoshi (PN Japan A&PS Delivery) (#17)
Re: SLRU statistics

On Thu, Apr 02, 2020 at 02:04:10AM +0000, Shinoda, Noriyoshi (PN Japan A&PS Delivery) wrote:

Hi,

Thank you for developing great features.
The attached patch is a small fix to the committed documentation for the data type name of blks_hit column.

Thank you for the patch, pushed.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#19Kuntal Ghosh
kuntalghosh.2007@gmail.com
In reply to: Tomas Vondra (#18)
1 attachment(s)
Re: SLRU statistics

Hello Tomas,

On Thu, Apr 2, 2020 at 5:59 PM Tomas Vondra
<tomas.vondra@2ndquadrant.com> wrote:

Thank you for the patch, pushed.

In SimpleLruReadPage_ReadOnly, we first try to find the SLRU page in
shared buffer under shared lock, then conditionally visit
SimpleLruReadPage if reading is necessary. IMHO, we should update
hit_count if we can find the buffer in SimpleLruReadPage_ReadOnly
directly. Am I missing something?

Attached a patch for the same.

--
Thanks & Regards,
Kuntal Ghosh
EnterpriseDB: http://www.enterprisedb.com

Attachments:

v1-0001-Update-stats-in-SimpleLruReadPage_ReadOnly.patchapplication/octet-stream; name=v1-0001-Update-stats-in-SimpleLruReadPage_ReadOnly.patchDownload
From a11a756546da1b17985d75bbe6bbb957787cc48d Mon Sep 17 00:00:00 2001
From: Kuntal Ghosh <kuntal.ghosh@enterprisedb.com>
Date: Tue, 7 Apr 2020 16:49:30 +0530
Subject: [PATCH v1] Update stats in SimpleLruReadPage_ReadOnly

---
 src/backend/access/transam/slru.c | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/src/backend/access/transam/slru.c b/src/backend/access/transam/slru.c
index de3c94105d..b2316af779 100644
--- a/src/backend/access/transam/slru.c
+++ b/src/backend/access/transam/slru.c
@@ -491,6 +491,10 @@ SimpleLruReadPage_ReadOnly(SlruCtl ctl, int pageno, TransactionId xid)
 		{
 			/* See comments for SlruRecentlyUsed macro */
 			SlruRecentlyUsed(shared, slotno);
+
+			/* update the stats counter of pages found in the SLRU */
+			pgstat_count_slru_page_hit(ctl);
+
 			return slotno;
 		}
 	}
-- 
2.17.1

#20Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Kuntal Ghosh (#19)
Re: SLRU statistics

On Tue, Apr 07, 2020 at 05:01:37PM +0530, Kuntal Ghosh wrote:

Hello Tomas,

On Thu, Apr 2, 2020 at 5:59 PM Tomas Vondra
<tomas.vondra@2ndquadrant.com> wrote:

Thank you for the patch, pushed.

In SimpleLruReadPage_ReadOnly, we first try to find the SLRU page in
shared buffer under shared lock, then conditionally visit
SimpleLruReadPage if reading is necessary. IMHO, we should update
hit_count if we can find the buffer in SimpleLruReadPage_ReadOnly
directly. Am I missing something?

Attached a patch for the same.

Yes, I think that's correct - without this we fail to account for
(possibly) a quite significant number of hits. Thanks for the report,
I'll get this pushed later today.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#21Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Tomas Vondra (#16)
1 attachment(s)
Re: SLRU statistics

On 2020/04/02 9:41, Tomas Vondra wrote:

Hi,

I've pushed this after some minor cleanup and improvements.

+static char *slru_names[] = {"async", "clog", "commit_timestamp",
+							  "multixact_offset", "multixact_member",
+							  "oldserxid", "pg_xact", "subtrans",
+							  "other" /* has to be last */};

When I tried pg_stat_slru, I found that it returns a row for "pg_xact".
But since there is no "pg_xact" slru ("clog" slru exists instead),
"pg_xact" should be removed? Patch attached.

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

Attachments:

remove_pg_xact_from_pg_stat_slru.patchtext/plain; charset=UTF-8; name=remove_pg_xact_from_pg_stat_slru.patch; x-mac-creator=0; x-mac-type=0Download
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 6562cc400b..ba6d8d2123 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -3483,7 +3483,7 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
        predefined list (<literal>async</literal>, <literal>clog</literal>,
        <literal>commit_timestamp</literal>, <literal>multixact_offset</literal>,
        <literal>multixact_member</literal>, <literal>oldserxid</literal>,
-       <literal>pg_xact</literal>, <literal>subtrans</literal> and
+       <literal>subtrans</literal> and
        <literal>other</literal>) resets counters for only that entry.
        Names not included in this list are treated as <literal>other</literal>.
       </entry>
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 50eea2e8a8..2ba3858d31 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -152,7 +152,7 @@ PgStat_MsgBgWriter BgWriterStats;
  */
 static char *slru_names[] = {"async", "clog", "commit_timestamp",
 							  "multixact_offset", "multixact_member",
-							  "oldserxid", "pg_xact", "subtrans",
+							  "oldserxid", "subtrans",
 							  "other" /* has to be last */};
 
 /* number of elemenents of slru_name array */
#22Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Fujii Masao (#21)
Re: SLRU statistics

On Fri, May 01, 2020 at 03:02:59AM +0900, Fujii Masao wrote:

On 2020/04/02 9:41, Tomas Vondra wrote:

Hi,

I've pushed this after some minor cleanup and improvements.

+static char *slru_names[] = {"async", "clog", "commit_timestamp",
+							  "multixact_offset", "multixact_member",
+							  "oldserxid", "pg_xact", "subtrans",
+							  "other" /* has to be last */};

When I tried pg_stat_slru, I found that it returns a row for "pg_xact".
But since there is no "pg_xact" slru ("clog" slru exists instead),
"pg_xact" should be removed? Patch attached.

Yeah, I think I got confused and accidentally added both "clog" and
"pg_xact". I'll get "pg_xact" removed.

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 6562cc400b..ba6d8d2123 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -3483,7 +3483,7 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
predefined list (<literal>async</literal>, <literal>clog</literal>,
<literal>commit_timestamp</literal>, <literal>multixact_offset</literal>,
<literal>multixact_member</literal>, <literal>oldserxid</literal>,
-       <literal>pg_xact</literal>, <literal>subtrans</literal> and
+       <literal>subtrans</literal> and
<literal>other</literal>) resets counters for only that entry.
Names not included in this list are treated as <literal>other</literal>.
</entry>
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 50eea2e8a8..2ba3858d31 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -152,7 +152,7 @@ PgStat_MsgBgWriter BgWriterStats;
*/
static char *slru_names[] = {"async", "clog", "commit_timestamp",
"multixact_offset", "multixact_member",
-							  "oldserxid", "pg_xact", "subtrans",
+							  "oldserxid", "subtrans",
"other" /* has to be last */};

/* number of elemenents of slru_name array */

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#23Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Tomas Vondra (#22)
Re: SLRU statistics

On 2020/05/01 3:19, Tomas Vondra wrote:

On Fri, May 01, 2020 at 03:02:59AM +0900, Fujii Masao wrote:

On 2020/04/02 9:41, Tomas Vondra wrote:

Hi,

I've pushed this after some minor cleanup and improvements.

+static char *slru_names[] = {"async", "clog", "commit_timestamp",
+                              "multixact_offset", "multixact_member",
+                              "oldserxid", "pg_xact", "subtrans",
+                              "other" /* has to be last */};

When I tried pg_stat_slru, I found that it returns a row for "pg_xact".
But since there is no "pg_xact" slru ("clog" slru exists instead),
"pg_xact" should be removed? Patch attached.

Yeah, I think I got confused and accidentally added both "clog" and
"pg_xact". I'll get "pg_xact" removed.

Thanks!

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

#24Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Fujii Masao (#23)
Re: SLRU statistics

On Fri, May 01, 2020 at 11:49:51AM +0900, Fujii Masao wrote:

On 2020/05/01 3:19, Tomas Vondra wrote:

On Fri, May 01, 2020 at 03:02:59AM +0900, Fujii Masao wrote:

On 2020/04/02 9:41, Tomas Vondra wrote:

Hi,

I've pushed this after some minor cleanup and improvements.

+static char *slru_names[] = {"async", "clog", "commit_timestamp",
+����������������������������� "multixact_offset", "multixact_member",
+����������������������������� "oldserxid", "pg_xact", "subtrans",
+����������������������������� "other" /* has to be last */};

When I tried pg_stat_slru, I found that it returns a row for "pg_xact".
But since there is no "pg_xact" slru ("clog" slru exists instead),
"pg_xact" should be removed? Patch attached.

Yeah, I think I got confused and accidentally added both "clog" and
"pg_xact". I'll get "pg_xact" removed.

Thanks!

OK, pushed. Thanks!

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#25Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Tomas Vondra (#24)
Re: SLRU statistics

On 2020/05/02 9:08, Tomas Vondra wrote:

On Fri, May 01, 2020 at 11:49:51AM +0900, Fujii Masao wrote:

On 2020/05/01 3:19, Tomas Vondra wrote:

On Fri, May 01, 2020 at 03:02:59AM +0900, Fujii Masao wrote:

On 2020/04/02 9:41, Tomas Vondra wrote:

Hi,

I've pushed this after some minor cleanup and improvements.

+static char *slru_names[] = {"async", "clog", "commit_timestamp",
+                              "multixact_offset", "multixact_member",
+                              "oldserxid", "pg_xact", "subtrans",
+                              "other" /* has to be last */};

When I tried pg_stat_slru, I found that it returns a row for "pg_xact".
But since there is no "pg_xact" slru ("clog" slru exists instead),
"pg_xact" should be removed? Patch attached.

Yeah, I think I got confused and accidentally added both "clog" and
"pg_xact". I'll get "pg_xact" removed.

Thanks!

OK, pushed. Thanks!

Thanks a lot!

But, like the patch that I attached in the previous email does,
"pg_xact" should be removed from the description of pg_stat_reset_slru()
in monitoring.sgml.

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

Yes.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

Checkpointer sends off activity statistics to the stats collector in
two places, by calling pgstat_send_bgwriter(). What about calling
pgstat_send_slru() just after pgstat_send_bgwriter()?

In previous email, I mentioned checkpointer just as an example.
So probably we need to investigate what process should send slru stats,
other than checkpointer. I guess that at least autovacuum worker,
logical replication walsender and parallel query worker (maybe this has
been already covered by parallel query some mechanisms. Sorry I've
not checked that) would need to send its slru stats.

Atsushi-san reported another issue in pg_stat_slru.
You're planning to work on that?
/messages/by-id/CACZ0uYFe16pjZxQYaTn53mspyM7dgMPYL3DJLjjPw69GMCC2Ow@mail.gmail.com

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

#26Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Fujii Masao (#25)
Re: SLRU statistics

On Sat, May 02, 2020 at 03:56:07PM +0900, Fujii Masao wrote:

On 2020/05/02 9:08, Tomas Vondra wrote:

On Fri, May 01, 2020 at 11:49:51AM +0900, Fujii Masao wrote:

On 2020/05/01 3:19, Tomas Vondra wrote:

On Fri, May 01, 2020 at 03:02:59AM +0900, Fujii Masao wrote:

On 2020/04/02 9:41, Tomas Vondra wrote:

Hi,

I've pushed this after some minor cleanup and improvements.

+static char *slru_names[] = {"async", "clog", "commit_timestamp",
+����������������������������� "multixact_offset", "multixact_member",
+����������������������������� "oldserxid", "pg_xact", "subtrans",
+����������������������������� "other" /* has to be last */};

When I tried pg_stat_slru, I found that it returns a row for "pg_xact".
But since there is no "pg_xact" slru ("clog" slru exists instead),
"pg_xact" should be removed? Patch attached.

Yeah, I think I got confused and accidentally added both "clog" and
"pg_xact". I'll get "pg_xact" removed.

Thanks!

OK, pushed. Thanks!

Thanks a lot!

But, like the patch that I attached in the previous email does,
"pg_xact" should be removed from the description of pg_stat_reset_slru()
in monitoring.sgml.

Whooops. My bad, will fix.

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

Yes.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

Checkpointer sends off activity statistics to the stats collector in
two places, by calling pgstat_send_bgwriter(). What about calling
pgstat_send_slru() just after pgstat_send_bgwriter()?

Yep, that's what I proposed.

In previous email, I mentioned checkpointer just as an example.
So probably we need to investigate what process should send slru stats,
other than checkpointer. I guess that at least autovacuum worker,
logical replication walsender and parallel query worker (maybe this has
been already covered by parallel query some mechanisms. Sorry I've
not checked that) would need to send its slru stats.

Probably. Do you have any other process type in mind?

Atsushi-san reported another issue in pg_stat_slru.
You're planning to work on that?
/messages/by-id/CACZ0uYFe16pjZxQYaTn53mspyM7dgMPYL3DJLjjPw69GMCC2Ow@mail.gmail.com

Yes, I'll investigate.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#27Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Tomas Vondra (#26)
Re: SLRU statistics

On Sat, May 02, 2020 at 12:55:00PM +0200, Tomas Vondra wrote:

On Sat, May 02, 2020 at 03:56:07PM +0900, Fujii Masao wrote:

...

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

Yes.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

Checkpointer sends off activity statistics to the stats collector in
two places, by calling pgstat_send_bgwriter(). What about calling
pgstat_send_slru() just after pgstat_send_bgwriter()?

Yep, that's what I proposed.

In previous email, I mentioned checkpointer just as an example.
So probably we need to investigate what process should send slru stats,
other than checkpointer. I guess that at least autovacuum worker,
logical replication walsender and parallel query worker (maybe this has
been already covered by parallel query some mechanisms. Sorry I've
not checked that) would need to send its slru stats.

Probably. Do you have any other process type in mind?

I've looked at places calling pgstat_send_* functions, and I found
thsese places:

src/backend/postmaster/bgwriter.c

- AFAIK it merely writes out dirty shared buffers, so likely irrelevant.

src/backend/postmaster/checkpointer.c

- This is what we're already discussing here.

src/backend/postmaster/pgarch.c

- Seems irrelevant.

I'm a bit puzzled why we're not sending any stats from walsender, which
I suppose could do various stuff during logical decoding.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#28Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Tomas Vondra (#27)
1 attachment(s)
Re: SLRU statistics

On 2020/05/03 1:59, Tomas Vondra wrote:

On Sat, May 02, 2020 at 12:55:00PM +0200, Tomas Vondra wrote:

On Sat, May 02, 2020 at 03:56:07PM +0900, Fujii Masao wrote:

...

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

Yes.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

Checkpointer sends off activity statistics to the stats collector in
two places, by calling pgstat_send_bgwriter(). What about calling
pgstat_send_slru() just after pgstat_send_bgwriter()?

Yep, that's what I proposed.

In previous email, I mentioned checkpointer just as an example.
So probably we need to investigate what process should send slru stats,
other than checkpointer. I guess that at least autovacuum worker,
logical replication walsender and parallel query worker (maybe this has
been already covered by parallel query some mechanisms. Sorry I've
not checked that) would need to send its slru stats.

Probably. Do you have any other process type in mind?

No. For now what I'm in mind are just checkpointer, autovacuum worker,
logical replication walsender and parallel query worker. Seems logical
replication worker and syncer have sent slru stats via pgstat_report_stat().

I've looked at places calling pgstat_send_* functions, and I found
thsese places:

src/backend/postmaster/bgwriter.c

- AFAIK it merely writes out dirty shared buffers, so likely irrelevant.

src/backend/postmaster/checkpointer.c

- This is what we're already discussing here.

src/backend/postmaster/pgarch.c

- Seems irrelevant.

I'm a bit puzzled why we're not sending any stats from walsender, which
I suppose could do various stuff during logical decoding.

Not sure why, but that seems an oversight...

Also I found another minor issue; SLRUStats has not been initialized to 0
and which could update the counters unexpectedly. Attached patch fixes
this issue.

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

Attachments:

init_slru_stats.patchtext/plain; charset=UTF-8; name=init_slru_stats.patch; x-mac-creator=0; x-mac-type=0Download
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 3f8105c6eb..416f86fbd6 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -2900,6 +2900,9 @@ pgstat_initialize(void)
 		MyBEEntry = &BackendStatusArray[MaxBackends + MyAuxProcType];
 	}
 
+	/* Initialize SLRU statistics to zero */
+	memset(&SLRUStats, 0, sizeof(SLRUStats));
+
 	/* Set up a process-exit hook to clean up */
 	on_shmem_exit(pgstat_beshutdown_hook, 0);
 }
#29Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Fujii Masao (#28)
1 attachment(s)
Re: SLRU statistics

On 2020/05/07 13:47, Fujii Masao wrote:

On 2020/05/03 1:59, Tomas Vondra wrote:

On Sat, May 02, 2020 at 12:55:00PM +0200, Tomas Vondra wrote:

On Sat, May 02, 2020 at 03:56:07PM +0900, Fujii Masao wrote:

...

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

Yes.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

Checkpointer sends off activity statistics to the stats collector in
two places, by calling pgstat_send_bgwriter(). What about calling
pgstat_send_slru() just after pgstat_send_bgwriter()?

Yep, that's what I proposed.

In previous email, I mentioned checkpointer just as an example.
So probably we need to investigate what process should send slru stats,
other than checkpointer. I guess that at least autovacuum worker,
logical replication walsender and parallel query worker (maybe this has
been already covered by parallel query some mechanisms. Sorry I've
not checked that) would need to send its slru stats.

Probably. Do you have any other process type in mind?

No. For now what I'm in mind are just checkpointer, autovacuum worker,
logical replication walsender and parallel query worker. Seems logical
replication worker and syncer have sent slru stats via pgstat_report_stat().

I've looked at places calling pgstat_send_* functions, and I found
thsese places:

src/backend/postmaster/bgwriter.c

- AFAIK it merely writes out dirty shared buffers, so likely irrelevant.

src/backend/postmaster/checkpointer.c

- This is what we're already discussing here.

src/backend/postmaster/pgarch.c

- Seems irrelevant.

I'm a bit puzzled why we're not sending any stats from walsender, which
I suppose could do various stuff during logical decoding.

Not sure why, but that seems an oversight...

Also I found another minor issue; SLRUStats has not been initialized to 0
and which could update the counters unexpectedly. Attached patch fixes
this issue.

This is minor issue, but basically it's better to fix that before
v13 beta1 release. So barring any objection, I will commit the patch.

+ values[8] = Int64GetDatum(stat.stat_reset_timestamp);

Also I found another small issue: pg_stat_get_slru() returns the timestamp
when pg_stat_slru was reset by using Int64GetDatum(). This works maybe
because the timestamp is also int64. But TimestampTzGetDatum() should
be used here, instead. Patch attached. Thought?

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

Attachments:

pg_stat_slru_reset_timestamp.patchtext/plain; charset=UTF-8; name=pg_stat_slru_reset_timestamp.patch; x-mac-creator=0; x-mac-type=0Download
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 446044609e..6b47617328 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1757,7 +1757,7 @@ pg_stat_get_slru(PG_FUNCTION_ARGS)
 		values[5] = Int64GetDatum(stat.blocks_exists);
 		values[6] = Int64GetDatum(stat.flush);
 		values[7] = Int64GetDatum(stat.truncate);
-		values[8] = Int64GetDatum(stat.stat_reset_timestamp);
+		values[8] = TimestampTzGetDatum(stat.stat_reset_timestamp);
 
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
 	}
#30Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Fujii Masao (#29)
Re: SLRU statistics

On Wed, May 13, 2020 at 04:10:30PM +0900, Fujii Masao wrote:

On 2020/05/07 13:47, Fujii Masao wrote:

On 2020/05/03 1:59, Tomas Vondra wrote:

On Sat, May 02, 2020 at 12:55:00PM +0200, Tomas Vondra wrote:

On Sat, May 02, 2020 at 03:56:07PM +0900, Fujii Masao wrote:

...

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

Yes.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

Checkpointer sends off activity statistics to the stats collector in
two places, by calling pgstat_send_bgwriter(). What about calling
pgstat_send_slru() just after pgstat_send_bgwriter()?

Yep, that's what I proposed.

In previous email, I mentioned checkpointer just as an example.
So probably we need to investigate what process should send slru stats,
other than checkpointer. I guess that at least autovacuum worker,
logical replication walsender and parallel query worker (maybe this has
been already covered by parallel query some mechanisms. Sorry I've
not checked that) would need to send its slru stats.

Probably. Do you have any other process type in mind?

No. For now what I'm in mind are just checkpointer, autovacuum worker,
logical replication walsender and parallel query worker. Seems logical
replication worker and syncer have sent slru stats via pgstat_report_stat().

I've looked at places calling pgstat_send_* functions, and I found
thsese places:

src/backend/postmaster/bgwriter.c

- AFAIK it merely writes out dirty shared buffers, so likely irrelevant.

src/backend/postmaster/checkpointer.c

- This is what we're already discussing here.

src/backend/postmaster/pgarch.c

- Seems irrelevant.

I'm a bit puzzled why we're not sending any stats from walsender, which
I suppose could do various stuff during logical decoding.

Not sure why, but that seems an oversight...

Also I found another minor issue; SLRUStats has not been initialized to 0
and which could update the counters unexpectedly. Attached patch fixes
this issue.

This is minor issue, but basically it's better to fix that before
v13 beta1 release. So barring any objection, I will commit the patch.

+ values[8] = Int64GetDatum(stat.stat_reset_timestamp);

Also I found another small issue: pg_stat_get_slru() returns the timestamp
when pg_stat_slru was reset by using Int64GetDatum(). This works maybe
because the timestamp is also int64. But TimestampTzGetDatum() should
be used here, instead. Patch attached. Thought?

I agree with both fixes.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#31Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Tomas Vondra (#30)
Re: SLRU statistics

On 2020/05/13 17:21, Tomas Vondra wrote:

On Wed, May 13, 2020 at 04:10:30PM +0900, Fujii Masao wrote:

On 2020/05/07 13:47, Fujii Masao wrote:

On 2020/05/03 1:59, Tomas Vondra wrote:

On Sat, May 02, 2020 at 12:55:00PM +0200, Tomas Vondra wrote:

On Sat, May 02, 2020 at 03:56:07PM +0900, Fujii Masao wrote:

...

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

Yes.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

Checkpointer sends off activity statistics to the stats collector in
two places, by calling pgstat_send_bgwriter(). What about calling
pgstat_send_slru() just after pgstat_send_bgwriter()?

Yep, that's what I proposed.

In previous email, I mentioned checkpointer just as an example.
So probably we need to investigate what process should send slru stats,
other than checkpointer. I guess that at least autovacuum worker,
logical replication walsender and parallel query worker (maybe this has
been already covered by parallel query some mechanisms. Sorry I've
not checked that) would need to send its slru stats.

Probably. Do you have any other process type in mind?

No. For now what I'm in mind are just checkpointer, autovacuum worker,
logical replication walsender and parallel query worker. Seems logical
replication worker and syncer have sent slru stats via pgstat_report_stat().

I've looked at places calling pgstat_send_* functions, and I found
thsese places:

src/backend/postmaster/bgwriter.c

- AFAIK it merely writes out dirty shared buffers, so likely irrelevant.

src/backend/postmaster/checkpointer.c

- This is what we're already discussing here.

src/backend/postmaster/pgarch.c

- Seems irrelevant.

I'm a bit puzzled why we're not sending any stats from walsender, which
I suppose could do various stuff during logical decoding.

Not sure why, but that seems an oversight...

Also I found another minor issue; SLRUStats has not been initialized to 0
and which could update the counters unexpectedly. Attached patch fixes
this issue.

This is minor issue, but basically it's better to fix that before
v13 beta1 release. So barring any objection, I will commit the patch.

+        values[8] = Int64GetDatum(stat.stat_reset_timestamp);

Also I found another small issue: pg_stat_get_slru() returns the timestamp
when pg_stat_slru was reset by using Int64GetDatum(). This works maybe
because the timestamp is also int64. But TimestampTzGetDatum() should
be used here, instead. Patch attached. Thought?

I agree with both fixes.

Pushed both. Thanks!

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

#32Tom Lane
tgl@sss.pgh.pa.us
In reply to: Fujii Masao (#31)
Re: SLRU statistics

Fujii Masao <masao.fujii@oss.nttdata.com> writes:

On 2020/05/13 17:21, Tomas Vondra wrote:

On Wed, May 13, 2020 at 04:10:30PM +0900, Fujii Masao wrote:

Also I found another minor issue; SLRUStats has not been initialized to 0
and which could update the counters unexpectedly. Attached patch fixes
this issue.

Pushed both. Thanks!

Why is that necessary? A static variable is defined by C to start off
as zeroes.

regards, tom lane

#33Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Tom Lane (#32)
Re: SLRU statistics

On 2020/05/13 23:26, Tom Lane wrote:

Fujii Masao <masao.fujii@oss.nttdata.com> writes:

On 2020/05/13 17:21, Tomas Vondra wrote:

On Wed, May 13, 2020 at 04:10:30PM +0900, Fujii Masao wrote:

Also I found another minor issue; SLRUStats has not been initialized to 0
and which could update the counters unexpectedly. Attached patch fixes
this issue.

Pushed both. Thanks!

Why is that necessary? A static variable is defined by C to start off
as zeroes.

Because SLRUStats is not a static variable. No?

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

#34Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Tom Lane (#32)
Re: SLRU statistics

On Wed, May 13, 2020 at 10:26:39AM -0400, Tom Lane wrote:

Fujii Masao <masao.fujii@oss.nttdata.com> writes:

On 2020/05/13 17:21, Tomas Vondra wrote:

On Wed, May 13, 2020 at 04:10:30PM +0900, Fujii Masao wrote:

Also I found another minor issue; SLRUStats has not been initialized to 0
and which could update the counters unexpectedly. Attached patch fixes
this issue.

Pushed both. Thanks!

Why is that necessary? A static variable is defined by C to start off
as zeroes.

But is it a static variable? It's not declared as 'static' but maybe we
can assume it inits to zeroes anyway? I see we do that for
BgWriterStats.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#35Tom Lane
tgl@sss.pgh.pa.us
In reply to: Tomas Vondra (#34)
Re: SLRU statistics

Tomas Vondra <tomas.vondra@2ndquadrant.com> writes:

On Wed, May 13, 2020 at 10:26:39AM -0400, Tom Lane wrote:

Why is that necessary? A static variable is defined by C to start off
as zeroes.

But is it a static variable? It's not declared as 'static' but maybe we
can assume it inits to zeroes anyway? I see we do that for
BgWriterStats.

Sorry, by "static" I meant "statically allocated", not "private to
this module". I'm sure the C standard has some more precise terminology
for this distinction, but I forget what it is.

regards, tom lane

#36Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Tom Lane (#35)
Re: SLRU statistics

On Wed, May 13, 2020 at 11:01:47AM -0400, Tom Lane wrote:

Tomas Vondra <tomas.vondra@2ndquadrant.com> writes:

On Wed, May 13, 2020 at 10:26:39AM -0400, Tom Lane wrote:

Why is that necessary? A static variable is defined by C to start off
as zeroes.

But is it a static variable? It's not declared as 'static' but maybe we
can assume it inits to zeroes anyway? I see we do that for
BgWriterStats.

Sorry, by "static" I meant "statically allocated", not "private to
this module". I'm sure the C standard has some more precise terminology
for this distinction, but I forget what it is.

Ah, I see. I'm no expert in reading C standard (or any other standard),
but a quick google search yielded this section of C99 standard:

-------------------------------------------------------------------------
If an object that has static storage duration is not initialized
explicitly, then:

- if it has pointer type, it is initialized to a null pointer;

- if it has arithmetic type, it is initialized to (positive or unsigned)
zero;

- if it is an aggregate, every member is initialized (recursively)
according to these rules;

- if it is au nion, the first named member is initialized (recursively)
according to these rules
-------------------------------------------------------------------------

I assume the SLRU variable counts as aggregate, with members having
arithmetic types. In which case it really should be initialized to 0.

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#37Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Fujii Masao (#33)
Re: SLRU statistics

On Wed, May 13, 2020 at 11:46:30PM +0900, Fujii Masao wrote:

On 2020/05/13 23:26, Tom Lane wrote:

Fujii Masao <masao.fujii@oss.nttdata.com> writes:

On 2020/05/13 17:21, Tomas Vondra wrote:

On Wed, May 13, 2020 at 04:10:30PM +0900, Fujii Masao wrote:

Also I found another minor issue; SLRUStats has not been initialized to 0
and which could update the counters unexpectedly. Attached patch fixes
this issue.

Pushed both. Thanks!

Why is that necessary? A static variable is defined by C to start off
as zeroes.

Because SLRUStats is not a static variable. No?

I think it counts as a variable with "static storage duration" per 6.7.8
(para 10), see [1]http://www.open-std.org/jtc1/sc22/wg14/www/docs/n1256.pdf. I wasn't aware of this either, but it probably means
the memset is unnecessary.

Also, it seems a bit strange/confusing to handle this differently from
BgWriterStats. And that worked fine without the init for years ...

[1]: http://www.open-std.org/jtc1/sc22/wg14/www/docs/n1256.pdf

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#38Tom Lane
tgl@sss.pgh.pa.us
In reply to: Tomas Vondra (#37)
Re: SLRU statistics

Tomas Vondra <tomas.vondra@2ndquadrant.com> writes:

I think it counts as a variable with "static storage duration" per 6.7.8
(para 10), see [1]. I wasn't aware of this either, but it probably means
the memset is unnecessary.
Also, it seems a bit strange/confusing to handle this differently from
BgWriterStats. And that worked fine without the init for years ...

Yeah, exactly.

There might be merit in memsetting it if we thought that it could have
become nonzero in the postmaster during a previous shmem cycle-of-life.
But the postmaster really shouldn't be accumulating such counts; and
if it is, then we have a bigger problem, because child processes would
be inheriting those counts via fork.

I think this change is unnecessary and should be reverted to avoid
future confusion about whether somehow it is necessary.

regards, tom lane

#39Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Tom Lane (#38)
Re: SLRU statistics

On 2020/05/14 0:38, Tom Lane wrote:

Tomas Vondra <tomas.vondra@2ndquadrant.com> writes:

I think it counts as a variable with "static storage duration" per 6.7.8
(para 10), see [1]. I wasn't aware of this either, but it probably means
the memset is unnecessary.
Also, it seems a bit strange/confusing to handle this differently from
BgWriterStats. And that worked fine without the init for years ...

Yeah, exactly.

There might be merit in memsetting it if we thought that it could have
become nonzero in the postmaster during a previous shmem cycle-of-life.
But the postmaster really shouldn't be accumulating such counts; and
if it is, then we have a bigger problem, because child processes would
be inheriting those counts via fork.

In my previous test, I thought I observed that the counters are already
updated at the beginning of some processes. So I thought that
the counters need to be initialized. Sorry, that's my fault...

So I tried the similar test again and found that postmaster seems to be
able to increment the counters unless I'm missing something.
For example,

frame #2: 0x000000010d93845f postgres`pgstat_count_slru_page_zeroed(ctl=0x000000010de27320) at pgstat.c:6739:2
frame #3: 0x000000010d5922ba postgres`SimpleLruZeroPage(ctl=0x000000010de27320, pageno=0) at slru.c:290:2
frame #4: 0x000000010d6b9ae2 postgres`AsyncShmemInit at async.c:568:12
frame #5: 0x000000010d9da9a6 postgres`CreateSharedMemoryAndSemaphores at ipci.c:265:2
frame #6: 0x000000010d93f679 postgres`reset_shared at postmaster.c:2664:2
frame #7: 0x000000010d93d253 postgres`PostmasterMain(argc=3, argv=0x00007fad56402e00) at postmaster.c:1008:2

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

#40Alvaro Herrera
alvherre@2ndquadrant.com
In reply to: Fujii Masao (#39)
Re: SLRU statistics

On 2020-May-14, Fujii Masao wrote:

So I tried the similar test again and found that postmaster seems to be
able to increment the counters unless I'm missing something.
For example,

frame #2: 0x000000010d93845f postgres`pgstat_count_slru_page_zeroed(ctl=0x000000010de27320) at pgstat.c:6739:2
frame #3: 0x000000010d5922ba postgres`SimpleLruZeroPage(ctl=0x000000010de27320, pageno=0) at slru.c:290:2
frame #4: 0x000000010d6b9ae2 postgres`AsyncShmemInit at async.c:568:12
frame #5: 0x000000010d9da9a6 postgres`CreateSharedMemoryAndSemaphores at ipci.c:265:2
frame #6: 0x000000010d93f679 postgres`reset_shared at postmaster.c:2664:2
frame #7: 0x000000010d93d253 postgres`PostmasterMain(argc=3, argv=0x00007fad56402e00) at postmaster.c:1008:2

Umm. I have the feeling that we'd rather avoid these updates in
postmaster, per our general rule that postmaster should not touch shared
memory. However, it might be that it's okay in this case, as it only
happens just as shmem is being "created", so other processes have not
yet had any time to mess things up. (IIRC only the Async module is
doing that.)

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#41Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Alvaro Herrera (#40)
Re: SLRU statistics

On 2020/05/14 1:14, Alvaro Herrera wrote:

On 2020-May-14, Fujii Masao wrote:

So I tried the similar test again and found that postmaster seems to be
able to increment the counters unless I'm missing something.
For example,

frame #2: 0x000000010d93845f postgres`pgstat_count_slru_page_zeroed(ctl=0x000000010de27320) at pgstat.c:6739:2
frame #3: 0x000000010d5922ba postgres`SimpleLruZeroPage(ctl=0x000000010de27320, pageno=0) at slru.c:290:2
frame #4: 0x000000010d6b9ae2 postgres`AsyncShmemInit at async.c:568:12
frame #5: 0x000000010d9da9a6 postgres`CreateSharedMemoryAndSemaphores at ipci.c:265:2
frame #6: 0x000000010d93f679 postgres`reset_shared at postmaster.c:2664:2
frame #7: 0x000000010d93d253 postgres`PostmasterMain(argc=3, argv=0x00007fad56402e00) at postmaster.c:1008:2

Umm. I have the feeling that we'd rather avoid these updates in
postmaster, per our general rule that postmaster should not touch shared
memory. However, it might be that it's okay in this case, as it only
happens just as shmem is being "created", so other processes have not
yet had any time to mess things up.

But since the counter that postmaster incremented is propagated to
child processes via fork, it should be zeroed at postmaster or the
beginning of child process? Otherwise that counter always starts
with non-zero in child process.

(IIRC only the Async module is
doing that.)

Yes, as far as I do the test.

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

#42Tom Lane
tgl@sss.pgh.pa.us
In reply to: Fujii Masao (#41)
Re: SLRU statistics

Fujii Masao <masao.fujii@oss.nttdata.com> writes:

But since the counter that postmaster incremented is propagated to
child processes via fork, it should be zeroed at postmaster or the
beginning of child process? Otherwise that counter always starts
with non-zero in child process.

Yes, if the postmaster is incrementing these counts then we would
have to reset them at the start of each child process. I share
Alvaro's feeling that that's bad and we don't want to do it.

(IIRC only the Async module is doing that.)

Hm, maybe we can fix that.

regards, tom lane

#43Tom Lane
tgl@sss.pgh.pa.us
In reply to: Tom Lane (#42)
1 attachment(s)
Re: SLRU statistics

I wrote:

(IIRC only the Async module is doing that.)

Hm, maybe we can fix that.

Yeah, it's quite easy to make async.c postpone its first write to the
async SLRU. This seems like a win all around, because many installations
don't use NOTIFY and so will never need to do that work at all. In
installations that do use notify, this costs an extra instruction or
two per NOTIFY, but that's down in the noise.

I got through check-world with the assertion shown that we are not
counting any SLRU operations in the postmaster. Don't know if we
want to commit that or not --- any thoughts?

regards, tom lane

Attachments:

dont-do-slru-ops-in-postmaster.patchtext/x-diff; charset=us-ascii; name=dont-do-slru-ops-in-postmaster.patchDownload
diff --git a/src/backend/commands/async.c b/src/backend/commands/async.c
index 0c9d20e..6ecea01 100644
--- a/src/backend/commands/async.c
+++ b/src/backend/commands/async.c
@@ -200,7 +200,10 @@ typedef struct QueuePosition
 	} while (0)
 
 #define QUEUE_POS_EQUAL(x,y) \
-	 ((x).page == (y).page && (x).offset == (y).offset)
+	((x).page == (y).page && (x).offset == (y).offset)
+
+#define QUEUE_POS_IS_ZERO(x) \
+	((x).page == 0 && (x).offset == 0)
 
 /* choose logically smaller QueuePosition */
 #define QUEUE_POS_MIN(x,y) \
@@ -515,7 +518,6 @@ void
 AsyncShmemInit(void)
 {
 	bool		found;
-	int			slotno;
 	Size		size;
 
 	/*
@@ -562,13 +564,6 @@ AsyncShmemInit(void)
 		 * During start or reboot, clean out the pg_notify directory.
 		 */
 		(void) SlruScanDirectory(AsyncCtl, SlruScanDirCbDeleteAll, NULL);
-
-		/* Now initialize page zero to empty */
-		LWLockAcquire(AsyncCtlLock, LW_EXCLUSIVE);
-		slotno = SimpleLruZeroPage(AsyncCtl, QUEUE_POS_PAGE(QUEUE_HEAD));
-		/* This write is just to verify that pg_notify/ is writable */
-		SimpleLruWritePage(AsyncCtl, slotno);
-		LWLockRelease(AsyncCtlLock);
 	}
 }
 
@@ -1470,9 +1465,18 @@ asyncQueueAddEntries(ListCell *nextNotify)
 	 */
 	queue_head = QUEUE_HEAD;
 
-	/* Fetch the current page */
+	/*
+	 * If this is the first write since the postmaster started, we need to
+	 * initialize the first page of the async SLRU.  Otherwise, the current
+	 * page should be initialized already, so just fetch it.
+	 */
 	pageno = QUEUE_POS_PAGE(queue_head);
-	slotno = SimpleLruReadPage(AsyncCtl, pageno, true, InvalidTransactionId);
+	if (QUEUE_POS_IS_ZERO(queue_head))
+		slotno = SimpleLruZeroPage(AsyncCtl, pageno);
+	else
+		slotno = SimpleLruReadPage(AsyncCtl, pageno, true,
+								   InvalidTransactionId);
+
 	/* Note we mark the page dirty before writing in it */
 	AsyncCtl->shared->page_dirty[slotno] = true;
 
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 80a06e5..e3c808b 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -6729,6 +6729,8 @@ slru_entry(int slru_idx)
 {
 	Assert((slru_idx >= 0) && (slru_idx < SLRU_NUM_ELEMENTS));
 
+	Assert(IsUnderPostmaster || !IsPostmasterEnvironment);
+
 	return &SLRUStats[slru_idx];
 }
 
#44Ranier Vilela
ranier.vf@gmail.com
In reply to: Fujii Masao (#33)
Re: SLRU statistics

Em qua., 13 de mai. de 2020 às 11:46, Fujii Masao <
masao.fujii@oss.nttdata.com> escreveu:

On 2020/05/13 23:26, Tom Lane wrote:

Fujii Masao <masao.fujii@oss.nttdata.com> writes:

On 2020/05/13 17:21, Tomas Vondra wrote:

On Wed, May 13, 2020 at 04:10:30PM +0900, Fujii Masao wrote:

Also I found another minor issue; SLRUStats has not been initialized

to 0

and which could update the counters unexpectedly. Attached patch fixes
this issue.

Pushed both. Thanks!

Why is that necessary? A static variable is defined by C to start off
as zeroes.

Because SLRUStats is not a static variable. No?

IMHO, BgWriterStats have the same problem, shouldn't the same be done?

/* Initialize BgWriterStats to zero */
MemSet(&BgWriterStats, 0, sizeof(BgWriterStats));

/* Initialize SLRU statistics to zero */
memset(&SLRUStats, 0, sizeof(SLRUStats));

regards,
Ranier Vilela

#45Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Tom Lane (#43)
Re: SLRU statistics

On 2020/05/14 2:44, Tom Lane wrote:

I wrote:

(IIRC only the Async module is doing that.)

Hm, maybe we can fix that.

Yeah, it's quite easy to make async.c postpone its first write to the
async SLRU. This seems like a win all around, because many installations
don't use NOTIFY and so will never need to do that work at all. In
installations that do use notify, this costs an extra instruction or
two per NOTIFY, but that's down in the noise.

Looks good to me. Thanks for the patch!

I got through check-world with the assertion shown that we are not
counting any SLRU operations in the postmaster. Don't know if we
want to commit that or not --- any thoughts?

+1 to add this assertion because basically it's not good thing
to access to SLRU at postmaster and we may want to fix that if found.
At least if we get rid of the SLRUStats initialization code,
IMO it's better to add this assertion and ensure that postmaster
doesn't update the SLRU stats counters.

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

#46Tom Lane
tgl@sss.pgh.pa.us
In reply to: Fujii Masao (#45)
Re: SLRU statistics

Fujii Masao <masao.fujii@oss.nttdata.com> writes:

On 2020/05/14 2:44, Tom Lane wrote:

I got through check-world with the assertion shown that we are not
counting any SLRU operations in the postmaster. Don't know if we
want to commit that or not --- any thoughts?

+1 to add this assertion because basically it's not good thing
to access to SLRU at postmaster and we may want to fix that if found.
At least if we get rid of the SLRUStats initialization code,
IMO it's better to add this assertion and ensure that postmaster
doesn't update the SLRU stats counters.

Seems reasonable --- I'll include it.

It might be nice to have similar assertions protecting BgWriterStats.
But given that we've made that public to be hacked on directly by several
different modules, I'm not sure that there's any simple way to do that.

regards, tom lane

#47Fujii Masao
masao.fujii@oss.nttdata.com
In reply to: Fujii Masao (#28)
Re: SLRU statistics

On 2020/05/07 13:47, Fujii Masao wrote:

On 2020/05/03 1:59, Tomas Vondra wrote:

On Sat, May 02, 2020 at 12:55:00PM +0200, Tomas Vondra wrote:

On Sat, May 02, 2020 at 03:56:07PM +0900, Fujii Masao wrote:

...

Another thing I found is; pgstat_send_slru() should be called also by
other processes than backend? For example, since clog data is flushed
basically by checkpointer, checkpointer seems to need to send slru stats.
Otherwise, pg_stat_slru.flushes would not be updated.

Hmmm, that's a good point. If I understand the issue correctly, the
checkpointer accumulates the stats but never really sends them because
it never calls pgstat_report_stat/pgstat_send_slru. That's only called
from PostgresMain, but not from CheckpointerMain.

Yes.

I think we could simply add pgstat_send_slru() right after the existing
call in CheckpointerMain, right?

Checkpointer sends off activity statistics to the stats collector in
two places, by calling pgstat_send_bgwriter(). What about calling
pgstat_send_slru() just after pgstat_send_bgwriter()?

Yep, that's what I proposed.

In previous email, I mentioned checkpointer just as an example.
So probably we need to investigate what process should send slru stats,
other than checkpointer. I guess that at least autovacuum worker,
logical replication walsender and parallel query worker (maybe this has
been already covered by parallel query some mechanisms. Sorry I've
not checked that) would need to send its slru stats.

Probably. Do you have any other process type in mind?

No. For now what I'm in mind are just checkpointer, autovacuum worker,
logical replication walsender and parallel query worker. Seems logical
replication worker and syncer have sent slru stats via pgstat_report_stat().

Let me go back to this topic. As far as I read the code again, logical
walsender reports the stats at the exit via pgstat_beshutdown_hook()
process-exit callback. But it doesn't report the stats while it's running.
This is not the problem only for SLRU stats. We would need to consider
how to handle the stats by logical walsender, separately from SLRU stats.

Autovacuum worker reports the stats at the exit via pgstat_beshutdown_hook(),
too. Unlike logical walsender, autovacuum worker is not the process that
basically keeps running during the service. It exits after it does vacuum or
analyze. So it's not bad to report the stats only at the exit, in autovacuum
worker case. There is no need to add extra code for SLRU stats report by
autovacuum worker.

Parallel worker is in the same situation as autovacuum worker. Its lifetime
is basically short and its stats is reported at the exit via
pgstat_beshutdown_hook().

pgstat_beshutdown_hook() reports the stats only when MyDatabaseId is valid.
Checkpointer calls pgstat_beshutdown_hook() at the exit, but doesn't report
the stats because its MyDatabaseId is invalid. Also it doesn't report the SLRU
stats while it's running. As we discussed upthread, we need to make
checkpointer call pgstat_send_slru() just after pgstat_send_bgwriter().

However even if we do this, the stats updated during the last checkpointer's
activity (e.g., shutdown checkpoint) seems not reported because
pgstat_beshutdown_hook() doesn't report the stats in checkpointer case.
Do we need to address this issue? If yes, we would need to change
pgstat_beshutdown_hook() or register another checkpointer-exit callback
that sends the stats. Thought?

Startup process is in the same situation as checkpointer process. It reports
the stats neither at the exit nor whle it's running. But, like logical
walsender, this seems not the problem only for SLRU stats. We would need to
consider how to handle the stats by startup process, separately from SLRU
stats.

Therefore what we can do right now seems to make checkpointer report the SLRU
stats while it's running. Other issues need more time to investigate...
Thought?

Regards,

--
Fujii Masao
Advanced Computing Technology Center
Research and Development Headquarters
NTT DATA CORPORATION

#48Robert Haas
robertmhaas@gmail.com
In reply to: Fujii Masao (#47)
Re: SLRU statistics

On Thu, May 14, 2020 at 2:27 AM Fujii Masao <masao.fujii@oss.nttdata.com> wrote:

Therefore what we can do right now seems to make checkpointer report the SLRU
stats while it's running. Other issues need more time to investigate...
Thought?

I'm confused by why SLRU statistics are reported by messages sent to
the stats collector rather than by just directly updating shared
memory. For database or table statistics there can be any number of
objects and we can't know in advance how many there will be, so we
can't set aside shared memory for the stats in advance. For SLRUs,
there's no such problem. Just having the individual backends
periodically merge their accumulated backend-local counters into the
shared counters seems like it would be way simpler and more
performant.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#49Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#48)
Re: SLRU statistics

Robert Haas <robertmhaas@gmail.com> writes:

I'm confused by why SLRU statistics are reported by messages sent to
the stats collector rather than by just directly updating shared
memory.

It would be better to consider that as an aspect of the WIP stats
collector redesign, rather than inventing a bespoke mechanism for
SLRU stats that's outside the stats collector (and, no doubt,
would have its own set of bugs). We don't need to invent even more
pathways for this sort of data.

regards, tom lane