From 851f3888098ec2b63c24489a69ba3be3b7be5ff7 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Wed, 2 Mar 2022 18:02:03 -0800
Subject: [PATCH v65 05/11] pgstat: Split different types of stats into
 separate files.

pgstat.c is very long, and it's hard to find an order that makes sense
and is likely to be maintained over time. Splitting the different
pieces into seperate files seems like it'll make that easier.
---
 src/include/utils/pgstat_internal.h           |  162 ++
 src/backend/postmaster/pgstat.c               | 2024 +----------------
 src/backend/utils/activity/Makefile           |    5 +
 src/backend/utils/activity/pgstat_database.c  |  279 +++
 src/backend/utils/activity/pgstat_function.c  |  225 ++
 src/backend/utils/activity/pgstat_global.c    |  548 +++++
 src/backend/utils/activity/pgstat_relation.c  |  953 ++++++++
 .../utils/activity/pgstat_subscription.c      |   78 +
 8 files changed, 2259 insertions(+), 2015 deletions(-)
 create mode 100644 src/include/utils/pgstat_internal.h
 create mode 100644 src/backend/utils/activity/pgstat_database.c
 create mode 100644 src/backend/utils/activity/pgstat_function.c
 create mode 100644 src/backend/utils/activity/pgstat_global.c
 create mode 100644 src/backend/utils/activity/pgstat_relation.c
 create mode 100644 src/backend/utils/activity/pgstat_subscription.c

diff --git a/src/include/utils/pgstat_internal.h b/src/include/utils/pgstat_internal.h
new file mode 100644
index 00000000000..51c0d06447b
--- /dev/null
+++ b/src/include/utils/pgstat_internal.h
@@ -0,0 +1,162 @@
+/* ----------
+ * pgstat_internal.h
+ *
+ * Definitions for the PostgreSQL activity statistics facility that should
+ * only be needed by files implementing statistics support (rather than ones
+ * reporting stats).
+ *
+ * Copyright (c) 2001-2021, PostgreSQL Global Development Group
+ *
+ * src/include/utils/pgstat_internal.h
+ * ----------
+ */
+#ifndef PGSTAT_INTERNAL_H
+#define PGSTAT_INTERNAL_H
+
+
+#include "pgstat.h"
+#include "executor/instrument.h"
+
+
+#define PGSTAT_STAT_INTERVAL	500 /* Minimum time between stats file
+									 * updates; in milliseconds. */
+
+/* ----------
+ * The initial size hints for the hash tables used in the collector.
+ * ----------
+ */
+#define PGSTAT_DB_HASH_SIZE		16
+#define PGSTAT_TAB_HASH_SIZE	512
+#define PGSTAT_FUNCTION_HASH_SIZE	512
+#define PGSTAT_SUBSCRIPTION_HASH_SIZE	32
+#define PGSTAT_REPLSLOT_HASH_SIZE	32
+
+
+/*
+ * Some stats changes are transactional. To maintain those, a stack of
+ * PgStat_SubXactStatus entries is maintained, which contain data pertaining
+ * to the current transaction and its active subtransactions.
+ */
+typedef struct PgStat_SubXactStatus
+{
+	int			nest_level;		/* subtransaction nest level */
+
+	struct PgStat_SubXactStatus *prev;	/* higher-level subxact if any */
+
+	/*
+	 * Tuple insertion/deletion counts for an open transaction can't be
+	 * propagated into PgStat_TableStatus counters until we know if it is
+	 * going to commit or abort.  Hence, we keep these counts in per-subxact
+	 * structs that live in TopTransactionContext.  This data structure is
+	 * designed on the assumption that subxacts won't usually modify very many
+	 * tables.
+	 */
+	PgStat_TableXactStatus *first;	/* head of list for this subxact */
+} PgStat_SubXactStatus;
+
+
+/*
+ * List of SLRU names that we keep stats for.  There is no central registry of
+ * SLRUs, so we use this fixed list instead.  The "other" entry is used for
+ * all SLRUs without an explicit entry (e.g. SLRUs in extensions).
+ */
+static const char *const slru_names[] = {
+	"CommitTs",
+	"MultiXactMember",
+	"MultiXactOffset",
+	"Notify",
+	"Serial",
+	"Subtrans",
+	"Xact",
+	"other"						/* has to be last */
+};
+
+#define SLRU_NUM_ELEMENTS	lengthof(slru_names)
+
+
+/*
+ * Functions in pgstat.c
+ */
+
+extern PgStat_SubXactStatus *pgstat_xact_stack_level_get(int nest_level);
+extern void pgstat_setheader(PgStat_MsgHdr *hdr, StatMsgType mtype);
+extern void pgstat_send(void *msg, int len);
+#ifdef USE_ASSERT_CHECKING
+extern void pgstat_assert_is_up(void);
+#else
+#define pgstat_assert_is_up() ((void)true)
+#endif
+
+
+/*
+ * Functions in pgstat_database.c
+ */
+
+extern void AtEOXact_PgStat_Database(bool isCommit, bool parallel);
+extern void pgstat_update_dbstats(PgStat_MsgTabstat *tsmsg, TimestampTz now);
+extern void pgstat_report_disconnect(Oid dboid);
+
+
+/*
+ * Functions in pgstat_function.c
+ */
+
+extern void pgstat_send_funcstats(void);
+
+
+/*
+ * Functions in pgstat_global.c
+ */
+
+extern void pgstat_send_slru(void);
+extern void pgstat_wal_initialize(void);
+
+
+/*
+ * Functions in pgstat_relation.c
+ */
+
+extern void pgstat_send_tabstats(TimestampTz now, bool disconnect);
+extern void AtEOXact_PgStat_Relations(PgStat_SubXactStatus *xact_state, bool isCommit);
+extern void AtEOSubXact_PgStat_Relations(PgStat_SubXactStatus *xact_state, bool isCommit, int nestDepth);
+extern void AtPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state);
+extern void PostPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state);
+
+
+/*
+ * Variables in pgstat.c
+ */
+
+extern pgsocket pgStatSock;
+
+
+/*
+ * Variables in pgstat_database.c
+ */
+
+extern int	pgStatXactCommit;
+extern int	pgStatXactRollback;
+
+
+/*
+ * Variables in pgstat_functions.c
+ */
+
+extern bool have_function_stats;
+
+
+/*
+ * Variables in pgstat_database.c
+ */
+
+extern WalUsage prevWalUsage;
+
+
+/*
+ * Variables in pgstat_relation.c
+ */
+
+extern bool have_relation_stats;
+
+
+#endif							/* PGSTAT_INTERNAL_H */
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index 16fe2df4ec9..3b693460a93 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -36,14 +36,12 @@
 #include "access/htup_details.h"
 #include "access/tableam.h"
 #include "access/transam.h"
-#include "access/twophase_rmgr.h"
 #include "access/xact.h"
 #include "catalog/catalog.h"
 #include "catalog/pg_database.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_subscription.h"
 #include "common/ip.h"
-#include "executor/instrument.h"
 #include "libpq/libpq.h"
 #include "libpq/pqsignal.h"
 #include "mb/pg_wchar.h"
@@ -67,6 +65,7 @@
 #include "utils/builtins.h"
 #include "utils/guc.h"
 #include "utils/memutils.h"
+#include "utils/pgstat_internal.h"
 #include "utils/ps_status.h"
 #include "utils/rel.h"
 #include "utils/snapmgr.h"
@@ -76,9 +75,6 @@
  * Timer definitions.
  * ----------
  */
-#define PGSTAT_STAT_INTERVAL	500 /* Minimum time between stats file
-									 * updates; in milliseconds. */
-
 #define PGSTAT_RETRY_DELAY		10	/* How long to wait between checks for a
 									 * new file; in milliseconds. */
 
@@ -99,23 +95,11 @@
 #define PGSTAT_MIN_RCVBUF		(100 * 1024)
 
 
-/* ----------
- * The initial size hints for the hash tables used in the collector.
- * ----------
- */
-#define PGSTAT_DB_HASH_SIZE		16
-#define PGSTAT_TAB_HASH_SIZE	512
-#define PGSTAT_FUNCTION_HASH_SIZE	512
-#define PGSTAT_SUBSCRIPTION_HASH_SIZE	32
-#define PGSTAT_REPLSLOT_HASH_SIZE	32
-
-
 /* ----------
  * GUC parameters
  * ----------
  */
 bool		pgstat_track_counts = false;
-int			pgstat_track_functions = TRACK_FUNC_OFF;
 
 /* ----------
  * Built from GUC parameter
@@ -125,163 +109,21 @@ char	   *pgstat_stat_directory = NULL;
 char	   *pgstat_stat_filename = NULL;
 char	   *pgstat_stat_tmpname = NULL;
 
-/*
- * BgWriter and WAL global statistics counters.
- * Stored directly in a stats message structure so they can be sent
- * without needing to copy things around.  We assume these init to zeroes.
- */
-PgStat_MsgBgWriter PendingBgWriterStats;
-PgStat_MsgCheckpointer PendingCheckpointerStats;
-PgStat_MsgWal WalStats;
-
-/*
- * WAL usage counters saved from pgWALUsage at the previous call to
- * pgstat_send_wal(). This is used to calculate how much WAL usage
- * happens between pgstat_send_wal() calls, by subtracting
- * the previous counters from the current ones.
- */
-static WalUsage prevWalUsage;
-
-/*
- * List of SLRU names that we keep stats for.  There is no central registry of
- * SLRUs, so we use this fixed list instead.  The "other" entry is used for
- * all SLRUs without an explicit entry (e.g. SLRUs in extensions).
- */
-static const char *const slru_names[] = {
-	"CommitTs",
-	"MultiXactMember",
-	"MultiXactOffset",
-	"Notify",
-	"Serial",
-	"Subtrans",
-	"Xact",
-	"other"						/* has to be last */
-};
-
-#define SLRU_NUM_ELEMENTS	lengthof(slru_names)
-
-/*
- * SLRU statistics counts waiting to be sent to the collector.  These are
- * stored directly in stats message format so they can be sent without needing
- * to copy things around.  We assume this variable inits to zeroes.  Entries
- * are one-to-one with slru_names[].
- */
-static PgStat_MsgSLRU SLRUStats[SLRU_NUM_ELEMENTS];
-
 /* ----------
  * Local data
  * ----------
  */
-NON_EXEC_STATIC pgsocket pgStatSock = PGINVALID_SOCKET;
 
 static struct sockaddr_storage pgStatAddr;
 
+pgsocket pgStatSock = PGINVALID_SOCKET;
+
 static time_t last_pgstat_start_time;
 
 static bool pgStatRunningInCollector = false;
 
-/*
- * Structures in which backends store per-table info that's waiting to be
- * sent to the collector.
- *
- * NOTE: once allocated, TabStatusArray structures are never moved or deleted
- * for the life of the backend.  Also, we zero out the t_id fields of the
- * contained PgStat_TableStatus structs whenever they are not actively in use.
- * This allows relcache pgstat_info pointers to be treated as long-lived data,
- * avoiding repeated searches in pgstat_relation_assoc() when a relation is
- * repeatedly opened during a transaction.
- */
-#define TABSTAT_QUANTUM		100 /* we alloc this many at a time */
-
-typedef struct TabStatusArray
-{
-	struct TabStatusArray *tsa_next;	/* link to next array, if any */
-	int			tsa_used;		/* # entries currently used */
-	PgStat_TableStatus tsa_entries[TABSTAT_QUANTUM];	/* per-table data */
-} TabStatusArray;
-
-static TabStatusArray *pgStatTabList = NULL;
-
-/*
- * pgStatTabHash entry: map from relation OID to PgStat_TableStatus pointer
- */
-typedef struct TabStatHashEntry
-{
-	Oid			t_id;
-	PgStat_TableStatus *tsa_entry;
-} TabStatHashEntry;
-
-/*
- * Hash table for O(1) t_id -> tsa_entry lookup
- */
-static HTAB *pgStatTabHash = NULL;
-
-/*
- * Backends store per-function info that's waiting to be sent to the collector
- * in this hash table (indexed by function OID).
- */
-static HTAB *pgStatFunctions = NULL;
-
-/*
- * Indicates if backend has some relation stats that it hasn't yet
- * sent to the collector.
- */
-static bool have_relation_stats = false;
-
-/*
- * Indicates if backend has some function stats that it hasn't yet
- * sent to the collector.
- */
-static bool have_function_stats = false;
-
-/*
- * Some stats changes are transactional. To maintain those, a stack of
- * PgStat_SubXactStatus entries is maintained, which contain data pertaining
- * to the current transaction and its active subtransactions.
- */
-typedef struct PgStat_SubXactStatus
-{
-	int			nest_level;		/* subtransaction nest level */
-
-	struct PgStat_SubXactStatus *prev;	/* higher-level subxact if any */
-
-	/*
-	 * Tuple insertion/deletion counts for an open transaction can't be
-	 * propagated into PgStat_TableStatus counters until we know if it is
-	 * going to commit or abort.  Hence, we keep these counts in per-subxact
-	 * structs that live in TopTransactionContext.  This data structure is
-	 * designed on the assumption that subxacts won't usually modify very many
-	 * tables.
-	 */
-	PgStat_TableXactStatus *first;	/* head of list for this subxact */
-} PgStat_SubXactStatus;
-
 static PgStat_SubXactStatus *pgStatXactStack = NULL;
 
-static int	pgStatXactCommit = 0;
-static int	pgStatXactRollback = 0;
-PgStat_Counter pgStatBlockReadTime = 0;
-PgStat_Counter pgStatBlockWriteTime = 0;
-static PgStat_Counter pgLastSessionReportTime = 0;
-PgStat_Counter pgStatActiveTime = 0;
-PgStat_Counter pgStatTransactionIdleTime = 0;
-SessionEndType pgStatSessionEndCause = DISCONNECT_NORMAL;
-
-/* Record that's written to 2PC state file when pgstat state is persisted */
-typedef struct TwoPhasePgStatRecord
-{
-	PgStat_Counter tuples_inserted; /* tuples inserted in xact */
-	PgStat_Counter tuples_updated;	/* tuples updated in xact */
-	PgStat_Counter tuples_deleted;	/* tuples deleted in xact */
-	/* tuples i/u/d prior to truncate/drop */
-	PgStat_Counter inserted_pre_truncdrop;
-	PgStat_Counter updated_pre_truncdrop;
-	PgStat_Counter deleted_pre_truncdrop;
-	Oid			t_id;			/* table's OID */
-	bool		t_shared;		/* is it a shared catalog? */
-	bool		t_truncdropped;	/* was the relation truncated/dropped? */
-} TwoPhasePgStatRecord;
-
 /*
  * Info about current "snapshot" of stats file
  */
@@ -307,13 +149,6 @@ static HTAB *subscriptionStatHash = NULL;
  */
 static List *pending_write_requests = NIL;
 
-/*
- * Total time charged to functions so far in the current backend.
- * We use this to help separate "self" and "other" time charges.
- * (We assume this initializes to zero.)
- */
-static instr_time total_func_time;
-
 /*
  * For assertions that check pgstat is not used before initialization / after
  * shutdown.
@@ -352,23 +187,10 @@ static bool pgstat_db_requested(Oid databaseid);
 static PgStat_StatReplSlotEntry *pgstat_get_replslot_entry(NameData name, bool create_it);
 static void pgstat_reset_replslot(PgStat_StatReplSlotEntry *slotstats, TimestampTz ts);
 
-static void pgstat_send_tabstats(TimestampTz now, bool disconnect);
-static void pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg, TimestampTz now);
-static void pgstat_update_dbstats(PgStat_MsgTabstat *tsmsg, TimestampTz now);
-static void pgstat_send_funcstats(void);
-static void pgstat_wal_initialize(void);
-static void pgstat_send_slru(void);
 static HTAB *pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid);
-static bool pgstat_should_report_connstat(void);
-static void pgstat_report_disconnect(Oid dboid);
-
-static PgStat_TableStatus *get_tabstat_entry(Oid rel_id, bool isshared);
 
 static void pgstat_setup_memcxt(void);
-static void pgstat_assert_is_up(void);
 
-static void pgstat_setheader(PgStat_MsgHdr *hdr, StatMsgType mtype);
-static void pgstat_send(void *msg, int len);
 
 static void pgstat_recv_inquiry(PgStat_MsgInquiry *msg, int len);
 static void pgstat_recv_tabstat(PgStat_MsgTabstat *msg, int len);
@@ -932,231 +754,6 @@ pgstat_report_stat(bool disconnect)
 	pgstat_send_slru();
 }
 
-/*
- * Subroutine for pgstat_report_stat: Send relation statistics
- */
-static void
-pgstat_send_tabstats(TimestampTz now, bool disconnect)
-{
-	/* we assume this inits to all zeroes: */
-	static const PgStat_TableCounts all_zeroes;
-	PgStat_MsgTabstat regular_msg;
-	PgStat_MsgTabstat shared_msg;
-	TabStatusArray *tsa;
-	int			i;
-
-	/*
-	 * Destroy pgStatTabHash before we start invalidating PgStat_TableEntry
-	 * entries it points to.  (Should we fail partway through the loop below,
-	 * it's okay to have removed the hashtable already --- the only
-	 * consequence is we'd get multiple entries for the same table in the
-	 * pgStatTabList, and that's safe.)
-	 */
-	if (pgStatTabHash)
-		hash_destroy(pgStatTabHash);
-	pgStatTabHash = NULL;
-
-	/*
-	 * Scan through the TabStatusArray struct(s) to find tables that actually
-	 * have counts, and build messages to send.  We have to separate shared
-	 * relations from regular ones because the databaseid field in the message
-	 * header has to depend on that.
-	 */
-	regular_msg.m_databaseid = MyDatabaseId;
-	shared_msg.m_databaseid = InvalidOid;
-	regular_msg.m_nentries = 0;
-	shared_msg.m_nentries = 0;
-
-	for (tsa = pgStatTabList; tsa != NULL; tsa = tsa->tsa_next)
-	{
-		for (i = 0; i < tsa->tsa_used; i++)
-		{
-			PgStat_TableStatus *entry = &tsa->tsa_entries[i];
-			PgStat_MsgTabstat *this_msg;
-			PgStat_TableEntry *this_ent;
-
-			/* Shouldn't have any pending transaction-dependent counts */
-			Assert(entry->trans == NULL);
-
-			/*
-			 * Ignore entries that didn't accumulate any actual counts, such
-			 * as indexes that were opened by the planner but not used.
-			 */
-			if (memcmp(&entry->t_counts, &all_zeroes,
-					   sizeof(PgStat_TableCounts)) == 0)
-				continue;
-
-			/*
-			 * OK, insert data into the appropriate message, and send if full.
-			 */
-			this_msg = entry->t_shared ? &shared_msg : &regular_msg;
-			this_ent = &this_msg->m_entry[this_msg->m_nentries];
-			this_ent->t_id = entry->t_id;
-			memcpy(&this_ent->t_counts, &entry->t_counts,
-				   sizeof(PgStat_TableCounts));
-			if (++this_msg->m_nentries >= PGSTAT_NUM_TABENTRIES)
-			{
-				pgstat_send_tabstat(this_msg, now);
-				this_msg->m_nentries = 0;
-			}
-		}
-		/* zero out PgStat_TableStatus structs after use */
-		MemSet(tsa->tsa_entries, 0,
-			   tsa->tsa_used * sizeof(PgStat_TableStatus));
-		tsa->tsa_used = 0;
-	}
-
-	/*
-	 * Send partial messages.  Make sure that any pending xact commit/abort
-	 * and connection stats get counted, even if there are no table stats to
-	 * send.
-	 */
-	if (regular_msg.m_nentries > 0 ||
-		pgStatXactCommit > 0 || pgStatXactRollback > 0 || disconnect)
-		pgstat_send_tabstat(&regular_msg, now);
-	if (shared_msg.m_nentries > 0)
-		pgstat_send_tabstat(&shared_msg, now);
-
-	have_relation_stats = false;
-}
-
-/*
- * Subroutine for pgstat_send_tabstats: finish and send one tabstat message
- */
-static void
-pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg, TimestampTz now)
-{
-	int			n;
-	int			len;
-
-	/* It's unlikely we'd get here with no socket, but maybe not impossible */
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	/*
-	 * Report and reset accumulated xact commit/rollback and I/O timings
-	 * whenever we send a normal tabstat message
-	 */
-	pgstat_update_dbstats(tsmsg, now);
-
-	n = tsmsg->m_nentries;
-	len = offsetof(PgStat_MsgTabstat, m_entry[0]) +
-		n * sizeof(PgStat_TableEntry);
-
-	pgstat_setheader(&tsmsg->m_hdr, PGSTAT_MTYPE_TABSTAT);
-	pgstat_send(tsmsg, len);
-}
-
-/*
- * Subroutine for pgstat_send_tabstat: Handle xact commit/rollback and I/O
- * timings.
- */
-static void
-pgstat_update_dbstats(PgStat_MsgTabstat *tsmsg, TimestampTz now)
-{
-	if (OidIsValid(tsmsg->m_databaseid))
-	{
-		tsmsg->m_xact_commit = pgStatXactCommit;
-		tsmsg->m_xact_rollback = pgStatXactRollback;
-		tsmsg->m_block_read_time = pgStatBlockReadTime;
-		tsmsg->m_block_write_time = pgStatBlockWriteTime;
-
-		if (pgstat_should_report_connstat())
-		{
-			long		secs;
-			int			usecs;
-
-			/*
-			 * pgLastSessionReportTime is initialized to MyStartTimestamp by
-			 * pgstat_report_connect().
-			 */
-			TimestampDifference(pgLastSessionReportTime, now, &secs, &usecs);
-			pgLastSessionReportTime = now;
-			tsmsg->m_session_time = (PgStat_Counter) secs * 1000000 + usecs;
-			tsmsg->m_active_time = pgStatActiveTime;
-			tsmsg->m_idle_in_xact_time = pgStatTransactionIdleTime;
-		}
-		else
-		{
-			tsmsg->m_session_time = 0;
-			tsmsg->m_active_time = 0;
-			tsmsg->m_idle_in_xact_time = 0;
-		}
-		pgStatXactCommit = 0;
-		pgStatXactRollback = 0;
-		pgStatBlockReadTime = 0;
-		pgStatBlockWriteTime = 0;
-		pgStatActiveTime = 0;
-		pgStatTransactionIdleTime = 0;
-	}
-	else
-	{
-		tsmsg->m_xact_commit = 0;
-		tsmsg->m_xact_rollback = 0;
-		tsmsg->m_block_read_time = 0;
-		tsmsg->m_block_write_time = 0;
-		tsmsg->m_session_time = 0;
-		tsmsg->m_active_time = 0;
-		tsmsg->m_idle_in_xact_time = 0;
-	}
-}
-
-/*
- * Subroutine for pgstat_report_stat: populate and send a function stat message
- */
-static void
-pgstat_send_funcstats(void)
-{
-	/* we assume this inits to all zeroes: */
-	static const PgStat_FunctionCounts all_zeroes;
-
-	PgStat_MsgFuncstat msg;
-	PgStat_BackendFunctionEntry *entry;
-	HASH_SEQ_STATUS fstat;
-
-	if (pgStatFunctions == NULL)
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_FUNCSTAT);
-	msg.m_databaseid = MyDatabaseId;
-	msg.m_nentries = 0;
-
-	hash_seq_init(&fstat, pgStatFunctions);
-	while ((entry = (PgStat_BackendFunctionEntry *) hash_seq_search(&fstat)) != NULL)
-	{
-		PgStat_FunctionEntry *m_ent;
-
-		/* Skip it if no counts accumulated since last time */
-		if (memcmp(&entry->f_counts, &all_zeroes,
-				   sizeof(PgStat_FunctionCounts)) == 0)
-			continue;
-
-		/* need to convert format of time accumulators */
-		m_ent = &msg.m_entry[msg.m_nentries];
-		m_ent->f_id = entry->f_id;
-		m_ent->f_numcalls = entry->f_counts.f_numcalls;
-		m_ent->f_total_time = INSTR_TIME_GET_MICROSEC(entry->f_counts.f_total_time);
-		m_ent->f_self_time = INSTR_TIME_GET_MICROSEC(entry->f_counts.f_self_time);
-
-		if (++msg.m_nentries >= PGSTAT_NUM_FUNCENTRIES)
-		{
-			pgstat_send(&msg, offsetof(PgStat_MsgFuncstat, m_entry[0]) +
-						msg.m_nentries * sizeof(PgStat_FunctionEntry));
-			msg.m_nentries = 0;
-		}
-
-		/* reset the entry's counts */
-		MemSet(&entry->f_counts, 0, sizeof(PgStat_FunctionCounts));
-	}
-
-	if (msg.m_nentries > 0)
-		pgstat_send(&msg, offsetof(PgStat_MsgFuncstat, m_entry[0]) +
-					msg.m_nentries * sizeof(PgStat_FunctionEntry));
-
-	have_function_stats = false;
-}
-
-
 /* ----------
  * pgstat_vacuum_stat() -
  *
@@ -1429,61 +1026,6 @@ pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid)
 	return htab;
 }
 
-
-/* ----------
- * pgstat_drop_database() -
- *
- *	Tell the collector that we just dropped a database.
- *	(If the message gets lost, we will still clean the dead DB eventually
- *	via future invocations of pgstat_vacuum_stat().)
- * ----------
- */
-void
-pgstat_drop_database(Oid databaseid)
-{
-	PgStat_MsgDropdb msg;
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_DROPDB);
-	msg.m_databaseid = databaseid;
-	pgstat_send(&msg, sizeof(msg));
-}
-
-
-/* ----------
- * pgstat_drop_relation() -
- *
- *	Tell the collector that we just dropped a relation.
- *	(If the message gets lost, we will still clean the dead entry eventually
- *	via future invocations of pgstat_vacuum_stat().)
- *
- *	Currently not used for lack of any good place to call it; we rely
- *	entirely on pgstat_vacuum_stat() to clean out stats for dead rels.
- * ----------
- */
-#ifdef NOT_USED
-void
-pgstat_drop_relation(Oid relid)
-{
-	PgStat_MsgTabpurge msg;
-	int			len;
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	msg.m_tableid[0] = relid;
-	msg.m_nentries = 1;
-
-	len = offsetof(PgStat_MsgTabpurge, m_tableid[0]) + sizeof(Oid);
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_TABPURGE);
-	msg.m_databaseid = MyDatabaseId;
-	pgstat_send(&msg, len);
-}
-#endif							/* NOT_USED */
-
 /* ----------
  * pgstat_reset_counters() -
  *
@@ -1506,39 +1048,6 @@ pgstat_reset_counters(void)
 	pgstat_send(&msg, sizeof(msg));
 }
 
-/* ----------
- * pgstat_reset_shared_counters() -
- *
- *	Tell the statistics collector to reset cluster-wide shared counters.
- *
- *	Permission checking for this function is managed through the normal
- *	GRANT system.
- * ----------
- */
-void
-pgstat_reset_shared_counters(const char *target)
-{
-	PgStat_MsgResetsharedcounter msg;
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	if (strcmp(target, "archiver") == 0)
-		msg.m_resettarget = RESET_ARCHIVER;
-	else if (strcmp(target, "bgwriter") == 0)
-		msg.m_resettarget = RESET_BGWRITER;
-	else if (strcmp(target, "wal") == 0)
-		msg.m_resettarget = RESET_WAL;
-	else
-		ereport(ERROR,
-				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
-				 errmsg("unrecognized reset target: \"%s\"", target),
-				 errhint("Target must be \"archiver\", \"bgwriter\", or \"wal\".")));
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETSHAREDCOUNTER);
-	pgstat_send(&msg, sizeof(msg));
-}
-
 /* ----------
  * pgstat_reset_single_counter() -
  *
@@ -1564,443 +1073,6 @@ 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_reset_replslot_counter() -
- *
- *	Tell the statistics collector to reset a single replication slot
- *	counter, or all replication slots counters (when name is null).
- *
- *	Permission checking for this function is managed through the normal
- *	GRANT system.
- * ----------
- */
-void
-pgstat_reset_replslot_counter(const char *name)
-{
-	PgStat_MsgResetreplslotcounter msg;
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	if (name)
-	{
-		namestrcpy(&msg.m_slotname, name);
-		msg.clearall = false;
-	}
-	else
-		msg.clearall = true;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETREPLSLOTCOUNTER);
-
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* ----------
- * pgstat_reset_subscription_counter() -
- *
- *	Tell the statistics collector to reset a single subscription
- *	counter, or all subscription counters (when subid is InvalidOid).
- *
- *	Permission checking for this function is managed through the normal
- *	GRANT system.
- * ----------
- */
-void
-pgstat_reset_subscription_counter(Oid subid)
-{
-	PgStat_MsgResetsubcounter msg;
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	msg.m_subid = subid;
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETSUBCOUNTER);
-
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* ----------
- * pgstat_report_autovac() -
- *
- *	Called from autovacuum.c to report startup of an autovacuum process.
- *	We are called before InitPostgres is done, so can't rely on MyDatabaseId;
- *	the db OID must be passed in, instead.
- * ----------
- */
-void
-pgstat_report_autovac(Oid dboid)
-{
-	PgStat_MsgAutovacStart msg;
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_AUTOVAC_START);
-	msg.m_databaseid = dboid;
-	msg.m_start_time = GetCurrentTimestamp();
-
-	pgstat_send(&msg, sizeof(msg));
-}
-
-
-/* ---------
- * pgstat_report_vacuum() -
- *
- *	Tell the collector about the table we just vacuumed.
- * ---------
- */
-void
-pgstat_report_vacuum(Oid tableoid, bool shared,
-					 PgStat_Counter livetuples, PgStat_Counter deadtuples)
-{
-	PgStat_MsgVacuum msg;
-
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_VACUUM);
-	msg.m_databaseid = shared ? InvalidOid : MyDatabaseId;
-	msg.m_tableoid = tableoid;
-	msg.m_autovacuum = IsAutoVacuumWorkerProcess();
-	msg.m_vacuumtime = GetCurrentTimestamp();
-	msg.m_live_tuples = livetuples;
-	msg.m_dead_tuples = deadtuples;
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* --------
- * pgstat_report_analyze() -
- *
- *	Tell the collector about the table we just analyzed.
- *
- * Caller must provide new live- and dead-tuples estimates, as well as a
- * flag indicating whether to reset the changes_since_analyze counter.
- * --------
- */
-void
-pgstat_report_analyze(Relation rel,
-					  PgStat_Counter livetuples, PgStat_Counter deadtuples,
-					  bool resetcounter)
-{
-	PgStat_MsgAnalyze msg;
-
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
-		return;
-
-	/*
-	 * Unlike VACUUM, ANALYZE might be running inside a transaction that has
-	 * already inserted and/or deleted rows in the target table. ANALYZE will
-	 * have counted such rows as live or dead respectively. Because we will
-	 * report our counts of such rows at transaction end, we should subtract
-	 * off these counts from what we send to the collector now, else they'll
-	 * be double-counted after commit.  (This approach also ensures that the
-	 * collector ends up with the right numbers if we abort instead of
-	 * committing.)
-	 *
-	 * Waste no time on partitioned tables, though.
-	 */
-	if (pgstat_relation_should_count(rel) &&
-		rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
-	{
-		PgStat_TableXactStatus *trans;
-
-		for (trans = rel->pgstat_info->trans; trans; trans = trans->upper)
-		{
-			livetuples -= trans->tuples_inserted - trans->tuples_deleted;
-			deadtuples -= trans->tuples_updated + trans->tuples_deleted;
-		}
-		/* count stuff inserted by already-aborted subxacts, too */
-		deadtuples -= rel->pgstat_info->t_counts.t_delta_dead_tuples;
-		/* Since ANALYZE's counts are estimates, we could have underflowed */
-		livetuples = Max(livetuples, 0);
-		deadtuples = Max(deadtuples, 0);
-	}
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_ANALYZE);
-	msg.m_databaseid = rel->rd_rel->relisshared ? InvalidOid : MyDatabaseId;
-	msg.m_tableoid = RelationGetRelid(rel);
-	msg.m_autovacuum = IsAutoVacuumWorkerProcess();
-	msg.m_resetcounter = resetcounter;
-	msg.m_analyzetime = GetCurrentTimestamp();
-	msg.m_live_tuples = livetuples;
-	msg.m_dead_tuples = deadtuples;
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* --------
- * pgstat_report_recovery_conflict() -
- *
- *	Tell the collector about a Hot Standby recovery conflict.
- * --------
- */
-void
-pgstat_report_recovery_conflict(int reason)
-{
-	PgStat_MsgRecoveryConflict msg;
-
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RECOVERYCONFLICT);
-	msg.m_databaseid = MyDatabaseId;
-	msg.m_reason = reason;
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* --------
- * pgstat_report_deadlock() -
- *
- *	Tell the collector about a deadlock detected.
- * --------
- */
-void
-pgstat_report_deadlock(void)
-{
-	PgStat_MsgDeadlock msg;
-
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_DEADLOCK);
-	msg.m_databaseid = MyDatabaseId;
-	pgstat_send(&msg, sizeof(msg));
-}
-
-
-
-/* --------
- * pgstat_report_checksum_failures_in_db() -
- *
- *	Tell the collector about one or more checksum failures.
- * --------
- */
-void
-pgstat_report_checksum_failures_in_db(Oid dboid, int failurecount)
-{
-	PgStat_MsgChecksumFailure msg;
-
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_CHECKSUMFAILURE);
-	msg.m_databaseid = dboid;
-	msg.m_failurecount = failurecount;
-	msg.m_failure_time = GetCurrentTimestamp();
-
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* --------
- * pgstat_report_checksum_failure() -
- *
- *	Tell the collector about a checksum failure.
- * --------
- */
-void
-pgstat_report_checksum_failure(void)
-{
-	pgstat_report_checksum_failures_in_db(MyDatabaseId, 1);
-}
-
-/* --------
- * pgstat_report_tempfile() -
- *
- *	Tell the collector about a temporary file.
- * --------
- */
-void
-pgstat_report_tempfile(size_t filesize)
-{
-	PgStat_MsgTempFile msg;
-
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_TEMPFILE);
-	msg.m_databaseid = MyDatabaseId;
-	msg.m_filesize = filesize;
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* --------
- * pgstat_report_connect() -
- *
- *	Tell the collector about a new connection.
- * --------
- */
-void
-pgstat_report_connect(Oid dboid)
-{
-	PgStat_MsgConnect msg;
-
-	if (!pgstat_should_report_connstat())
-		return;
-
-	pgLastSessionReportTime = MyStartTimestamp;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_CONNECT);
-	msg.m_databaseid = MyDatabaseId;
-	pgstat_send(&msg, sizeof(PgStat_MsgConnect));
-}
-
-/* --------
- * pgstat_report_disconnect() -
- *
- *	Tell the collector about a disconnect.
- * --------
- */
-static void
-pgstat_report_disconnect(Oid dboid)
-{
-	PgStat_MsgDisconnect msg;
-
-	if (!pgstat_should_report_connstat())
-		return;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_DISCONNECT);
-	msg.m_databaseid = MyDatabaseId;
-	msg.m_cause = pgStatSessionEndCause;
-	pgstat_send(&msg, sizeof(PgStat_MsgDisconnect));
-}
-
-/* --------
- * pgstat_should_report_connstats() -
- *
- *	We report session statistics only for normal backend processes.  Parallel
- *	workers run in parallel, so they don't contribute to session times, even
- *	though they use CPU time. Walsender processes could be considered here,
- *	but they have different session characteristics from normal backends (for
- *	example, they are always "active"), so they would skew session statistics.
- * ----------
- */
-static bool
-pgstat_should_report_connstat(void)
-{
-	return MyBackendType == B_BACKEND;
-}
-
-/* ----------
- * pgstat_report_replslot() -
- *
- *	Tell the collector about replication slot statistics.
- * ----------
- */
-void
-pgstat_report_replslot(const PgStat_StatReplSlotEntry *repSlotStat)
-{
-	PgStat_MsgReplSlot msg;
-
-	/*
-	 * Prepare and send the message
-	 */
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_REPLSLOT);
-	namestrcpy(&msg.m_slotname, NameStr(repSlotStat->slotname));
-	msg.m_create = false;
-	msg.m_drop = false;
-	msg.m_spill_txns = repSlotStat->spill_txns;
-	msg.m_spill_count = repSlotStat->spill_count;
-	msg.m_spill_bytes = repSlotStat->spill_bytes;
-	msg.m_stream_txns = repSlotStat->stream_txns;
-	msg.m_stream_count = repSlotStat->stream_count;
-	msg.m_stream_bytes = repSlotStat->stream_bytes;
-	msg.m_total_txns = repSlotStat->total_txns;
-	msg.m_total_bytes = repSlotStat->total_bytes;
-	pgstat_send(&msg, sizeof(PgStat_MsgReplSlot));
-}
-
-/* ----------
- * pgstat_report_replslot_create() -
- *
- *	Tell the collector about creating the replication slot.
- * ----------
- */
-void
-pgstat_report_replslot_create(const char *slotname)
-{
-	PgStat_MsgReplSlot msg;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_REPLSLOT);
-	namestrcpy(&msg.m_slotname, slotname);
-	msg.m_create = true;
-	msg.m_drop = false;
-	pgstat_send(&msg, sizeof(PgStat_MsgReplSlot));
-}
-
-/* ----------
- * pgstat_report_replslot_drop() -
- *
- *	Tell the collector about dropping the replication slot.
- * ----------
- */
-void
-pgstat_report_replslot_drop(const char *slotname)
-{
-	PgStat_MsgReplSlot msg;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_REPLSLOT);
-	namestrcpy(&msg.m_slotname, slotname);
-	msg.m_create = false;
-	msg.m_drop = true;
-	pgstat_send(&msg, sizeof(PgStat_MsgReplSlot));
-}
-
-/* ----------
- * pgstat_report_subscription_error() -
- *
- *	Tell the collector about the subscription error.
- * ----------
- */
-void
-pgstat_report_subscription_error(Oid subid, bool is_apply_error)
-{
-	PgStat_MsgSubscriptionError msg;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_SUBSCRIPTIONERROR);
-	msg.m_subid = subid;
-	msg.m_is_apply_error = is_apply_error;
-	pgstat_send(&msg, sizeof(PgStat_MsgSubscriptionError));
-}
-
-/* ----------
- * pgstat_report_subscription_drop() -
- *
- *	Tell the collector about dropping the subscription.
- * ----------
- */
-void
-pgstat_report_subscription_drop(Oid subid)
-{
-	PgStat_MsgSubscriptionDrop msg;
-
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_SUBSCRIPTIONDROP);
-	msg.m_subid = subid;
-	pgstat_send(&msg, sizeof(PgStat_MsgSubscriptionDrop));
-}
-
 /* ----------
  * pgstat_ping() -
  *
@@ -2037,305 +1109,11 @@ pgstat_send_inquiry(TimestampTz clock_time, TimestampTz cutoff_time, Oid databas
 	pgstat_send(&msg, sizeof(msg));
 }
 
-
-/*
- * Initialize function call usage data.
- * Called by the executor before invoking a function.
- */
-void
-pgstat_init_function_usage(FunctionCallInfo fcinfo,
-						   PgStat_FunctionCallUsage *fcu)
-{
-	PgStat_BackendFunctionEntry *htabent;
-	bool		found;
-
-	if (pgstat_track_functions <= fcinfo->flinfo->fn_stats)
-	{
-		/* stats not wanted */
-		fcu->fs = NULL;
-		return;
-	}
-
-	if (!pgStatFunctions)
-	{
-		/* First time through - initialize function stat table */
-		HASHCTL		hash_ctl;
-
-		hash_ctl.keysize = sizeof(Oid);
-		hash_ctl.entrysize = sizeof(PgStat_BackendFunctionEntry);
-		pgStatFunctions = hash_create("Function stat entries",
-									  PGSTAT_FUNCTION_HASH_SIZE,
-									  &hash_ctl,
-									  HASH_ELEM | HASH_BLOBS);
-	}
-
-	/* Get the stats entry for this function, create if necessary */
-	htabent = hash_search(pgStatFunctions, &fcinfo->flinfo->fn_oid,
-						  HASH_ENTER, &found);
-	if (!found)
-		MemSet(&htabent->f_counts, 0, sizeof(PgStat_FunctionCounts));
-
-	fcu->fs = &htabent->f_counts;
-
-	/* save stats for this function, later used to compensate for recursion */
-	fcu->save_f_total_time = htabent->f_counts.f_total_time;
-
-	/* save current backend-wide total time */
-	fcu->save_total = total_func_time;
-
-	/* get clock time as of function start */
-	INSTR_TIME_SET_CURRENT(fcu->f_start);
-}
-
-/*
- * find_funcstat_entry - find any existing PgStat_BackendFunctionEntry entry
- *		for specified function
- *
- * If no entry, return NULL, don't create a new one
- */
-PgStat_BackendFunctionEntry *
-find_funcstat_entry(Oid func_id)
-{
-	pgstat_assert_is_up();
-
-	if (pgStatFunctions == NULL)
-		return NULL;
-
-	return (PgStat_BackendFunctionEntry *) hash_search(pgStatFunctions,
-													   (void *) &func_id,
-													   HASH_FIND, NULL);
-}
-
-/*
- * Calculate function call usage and update stat counters.
- * Called by the executor after invoking a function.
- *
- * In the case of a set-returning function that runs in value-per-call mode,
- * we will see multiple pgstat_init_function_usage/pgstat_end_function_usage
- * calls for what the user considers a single call of the function.  The
- * finalize flag should be TRUE on the last call.
- */
-void
-pgstat_end_function_usage(PgStat_FunctionCallUsage *fcu, bool finalize)
-{
-	PgStat_FunctionCounts *fs = fcu->fs;
-	instr_time	f_total;
-	instr_time	f_others;
-	instr_time	f_self;
-
-	/* stats not wanted? */
-	if (fs == NULL)
-		return;
-
-	/* total elapsed time in this function call */
-	INSTR_TIME_SET_CURRENT(f_total);
-	INSTR_TIME_SUBTRACT(f_total, fcu->f_start);
-
-	/* self usage: elapsed minus anything already charged to other calls */
-	f_others = total_func_time;
-	INSTR_TIME_SUBTRACT(f_others, fcu->save_total);
-	f_self = f_total;
-	INSTR_TIME_SUBTRACT(f_self, f_others);
-
-	/* update backend-wide total time */
-	INSTR_TIME_ADD(total_func_time, f_self);
-
-	/*
-	 * Compute the new f_total_time as the total elapsed time added to the
-	 * pre-call value of f_total_time.  This is necessary to avoid
-	 * double-counting any time taken by recursive calls of myself.  (We do
-	 * not need any similar kluge for self time, since that already excludes
-	 * any recursive calls.)
-	 */
-	INSTR_TIME_ADD(f_total, fcu->save_f_total_time);
-
-	/* update counters in function stats table */
-	if (finalize)
-		fs->f_numcalls++;
-	fs->f_total_time = f_total;
-	INSTR_TIME_ADD(fs->f_self_time, f_self);
-
-	/* indicate that we have something to send */
-	have_function_stats = true;
-}
-
-
-/* ----------
- * pgstat_relation_init() -
- *
- *	Initialize a relcache entry to count access statistics.
- *	Called whenever a relation is opened.
- *
- *	We assume that a relcache entry's pgstat_info field is zeroed by
- *	relcache.c when the relcache entry is made; thereafter it is long-lived
- *	data.  We can avoid repeated searches of the TabStatus arrays when the
- *	same relation is touched repeatedly within a transaction.
- * ----------
- */
-void
-pgstat_relation_init(Relation rel)
-{
-	Oid			rel_id = rel->rd_id;
-	char		relkind = rel->rd_rel->relkind;
-
-	/*
-	 * We only count stats for relations with storage and partitioned tables
-	 */
-	if (!RELKIND_HAS_STORAGE(relkind) && relkind != RELKIND_PARTITIONED_TABLE)
-	{
-		rel->pgstat_enabled = false;
-		rel->pgstat_info = NULL;
-		return;
-	}
-
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
-	{
-		/* We're not counting at all */
-		rel->pgstat_enabled = false;
-		rel->pgstat_info = NULL;
-		return;
-	}
-
-	/*
-	 * If we already set up this relation in the current transaction, nothing
-	 * to do.
-	 */
-	if (rel->pgstat_info &&
-		rel->pgstat_info->t_id == rel_id)
-		return;
-
-	rel->pgstat_enabled = true;
-}
-
-void
-pgstat_relation_assoc(Relation rel)
-{
-	Oid			rel_id = rel->rd_id;
-
-	Assert(rel->pgstat_enabled);
-	Assert(rel->pgstat_info == NULL);
-
-	/* find or make the PgStat_TableStatus entry, and update link */
-	rel->pgstat_info = get_tabstat_entry(rel_id, rel->rd_rel->relisshared);
-}
-
-/*
- * get_tabstat_entry - find or create a PgStat_TableStatus entry for rel
- */
-static PgStat_TableStatus *
-get_tabstat_entry(Oid rel_id, bool isshared)
-{
-	TabStatHashEntry *hash_entry;
-	PgStat_TableStatus *entry;
-	TabStatusArray *tsa;
-	bool		found;
-
-	pgstat_assert_is_up();
-
-	have_relation_stats = true;
-
-	/*
-	 * Create hash table if we don't have it already.
-	 */
-	if (pgStatTabHash == NULL)
-	{
-		HASHCTL		ctl;
-
-		ctl.keysize = sizeof(Oid);
-		ctl.entrysize = sizeof(TabStatHashEntry);
-
-		pgStatTabHash = hash_create("pgstat TabStatusArray lookup hash table",
-									TABSTAT_QUANTUM,
-									&ctl,
-									HASH_ELEM | HASH_BLOBS);
-	}
-
-	/*
-	 * Find an entry or create a new one.
-	 */
-	hash_entry = hash_search(pgStatTabHash, &rel_id, HASH_ENTER, &found);
-	if (!found)
-	{
-		/* initialize new entry with null pointer */
-		hash_entry->tsa_entry = NULL;
-	}
-
-	/*
-	 * If entry is already valid, we're done.
-	 */
-	if (hash_entry->tsa_entry)
-		return hash_entry->tsa_entry;
-
-	/*
-	 * Locate the first pgStatTabList entry with free space, making a new list
-	 * entry if needed.  Note that we could get an OOM failure here, but if so
-	 * we have left the hashtable and the list in a consistent state.
-	 */
-	if (pgStatTabList == NULL)
-	{
-		/* Set up first pgStatTabList entry */
-		pgStatTabList = (TabStatusArray *)
-			MemoryContextAllocZero(TopMemoryContext,
-								   sizeof(TabStatusArray));
-	}
-
-	tsa = pgStatTabList;
-	while (tsa->tsa_used >= TABSTAT_QUANTUM)
-	{
-		if (tsa->tsa_next == NULL)
-			tsa->tsa_next = (TabStatusArray *)
-				MemoryContextAllocZero(TopMemoryContext,
-									   sizeof(TabStatusArray));
-		tsa = tsa->tsa_next;
-	}
-
-	/*
-	 * Allocate a PgStat_TableStatus entry within this list entry.  We assume
-	 * the entry was already zeroed, either at creation or after last use.
-	 */
-	entry = &tsa->tsa_entries[tsa->tsa_used++];
-	entry->t_id = rel_id;
-	entry->t_shared = isshared;
-
-	/*
-	 * Now we can fill the entry in pgStatTabHash.
-	 */
-	hash_entry->tsa_entry = entry;
-
-	return entry;
-}
-
-/*
- * find_tabstat_entry - find any existing PgStat_TableStatus entry for rel
- *
- * If no entry, return NULL, don't create a new one
- *
- * Note: if we got an error in the most recent execution of pgstat_report_stat,
- * it's possible that an entry exists but there's no hashtable entry for it.
- * That's okay, we'll treat this case as "doesn't exist".
- */
-PgStat_TableStatus *
-find_tabstat_entry(Oid rel_id)
-{
-	TabStatHashEntry *hash_entry;
-
-	/* If hashtable doesn't exist, there are no entries at all */
-	if (!pgStatTabHash)
-		return NULL;
-
-	hash_entry = hash_search(pgStatTabHash, &rel_id, HASH_FIND, NULL);
-	if (!hash_entry)
-		return NULL;
-
-	/* Note that this step could also return NULL, but that's correct */
-	return hash_entry->tsa_entry;
-}
-
 /*
  * Ensure (sub)transaction stack entry for the given nest_level exists, adding
  * it if needed.
  */
-static PgStat_SubXactStatus *
+PgStat_SubXactStatus *
 pgstat_xact_stack_level_get(int nest_level)
 {
 	PgStat_SubXactStatus *xact_state;
@@ -2354,249 +1132,6 @@ pgstat_xact_stack_level_get(int nest_level)
 	return xact_state;
 }
 
-/*
- * add_tabstat_xact_level - add a new (sub)transaction state record
- */
-static void
-add_tabstat_xact_level(PgStat_TableStatus *pgstat_info, int nest_level)
-{
-	PgStat_SubXactStatus *xact_state;
-	PgStat_TableXactStatus *trans;
-
-	/*
-	 * If this is the first rel to be modified at the current nest level, we
-	 * first have to push a transaction stack entry.
-	 */
-	xact_state = pgstat_xact_stack_level_get(nest_level);
-
-	/* Now make a per-table stack entry */
-	trans = (PgStat_TableXactStatus *)
-		MemoryContextAllocZero(TopTransactionContext,
-							   sizeof(PgStat_TableXactStatus));
-	trans->nest_level = nest_level;
-	trans->upper = pgstat_info->trans;
-	trans->parent = pgstat_info;
-	trans->next = xact_state->first;
-	xact_state->first = trans;
-	pgstat_info->trans = trans;
-}
-
-/*
- * Add a new (sub)transaction record if needed.
- */
-static void
-ensure_tabstat_xact_level(PgStat_TableStatus *pgstat_info)
-{
-	int			nest_level = GetCurrentTransactionNestLevel();
-
-	if (pgstat_info->trans == NULL ||
-		pgstat_info->trans->nest_level != nest_level)
-		add_tabstat_xact_level(pgstat_info, nest_level);
-}
-
-/*
- * pgstat_count_heap_insert - count a tuple insertion of n tuples
- */
-void
-pgstat_count_heap_insert(Relation rel, PgStat_Counter n)
-{
-
-	if (pgstat_relation_should_count(rel))
-	{
-		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
-
-		ensure_tabstat_xact_level(pgstat_info);
-		pgstat_info->trans->tuples_inserted += n;
-	}
-}
-
-/*
- * pgstat_count_heap_update - count a tuple update
- */
-void
-pgstat_count_heap_update(Relation rel, bool hot)
-{
-
-	if (pgstat_relation_should_count(rel))
-	{
-		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
-
-		ensure_tabstat_xact_level(pgstat_info);
-		pgstat_info->trans->tuples_updated++;
-
-		/* t_tuples_hot_updated is nontransactional, so just advance it */
-		if (hot)
-			pgstat_info->t_counts.t_tuples_hot_updated++;
-	}
-}
-
-/*
- * pgstat_count_heap_delete - count a tuple deletion
- */
-void
-pgstat_count_heap_delete(Relation rel)
-{
-
-	if (pgstat_relation_should_count(rel))
-	{
-		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
-
-		ensure_tabstat_xact_level(pgstat_info);
-		pgstat_info->trans->tuples_deleted++;
-	}
-}
-
-/*
- * pgstat_truncdrop_save_counters
- *
- * Whenever a table is truncated/dropped, we save its i/u/d counters so that
- * they can be cleared, and if the (sub)xact that executed the truncate/drop
- * later aborts, the counters can be restored to the saved (pre-truncate/drop)
- * values.
- *
- * Note that for truncate we do this on the first truncate in any particular
- * subxact level only.
- */
-static void
-pgstat_truncdrop_save_counters(PgStat_TableXactStatus *trans, bool is_drop)
-{
-	if (!trans->truncdropped || is_drop)
-	{
-		trans->inserted_pre_truncdrop = trans->tuples_inserted;
-		trans->updated_pre_truncdrop = trans->tuples_updated;
-		trans->deleted_pre_truncdrop = trans->tuples_deleted;
-		trans->truncdropped = true;
-	}
-}
-
-/*
- * pgstat_truncdrop_restore_counters - restore counters when a truncate aborts
- */
-static void
-pgstat_truncdrop_restore_counters(PgStat_TableXactStatus *trans)
-{
-	if (trans->truncdropped)
-	{
-		trans->tuples_inserted = trans->inserted_pre_truncdrop;
-		trans->tuples_updated = trans->updated_pre_truncdrop;
-		trans->tuples_deleted = trans->deleted_pre_truncdrop;
-	}
-}
-
-/*
- * pgstat_count_truncate - update tuple counters due to truncate
- */
-void
-pgstat_count_truncate(Relation rel)
-{
-
-	if (pgstat_relation_should_count(rel))
-	{
-		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
-
-		ensure_tabstat_xact_level(pgstat_info);
-		pgstat_truncdrop_save_counters(pgstat_info->trans, false);
-		pgstat_info->trans->tuples_inserted = 0;
-		pgstat_info->trans->tuples_updated = 0;
-		pgstat_info->trans->tuples_deleted = 0;
-	}
-}
-
-/*
- * pgstat_update_heap_dead_tuples - update dead-tuples count
- *
- * The semantics of this are that we are reporting the nontransactional
- * recovery of "delta" dead tuples; so t_delta_dead_tuples decreases
- * rather than increasing, and the change goes straight into the per-table
- * counter, not into transactional state.
- */
-void
-pgstat_update_heap_dead_tuples(Relation rel, int delta)
-{
-	if (pgstat_relation_should_count(rel))
-	{
-		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
-
-		pgstat_info->t_counts.t_delta_dead_tuples -= delta;
-	}
-}
-
-/*
- * Perform relation stats specific end-of-transaction work. Helper for
- * AtEOXact_PgStat.
- *
- * Transfer transactional insert/update counts into the base tabstat entries.
- * We don't bother to free any of the transactional state, since it's all in
- * TopTransactionContext and will go away anyway.
- */
-static void
-AtEOXact_PgStat_Relations(PgStat_SubXactStatus *xact_state, bool isCommit)
-{
-	PgStat_TableXactStatus *trans;
-
-	for (trans = xact_state->first; trans != NULL; trans = trans->next)
-	{
-		PgStat_TableStatus *tabstat;
-
-		Assert(trans->nest_level == 1);
-		Assert(trans->upper == NULL);
-		tabstat = trans->parent;
-		Assert(tabstat->trans == trans);
-		/* restore pre-truncate/drop stats (if any) in case of aborted xact */
-		if (!isCommit)
-			pgstat_truncdrop_restore_counters(trans);
-		/* count attempted actions regardless of commit/abort */
-		tabstat->t_counts.t_tuples_inserted += trans->tuples_inserted;
-		tabstat->t_counts.t_tuples_updated += trans->tuples_updated;
-		tabstat->t_counts.t_tuples_deleted += trans->tuples_deleted;
-		if (isCommit)
-		{
-			tabstat->t_counts.t_truncdropped = trans->truncdropped;
-			if (trans->truncdropped)
-			{
-				/* forget live/dead stats seen by backend thus far */
-				tabstat->t_counts.t_delta_live_tuples = 0;
-				tabstat->t_counts.t_delta_dead_tuples = 0;
-			}
-			/* insert adds a live tuple, delete removes one */
-			tabstat->t_counts.t_delta_live_tuples +=
-				trans->tuples_inserted - trans->tuples_deleted;
-			/* update and delete each create a dead tuple */
-			tabstat->t_counts.t_delta_dead_tuples +=
-				trans->tuples_updated + trans->tuples_deleted;
-			/* insert, update, delete each count as one change event */
-			tabstat->t_counts.t_changed_tuples +=
-				trans->tuples_inserted + trans->tuples_updated +
-				trans->tuples_deleted;
-		}
-		else
-		{
-			/* inserted tuples are dead, deleted tuples are unaffected */
-			tabstat->t_counts.t_delta_dead_tuples +=
-				trans->tuples_inserted + trans->tuples_updated;
-			/* an aborted xact generates no changed_tuple events */
-		}
-		tabstat->trans = NULL;
-	}
-}
-
-static void
-AtEOXact_PgStat_Database(bool isCommit, bool parallel)
-{
-	/* Don't count parallel worker transaction stats */
-	if (!parallel)
-	{
-		/*
-		 * Count transaction commit or abort.  (We use counters, not just
-		 * bools, in case the reporting message isn't sent right away.)
-		 */
-		if (isCommit)
-			pgStatXactCommit++;
-		else
-			pgStatXactRollback++;
-	}
-}
-
 /* ----------
  * AtEOXact_PgStat
  *
@@ -2625,90 +1160,6 @@ AtEOXact_PgStat(bool isCommit, bool parallel)
 	pgstat_clear_snapshot();
 }
 
-/*
- * Perform relation stats specific end-of-sub-transaction work. Helper for
- * AtEOSubXact_PgStat.
- *
- * Transfer transactional insert/update counts into the next higher
- * subtransaction state.
- */
-static void
-AtEOSubXact_PgStat_Relations(PgStat_SubXactStatus *xact_state, bool isCommit, int nestDepth)
-{
-	PgStat_TableXactStatus *trans;
-	PgStat_TableXactStatus *next_trans;
-
-	for (trans = xact_state->first; trans != NULL; trans = next_trans)
-	{
-		PgStat_TableStatus *tabstat;
-
-		next_trans = trans->next;
-		Assert(trans->nest_level == nestDepth);
-		tabstat = trans->parent;
-		Assert(tabstat->trans == trans);
-
-		if (isCommit)
-		{
-			if (trans->upper && trans->upper->nest_level == nestDepth - 1)
-			{
-				if (trans->truncdropped)
-				{
-					/* propagate the truncate/drop status one level up */
-					pgstat_truncdrop_save_counters(trans->upper, false);
-					/* replace upper xact stats with ours */
-					trans->upper->tuples_inserted = trans->tuples_inserted;
-					trans->upper->tuples_updated = trans->tuples_updated;
-					trans->upper->tuples_deleted = trans->tuples_deleted;
-				}
-				else
-				{
-					trans->upper->tuples_inserted += trans->tuples_inserted;
-					trans->upper->tuples_updated += trans->tuples_updated;
-					trans->upper->tuples_deleted += trans->tuples_deleted;
-				}
-				tabstat->trans = trans->upper;
-				pfree(trans);
-			}
-			else
-			{
-				/*
-				 * When there isn't an immediate parent state, we can just
-				 * reuse the record instead of going through a
-				 * palloc/pfree pushup (this works since it's all in
-				 * TopTransactionContext anyway).  We have to re-link it
-				 * into the parent level, though, and that might mean
-				 * pushing a new entry into the pgStatXactStack.
-				 */
-				PgStat_SubXactStatus *upper_xact_state;
-
-				upper_xact_state = pgstat_xact_stack_level_get(nestDepth - 1);
-				trans->next = upper_xact_state->first;
-				upper_xact_state->first = trans;
-				trans->nest_level = nestDepth - 1;
-			}
-		}
-		else
-		{
-			/*
-			 * On abort, update top-level tabstat counts, then forget the
-			 * subtransaction
-			 */
-
-			/* first restore values obliterated by truncate/drop */
-			pgstat_truncdrop_restore_counters(trans);
-			/* count attempted actions regardless of commit/abort */
-			tabstat->t_counts.t_tuples_inserted += trans->tuples_inserted;
-			tabstat->t_counts.t_tuples_updated += trans->tuples_updated;
-			tabstat->t_counts.t_tuples_deleted += trans->tuples_deleted;
-			/* inserted tuples are dead, deleted tuples are unaffected */
-			tabstat->t_counts.t_delta_dead_tuples +=
-				trans->tuples_inserted + trans->tuples_updated;
-			tabstat->trans = trans->upper;
-			pfree(trans);
-		}
-	}
-}
-
 /* ----------
  * AtEOSubXact_PgStat
  *
@@ -2734,40 +1185,6 @@ AtEOSubXact_PgStat(bool isCommit, int nestDepth)
 	}
 }
 
-/*
- * Generate 2PC records for all the pending transaction-dependent relation
- * stats.
- */
-static void
-AtPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state)
-{
-	PgStat_TableXactStatus *trans;
-
-	for (trans = xact_state->first; trans != NULL; trans = trans->next)
-	{
-		PgStat_TableStatus *tabstat;
-		TwoPhasePgStatRecord record;
-
-		Assert(trans->nest_level == 1);
-		Assert(trans->upper == NULL);
-		tabstat = trans->parent;
-		Assert(tabstat->trans == trans);
-
-		record.tuples_inserted = trans->tuples_inserted;
-		record.tuples_updated = trans->tuples_updated;
-		record.tuples_deleted = trans->tuples_deleted;
-		record.inserted_pre_truncdrop = trans->inserted_pre_truncdrop;
-		record.updated_pre_truncdrop = trans->updated_pre_truncdrop;
-		record.deleted_pre_truncdrop = trans->deleted_pre_truncdrop;
-		record.t_id = tabstat->t_id;
-		record.t_shared = tabstat->t_shared;
-		record.t_truncdropped = trans->truncdropped;
-
-		RegisterTwoPhaseRecord(TWOPHASE_RM_PGSTAT_ID, 0,
-							   &record, sizeof(TwoPhasePgStatRecord));
-	}
-}
-
 /*
  * AtPrepare_PgStat
  *		Save the transactional stats state at 2PC transaction prepare.
@@ -2787,28 +1204,6 @@ AtPrepare_PgStat(void)
 	}
 }
 
-/*
- * All we need do here is unlink the transaction stats state from the
- * nontransactional state.  The nontransactional action counts will be
- * reported to the stats collector immediately, while the effects on
- * live and dead tuple counts are preserved in the 2PC state file.
- *
- * Note: AtEOXact_PgStat_Relations is not called during PREPARE.
- */
-static void
-PostPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state)
-{
-	PgStat_TableXactStatus *trans;
-
-	for (trans = xact_state->first; trans != NULL; trans = trans->next)
-	{
-		PgStat_TableStatus *tabstat;
-
-		tabstat = trans->parent;
-		tabstat->trans = NULL;
-	}
-}
-
 /*
  * PostPrepare_PgStat
  *		Clean up after successful PREPARE.
@@ -2838,72 +1233,6 @@ PostPrepare_PgStat(void)
 	pgstat_clear_snapshot();
 }
 
-/*
- * 2PC processing routine for COMMIT PREPARED case.
- *
- * Load the saved counts into our local pgstats state.
- */
-void
-pgstat_twophase_postcommit(TransactionId xid, uint16 info,
-						   void *recdata, uint32 len)
-{
-	TwoPhasePgStatRecord *rec = (TwoPhasePgStatRecord *) recdata;
-	PgStat_TableStatus *pgstat_info;
-
-	/* Find or create a tabstat entry for the rel */
-	pgstat_info = get_tabstat_entry(rec->t_id, rec->t_shared);
-
-	/* Same math as in AtEOXact_PgStat, commit case */
-	pgstat_info->t_counts.t_tuples_inserted += rec->tuples_inserted;
-	pgstat_info->t_counts.t_tuples_updated += rec->tuples_updated;
-	pgstat_info->t_counts.t_tuples_deleted += rec->tuples_deleted;
-	pgstat_info->t_counts.t_truncdropped = rec->t_truncdropped;
-	if (rec->t_truncdropped)
-	{
-		/* forget live/dead stats seen by backend thus far */
-		pgstat_info->t_counts.t_delta_live_tuples = 0;
-		pgstat_info->t_counts.t_delta_dead_tuples = 0;
-	}
-	pgstat_info->t_counts.t_delta_live_tuples +=
-		rec->tuples_inserted - rec->tuples_deleted;
-	pgstat_info->t_counts.t_delta_dead_tuples +=
-		rec->tuples_updated + rec->tuples_deleted;
-	pgstat_info->t_counts.t_changed_tuples +=
-		rec->tuples_inserted + rec->tuples_updated +
-		rec->tuples_deleted;
-}
-
-/*
- * 2PC processing routine for ROLLBACK PREPARED case.
- *
- * Load the saved counts into our local pgstats state, but treat them
- * as aborted.
- */
-void
-pgstat_twophase_postabort(TransactionId xid, uint16 info,
-						  void *recdata, uint32 len)
-{
-	TwoPhasePgStatRecord *rec = (TwoPhasePgStatRecord *) recdata;
-	PgStat_TableStatus *pgstat_info;
-
-	/* Find or create a tabstat entry for the rel */
-	pgstat_info = get_tabstat_entry(rec->t_id, rec->t_shared);
-
-	/* Same math as in AtEOXact_PgStat, abort case */
-	if (rec->t_truncdropped)
-	{
-		rec->tuples_inserted = rec->inserted_pre_truncdrop;
-		rec->tuples_updated = rec->updated_pre_truncdrop;
-		rec->tuples_deleted = rec->deleted_pre_truncdrop;
-	}
-	pgstat_info->t_counts.t_tuples_inserted += rec->tuples_inserted;
-	pgstat_info->t_counts.t_tuples_updated += rec->tuples_updated;
-	pgstat_info->t_counts.t_tuples_deleted += rec->tuples_deleted;
-	pgstat_info->t_counts.t_delta_dead_tuples +=
-		rec->tuples_inserted + rec->tuples_updated;
-}
-
-
 /* ----------
  * pgstat_fetch_stat_dbentry() -
  *
@@ -3208,7 +1537,7 @@ pgstat_initialize(void)
  *		Set common header fields in a statistics message
  * ----------
  */
-static void
+void
 pgstat_setheader(PgStat_MsgHdr *hdr, StatMsgType mtype)
 {
 	hdr->m_type = mtype;
@@ -3221,7 +1550,7 @@ pgstat_setheader(PgStat_MsgHdr *hdr, StatMsgType mtype)
  *		Send out one statistics message to the collector
  * ----------
  */
-static void
+void
 pgstat_send(void *msg, int len)
 {
 	int			rc;
@@ -3246,237 +1575,6 @@ pgstat_send(void *msg, int len)
 #endif
 }
 
-/* ----------
- * pgstat_send_archiver() -
- *
- *	Tell the collector about the WAL file that we successfully
- *	archived or failed to archive.
- * ----------
- */
-void
-pgstat_send_archiver(const char *xlog, bool failed)
-{
-	PgStat_MsgArchiver msg;
-
-	/*
-	 * Prepare and send the message
-	 */
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_ARCHIVER);
-	msg.m_failed = failed;
-	strlcpy(msg.m_xlog, xlog, sizeof(msg.m_xlog));
-	msg.m_timestamp = GetCurrentTimestamp();
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* ----------
- * pgstat_send_bgwriter() -
- *
- *		Send bgwriter statistics to the collector
- * ----------
- */
-void
-pgstat_send_bgwriter(void)
-{
-	/* We assume this initializes to zeroes */
-	static const PgStat_MsgBgWriter all_zeroes;
-
-	pgstat_assert_is_up();
-
-	/*
-	 * 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(&PendingBgWriterStats, &all_zeroes, sizeof(PgStat_MsgBgWriter)) == 0)
-		return;
-
-	/*
-	 * Prepare and send the message
-	 */
-	pgstat_setheader(&PendingBgWriterStats.m_hdr, PGSTAT_MTYPE_BGWRITER);
-	pgstat_send(&PendingBgWriterStats, sizeof(PendingBgWriterStats));
-
-	/*
-	 * Clear out the statistics buffer, so it can be re-used.
-	 */
-	MemSet(&PendingBgWriterStats, 0, sizeof(PendingBgWriterStats));
-}
-
-/* ----------
- * pgstat_send_checkpointer() -
- *
- *		Send checkpointer statistics to the collector
- * ----------
- */
-void
-pgstat_send_checkpointer(void)
-{
-	/* We assume this initializes to zeroes */
-	static const PgStat_MsgCheckpointer all_zeroes;
-
-	/*
-	 * 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(&PendingCheckpointerStats, &all_zeroes, sizeof(PgStat_MsgCheckpointer)) == 0)
-		return;
-
-	/*
-	 * Prepare and send the message
-	 */
-	pgstat_setheader(&PendingCheckpointerStats.m_hdr, PGSTAT_MTYPE_CHECKPOINTER);
-	pgstat_send(&PendingCheckpointerStats, sizeof(PendingCheckpointerStats));
-
-	/*
-	 * Clear out the statistics buffer, so it can be re-used.
-	 */
-	MemSet(&PendingCheckpointerStats, 0, sizeof(PendingCheckpointerStats));
-}
-
-/* ----------
- * pgstat_send_wal() -
- *
- *	Send WAL statistics to the collector.
- *
- * If 'force' is not set, WAL stats message is only sent if enough time has
- * passed since last one was sent to reach PGSTAT_STAT_INTERVAL.
- * ----------
- */
-void
-pgstat_send_wal(bool force)
-{
-	static TimestampTz sendTime = 0;
-
-	/*
-	 * 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.
-	 *
-	 * Check wal_records counter to determine whether any WAL activity has
-	 * happened since last time. Note that other WalUsage counters don't need
-	 * to be checked because they are incremented always together with
-	 * wal_records counter.
-	 *
-	 * m_wal_buffers_full also doesn't need to be checked because it's
-	 * incremented only when at least one WAL record is generated (i.e.,
-	 * wal_records counter is incremented). But for safely, we assert that
-	 * m_wal_buffers_full is always zero when no WAL record is generated
-	 *
-	 * This function can be called by a process like walwriter that normally
-	 * generates no WAL records. To determine whether any WAL activity has
-	 * happened at that process since the last time, the numbers of WAL writes
-	 * and syncs are also checked.
-	 */
-	if (pgWalUsage.wal_records == prevWalUsage.wal_records &&
-		WalStats.m_wal_write == 0 && WalStats.m_wal_sync == 0)
-	{
-		Assert(WalStats.m_wal_buffers_full == 0);
-		return;
-	}
-
-	if (!force)
-	{
-		TimestampTz now = GetCurrentTimestamp();
-
-		/*
-		 * Don't send a message unless it's been at least PGSTAT_STAT_INTERVAL
-		 * msec since we last sent one to avoid overloading the stats
-		 * collector.
-		 */
-		if (!TimestampDifferenceExceeds(sendTime, now, PGSTAT_STAT_INTERVAL))
-			return;
-		sendTime = now;
-	}
-
-	/*
-	 * Set the counters related to generated WAL data if the counters were
-	 * updated.
-	 */
-	if (pgWalUsage.wal_records != prevWalUsage.wal_records)
-	{
-		WalUsage	walusage;
-
-		/*
-		 * Calculate how much WAL usage counters were increased by
-		 * subtracting the previous counters from the current ones. Fill the
-		 * results in WAL stats message.
-		 */
-		MemSet(&walusage, 0, sizeof(WalUsage));
-		WalUsageAccumDiff(&walusage, &pgWalUsage, &prevWalUsage);
-
-		WalStats.m_wal_records = walusage.wal_records;
-		WalStats.m_wal_fpi = walusage.wal_fpi;
-		WalStats.m_wal_bytes = walusage.wal_bytes;
-
-		/*
-		 * Save the current counters for the subsequent calculation of WAL
-		 * usage.
-		 */
-		prevWalUsage = pgWalUsage;
-	}
-
-	/*
-	 * Prepare and send the message
-	 */
-	pgstat_setheader(&WalStats.m_hdr, PGSTAT_MTYPE_WAL);
-	pgstat_send(&WalStats, sizeof(WalStats));
-
-	/*
-	 * Clear out the statistics buffer, so it can be re-used.
-	 */
-	MemSet(&WalStats, 0, sizeof(WalStats));
-}
-
-static void
-pgstat_wal_initialize(void)
-{
-	/*
-	 * Initialize prevWalUsage with pgWalUsage so that pgstat_send_wal() can
-	 * calculate how much pgWalUsage counters are increased by subtracting
-	 * prevWalUsage from pgWalUsage.
-	 */
-	prevWalUsage = pgWalUsage;
-}
-
-/* ----------
- * pgstat_send_slru() -
- *
- *		Send SLRU statistics to the collector
- * ----------
- */
-static void
-pgstat_send_slru(void)
-{
-	/* We assume this initializes to zeroes */
-	static const PgStat_MsgSLRU all_zeroes;
-
-	for (int 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() -
  *
@@ -5093,11 +3191,13 @@ pgstat_setup_memcxt(void)
  * pgstat_shutdown(). This check is put in a few central places to catch
  * violations of this rule more easily.
  */
-static void
+#ifdef USE_ASSERT_CHECKING
+void
 pgstat_assert_is_up(void)
 {
 	Assert(pgstat_is_initialized && !pgstat_is_shutdown);
 }
+#endif
 
 
 /* ----------
@@ -6327,109 +4427,3 @@ pgstat_reset_subscription(PgStat_StatSubEntry *subentry, TimestampTz ts)
 	subentry->sync_error_count = 0;
 	subentry->stat_reset_timestamp = ts;
 }
-
-/*
- * 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 projects.
- */
-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.
- */
-const char *
-pgstat_slru_name(int slru_idx)
-{
-	if (slru_idx < 0 || slru_idx >= SLRU_NUM_ELEMENTS)
-		return NULL;
-
-	return slru_names[slru_idx];
-}
-
-/*
- * slru_entry
- *
- * Returns pointer to entry with counters for given SLRU (based on the name
- * stored in SlruCtl as lwlock tranche name).
- */
-static inline PgStat_MsgSLRU *
-slru_entry(int slru_idx)
-{
-	pgstat_assert_is_up();
-
-	/*
-	 * The postmaster should never register any SLRU statistics counts; if it
-	 * did, the counts would be duplicated into child processes via fork().
-	 */
-	Assert(IsUnderPostmaster || !IsPostmasterEnvironment);
-
-	Assert((slru_idx >= 0) && (slru_idx < SLRU_NUM_ELEMENTS));
-
-	return &SLRUStats[slru_idx];
-}
-
-/*
- * SLRU statistics count accumulation functions --- called from slru.c
- */
-
-void
-pgstat_count_slru_page_zeroed(int slru_idx)
-{
-	slru_entry(slru_idx)->m_blocks_zeroed += 1;
-}
-
-void
-pgstat_count_slru_page_hit(int slru_idx)
-{
-	slru_entry(slru_idx)->m_blocks_hit += 1;
-}
-
-void
-pgstat_count_slru_page_exists(int slru_idx)
-{
-	slru_entry(slru_idx)->m_blocks_exists += 1;
-}
-
-void
-pgstat_count_slru_page_read(int slru_idx)
-{
-	slru_entry(slru_idx)->m_blocks_read += 1;
-}
-
-void
-pgstat_count_slru_page_written(int slru_idx)
-{
-	slru_entry(slru_idx)->m_blocks_written += 1;
-}
-
-void
-pgstat_count_slru_flush(int slru_idx)
-{
-	slru_entry(slru_idx)->m_flush += 1;
-}
-
-void
-pgstat_count_slru_truncate(int slru_idx)
-{
-	slru_entry(slru_idx)->m_truncate += 1;
-}
diff --git a/src/backend/utils/activity/Makefile b/src/backend/utils/activity/Makefile
index 425f1d11b84..575de7d507b 100644
--- a/src/backend/utils/activity/Makefile
+++ b/src/backend/utils/activity/Makefile
@@ -16,6 +16,11 @@ include $(top_builddir)/src/Makefile.global
 OBJS = \
 	backend_progress.o \
 	backend_status.o \
+	pgstat_database.o \
+	pgstat_function.o \
+	pgstat_global.o \
+	pgstat_relation.o \
+	pgstat_subscription.o \
 	wait_event.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/utils/activity/pgstat_database.c b/src/backend/utils/activity/pgstat_database.c
new file mode 100644
index 00000000000..7850829e36f
--- /dev/null
+++ b/src/backend/utils/activity/pgstat_database.c
@@ -0,0 +1,279 @@
+/* -------------------------------------------------------------------------
+ *
+ * pgstat_database.c
+ *	  Implementation of database statistics.
+ *
+ * This file contains the implementation of function database. It is kept
+ * separate from pgstat.c to enforce the line between the statistics access /
+ * storage implementation and the details about individual types of
+ * statistics.
+ *
+ * Copyright (c) 2001-2021, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/backend/utils/activity/pgstat_database.c
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "utils/pgstat_internal.h"
+#include "utils/timestamp.h"
+
+
+static bool pgstat_should_report_connstat(void);
+
+
+int	pgStatXactCommit = 0;
+int	pgStatXactRollback = 0;
+PgStat_Counter pgStatBlockReadTime = 0;
+PgStat_Counter pgStatBlockWriteTime = 0;
+PgStat_Counter pgStatActiveTime = 0;
+PgStat_Counter pgStatTransactionIdleTime = 0;
+SessionEndType pgStatSessionEndCause = DISCONNECT_NORMAL;
+
+
+static PgStat_Counter pgLastSessionReportTime = 0;
+
+
+/* ----------
+ * pgstat_drop_database() -
+ *
+ *	Tell the collector that we just dropped a database.
+ *	(If the message gets lost, we will still clean the dead DB eventually
+ *	via future invocations of pgstat_vacuum_stat().)
+ * ----------
+ */
+void
+pgstat_drop_database(Oid databaseid)
+{
+	PgStat_MsgDropdb msg;
+
+	if (pgStatSock == PGINVALID_SOCKET)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_DROPDB);
+	msg.m_databaseid = databaseid;
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* --------
+ * pgstat_report_recovery_conflict() -
+ *
+ *	Tell the collector about a Hot Standby recovery conflict.
+ * --------
+ */
+void
+pgstat_report_recovery_conflict(int reason)
+{
+	PgStat_MsgRecoveryConflict msg;
+
+	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RECOVERYCONFLICT);
+	msg.m_databaseid = MyDatabaseId;
+	msg.m_reason = reason;
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* --------
+ * pgstat_report_deadlock() -
+ *
+ *	Tell the collector about a deadlock detected.
+ * --------
+ */
+void
+pgstat_report_deadlock(void)
+{
+	PgStat_MsgDeadlock msg;
+
+	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_DEADLOCK);
+	msg.m_databaseid = MyDatabaseId;
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* --------
+ * pgstat_report_checksum_failures_in_db() -
+ *
+ *	Tell the collector about one or more checksum failures.
+ * --------
+ */
+void
+pgstat_report_checksum_failures_in_db(Oid dboid, int failurecount)
+{
+	PgStat_MsgChecksumFailure msg;
+
+	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_CHECKSUMFAILURE);
+	msg.m_databaseid = dboid;
+	msg.m_failurecount = failurecount;
+	msg.m_failure_time = GetCurrentTimestamp();
+
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* --------
+ * pgstat_report_checksum_failure() -
+ *
+ *	Tell the collector about a checksum failure.
+ * --------
+ */
+void
+pgstat_report_checksum_failure(void)
+{
+	pgstat_report_checksum_failures_in_db(MyDatabaseId, 1);
+}
+
+/* --------
+ * pgstat_report_tempfile() -
+ *
+ *	Tell the collector about a temporary file.
+ * --------
+ */
+void
+pgstat_report_tempfile(size_t filesize)
+{
+	PgStat_MsgTempFile msg;
+
+	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_TEMPFILE);
+	msg.m_databaseid = MyDatabaseId;
+	msg.m_filesize = filesize;
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/*
+ * Subroutine for pgstat_send_tabstat: Handle xact commit/rollback and I/O
+ * timings.
+ */
+void
+pgstat_update_dbstats(PgStat_MsgTabstat *tsmsg, TimestampTz now)
+{
+	if (OidIsValid(tsmsg->m_databaseid))
+	{
+		tsmsg->m_xact_commit = pgStatXactCommit;
+		tsmsg->m_xact_rollback = pgStatXactRollback;
+		tsmsg->m_block_read_time = pgStatBlockReadTime;
+		tsmsg->m_block_write_time = pgStatBlockWriteTime;
+
+		if (pgstat_should_report_connstat())
+		{
+			long		secs;
+			int			usecs;
+
+			/*
+			 * pgLastSessionReportTime is initialized to MyStartTimestamp by
+			 * pgstat_report_connect().
+			 */
+			TimestampDifference(pgLastSessionReportTime, now, &secs, &usecs);
+			pgLastSessionReportTime = now;
+			tsmsg->m_session_time = (PgStat_Counter) secs * 1000000 + usecs;
+			tsmsg->m_active_time = pgStatActiveTime;
+			tsmsg->m_idle_in_xact_time = pgStatTransactionIdleTime;
+		}
+		else
+		{
+			tsmsg->m_session_time = 0;
+			tsmsg->m_active_time = 0;
+			tsmsg->m_idle_in_xact_time = 0;
+		}
+		pgStatXactCommit = 0;
+		pgStatXactRollback = 0;
+		pgStatBlockReadTime = 0;
+		pgStatBlockWriteTime = 0;
+		pgStatActiveTime = 0;
+		pgStatTransactionIdleTime = 0;
+	}
+	else
+	{
+		tsmsg->m_xact_commit = 0;
+		tsmsg->m_xact_rollback = 0;
+		tsmsg->m_block_read_time = 0;
+		tsmsg->m_block_write_time = 0;
+		tsmsg->m_session_time = 0;
+		tsmsg->m_active_time = 0;
+		tsmsg->m_idle_in_xact_time = 0;
+	}
+}
+
+/* --------
+ * pgstat_report_connect() -
+ *
+ *	Tell the collector about a new connection.
+ * --------
+ */
+void
+pgstat_report_connect(Oid dboid)
+{
+	PgStat_MsgConnect msg;
+
+	if (!pgstat_should_report_connstat())
+		return;
+
+	pgLastSessionReportTime = MyStartTimestamp;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_CONNECT);
+	msg.m_databaseid = MyDatabaseId;
+	pgstat_send(&msg, sizeof(PgStat_MsgConnect));
+}
+
+/* --------
+ * pgstat_report_disconnect() -
+ *
+ *	Tell the collector about a disconnect.
+ * --------
+ */
+void
+pgstat_report_disconnect(Oid dboid)
+{
+	PgStat_MsgDisconnect msg;
+
+	if (!pgstat_should_report_connstat())
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_DISCONNECT);
+	msg.m_databaseid = MyDatabaseId;
+	msg.m_cause = pgStatSessionEndCause;
+	pgstat_send(&msg, sizeof(PgStat_MsgDisconnect));
+}
+
+/* --------
+ * pgstat_should_report_connstats() -
+ *
+ *	We report session statistics only for normal backend processes.  Parallel
+ *	workers run in parallel, so they don't contribute to session times, even
+ *	though they use CPU time. Walsender processes could be considered here,
+ *	but they have different session characteristics from normal backends (for
+ *	example, they are always "active"), so they would skew session statistics.
+ * ----------
+ */
+static bool
+pgstat_should_report_connstat(void)
+{
+	return MyBackendType == B_BACKEND;
+}
+
+void
+AtEOXact_PgStat_Database(bool isCommit, bool parallel)
+{
+	/* Don't count parallel worker transaction stats */
+	if (!parallel)
+	{
+		/*
+		 * Count transaction commit or abort.  (We use counters, not just
+		 * bools, in case the reporting message isn't sent right away.)
+		 */
+		if (isCommit)
+			pgStatXactCommit++;
+		else
+			pgStatXactRollback++;
+	}
+}
diff --git a/src/backend/utils/activity/pgstat_function.c b/src/backend/utils/activity/pgstat_function.c
new file mode 100644
index 00000000000..728bc20b133
--- /dev/null
+++ b/src/backend/utils/activity/pgstat_function.c
@@ -0,0 +1,225 @@
+/* -------------------------------------------------------------------------
+ *
+ * pgstat_function.c
+ *	  Implementation of function statistics.
+ *
+ * This file contains the implementation of function statistics. It is kept
+ * separate from pgstat.c to enforce the line between the statistics access /
+ * storage implementation and the details about individual types of
+ * statistics.
+ *
+ * Copyright (c) 2001-2021, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/backend/utils/activity/pgstat_function.c
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "utils/pgstat_internal.h"
+#include "utils/timestamp.h"
+
+
+/* ----------
+ * GUC parameters
+ * ----------
+ */
+int			pgstat_track_functions = TRACK_FUNC_OFF;
+
+
+/*
+ * Indicates if backend has some function stats that it hasn't yet
+ * sent to the collector.
+ */
+bool have_function_stats = false;
+
+/*
+ * Backends store per-function info that's waiting to be sent to the collector
+ * in this hash table (indexed by function OID).
+ */
+static HTAB *pgStatFunctions = NULL;
+
+/*
+ * Total time charged to functions so far in the current backend.
+ * We use this to help separate "self" and "other" time charges.
+ * (We assume this initializes to zero.)
+ */
+static instr_time total_func_time;
+
+
+/*
+ * Initialize function call usage data.
+ * Called by the executor before invoking a function.
+ */
+void
+pgstat_init_function_usage(FunctionCallInfo fcinfo,
+						   PgStat_FunctionCallUsage *fcu)
+{
+	PgStat_BackendFunctionEntry *htabent;
+	bool		found;
+
+	if (pgstat_track_functions <= fcinfo->flinfo->fn_stats)
+	{
+		/* stats not wanted */
+		fcu->fs = NULL;
+		return;
+	}
+
+	if (!pgStatFunctions)
+	{
+		/* First time through - initialize function stat table */
+		HASHCTL		hash_ctl;
+
+		hash_ctl.keysize = sizeof(Oid);
+		hash_ctl.entrysize = sizeof(PgStat_BackendFunctionEntry);
+		pgStatFunctions = hash_create("Function stat entries",
+									  PGSTAT_FUNCTION_HASH_SIZE,
+									  &hash_ctl,
+									  HASH_ELEM | HASH_BLOBS);
+	}
+
+	/* Get the stats entry for this function, create if necessary */
+	htabent = hash_search(pgStatFunctions, &fcinfo->flinfo->fn_oid,
+						  HASH_ENTER, &found);
+	if (!found)
+		MemSet(&htabent->f_counts, 0, sizeof(PgStat_FunctionCounts));
+
+	fcu->fs = &htabent->f_counts;
+
+	/* save stats for this function, later used to compensate for recursion */
+	fcu->save_f_total_time = htabent->f_counts.f_total_time;
+
+	/* save current backend-wide total time */
+	fcu->save_total = total_func_time;
+
+	/* get clock time as of function start */
+	INSTR_TIME_SET_CURRENT(fcu->f_start);
+}
+
+/*
+ * Calculate function call usage and update stat counters.
+ * Called by the executor after invoking a function.
+ *
+ * In the case of a set-returning function that runs in value-per-call mode,
+ * we will see multiple pgstat_init_function_usage/pgstat_end_function_usage
+ * calls for what the user considers a single call of the function.  The
+ * finalize flag should be TRUE on the last call.
+ */
+void
+pgstat_end_function_usage(PgStat_FunctionCallUsage *fcu, bool finalize)
+{
+	PgStat_FunctionCounts *fs = fcu->fs;
+	instr_time	f_total;
+	instr_time	f_others;
+	instr_time	f_self;
+
+	/* stats not wanted? */
+	if (fs == NULL)
+		return;
+
+	/* total elapsed time in this function call */
+	INSTR_TIME_SET_CURRENT(f_total);
+	INSTR_TIME_SUBTRACT(f_total, fcu->f_start);
+
+	/* self usage: elapsed minus anything already charged to other calls */
+	f_others = total_func_time;
+	INSTR_TIME_SUBTRACT(f_others, fcu->save_total);
+	f_self = f_total;
+	INSTR_TIME_SUBTRACT(f_self, f_others);
+
+	/* update backend-wide total time */
+	INSTR_TIME_ADD(total_func_time, f_self);
+
+	/*
+	 * Compute the new f_total_time as the total elapsed time added to the
+	 * pre-call value of f_total_time.  This is necessary to avoid
+	 * double-counting any time taken by recursive calls of myself.  (We do
+	 * not need any similar kluge for self time, since that already excludes
+	 * any recursive calls.)
+	 */
+	INSTR_TIME_ADD(f_total, fcu->save_f_total_time);
+
+	/* update counters in function stats table */
+	if (finalize)
+		fs->f_numcalls++;
+	fs->f_total_time = f_total;
+	INSTR_TIME_ADD(fs->f_self_time, f_self);
+
+	/* indicate that we have something to send */
+	have_function_stats = true;
+}
+
+/*
+ * Subroutine for pgstat_report_stat: populate and send a function stat message
+ */
+void
+pgstat_send_funcstats(void)
+{
+	/* we assume this inits to all zeroes: */
+	static const PgStat_FunctionCounts all_zeroes;
+
+	PgStat_MsgFuncstat msg;
+	PgStat_BackendFunctionEntry *entry;
+	HASH_SEQ_STATUS fstat;
+
+	if (pgStatFunctions == NULL)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_FUNCSTAT);
+	msg.m_databaseid = MyDatabaseId;
+	msg.m_nentries = 0;
+
+	hash_seq_init(&fstat, pgStatFunctions);
+	while ((entry = (PgStat_BackendFunctionEntry *) hash_seq_search(&fstat)) != NULL)
+	{
+		PgStat_FunctionEntry *m_ent;
+
+		/* Skip it if no counts accumulated since last time */
+		if (memcmp(&entry->f_counts, &all_zeroes,
+				   sizeof(PgStat_FunctionCounts)) == 0)
+			continue;
+
+		/* need to convert format of time accumulators */
+		m_ent = &msg.m_entry[msg.m_nentries];
+		m_ent->f_id = entry->f_id;
+		m_ent->f_numcalls = entry->f_counts.f_numcalls;
+		m_ent->f_total_time = INSTR_TIME_GET_MICROSEC(entry->f_counts.f_total_time);
+		m_ent->f_self_time = INSTR_TIME_GET_MICROSEC(entry->f_counts.f_self_time);
+
+		if (++msg.m_nentries >= PGSTAT_NUM_FUNCENTRIES)
+		{
+			pgstat_send(&msg, offsetof(PgStat_MsgFuncstat, m_entry[0]) +
+						msg.m_nentries * sizeof(PgStat_FunctionEntry));
+			msg.m_nentries = 0;
+		}
+
+		/* reset the entry's counts */
+		MemSet(&entry->f_counts, 0, sizeof(PgStat_FunctionCounts));
+	}
+
+	if (msg.m_nentries > 0)
+		pgstat_send(&msg, offsetof(PgStat_MsgFuncstat, m_entry[0]) +
+					msg.m_nentries * sizeof(PgStat_FunctionEntry));
+
+	have_function_stats = false;
+}
+
+/*
+ * find_funcstat_entry - find any existing PgStat_BackendFunctionEntry entry
+ *		for specified function
+ *
+ * If no entry, return NULL, don't create a new one
+ */
+PgStat_BackendFunctionEntry *
+find_funcstat_entry(Oid func_id)
+{
+	pgstat_assert_is_up();
+
+	if (pgStatFunctions == NULL)
+		return NULL;
+
+	return (PgStat_BackendFunctionEntry *) hash_search(pgStatFunctions,
+													   (void *) &func_id,
+													   HASH_FIND, NULL);
+}
diff --git a/src/backend/utils/activity/pgstat_global.c b/src/backend/utils/activity/pgstat_global.c
new file mode 100644
index 00000000000..295763d8f3e
--- /dev/null
+++ b/src/backend/utils/activity/pgstat_global.c
@@ -0,0 +1,548 @@
+/* -------------------------------------------------------------------------
+ *
+ * pgstat_global.c
+ *	  Implementation of all global statistics.
+ *
+ * This file contains the implementation of global statistics. It is kept
+ * separate from pgstat.c to enforce the line between the statistics access /
+ * storage implementation and the details about individual types of
+ * statistics.
+ *
+ * Copyright (c) 2001-2021, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/backend/utils/activity/pgstat_global.c
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "executor/instrument.h"
+#include "replication/slot.h"
+#include "utils/builtins.h"
+#include "utils/pgstat_internal.h"
+#include "utils/timestamp.h"
+
+
+/* ----------
+ * pending stats state that is directly modified from outside the stats system
+ * ----------
+ */
+
+/*
+ * BgWriter and WAL global statistics counters.
+ * Stored directly in a stats message structure so they can be sent
+ * without needing to copy things around.  We assume these init to zeroes.
+ */
+PgStat_MsgBgWriter PendingBgWriterStats;
+PgStat_MsgCheckpointer PendingCheckpointerStats;
+PgStat_MsgWal WalStats;
+
+/*
+ * SLRU statistics counts waiting to be sent to the collector.  These are
+ * stored directly in stats message format so they can be sent without needing
+ * to copy things around.  We assume this variable inits to zeroes.  Entries
+ * are one-to-one with slru_names[].
+ */
+static PgStat_MsgSLRU SLRUStats[SLRU_NUM_ELEMENTS];
+
+/*
+ * WAL usage counters saved from pgWALUsage at the previous call to
+ * pgstat_send_wal(). This is used to calculate how much WAL usage
+ * happens between pgstat_send_wal() calls, by subtracting
+ * the previous counters from the current ones.
+ *
+ * FIXME: It'd be better if this weren't global.
+ */
+WalUsage prevWalUsage;
+
+
+/* ----------
+ * pgstat_reset_shared_counters() -
+ *
+ *	Tell the statistics collector to reset cluster-wide shared counters.
+ *
+ *	Permission checking for this function is managed through the normal
+ *	GRANT system.
+ * ----------
+ */
+void
+pgstat_reset_shared_counters(const char *target)
+{
+	PgStat_MsgResetsharedcounter msg;
+
+	if (pgStatSock == PGINVALID_SOCKET)
+		return;
+
+	if (strcmp(target, "archiver") == 0)
+		msg.m_resettarget = RESET_ARCHIVER;
+	else if (strcmp(target, "bgwriter") == 0)
+		msg.m_resettarget = RESET_BGWRITER;
+	else if (strcmp(target, "wal") == 0)
+		msg.m_resettarget = RESET_WAL;
+	else
+		ereport(ERROR,
+				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+				 errmsg("unrecognized reset target: \"%s\"", target),
+				 errhint("Target must be \"archiver\", \"bgwriter\", or \"wal\".")));
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETSHAREDCOUNTER);
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* ----------
+ * pgstat_send_archiver() -
+ *
+ *	Tell the collector about the WAL file that we successfully
+ *	archived or failed to archive.
+ * ----------
+ */
+void
+pgstat_send_archiver(const char *xlog, bool failed)
+{
+	PgStat_MsgArchiver msg;
+
+	/*
+	 * Prepare and send the message
+	 */
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_ARCHIVER);
+	msg.m_failed = failed;
+	strlcpy(msg.m_xlog, xlog, sizeof(msg.m_xlog));
+	msg.m_timestamp = GetCurrentTimestamp();
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* ----------
+ * pgstat_send_bgwriter() -
+ *
+ *		Send bgwriter statistics to the collector
+ * ----------
+ */
+void
+pgstat_send_bgwriter(void)
+{
+	/* We assume this initializes to zeroes */
+	static const PgStat_MsgBgWriter all_zeroes;
+
+	pgstat_assert_is_up();
+
+	/*
+	 * 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(&PendingBgWriterStats, &all_zeroes, sizeof(PgStat_MsgBgWriter)) == 0)
+		return;
+
+	/*
+	 * Prepare and send the message
+	 */
+	pgstat_setheader(&PendingBgWriterStats.m_hdr, PGSTAT_MTYPE_BGWRITER);
+	pgstat_send(&PendingBgWriterStats, sizeof(PendingBgWriterStats));
+
+	/*
+	 * Clear out the statistics buffer, so it can be re-used.
+	 */
+	MemSet(&PendingBgWriterStats, 0, sizeof(PendingBgWriterStats));
+}
+
+/* ----------
+ * pgstat_send_checkpointer() -
+ *
+ *		Send checkpointer statistics to the collector
+ * ----------
+ */
+void
+pgstat_send_checkpointer(void)
+{
+	/* We assume this initializes to zeroes */
+	static const PgStat_MsgCheckpointer all_zeroes;
+
+	/*
+	 * 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(&PendingCheckpointerStats, &all_zeroes, sizeof(PgStat_MsgCheckpointer)) == 0)
+		return;
+
+	/*
+	 * Prepare and send the message
+	 */
+	pgstat_setheader(&PendingCheckpointerStats.m_hdr, PGSTAT_MTYPE_CHECKPOINTER);
+	pgstat_send(&PendingCheckpointerStats, sizeof(PendingCheckpointerStats));
+
+	/*
+	 * Clear out the statistics buffer, so it can be re-used.
+	 */
+	MemSet(&PendingCheckpointerStats, 0, sizeof(PendingCheckpointerStats));
+}
+
+/* ----------
+ * pgstat_reset_replslot_counter() -
+ *
+ *	Tell the statistics collector to reset a single replication slot
+ *	counter, or all replication slots counters (when name is null).
+ *
+ *	Permission checking for this function is managed through the normal
+ *	GRANT system.
+ * ----------
+ */
+void
+pgstat_reset_replslot_counter(const char *name)
+{
+	PgStat_MsgResetreplslotcounter msg;
+
+	if (pgStatSock == PGINVALID_SOCKET)
+		return;
+
+	if (name)
+	{
+		namestrcpy(&msg.m_slotname, name);
+		msg.clearall = false;
+	}
+	else
+		msg.clearall = true;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETREPLSLOTCOUNTER);
+
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* ----------
+ * pgstat_report_replslot() -
+ *
+ *	Tell the collector about replication slot statistics.
+ * ----------
+ */
+void
+pgstat_report_replslot(const PgStat_StatReplSlotEntry *repSlotStat)
+{
+	PgStat_MsgReplSlot msg;
+
+	/*
+	 * Prepare and send the message
+	 */
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_REPLSLOT);
+	namestrcpy(&msg.m_slotname, NameStr(repSlotStat->slotname));
+	msg.m_create = false;
+	msg.m_drop = false;
+	msg.m_spill_txns = repSlotStat->spill_txns;
+	msg.m_spill_count = repSlotStat->spill_count;
+	msg.m_spill_bytes = repSlotStat->spill_bytes;
+	msg.m_stream_txns = repSlotStat->stream_txns;
+	msg.m_stream_count = repSlotStat->stream_count;
+	msg.m_stream_bytes = repSlotStat->stream_bytes;
+	msg.m_total_txns = repSlotStat->total_txns;
+	msg.m_total_bytes = repSlotStat->total_bytes;
+	pgstat_send(&msg, sizeof(PgStat_MsgReplSlot));
+}
+
+/* ----------
+ * pgstat_report_replslot_create() -
+ *
+ *	Tell the collector about creating the replication slot.
+ * ----------
+ */
+void
+pgstat_report_replslot_create(const char *slotname)
+{
+	PgStat_MsgReplSlot msg;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_REPLSLOT);
+	namestrcpy(&msg.m_slotname, slotname);
+	msg.m_create = true;
+	msg.m_drop = false;
+	pgstat_send(&msg, sizeof(PgStat_MsgReplSlot));
+}
+
+/* ----------
+ * pgstat_report_replslot_drop() -
+ *
+ *	Tell the collector about dropping the replication slot.
+ * ----------
+ */
+void
+pgstat_report_replslot_drop(const char *slotname)
+{
+	PgStat_MsgReplSlot msg;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_REPLSLOT);
+	namestrcpy(&msg.m_slotname, slotname);
+	msg.m_create = false;
+	msg.m_drop = true;
+	pgstat_send(&msg, sizeof(PgStat_MsgReplSlot));
+}
+
+/* ----------
+ * pgstat_send_slru() -
+ *
+ *		Send SLRU statistics to the collector
+ * ----------
+ */
+void
+pgstat_send_slru(void)
+{
+	/* We assume this initializes to zeroes */
+	static const PgStat_MsgSLRU all_zeroes;
+
+	for (int 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));
+	}
+}
+
+/* ----------
+ * 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));
+}
+
+/*
+ * slru_entry
+ *
+ * Returns pointer to entry with counters for given SLRU (based on the name
+ * stored in SlruCtl as lwlock tranche name).
+ */
+static inline PgStat_MsgSLRU *
+slru_entry(int slru_idx)
+{
+	pgstat_assert_is_up();
+
+	/*
+	 * The postmaster should never register any SLRU statistics counts; if it
+	 * did, the counts would be duplicated into child processes via fork().
+	 */
+	Assert(IsUnderPostmaster || !IsPostmasterEnvironment);
+
+	Assert((slru_idx >= 0) && (slru_idx < SLRU_NUM_ELEMENTS));
+
+	return &SLRUStats[slru_idx];
+}
+
+/*
+ * SLRU statistics count accumulation functions --- called from slru.c
+ */
+
+void
+pgstat_count_slru_page_zeroed(int slru_idx)
+{
+	slru_entry(slru_idx)->m_blocks_zeroed += 1;
+}
+
+void
+pgstat_count_slru_page_hit(int slru_idx)
+{
+	slru_entry(slru_idx)->m_blocks_hit += 1;
+}
+
+void
+pgstat_count_slru_page_exists(int slru_idx)
+{
+	slru_entry(slru_idx)->m_blocks_exists += 1;
+}
+
+void
+pgstat_count_slru_page_read(int slru_idx)
+{
+	slru_entry(slru_idx)->m_blocks_read += 1;
+}
+
+void
+pgstat_count_slru_page_written(int slru_idx)
+{
+	slru_entry(slru_idx)->m_blocks_written += 1;
+}
+
+void
+pgstat_count_slru_flush(int slru_idx)
+{
+	slru_entry(slru_idx)->m_flush += 1;
+}
+
+void
+pgstat_count_slru_truncate(int slru_idx)
+{
+	slru_entry(slru_idx)->m_truncate += 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.
+ */
+const char *
+pgstat_slru_name(int slru_idx)
+{
+	if (slru_idx < 0 || slru_idx >= SLRU_NUM_ELEMENTS)
+		return NULL;
+
+	return slru_names[slru_idx];
+}
+
+/*
+ * 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 projects.
+ */
+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_send_wal() -
+ *
+ *	Send WAL statistics to the collector.
+ *
+ * If 'force' is not set, WAL stats message is only sent if enough time has
+ * passed since last one was sent to reach PGSTAT_STAT_INTERVAL.
+ * ----------
+ */
+void
+pgstat_send_wal(bool force)
+{
+	static TimestampTz sendTime = 0;
+
+	/*
+	 * 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.
+	 *
+	 * Check wal_records counter to determine whether any WAL activity has
+	 * happened since last time. Note that other WalUsage counters don't need
+	 * to be checked because they are incremented always together with
+	 * wal_records counter.
+	 *
+	 * m_wal_buffers_full also doesn't need to be checked because it's
+	 * incremented only when at least one WAL record is generated (i.e.,
+	 * wal_records counter is incremented). But for safely, we assert that
+	 * m_wal_buffers_full is always zero when no WAL record is generated
+	 *
+	 * This function can be called by a process like walwriter that normally
+	 * generates no WAL records. To determine whether any WAL activity has
+	 * happened at that process since the last time, the numbers of WAL writes
+	 * and syncs are also checked.
+	 */
+	if (pgWalUsage.wal_records == prevWalUsage.wal_records &&
+		WalStats.m_wal_write == 0 && WalStats.m_wal_sync == 0)
+	{
+		Assert(WalStats.m_wal_buffers_full == 0);
+		return;
+	}
+
+	if (!force)
+	{
+		TimestampTz now = GetCurrentTimestamp();
+
+		/*
+		 * Don't send a message unless it's been at least PGSTAT_STAT_INTERVAL
+		 * msec since we last sent one to avoid overloading the stats
+		 * collector.
+		 */
+		if (!TimestampDifferenceExceeds(sendTime, now, PGSTAT_STAT_INTERVAL))
+			return;
+		sendTime = now;
+	}
+
+	/*
+	 * Set the counters related to generated WAL data if the counters were
+	 * updated.
+	 */
+	if (pgWalUsage.wal_records != prevWalUsage.wal_records)
+	{
+		WalUsage	walusage;
+
+		/*
+		 * Calculate how much WAL usage counters were increased by
+		 * subtracting the previous counters from the current ones. Fill the
+		 * results in WAL stats message.
+		 */
+		MemSet(&walusage, 0, sizeof(WalUsage));
+		WalUsageAccumDiff(&walusage, &pgWalUsage, &prevWalUsage);
+
+		WalStats.m_wal_records = walusage.wal_records;
+		WalStats.m_wal_fpi = walusage.wal_fpi;
+		WalStats.m_wal_bytes = walusage.wal_bytes;
+
+		/*
+		 * Save the current counters for the subsequent calculation of WAL
+		 * usage.
+		 */
+		prevWalUsage = pgWalUsage;
+	}
+
+	/*
+	 * Prepare and send the message
+	 */
+	pgstat_setheader(&WalStats.m_hdr, PGSTAT_MTYPE_WAL);
+	pgstat_send(&WalStats, sizeof(WalStats));
+
+	/*
+	 * Clear out the statistics buffer, so it can be re-used.
+	 */
+	MemSet(&WalStats, 0, sizeof(WalStats));
+}
+
+void
+pgstat_wal_initialize(void)
+{
+	/*
+	 * Initialize prevWalUsage with pgWalUsage so that pgstat_send_wal() can
+	 * calculate how much pgWalUsage counters are increased by subtracting
+	 * prevWalUsage from pgWalUsage.
+	 */
+	prevWalUsage = pgWalUsage;
+}
diff --git a/src/backend/utils/activity/pgstat_relation.c b/src/backend/utils/activity/pgstat_relation.c
new file mode 100644
index 00000000000..018f6a4d68d
--- /dev/null
+++ b/src/backend/utils/activity/pgstat_relation.c
@@ -0,0 +1,953 @@
+/* -------------------------------------------------------------------------
+ *
+ * pgstat_relation.c
+ *	  Implementation of relation statistics.
+ *
+ * This file contains the implementation of function relation. It is kept
+ * separate from pgstat.c to enforce the line between the statistics access /
+ * storage implementation and the details about individual types of
+ * statistics.
+ *
+ * Copyright (c) 2001-2021, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/backend/utils/activity/pgstat_relation.c
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/twophase_rmgr.h"
+#include "access/xact.h"
+#include "catalog/partition.h"
+#include "postmaster/autovacuum.h"
+#include "utils/memutils.h"
+#include "utils/pgstat_internal.h"
+#include "utils/rel.h"
+#include "utils/timestamp.h"
+
+
+/*
+ * Structures in which backends store per-table info that's waiting to be
+ * sent to the collector.
+ *
+ * NOTE: once allocated, TabStatusArray structures are never moved or deleted
+ * for the life of the backend.  Also, we zero out the t_id fields of the
+ * contained PgStat_TableStatus structs whenever they are not actively in use.
+ * This allows relcache pgstat_info pointers to be treated as long-lived data,
+ * avoiding repeated searches in pgstat_relation_assoc() when a relation is
+ * repeatedly opened during a transaction.
+ */
+#define TABSTAT_QUANTUM		100 /* we alloc this many at a time */
+
+
+typedef struct TabStatusArray
+{
+	struct TabStatusArray *tsa_next;	/* link to next array, if any */
+	int			tsa_used;		/* # entries currently used */
+	PgStat_TableStatus tsa_entries[TABSTAT_QUANTUM];	/* per-table data */
+} TabStatusArray;
+
+static TabStatusArray *pgStatTabList = NULL;
+
+/*
+ * pgStatTabHash entry: map from relation OID to PgStat_TableStatus pointer
+ */
+typedef struct TabStatHashEntry
+{
+	Oid			t_id;
+	PgStat_TableStatus *tsa_entry;
+} TabStatHashEntry;
+
+/* Record that's written to 2PC state file when pgstat state is persisted */
+typedef struct TwoPhasePgStatRecord
+{
+	PgStat_Counter tuples_inserted; /* tuples inserted in xact */
+	PgStat_Counter tuples_updated;	/* tuples updated in xact */
+	PgStat_Counter tuples_deleted;	/* tuples deleted in xact */
+	/* tuples i/u/d prior to truncate/drop */
+	PgStat_Counter inserted_pre_truncdrop;
+	PgStat_Counter updated_pre_truncdrop;
+	PgStat_Counter deleted_pre_truncdrop;
+	Oid			t_id;			/* table's OID */
+	bool		t_shared;		/* is it a shared catalog? */
+	bool		t_truncdropped;	/* was the relation truncated/dropped? */
+} TwoPhasePgStatRecord;
+
+
+static PgStat_TableStatus *get_tabstat_entry(Oid rel_id, bool isshared);
+static void pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg, TimestampTz now);
+static void add_tabstat_xact_level(PgStat_TableStatus *pgstat_info, int nest_level);
+static void ensure_tabstat_xact_level(PgStat_TableStatus *pgstat_info);
+static void pgstat_truncdrop_save_counters(PgStat_TableXactStatus *trans, bool is_drop);
+static void pgstat_truncdrop_restore_counters(PgStat_TableXactStatus *trans);
+
+
+/*
+ * Indicates if backend has some relation stats that it hasn't yet
+ * sent to the collector.
+ */
+bool have_relation_stats;
+
+
+/*
+ * Hash table for O(1) t_id -> tsa_entry lookup
+ */
+static HTAB *pgStatTabHash = NULL;
+
+
+/* ----------
+ * pgstat_relation_init() -
+ *
+ *	Initialize a relcache entry to count access statistics.
+ *	Called whenever a relation is opened.
+ *
+ *	We assume that a relcache entry's pgstat_info field is zeroed by
+ *	relcache.c when the relcache entry is made; thereafter it is long-lived
+ *	data.  We can avoid repeated searches of the TabStatus arrays when the
+ *	same relation is touched repeatedly within a transaction.
+ * ----------
+ */
+void
+pgstat_relation_init(Relation rel)
+{
+	Oid			rel_id = rel->rd_id;
+	char		relkind = rel->rd_rel->relkind;
+
+	/*
+	 * We only count stats for relations with storage and partitioned tables
+	 */
+	if (!RELKIND_HAS_STORAGE(relkind) && relkind != RELKIND_PARTITIONED_TABLE)
+	{
+		rel->pgstat_enabled = false;
+		rel->pgstat_info = NULL;
+		return;
+	}
+
+	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+	{
+		/* We're not counting at all */
+		rel->pgstat_enabled = false;
+		rel->pgstat_info = NULL;
+		return;
+	}
+
+	/*
+	 * If we already set up this relation in the current transaction, nothing
+	 * to do.
+	 */
+	if (rel->pgstat_info &&
+		rel->pgstat_info->t_id == rel_id)
+		return;
+
+	rel->pgstat_enabled = true;
+}
+
+void
+pgstat_relation_assoc(Relation rel)
+{
+	Oid			rel_id = rel->rd_id;
+
+	Assert(rel->pgstat_enabled);
+	Assert(rel->pgstat_info == NULL);
+
+	/* find or make the PgStat_TableStatus entry, and update link */
+	rel->pgstat_info = get_tabstat_entry(rel_id, rel->rd_rel->relisshared);
+}
+
+/* ----------
+ * pgstat_drop_relation() -
+ *
+ *	Tell the collector that we just dropped a relation.
+ *	(If the message gets lost, we will still clean the dead entry eventually
+ *	via future invocations of pgstat_vacuum_stat().)
+ *
+ *	Currently not used for lack of any good place to call it; we rely
+ *	entirely on pgstat_vacuum_stat() to clean out stats for dead rels.
+ * ----------
+ */
+#ifdef NOT_USED
+void
+pgstat_drop_relation(Oid relid)
+{
+	PgStat_MsgTabpurge msg;
+	int			len;
+
+	if (pgStatSock == PGINVALID_SOCKET)
+		return;
+
+	msg.m_tableid[0] = relid;
+	msg.m_nentries = 1;
+
+	len = offsetof(PgStat_MsgTabpurge, m_tableid[0]) + sizeof(Oid);
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_TABPURGE);
+	msg.m_databaseid = MyDatabaseId;
+	pgstat_send(&msg, len);
+}
+#endif							/* NOT_USED */
+
+/* ----------
+ * pgstat_report_autovac() -
+ *
+ *	Called from autovacuum.c to report startup of an autovacuum process.
+ *	We are called before InitPostgres is done, so can't rely on MyDatabaseId;
+ *	the db OID must be passed in, instead.
+ * ----------
+ */
+void
+pgstat_report_autovac(Oid dboid)
+{
+	PgStat_MsgAutovacStart msg;
+
+	if (pgStatSock == PGINVALID_SOCKET)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_AUTOVAC_START);
+	msg.m_databaseid = dboid;
+	msg.m_start_time = GetCurrentTimestamp();
+
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* ---------
+ * pgstat_report_vacuum() -
+ *
+ *	Tell the collector about the table we just vacuumed.
+ * ---------
+ */
+void
+pgstat_report_vacuum(Oid tableoid, bool shared,
+					 PgStat_Counter livetuples, PgStat_Counter deadtuples)
+{
+	PgStat_MsgVacuum msg;
+
+	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+		return;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_VACUUM);
+	msg.m_databaseid = shared ? InvalidOid : MyDatabaseId;
+	msg.m_tableoid = tableoid;
+	msg.m_autovacuum = IsAutoVacuumWorkerProcess();
+	msg.m_vacuumtime = GetCurrentTimestamp();
+	msg.m_live_tuples = livetuples;
+	msg.m_dead_tuples = deadtuples;
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/* --------
+ * pgstat_report_analyze() -
+ *
+ *	Tell the collector about the table we just analyzed.
+ *
+ * Caller must provide new live- and dead-tuples estimates, as well as a
+ * flag indicating whether to reset the changes_since_analyze counter.
+ * --------
+ */
+void
+pgstat_report_analyze(Relation rel,
+					  PgStat_Counter livetuples, PgStat_Counter deadtuples,
+					  bool resetcounter)
+{
+	PgStat_MsgAnalyze msg;
+
+	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+		return;
+
+	/*
+	 * Unlike VACUUM, ANALYZE might be running inside a transaction that has
+	 * already inserted and/or deleted rows in the target table. ANALYZE will
+	 * have counted such rows as live or dead respectively. Because we will
+	 * report our counts of such rows at transaction end, we should subtract
+	 * off these counts from what we send to the collector now, else they'll
+	 * be double-counted after commit.  (This approach also ensures that the
+	 * collector ends up with the right numbers if we abort instead of
+	 * committing.)
+	 *
+	 * Waste no time on partitioned tables, though.
+	 */
+	if (pgstat_relation_should_count(rel) &&
+		rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+	{
+		PgStat_TableXactStatus *trans;
+
+		for (trans = rel->pgstat_info->trans; trans; trans = trans->upper)
+		{
+			livetuples -= trans->tuples_inserted - trans->tuples_deleted;
+			deadtuples -= trans->tuples_updated + trans->tuples_deleted;
+		}
+		/* count stuff inserted by already-aborted subxacts, too */
+		deadtuples -= rel->pgstat_info->t_counts.t_delta_dead_tuples;
+		/* Since ANALYZE's counts are estimates, we could have underflowed */
+		livetuples = Max(livetuples, 0);
+		deadtuples = Max(deadtuples, 0);
+	}
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_ANALYZE);
+	msg.m_databaseid = rel->rd_rel->relisshared ? InvalidOid : MyDatabaseId;
+	msg.m_tableoid = RelationGetRelid(rel);
+	msg.m_autovacuum = IsAutoVacuumWorkerProcess();
+	msg.m_resetcounter = resetcounter;
+	msg.m_analyzetime = GetCurrentTimestamp();
+	msg.m_live_tuples = livetuples;
+	msg.m_dead_tuples = deadtuples;
+	pgstat_send(&msg, sizeof(msg));
+}
+
+/*
+ * Subroutine for pgstat_report_stat: Send relation statistics
+ */
+void
+pgstat_send_tabstats(TimestampTz now, bool disconnect)
+{
+	/* we assume this inits to all zeroes: */
+	static const PgStat_TableCounts all_zeroes;
+	PgStat_MsgTabstat regular_msg;
+	PgStat_MsgTabstat shared_msg;
+	TabStatusArray *tsa;
+	int			i;
+
+	/*
+	 * Destroy pgStatTabHash before we start invalidating PgStat_TableEntry
+	 * entries it points to.  (Should we fail partway through the loop below,
+	 * it's okay to have removed the hashtable already --- the only
+	 * consequence is we'd get multiple entries for the same table in the
+	 * pgStatTabList, and that's safe.)
+	 */
+	if (pgStatTabHash)
+		hash_destroy(pgStatTabHash);
+	pgStatTabHash = NULL;
+
+	/*
+	 * Scan through the TabStatusArray struct(s) to find tables that actually
+	 * have counts, and build messages to send.  We have to separate shared
+	 * relations from regular ones because the databaseid field in the message
+	 * header has to depend on that.
+	 */
+	regular_msg.m_databaseid = MyDatabaseId;
+	shared_msg.m_databaseid = InvalidOid;
+	regular_msg.m_nentries = 0;
+	shared_msg.m_nentries = 0;
+
+	for (tsa = pgStatTabList; tsa != NULL; tsa = tsa->tsa_next)
+	{
+		for (i = 0; i < tsa->tsa_used; i++)
+		{
+			PgStat_TableStatus *entry = &tsa->tsa_entries[i];
+			PgStat_MsgTabstat *this_msg;
+			PgStat_TableEntry *this_ent;
+
+			/* Shouldn't have any pending transaction-dependent counts */
+			Assert(entry->trans == NULL);
+
+			/*
+			 * Ignore entries that didn't accumulate any actual counts, such
+			 * as indexes that were opened by the planner but not used.
+			 */
+			if (memcmp(&entry->t_counts, &all_zeroes,
+					   sizeof(PgStat_TableCounts)) == 0)
+				continue;
+
+			/*
+			 * OK, insert data into the appropriate message, and send if full.
+			 */
+			this_msg = entry->t_shared ? &shared_msg : &regular_msg;
+			this_ent = &this_msg->m_entry[this_msg->m_nentries];
+			this_ent->t_id = entry->t_id;
+			memcpy(&this_ent->t_counts, &entry->t_counts,
+				   sizeof(PgStat_TableCounts));
+			if (++this_msg->m_nentries >= PGSTAT_NUM_TABENTRIES)
+			{
+				pgstat_send_tabstat(this_msg, now);
+				this_msg->m_nentries = 0;
+			}
+		}
+		/* zero out PgStat_TableStatus structs after use */
+		MemSet(tsa->tsa_entries, 0,
+			   tsa->tsa_used * sizeof(PgStat_TableStatus));
+		tsa->tsa_used = 0;
+	}
+
+	/*
+	 * Send partial messages.  Make sure that any pending xact commit/abort
+	 * and connection stats get counted, even if there are no table stats to
+	 * send.
+	 */
+	if (regular_msg.m_nentries > 0 ||
+		pgStatXactCommit > 0 || pgStatXactRollback > 0 || disconnect)
+		pgstat_send_tabstat(&regular_msg, now);
+	if (shared_msg.m_nentries > 0)
+		pgstat_send_tabstat(&shared_msg, now);
+
+	have_relation_stats = false;
+}
+
+/*
+ * Subroutine for pgstat_send_tabstats: finish and send one tabstat message
+ */
+static void
+pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg, TimestampTz now)
+{
+	int			n;
+	int			len;
+
+	/* It's unlikely we'd get here with no socket, but maybe not impossible */
+	if (pgStatSock == PGINVALID_SOCKET)
+		return;
+
+	/*
+	 * Report and reset accumulated xact commit/rollback and I/O timings
+	 * whenever we send a normal tabstat message
+	 */
+	pgstat_update_dbstats(tsmsg, now);
+
+	n = tsmsg->m_nentries;
+	len = offsetof(PgStat_MsgTabstat, m_entry[0]) +
+		n * sizeof(PgStat_TableEntry);
+
+	pgstat_setheader(&tsmsg->m_hdr, PGSTAT_MTYPE_TABSTAT);
+	pgstat_send(tsmsg, len);
+}
+
+/*
+ * pgstat_count_heap_insert - count a tuple insertion of n tuples
+ */
+void
+pgstat_count_heap_insert(Relation rel, PgStat_Counter n)
+{
+
+	if (pgstat_relation_should_count(rel))
+	{
+		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
+
+		ensure_tabstat_xact_level(pgstat_info);
+		pgstat_info->trans->tuples_inserted += n;
+	}
+}
+
+/*
+ * pgstat_count_heap_update - count a tuple update
+ */
+void
+pgstat_count_heap_update(Relation rel, bool hot)
+{
+
+	if (pgstat_relation_should_count(rel))
+	{
+		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
+
+		ensure_tabstat_xact_level(pgstat_info);
+		pgstat_info->trans->tuples_updated++;
+
+		/* t_tuples_hot_updated is nontransactional, so just advance it */
+		if (hot)
+			pgstat_info->t_counts.t_tuples_hot_updated++;
+	}
+}
+
+/*
+ * pgstat_count_heap_delete - count a tuple deletion
+ */
+void
+pgstat_count_heap_delete(Relation rel)
+{
+
+	if (pgstat_relation_should_count(rel))
+	{
+		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
+
+		ensure_tabstat_xact_level(pgstat_info);
+		pgstat_info->trans->tuples_deleted++;
+	}
+}
+
+/*
+ * pgstat_count_truncate - update tuple counters due to truncate
+ */
+void
+pgstat_count_truncate(Relation rel)
+{
+
+	if (pgstat_relation_should_count(rel))
+	{
+		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
+
+		ensure_tabstat_xact_level(pgstat_info);
+		pgstat_truncdrop_save_counters(pgstat_info->trans, false);
+		pgstat_info->trans->tuples_inserted = 0;
+		pgstat_info->trans->tuples_updated = 0;
+		pgstat_info->trans->tuples_deleted = 0;
+	}
+}
+
+/*
+ * pgstat_update_heap_dead_tuples - update dead-tuples count
+ *
+ * The semantics of this are that we are reporting the nontransactional
+ * recovery of "delta" dead tuples; so t_delta_dead_tuples decreases
+ * rather than increasing, and the change goes straight into the per-table
+ * counter, not into transactional state.
+ */
+void
+pgstat_update_heap_dead_tuples(Relation rel, int delta)
+{
+	if (pgstat_relation_should_count(rel))
+	{
+		PgStat_TableStatus *pgstat_info = rel->pgstat_info;
+
+		pgstat_info->t_counts.t_delta_dead_tuples -= delta;
+	}
+}
+
+/*
+ * Perform relation stats specific end-of-transaction work. Helper for
+ * AtEOXact_PgStat.
+ *
+ * Transfer transactional insert/update counts into the base tabstat entries.
+ * We don't bother to free any of the transactional state, since it's all in
+ * TopTransactionContext and will go away anyway.
+ */
+void
+AtEOXact_PgStat_Relations(PgStat_SubXactStatus *xact_state, bool isCommit)
+{
+	PgStat_TableXactStatus *trans;
+
+	for (trans = xact_state->first; trans != NULL; trans = trans->next)
+	{
+		PgStat_TableStatus *tabstat;
+
+		Assert(trans->nest_level == 1);
+		Assert(trans->upper == NULL);
+		tabstat = trans->parent;
+		Assert(tabstat->trans == trans);
+		/* restore pre-truncate/drop stats (if any) in case of aborted xact */
+		if (!isCommit)
+			pgstat_truncdrop_restore_counters(trans);
+		/* count attempted actions regardless of commit/abort */
+		tabstat->t_counts.t_tuples_inserted += trans->tuples_inserted;
+		tabstat->t_counts.t_tuples_updated += trans->tuples_updated;
+		tabstat->t_counts.t_tuples_deleted += trans->tuples_deleted;
+		if (isCommit)
+		{
+			tabstat->t_counts.t_truncdropped = trans->truncdropped;
+			if (trans->truncdropped)
+			{
+				/* forget live/dead stats seen by backend thus far */
+				tabstat->t_counts.t_delta_live_tuples = 0;
+				tabstat->t_counts.t_delta_dead_tuples = 0;
+			}
+			/* insert adds a live tuple, delete removes one */
+			tabstat->t_counts.t_delta_live_tuples +=
+				trans->tuples_inserted - trans->tuples_deleted;
+			/* update and delete each create a dead tuple */
+			tabstat->t_counts.t_delta_dead_tuples +=
+				trans->tuples_updated + trans->tuples_deleted;
+			/* insert, update, delete each count as one change event */
+			tabstat->t_counts.t_changed_tuples +=
+				trans->tuples_inserted + trans->tuples_updated +
+				trans->tuples_deleted;
+		}
+		else
+		{
+			/* inserted tuples are dead, deleted tuples are unaffected */
+			tabstat->t_counts.t_delta_dead_tuples +=
+				trans->tuples_inserted + trans->tuples_updated;
+			/* an aborted xact generates no changed_tuple events */
+		}
+		tabstat->trans = NULL;
+	}
+}
+
+/*
+ * Perform relation stats specific end-of-sub-transaction work. Helper for
+ * AtEOSubXact_PgStat.
+ *
+ * Transfer transactional insert/update counts into the next higher
+ * subtransaction state.
+ */
+void
+AtEOSubXact_PgStat_Relations(PgStat_SubXactStatus *xact_state, bool isCommit, int nestDepth)
+{
+	PgStat_TableXactStatus *trans;
+	PgStat_TableXactStatus *next_trans;
+
+	for (trans = xact_state->first; trans != NULL; trans = next_trans)
+	{
+		PgStat_TableStatus *tabstat;
+
+		next_trans = trans->next;
+		Assert(trans->nest_level == nestDepth);
+		tabstat = trans->parent;
+		Assert(tabstat->trans == trans);
+
+		if (isCommit)
+		{
+			if (trans->upper && trans->upper->nest_level == nestDepth - 1)
+			{
+				if (trans->truncdropped)
+				{
+					/* propagate the truncate/drop status one level up */
+					pgstat_truncdrop_save_counters(trans->upper, false);
+					/* replace upper xact stats with ours */
+					trans->upper->tuples_inserted = trans->tuples_inserted;
+					trans->upper->tuples_updated = trans->tuples_updated;
+					trans->upper->tuples_deleted = trans->tuples_deleted;
+				}
+				else
+				{
+					trans->upper->tuples_inserted += trans->tuples_inserted;
+					trans->upper->tuples_updated += trans->tuples_updated;
+					trans->upper->tuples_deleted += trans->tuples_deleted;
+				}
+				tabstat->trans = trans->upper;
+				pfree(trans);
+			}
+			else
+			{
+				/*
+				 * When there isn't an immediate parent state, we can just
+				 * reuse the record instead of going through a
+				 * palloc/pfree pushup (this works since it's all in
+				 * TopTransactionContext anyway).  We have to re-link it
+				 * into the parent level, though, and that might mean
+				 * pushing a new entry into the pgStatXactStack.
+				 */
+				PgStat_SubXactStatus *upper_xact_state;
+
+				upper_xact_state = pgstat_xact_stack_level_get(nestDepth - 1);
+				trans->next = upper_xact_state->first;
+				upper_xact_state->first = trans;
+				trans->nest_level = nestDepth - 1;
+			}
+		}
+		else
+		{
+			/*
+			 * On abort, update top-level tabstat counts, then forget the
+			 * subtransaction
+			 */
+
+			/* first restore values obliterated by truncate/drop */
+			pgstat_truncdrop_restore_counters(trans);
+			/* count attempted actions regardless of commit/abort */
+			tabstat->t_counts.t_tuples_inserted += trans->tuples_inserted;
+			tabstat->t_counts.t_tuples_updated += trans->tuples_updated;
+			tabstat->t_counts.t_tuples_deleted += trans->tuples_deleted;
+			/* inserted tuples are dead, deleted tuples are unaffected */
+			tabstat->t_counts.t_delta_dead_tuples +=
+				trans->tuples_inserted + trans->tuples_updated;
+			tabstat->trans = trans->upper;
+			pfree(trans);
+		}
+	}
+}
+
+/*
+ * Generate 2PC records for all the pending transaction-dependent relation
+ * stats.
+ */
+void
+AtPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state)
+{
+	PgStat_TableXactStatus *trans;
+
+	for (trans = xact_state->first; trans != NULL; trans = trans->next)
+	{
+		PgStat_TableStatus *tabstat;
+		TwoPhasePgStatRecord record;
+
+		Assert(trans->nest_level == 1);
+		Assert(trans->upper == NULL);
+		tabstat = trans->parent;
+		Assert(tabstat->trans == trans);
+
+		record.tuples_inserted = trans->tuples_inserted;
+		record.tuples_updated = trans->tuples_updated;
+		record.tuples_deleted = trans->tuples_deleted;
+		record.inserted_pre_truncdrop = trans->inserted_pre_truncdrop;
+		record.updated_pre_truncdrop = trans->updated_pre_truncdrop;
+		record.deleted_pre_truncdrop = trans->deleted_pre_truncdrop;
+		record.t_id = tabstat->t_id;
+		record.t_shared = tabstat->t_shared;
+		record.t_truncdropped = trans->truncdropped;
+
+		RegisterTwoPhaseRecord(TWOPHASE_RM_PGSTAT_ID, 0,
+							   &record, sizeof(TwoPhasePgStatRecord));
+	}
+}
+
+/*
+ * All we need do here is unlink the transaction stats state from the
+ * nontransactional state.  The nontransactional action counts will be
+ * reported to the stats collector immediately, while the effects on
+ * live and dead tuple counts are preserved in the 2PC state file.
+ *
+ * Note: AtEOXact_PgStat_Relations is not called during PREPARE.
+ */
+void
+PostPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state)
+{
+	PgStat_TableXactStatus *trans;
+
+	for (trans = xact_state->first; trans != NULL; trans = trans->next)
+	{
+		PgStat_TableStatus *tabstat;
+
+		tabstat = trans->parent;
+		tabstat->trans = NULL;
+	}
+}
+
+/*
+ * 2PC processing routine for COMMIT PREPARED case.
+ *
+ * Load the saved counts into our local pgstats state.
+ */
+void
+pgstat_twophase_postcommit(TransactionId xid, uint16 info,
+						   void *recdata, uint32 len)
+{
+	TwoPhasePgStatRecord *rec = (TwoPhasePgStatRecord *) recdata;
+	PgStat_TableStatus *pgstat_info;
+
+	/* Find or create a tabstat entry for the rel */
+	pgstat_info = get_tabstat_entry(rec->t_id, rec->t_shared);
+
+	/* Same math as in AtEOXact_PgStat, commit case */
+	pgstat_info->t_counts.t_tuples_inserted += rec->tuples_inserted;
+	pgstat_info->t_counts.t_tuples_updated += rec->tuples_updated;
+	pgstat_info->t_counts.t_tuples_deleted += rec->tuples_deleted;
+	pgstat_info->t_counts.t_truncdropped = rec->t_truncdropped;
+	if (rec->t_truncdropped)
+	{
+		/* forget live/dead stats seen by backend thus far */
+		pgstat_info->t_counts.t_delta_live_tuples = 0;
+		pgstat_info->t_counts.t_delta_dead_tuples = 0;
+	}
+	pgstat_info->t_counts.t_delta_live_tuples +=
+		rec->tuples_inserted - rec->tuples_deleted;
+	pgstat_info->t_counts.t_delta_dead_tuples +=
+		rec->tuples_updated + rec->tuples_deleted;
+	pgstat_info->t_counts.t_changed_tuples +=
+		rec->tuples_inserted + rec->tuples_updated +
+		rec->tuples_deleted;
+}
+
+/*
+ * 2PC processing routine for ROLLBACK PREPARED case.
+ *
+ * Load the saved counts into our local pgstats state, but treat them
+ * as aborted.
+ */
+void
+pgstat_twophase_postabort(TransactionId xid, uint16 info,
+						  void *recdata, uint32 len)
+{
+	TwoPhasePgStatRecord *rec = (TwoPhasePgStatRecord *) recdata;
+	PgStat_TableStatus *pgstat_info;
+
+	/* Find or create a tabstat entry for the rel */
+	pgstat_info = get_tabstat_entry(rec->t_id, rec->t_shared);
+
+	/* Same math as in AtEOXact_PgStat, abort case */
+	if (rec->t_truncdropped)
+	{
+		rec->tuples_inserted = rec->inserted_pre_truncdrop;
+		rec->tuples_updated = rec->updated_pre_truncdrop;
+		rec->tuples_deleted = rec->deleted_pre_truncdrop;
+	}
+	pgstat_info->t_counts.t_tuples_inserted += rec->tuples_inserted;
+	pgstat_info->t_counts.t_tuples_updated += rec->tuples_updated;
+	pgstat_info->t_counts.t_tuples_deleted += rec->tuples_deleted;
+	pgstat_info->t_counts.t_delta_dead_tuples +=
+		rec->tuples_inserted + rec->tuples_updated;
+}
+
+/*
+ * get_tabstat_entry - find or create a PgStat_TableStatus entry for rel
+ */
+static PgStat_TableStatus *
+get_tabstat_entry(Oid rel_id, bool isshared)
+{
+	TabStatHashEntry *hash_entry;
+	PgStat_TableStatus *entry;
+	TabStatusArray *tsa;
+	bool		found;
+
+	pgstat_assert_is_up();
+
+	have_relation_stats = true;
+
+	/*
+	 * Create hash table if we don't have it already.
+	 */
+	if (pgStatTabHash == NULL)
+	{
+		HASHCTL		ctl;
+
+		ctl.keysize = sizeof(Oid);
+		ctl.entrysize = sizeof(TabStatHashEntry);
+
+		pgStatTabHash = hash_create("pgstat TabStatusArray lookup hash table",
+									TABSTAT_QUANTUM,
+									&ctl,
+									HASH_ELEM | HASH_BLOBS);
+	}
+
+	/*
+	 * Find an entry or create a new one.
+	 */
+	hash_entry = hash_search(pgStatTabHash, &rel_id, HASH_ENTER, &found);
+	if (!found)
+	{
+		/* initialize new entry with null pointer */
+		hash_entry->tsa_entry = NULL;
+	}
+
+	/*
+	 * If entry is already valid, we're done.
+	 */
+	if (hash_entry->tsa_entry)
+		return hash_entry->tsa_entry;
+
+	/*
+	 * Locate the first pgStatTabList entry with free space, making a new list
+	 * entry if needed.  Note that we could get an OOM failure here, but if so
+	 * we have left the hashtable and the list in a consistent state.
+	 */
+	if (pgStatTabList == NULL)
+	{
+		/* Set up first pgStatTabList entry */
+		pgStatTabList = (TabStatusArray *)
+			MemoryContextAllocZero(TopMemoryContext,
+								   sizeof(TabStatusArray));
+	}
+
+	tsa = pgStatTabList;
+	while (tsa->tsa_used >= TABSTAT_QUANTUM)
+	{
+		if (tsa->tsa_next == NULL)
+			tsa->tsa_next = (TabStatusArray *)
+				MemoryContextAllocZero(TopMemoryContext,
+									   sizeof(TabStatusArray));
+		tsa = tsa->tsa_next;
+	}
+
+	/*
+	 * Allocate a PgStat_TableStatus entry within this list entry.  We assume
+	 * the entry was already zeroed, either at creation or after last use.
+	 */
+	entry = &tsa->tsa_entries[tsa->tsa_used++];
+	entry->t_id = rel_id;
+	entry->t_shared = isshared;
+
+	/*
+	 * Now we can fill the entry in pgStatTabHash.
+	 */
+	hash_entry->tsa_entry = entry;
+
+	return entry;
+}
+
+/*
+ * find_tabstat_entry - find any existing PgStat_TableStatus entry for rel
+ *
+ * If no entry, return NULL, don't create a new one
+ *
+ * Note: if we got an error in the most recent execution of pgstat_report_stat,
+ * it's possible that an entry exists but there's no hashtable entry for it.
+ * That's okay, we'll treat this case as "doesn't exist".
+ */
+PgStat_TableStatus *
+find_tabstat_entry(Oid rel_id)
+{
+	TabStatHashEntry *hash_entry;
+
+	/* If hashtable doesn't exist, there are no entries at all */
+	if (!pgStatTabHash)
+		return NULL;
+
+	hash_entry = hash_search(pgStatTabHash, &rel_id, HASH_FIND, NULL);
+	if (!hash_entry)
+		return NULL;
+
+	/* Note that this step could also return NULL, but that's correct */
+	return hash_entry->tsa_entry;
+}
+
+/*
+ * add_tabstat_xact_level - add a new (sub)transaction state record
+ */
+static void
+add_tabstat_xact_level(PgStat_TableStatus *pgstat_info, int nest_level)
+{
+	PgStat_SubXactStatus *xact_state;
+	PgStat_TableXactStatus *trans;
+
+	/*
+	 * If this is the first rel to be modified at the current nest level, we
+	 * first have to push a transaction stack entry.
+	 */
+	xact_state = pgstat_xact_stack_level_get(nest_level);
+
+	/* Now make a per-table stack entry */
+	trans = (PgStat_TableXactStatus *)
+		MemoryContextAllocZero(TopTransactionContext,
+							   sizeof(PgStat_TableXactStatus));
+	trans->nest_level = nest_level;
+	trans->upper = pgstat_info->trans;
+	trans->parent = pgstat_info;
+	trans->next = xact_state->first;
+	xact_state->first = trans;
+	pgstat_info->trans = trans;
+}
+
+/*
+ * Add a new (sub)transaction record if needed.
+ */
+static void
+ensure_tabstat_xact_level(PgStat_TableStatus *pgstat_info)
+{
+	int			nest_level = GetCurrentTransactionNestLevel();
+
+	if (pgstat_info->trans == NULL ||
+		pgstat_info->trans->nest_level != nest_level)
+		add_tabstat_xact_level(pgstat_info, nest_level);
+}
+
+/*
+ * pgstat_truncdrop_save_counters
+ *
+ * Whenever a table is truncated/dropped, we save its i/u/d counters so that
+ * they can be cleared, and if the (sub)xact that executed the truncate/drop
+ * later aborts, the counters can be restored to the saved (pre-truncate/drop)
+ * values.
+ *
+ * Note that for truncate we do this on the first truncate in any particular
+ * subxact level only.
+ */
+static void
+pgstat_truncdrop_save_counters(PgStat_TableXactStatus *trans, bool is_drop)
+{
+	if (!trans->truncdropped || is_drop)
+	{
+		trans->inserted_pre_truncdrop = trans->tuples_inserted;
+		trans->updated_pre_truncdrop = trans->tuples_updated;
+		trans->deleted_pre_truncdrop = trans->tuples_deleted;
+		trans->truncdropped = true;
+	}
+}
+
+/*
+ * pgstat_truncdrop_restore_counters - restore counters when a truncate aborts
+ */
+static void
+pgstat_truncdrop_restore_counters(PgStat_TableXactStatus *trans)
+{
+	if (trans->truncdropped)
+	{
+		trans->tuples_inserted = trans->inserted_pre_truncdrop;
+		trans->tuples_updated = trans->updated_pre_truncdrop;
+		trans->tuples_deleted = trans->deleted_pre_truncdrop;
+	}
+}
diff --git a/src/backend/utils/activity/pgstat_subscription.c b/src/backend/utils/activity/pgstat_subscription.c
new file mode 100644
index 00000000000..0bfbe814d76
--- /dev/null
+++ b/src/backend/utils/activity/pgstat_subscription.c
@@ -0,0 +1,78 @@
+/* -------------------------------------------------------------------------
+ *
+ * pgstat_subscription.c
+ *	  Implementation of subscription statistics.
+ *
+ * This file contains the implementation of subscription statistics. It is kept
+ * separate from pgstat.c to enforce the line between the statistics access /
+ * storage implementation and the details about individual types of
+ * statistics.
+ *
+ * Copyright (c) 2001-2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/backend/utils/activity/pgstat_subscription.c
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "utils/pgstat_internal.h"
+
+
+/* ----------
+ * pgstat_report_subscription_error() -
+ *
+ *	Tell the collector about the subscription error.
+ * ----------
+ */
+void
+pgstat_report_subscription_error(Oid subid, bool is_apply_error)
+{
+	PgStat_MsgSubscriptionError msg;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_SUBSCRIPTIONERROR);
+	msg.m_subid = subid;
+	msg.m_is_apply_error = is_apply_error;
+	pgstat_send(&msg, sizeof(PgStat_MsgSubscriptionError));
+}
+
+/* ----------
+ * pgstat_report_subscription_drop() -
+ *
+ *	Tell the collector about dropping the subscription.
+ * ----------
+ */
+void
+pgstat_report_subscription_drop(Oid subid)
+{
+	PgStat_MsgSubscriptionDrop msg;
+
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_SUBSCRIPTIONDROP);
+	msg.m_subid = subid;
+	pgstat_send(&msg, sizeof(PgStat_MsgSubscriptionDrop));
+}
+
+/* ----------
+ * pgstat_reset_subscription_counter() -
+ *
+ *	Tell the statistics collector to reset a single subscription
+ *	counter, or all subscription counters (when subid is InvalidOid).
+ *
+ *	Permission checking for this function is managed through the normal
+ *	GRANT system.
+ * ----------
+ */
+void
+pgstat_reset_subscription_counter(Oid subid)
+{
+	PgStat_MsgResetsubcounter msg;
+
+	if (pgStatSock == PGINVALID_SOCKET)
+		return;
+
+	msg.m_subid = subid;
+	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETSUBCOUNTER);
+
+	pgstat_send(&msg, sizeof(msg));
+}
-- 
2.35.1.354.g715d08a9e5

