From eda01880ac7667e8bae76e5d9767a81f36ef9399 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 08/11] pgstat: store statistics in shared memory.

TODO:
- verify all AFIXMEs
- Tests for stats dropping:
  - write regression test verifying that BEGIN;CREATE;USE;DROP; doesn't
    leave stats behind (for functions, tables).
  - same, but with different subtransaction cases
  - write tap test checking stats being dropped on replicas
  - think about / document checkpointer race conditions
  - check what needs to be done for index (and matview, ...?) stats
- consistent naming around replacing "stats collector"
  references. Don't like "activity statistics".
- Document stats_fetch_consistency / adjust existing documentation
  around stats fetching

Previously activity statistics is collected via sockets and shared
among backends through files periodically. Such files reaches tens of
megabytes and are created at most every 1 second and such large data
is serialized by stats collector then de-serialized on every backend
periodically. To evade that large cost, this patch places activity
statistics data on shared memory. Each backend accumulates statistics
numbers locally then tries to move them onto the shared statistics at
every transaction end but with intervals not shorter than 10 seconds.
Until 60 second has elapsed since the last flushing to shared stats,
lock failure postpones stats flushing to try to alleviate lock
contention that slows down transactions.  Finally stats flush waits
for locks so that shared statistics doesn't get stale.

Author: Kyotaro Horiguchi <horikyota.ntt@gmail.com>
Author: Andres Freund <andres@anarazel.de>
---
 src/include/access/xact.h                     |   37 +-
 src/include/catalog/pg_proc.dat               |    5 +
 src/include/miscadmin.h                       |    2 +-
 src/include/pgstat.h                          |  725 +--
 src/include/storage/lwlock.h                  |    1 +
 src/include/utils/pgstat_internal.h           |  456 +-
 src/include/utils/timeout.h                   |    1 +
 src/include/utils/wait_event.h                |    2 +-
 src/backend/access/heap/heapam_handler.c      |    4 +-
 src/backend/access/heap/vacuumlazy.c          |    2 +-
 src/backend/access/rmgrdesc/xactdesc.c        |   52 +
 src/backend/access/transam/twophase.c         |   53 +
 src/backend/access/transam/xact.c             |   62 +-
 src/backend/access/transam/xlog.c             |   26 +-
 src/backend/catalog/heap.c                    |    7 +
 src/backend/catalog/pg_proc.c                 |    5 +
 src/backend/commands/dbcommands.c             |    2 +-
 src/backend/commands/functioncmds.c           |    2 +
 src/backend/commands/matview.c                |    8 +-
 src/backend/commands/vacuum.c                 |    4 +-
 src/backend/commands/vacuumparallel.c         |    2 +
 src/backend/postmaster/autovacuum.c           |   70 +-
 src/backend/postmaster/bgwriter.c             |    4 +-
 src/backend/postmaster/checkpointer.c         |   36 +-
 src/backend/postmaster/pgarch.c               |   12 +-
 src/backend/postmaster/pgstat.c               | 5794 ++++++-----------
 src/backend/postmaster/postmaster.c           |   89 +-
 src/backend/postmaster/walwriter.c            |    6 +-
 src/backend/replication/basebackup.c          |    4 +-
 src/backend/replication/logical/logical.c     |    3 +-
 src/backend/replication/slot.c                |   25 +-
 src/backend/storage/buffer/bufmgr.c           |    8 +-
 src/backend/storage/ipc/ipci.c                |    2 +
 src/backend/storage/lmgr/lwlock.c             |    4 +-
 src/backend/storage/lmgr/lwlocknames.txt      |    1 +
 src/backend/storage/smgr/smgr.c               |    5 -
 src/backend/tcop/postgres.c                   |   41 +-
 src/backend/utils/activity/pgstat_database.c  |  338 +-
 src/backend/utils/activity/pgstat_function.c  |  179 +-
 src/backend/utils/activity/pgstat_global.c    |  812 ++-
 src/backend/utils/activity/pgstat_relation.c  |  683 +-
 .../utils/activity/pgstat_subscription.c      |  113 +-
 src/backend/utils/activity/wait_event.c       |    4 +-
 src/backend/utils/adt/pgstatfuncs.c           |   26 +-
 src/backend/utils/cache/relcache.c            |    4 +
 src/backend/utils/init/globals.c              |    1 +
 src/backend/utils/init/miscinit.c             |    3 -
 src/backend/utils/init/postinit.c             |   12 +
 src/backend/utils/misc/guc.c                  |   20 +
 src/backend/utils/misc/postgresql.conf.sample |    3 +-
 src/bin/pg_basebackup/t/010_pg_basebackup.pl  |    2 +-
 doc/src/sgml/monitoring.sgml                  |    3 +-
 src/test/regress/expected/stats.out           |    8 +
 src/test/regress/sql/stats.sql                |   10 +
 src/tools/pgindent/typedefs.list              |   51 +-
 55 files changed, 4317 insertions(+), 5517 deletions(-)

diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 062cc7e17d8..a1b15918738 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -180,6 +180,7 @@ typedef struct SavedTransactionCharacteristics
 #define XACT_XINFO_HAS_ORIGIN			(1U << 5)
 #define XACT_XINFO_HAS_AE_LOCKS			(1U << 6)
 #define XACT_XINFO_HAS_GID				(1U << 7)
+#define XACT_XINFO_HAS_DROPPED_STATS	(1U << 8)
 
 /*
  * Also stored in xinfo, these indicating a variety of additional actions that
@@ -230,7 +231,7 @@ typedef struct xl_xact_assignment
 typedef struct xl_xact_xinfo
 {
 	/*
-	 * Even though we right now only require 1 byte of space in xinfo we use
+	 * Even though we right now only require two bytes of space in xinfo we use
 	 * four so following records don't have to care about alignment. Commit
 	 * records can be large, so copying large portions isn't attractive.
 	 */
@@ -257,6 +258,24 @@ typedef struct xl_xact_relfilenodes
 } xl_xact_relfilenodes;
 #define MinSizeOfXactRelfilenodes offsetof(xl_xact_relfilenodes, xnodes)
 
+/* FIXME: circular dependency issues */
+/*
+ * A transactionally dropped statistics entry.
+ */
+typedef struct PgStat_DroppedStatsItem
+{
+	int			kind;
+	Oid			dboid;
+	Oid			objoid;
+} PgStat_DroppedStatsItem;
+
+typedef struct xl_xact_dropped_stats
+{
+	int		ndropped;
+	PgStat_DroppedStatsItem dropped_stats[FLEXIBLE_ARRAY_MEMBER];
+} xl_xact_dropped_stats;
+#define MinSizeOfXactDroppedStats offsetof(xl_xact_dropped_stats, dropped_stats)
+
 typedef struct xl_xact_invals
 {
 	int			nmsgs;			/* number of shared inval msgs */
@@ -283,6 +302,7 @@ typedef struct xl_xact_commit
 	/* xl_xact_dbinfo follows if XINFO_HAS_DBINFO */
 	/* xl_xact_subxacts follows if XINFO_HAS_SUBXACT */
 	/* xl_xact_relfilenodes follows if XINFO_HAS_RELFILENODES */
+	/* xl_xact_dropped_stats follows if XINFO_HAS_DROPPED_STATS */
 	/* xl_xact_invals follows if XINFO_HAS_INVALS */
 	/* xl_xact_twophase follows if XINFO_HAS_TWOPHASE */
 	/* twophase_gid follows if XINFO_HAS_GID. As a null-terminated string. */
@@ -298,6 +318,7 @@ typedef struct xl_xact_abort
 	/* xl_xact_dbinfo follows if XINFO_HAS_DBINFO */
 	/* xl_xact_subxacts follows if XINFO_HAS_SUBXACT */
 	/* xl_xact_relfilenodes follows if XINFO_HAS_RELFILENODES */
+	/* xl_xact_dropped_stats follows if XINFO_HAS_DROPPED_STATS */
 	/* No invalidation messages needed. */
 	/* xl_xact_twophase follows if XINFO_HAS_TWOPHASE */
 	/* twophase_gid follows if XINFO_HAS_GID. As a null-terminated string. */
@@ -316,6 +337,8 @@ typedef struct xl_xact_prepare
 	int32		nsubxacts;		/* number of following subxact XIDs */
 	int32		ncommitrels;	/* number of delete-on-commit rels */
 	int32		nabortrels;		/* number of delete-on-abort rels */
+	int32		ncommitdroppedstats;  /* number of stats to drop on commit */
+	int32		nabortdroppedstats;  /* number of stats to drop on abort */
 	int32		ninvalmsgs;		/* number of cache invalidation messages */
 	bool		initfileinval;	/* does relcache init file need invalidation? */
 	uint16		gidlen;			/* length of the GID - GID follows the header */
@@ -342,6 +365,9 @@ typedef struct xl_xact_parsed_commit
 	int			nrels;
 	RelFileNode *xnodes;
 
+	int			ndroppedstats;
+	PgStat_DroppedStatsItem *droppedstats;
+
 	int			nmsgs;
 	SharedInvalidationMessage *msgs;
 
@@ -349,6 +375,8 @@ typedef struct xl_xact_parsed_commit
 	char		twophase_gid[GIDSIZE];	/* only for 2PC */
 	int			nabortrels;		/* only for 2PC */
 	RelFileNode *abortnodes;	/* only for 2PC */
+	int			nabortdroppedstats;		/* only for 2PC */
+	PgStat_DroppedStatsItem *abortdroppedstats; /* only for 2PC */
 
 	XLogRecPtr	origin_lsn;
 	TimestampTz origin_timestamp;
@@ -370,6 +398,9 @@ typedef struct xl_xact_parsed_abort
 	int			nrels;
 	RelFileNode *xnodes;
 
+	int			ndroppedstats;
+	PgStat_DroppedStatsItem *droppedstats;
+
 	TransactionId twophase_xid; /* only for 2PC */
 	char		twophase_gid[GIDSIZE];	/* only for 2PC */
 
@@ -449,6 +480,8 @@ extern int	xactGetCommittedChildren(TransactionId **ptr);
 extern XLogRecPtr XactLogCommitRecord(TimestampTz commit_time,
 									  int nsubxacts, TransactionId *subxacts,
 									  int nrels, RelFileNode *rels,
+									  int ndroppedstats,
+									  PgStat_DroppedStatsItem *droppedstats,
 									  int nmsgs, SharedInvalidationMessage *msgs,
 									  bool relcacheInval,
 									  int xactflags,
@@ -458,6 +491,8 @@ extern XLogRecPtr XactLogCommitRecord(TimestampTz commit_time,
 extern XLogRecPtr XactLogAbortRecord(TimestampTz abort_time,
 									 int nsubxacts, TransactionId *subxacts,
 									 int nrels, RelFileNode *rels,
+									 int ndroppedstats,
+									 PgStat_DroppedStatsItem *droppedstats,
 									 int xactflags, TransactionId twophase_xid,
 									 const char *twophase_gid);
 extern void xact_redo(XLogReaderState *record);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index bf888581716..bba33f593f0 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5744,6 +5744,11 @@
   proname => 'pg_stat_clear_snapshot', proisstrict => 'f', provolatile => 'v',
   proparallel => 'r', prorettype => 'void', proargtypes => '',
   prosrc => 'pg_stat_clear_snapshot' },
+{ oid => '2137',
+  descr => 'statistics: force stats to be flushed after the next commit',
+  proname => 'pg_stat_force_next_flush', proisstrict => 'f', provolatile => 'v',
+  proparallel => 'r', prorettype => 'void', proargtypes => '',
+  prosrc => 'pg_stat_force_next_flush' },
 { oid => '2274',
   descr => 'statistics: reset collected statistics for current database',
   proname => 'pg_stat_reset', proisstrict => 'f', provolatile => 'v',
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 0abc3ad5405..1d89b2759d2 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -94,6 +94,7 @@ extern PGDLLIMPORT volatile sig_atomic_t IdleInTransactionSessionTimeoutPending;
 extern PGDLLIMPORT volatile sig_atomic_t IdleSessionTimeoutPending;
 extern PGDLLIMPORT volatile sig_atomic_t ProcSignalBarrierPending;
 extern PGDLLIMPORT volatile sig_atomic_t LogMemoryContextPending;
+extern PGDLLIMPORT volatile sig_atomic_t IdleStatsUpdateTimeoutPending;
 
 extern PGDLLIMPORT volatile sig_atomic_t CheckClientConnectionPending;
 extern PGDLLIMPORT volatile sig_atomic_t ClientConnectionLost;
@@ -333,7 +334,6 @@ typedef enum BackendType
 	B_WAL_SENDER,
 	B_WAL_WRITER,
 	B_ARCHIVER,
-	B_STATS_COLLECTOR,
 	B_LOGGER,
 } BackendType;
 
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 0fbf198fdd1..a9a72d01f2b 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -1,7 +1,7 @@
 /* ----------
  *	pgstat.h
  *
- *	Definitions for the PostgreSQL statistics collector daemon.
+ *	Definitions for the PostgreSQL activity statistics facility.
  *
  *	Copyright (c) 2001-2022, PostgreSQL Global Development Group
  *
@@ -12,12 +12,11 @@
 #define PGSTAT_H
 
 #include "datatype/timestamp.h"
+#include "executor/instrument.h"
 #include "portability/instr_time.h"
 #include "postmaster/pgarch.h"	/* for MAX_XFN_CHARS */
-#include "replication/logicalproto.h"
 #include "utils/backend_progress.h" /* for backward compatibility */
 #include "utils/backend_status.h"	/* for backward compatibility */
-#include "utils/hsearch.h"
 #include "utils/relcache.h"
 #include "utils/wait_event.h"	/* for backward compatibility */
 
@@ -27,12 +26,30 @@
  * ----------
  */
 #define PGSTAT_STAT_PERMANENT_DIRECTORY		"pg_stat"
-#define PGSTAT_STAT_PERMANENT_FILENAME		"pg_stat/global.stat"
-#define PGSTAT_STAT_PERMANENT_TMPFILE		"pg_stat/global.tmp"
+#define PGSTAT_STAT_PERMANENT_FILENAME		"pg_stat/saved_stats"
+#define PGSTAT_STAT_PERMANENT_TMPFILE		"pg_stat/saved_stats.tmp"
 
 /* Default directory to store temporary statistics data in */
 #define PG_STAT_TMP_DIR		"pg_stat_tmp"
 
+/* The types of statistics entries */
+typedef enum PgStatKind
+{
+	/* stats with a variable number of entries */
+	PGSTAT_KIND_DB,				/* database-wide statistics */
+	PGSTAT_KIND_TABLE,			/* per-table statistics */
+	PGSTAT_KIND_FUNCTION,		/* per-function statistics */
+	PGSTAT_KIND_SUBSCRIPTION,	/* per-subscription statistics */
+
+	/* stats with a constant number of entries */
+	PGSTAT_KIND_ARCHIVER,
+	PGSTAT_KIND_BGWRITER,
+	PGSTAT_KIND_CHECKPOINTER,
+	PGSTAT_KIND_REPLSLOT,
+	PGSTAT_KIND_SLRU,
+	PGSTAT_KIND_WAL,
+} PgStatKind;
+#define PGSTAT_KIND_LAST PGSTAT_KIND_WAL
 
 /* Values for track_functions GUC variable --- order is significant! */
 typedef enum TrackFunctionsLevel
@@ -42,6 +59,13 @@ typedef enum TrackFunctionsLevel
 	TRACK_FUNC_ALL
 }			TrackFunctionsLevel;
 
+typedef enum PgStatsFetchConsistency
+{
+	STATS_FETCH_CONSISTENCY_NONE,
+	STATS_FETCH_CONSISTENCY_CACHE,
+	STATS_FETCH_CONSISTENCY_SNAPSHOT,
+} PgStatsFetchConsistency;
+
 /* Values to track the cause of session termination */
 typedef enum SessionEndType
 {
@@ -83,9 +107,8 @@ typedef enum PgStat_Single_Reset_Type
  * PgStat_TableCounts			The actual per-table counts kept by a backend
  *
  * This struct should contain only actual event counters, because we memcmp
- * it against zeroes to detect whether there are any counts to transmit.
- * It is a component of PgStat_TableStatus (within-backend state) and
- * PgStat_TableEntry (the transmitted message format).
+ * it against zeroes to detect whether there are any stats updates to apply.
+ * It is a component of PgStat_TableStatus (within-backend state).
  *
  * Note: for a table, tuples_returned is the number of tuples successfully
  * fetched by heap_getnext, while tuples_fetched is the number of tuples
@@ -138,10 +161,9 @@ typedef struct PgStat_TableCounts
  */
 typedef struct PgStat_TableStatus
 {
-	Oid			t_id;			/* table's OID */
-	bool		t_shared;		/* is it a shared catalog? */
 	struct PgStat_TableXactStatus *trans;	/* lowest subxact's counts */
 	PgStat_TableCounts t_counts;	/* event counts to be sent */
+	Relation	relation;			/* rel that is using this entry */
 } PgStat_TableStatus;
 
 /* ----------
@@ -172,10 +194,10 @@ typedef struct PgStat_TableXactStatus
  * PgStat_FunctionCounts	The actual per-function counts kept by a backend
  *
  * This struct should contain only actual event counters, because we memcmp
- * it against zeroes to detect whether there are any counts to transmit.
+ * it against zeroes to detect whether there are any counts to write.
  *
  * Note that the time counters are in instr_time format here.  We convert to
- * microseconds in PgStat_Counter format when transmitting to the collector.
+ * microseconds in PgStat_Counter format when updating the shared statistics.
  * ----------
  */
 typedef struct PgStat_FunctionCounts
@@ -186,12 +208,11 @@ typedef struct PgStat_FunctionCounts
 } PgStat_FunctionCounts;
 
 /* ----------
- * PgStat_BackendFunctionEntry	Entry in backend's per-function hash table
+ * PgStat_BackendFunctionEntry	Non-flushed function stats.
  * ----------
  */
 typedef struct PgStat_BackendFunctionEntry
 {
-	Oid			f_id;
 	PgStat_FunctionCounts f_counts;
 } PgStat_BackendFunctionEntry;
 
@@ -211,570 +232,19 @@ typedef struct PgStat_FunctionCallUsage
 	instr_time	f_start;
 } PgStat_FunctionCallUsage;
 
-
-/* ------------------------------------------------------------
- * Message formats follow
- * ------------------------------------------------------------
- */
-
 /* ----------
- * The types of backend -> collector messages
+ * PgStat_BackendSubEntry	Non-flushed subscription stats.
  * ----------
  */
-typedef enum StatMsgType
+typedef struct PgStat_BackendSubEntry
 {
-	PGSTAT_MTYPE_DUMMY,
-	PGSTAT_MTYPE_INQUIRY,
-	PGSTAT_MTYPE_TABSTAT,
-	PGSTAT_MTYPE_TABPURGE,
-	PGSTAT_MTYPE_DROPDB,
-	PGSTAT_MTYPE_RESETCOUNTER,
-	PGSTAT_MTYPE_RESETSHAREDCOUNTER,
-	PGSTAT_MTYPE_RESETSINGLECOUNTER,
-	PGSTAT_MTYPE_RESETSLRUCOUNTER,
-	PGSTAT_MTYPE_RESETREPLSLOTCOUNTER,
-	PGSTAT_MTYPE_RESETSUBCOUNTER,
-	PGSTAT_MTYPE_AUTOVAC_START,
-	PGSTAT_MTYPE_VACUUM,
-	PGSTAT_MTYPE_ANALYZE,
-	PGSTAT_MTYPE_ARCHIVER,
-	PGSTAT_MTYPE_BGWRITER,
-	PGSTAT_MTYPE_CHECKPOINTER,
-	PGSTAT_MTYPE_WAL,
-	PGSTAT_MTYPE_SLRU,
-	PGSTAT_MTYPE_FUNCSTAT,
-	PGSTAT_MTYPE_FUNCPURGE,
-	PGSTAT_MTYPE_RECOVERYCONFLICT,
-	PGSTAT_MTYPE_TEMPFILE,
-	PGSTAT_MTYPE_DEADLOCK,
-	PGSTAT_MTYPE_CHECKSUMFAILURE,
-	PGSTAT_MTYPE_REPLSLOT,
-	PGSTAT_MTYPE_CONNECT,
-	PGSTAT_MTYPE_DISCONNECT,
-	PGSTAT_MTYPE_SUBSCRIPTIONDROP,
-	PGSTAT_MTYPE_SUBSCRIPTIONERROR,
-} StatMsgType;
-
-/* ----------
- * PgStat_MsgHdr				The common message header
- * ----------
- */
-typedef struct PgStat_MsgHdr
-{
-	StatMsgType m_type;
-	int			m_size;
-} PgStat_MsgHdr;
-
-/* ----------
- * Space available in a message.  This will keep the UDP packets below 1K,
- * which should fit unfragmented into the MTU of the loopback interface.
- * (Larger values of PGSTAT_MAX_MSG_SIZE would work for that on most
- * platforms, but we're being conservative here.)
- * ----------
- */
-#define PGSTAT_MAX_MSG_SIZE 1000
-#define PGSTAT_MSG_PAYLOAD	(PGSTAT_MAX_MSG_SIZE - sizeof(PgStat_MsgHdr))
-
-
-/* ----------
- * PgStat_MsgDummy				A dummy message, ignored by the collector
- * ----------
- */
-typedef struct PgStat_MsgDummy
-{
-	PgStat_MsgHdr m_hdr;
-} PgStat_MsgDummy;
-
-
-/* ----------
- * PgStat_MsgInquiry			Sent by a backend to ask the collector
- *								to write the stats file(s).
- *
- * Ordinarily, an inquiry message prompts writing of the global stats file,
- * the stats file for shared catalogs, and the stats file for the specified
- * database.  If databaseid is InvalidOid, only the first two are written.
- *
- * New file(s) will be written only if the existing file has a timestamp
- * older than the specified cutoff_time; this prevents duplicated effort
- * when multiple requests arrive at nearly the same time, assuming that
- * backends send requests with cutoff_times a little bit in the past.
- *
- * clock_time should be the requestor's current local time; the collector
- * uses this to check for the system clock going backward, but it has no
- * effect unless that occurs.  We assume clock_time >= cutoff_time, though.
- * ----------
- */
-
-typedef struct PgStat_MsgInquiry
-{
-	PgStat_MsgHdr m_hdr;
-	TimestampTz clock_time;		/* observed local clock time */
-	TimestampTz cutoff_time;	/* minimum acceptable file timestamp */
-	Oid			databaseid;		/* requested DB (InvalidOid => shared only) */
-} PgStat_MsgInquiry;
-
-
-/* ----------
- * PgStat_TableEntry			Per-table info in a MsgTabstat
- * ----------
- */
-typedef struct PgStat_TableEntry
-{
-	Oid			t_id;
-	PgStat_TableCounts t_counts;
-} PgStat_TableEntry;
-
-/* ----------
- * PgStat_MsgTabstat			Sent by the backend to report table
- *								and buffer access statistics.
- * ----------
- */
-#define PGSTAT_NUM_TABENTRIES  \
-	((PGSTAT_MSG_PAYLOAD - sizeof(Oid) - 3 * sizeof(int) - 5 * sizeof(PgStat_Counter)) \
-	 / sizeof(PgStat_TableEntry))
-
-typedef struct PgStat_MsgTabstat
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	int			m_nentries;
-	int			m_xact_commit;
-	int			m_xact_rollback;
-	PgStat_Counter m_block_read_time;	/* times in microseconds */
-	PgStat_Counter m_block_write_time;
-	PgStat_Counter m_session_time;
-	PgStat_Counter m_active_time;
-	PgStat_Counter m_idle_in_xact_time;
-	PgStat_TableEntry m_entry[PGSTAT_NUM_TABENTRIES];
-} PgStat_MsgTabstat;
-
-/* ----------
- * PgStat_MsgTabpurge			Sent by the backend to tell the collector
- *								about dead tables.
- * ----------
- */
-#define PGSTAT_NUM_TABPURGE  \
-	((PGSTAT_MSG_PAYLOAD - sizeof(Oid) - sizeof(int))  \
-	 / sizeof(Oid))
-
-typedef struct PgStat_MsgTabpurge
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	int			m_nentries;
-	Oid			m_tableid[PGSTAT_NUM_TABPURGE];
-} PgStat_MsgTabpurge;
-
-/* ----------
- * PgStat_MsgDropdb				Sent by the backend to tell the collector
- *								about a dropped database
- * ----------
- */
-typedef struct PgStat_MsgDropdb
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-} PgStat_MsgDropdb;
-
-/* ----------
- * PgStat_MsgResetcounter		Sent by the backend to tell the collector
- *								to reset counters
- * ----------
- */
-typedef struct PgStat_MsgResetcounter
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-} PgStat_MsgResetcounter;
-
-/* ----------
- * PgStat_MsgResetsharedcounter Sent by the backend to tell the collector
- *								to reset a shared counter
- * ----------
- */
-typedef struct PgStat_MsgResetsharedcounter
-{
-	PgStat_MsgHdr m_hdr;
-	PgStat_Shared_Reset_Target m_resettarget;
-} PgStat_MsgResetsharedcounter;
-
-/* ----------
- * PgStat_MsgResetsinglecounter Sent by the backend to tell the collector
- *								to reset a single counter
- * ----------
- */
-typedef struct PgStat_MsgResetsinglecounter
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	PgStat_Single_Reset_Type m_resettype;
-	Oid			m_objectid;
-} PgStat_MsgResetsinglecounter;
-
-/* ----------
- * PgStat_MsgResetslrucounter Sent by the backend to tell the collector
- *								to reset a SLRU counter
- * ----------
- */
-typedef struct PgStat_MsgResetslrucounter
-{
-	PgStat_MsgHdr m_hdr;
-	int			m_index;
-} PgStat_MsgResetslrucounter;
-
-/* ----------
- * PgStat_MsgResetreplslotcounter Sent by the backend to tell the collector
- *								to reset replication slot counter(s)
- * ----------
- */
-typedef struct PgStat_MsgResetreplslotcounter
-{
-	PgStat_MsgHdr m_hdr;
-	NameData	m_slotname;
-	bool		clearall;
-} PgStat_MsgResetreplslotcounter;
-
-/* ----------
- * PgStat_MsgResetsubcounter Sent by the backend to tell the collector
- *								to reset subscription counter(s)
- * ----------
- */
-typedef struct PgStat_MsgResetsubcounter
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_subid;		/* InvalidOid means reset all subscription
-								 * stats */
-} PgStat_MsgResetsubcounter;
-
-
-/* ----------
- * PgStat_MsgAutovacStart		Sent by the autovacuum daemon to signal
- *								that a database is going to be processed
- * ----------
- */
-typedef struct PgStat_MsgAutovacStart
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	TimestampTz m_start_time;
-} PgStat_MsgAutovacStart;
-
-
-/* ----------
- * PgStat_MsgVacuum				Sent by the backend or autovacuum daemon
- *								after VACUUM
- * ----------
- */
-typedef struct PgStat_MsgVacuum
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	Oid			m_tableoid;
-	bool		m_autovacuum;
-	TimestampTz m_vacuumtime;
-	PgStat_Counter m_live_tuples;
-	PgStat_Counter m_dead_tuples;
-} PgStat_MsgVacuum;
-
-/* ----------
- * PgStat_MsgAnalyze			Sent by the backend or autovacuum daemon
- *								after ANALYZE
- * ----------
- */
-typedef struct PgStat_MsgAnalyze
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	Oid			m_tableoid;
-	bool		m_autovacuum;
-	bool		m_resetcounter;
-	TimestampTz m_analyzetime;
-	PgStat_Counter m_live_tuples;
-	PgStat_Counter m_dead_tuples;
-} PgStat_MsgAnalyze;
-
-
-/* ----------
- * PgStat_MsgArchiver			Sent by the archiver to update statistics.
- * ----------
- */
-typedef struct PgStat_MsgArchiver
-{
-	PgStat_MsgHdr m_hdr;
-	bool		m_failed;		/* Failed attempt */
-	char		m_xlog[MAX_XFN_CHARS + 1];
-	TimestampTz m_timestamp;
-} PgStat_MsgArchiver;
-
-/* ----------
- * PgStat_MsgBgWriter			Sent by the bgwriter to update statistics.
- * ----------
- */
-typedef struct PgStat_MsgBgWriter
-{
-	PgStat_MsgHdr m_hdr;
-
-	PgStat_Counter m_buf_written_clean;
-	PgStat_Counter m_maxwritten_clean;
-	PgStat_Counter m_buf_alloc;
-} PgStat_MsgBgWriter;
-
-/* ----------
- * PgStat_MsgCheckpointer  Sent by the checkpointer to update statistics.
- * ----------
- */
-typedef struct PgStat_MsgCheckpointer
-{
-	PgStat_MsgHdr m_hdr;
-
-	PgStat_Counter m_timed_checkpoints;
-	PgStat_Counter m_requested_checkpoints;
-	PgStat_Counter m_buf_written_checkpoints;
-	PgStat_Counter m_buf_written_backend;
-	PgStat_Counter m_buf_fsync_backend;
-	PgStat_Counter m_checkpoint_write_time; /* times in milliseconds */
-	PgStat_Counter m_checkpoint_sync_time;
-} PgStat_MsgCheckpointer;
-
-/* ----------
- * PgStat_MsgWal			Sent by backends and background processes to update WAL statistics.
- * ----------
- */
-typedef struct PgStat_MsgWal
-{
-	PgStat_MsgHdr m_hdr;
-	PgStat_Counter m_wal_records;
-	PgStat_Counter m_wal_fpi;
-	uint64		m_wal_bytes;
-	PgStat_Counter m_wal_buffers_full;
-	PgStat_Counter m_wal_write;
-	PgStat_Counter m_wal_sync;
-	PgStat_Counter m_wal_write_time;	/* time spent writing wal records in
-										 * microseconds */
-	PgStat_Counter m_wal_sync_time; /* time spent syncing wal records in
-									 * microseconds */
-} PgStat_MsgWal;
-
-/* ----------
- * PgStat_MsgSLRU			Sent by a backend to update SLRU statistics.
- * ----------
- */
-typedef struct PgStat_MsgSLRU
-{
-	PgStat_MsgHdr m_hdr;
-	PgStat_Counter m_index;
-	PgStat_Counter m_blocks_zeroed;
-	PgStat_Counter m_blocks_hit;
-	PgStat_Counter m_blocks_read;
-	PgStat_Counter m_blocks_written;
-	PgStat_Counter m_blocks_exists;
-	PgStat_Counter m_flush;
-	PgStat_Counter m_truncate;
-} PgStat_MsgSLRU;
-
-/* ----------
- * PgStat_MsgReplSlot	Sent by a backend or a wal sender to update replication
- *						slot statistics.
- * ----------
- */
-typedef struct PgStat_MsgReplSlot
-{
-	PgStat_MsgHdr m_hdr;
-	NameData	m_slotname;
-	bool		m_create;
-	bool		m_drop;
-	PgStat_Counter m_spill_txns;
-	PgStat_Counter m_spill_count;
-	PgStat_Counter m_spill_bytes;
-	PgStat_Counter m_stream_txns;
-	PgStat_Counter m_stream_count;
-	PgStat_Counter m_stream_bytes;
-	PgStat_Counter m_total_txns;
-	PgStat_Counter m_total_bytes;
-} PgStat_MsgReplSlot;
-
-/* ----------
- * PgStat_MsgSubscriptionDrop	Sent by the backend and autovacuum to tell the
- *								collector about the dead subscription.
- * ----------
- */
-typedef struct PgStat_MsgSubscriptionDrop
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_subid;
-} PgStat_MsgSubscriptionDrop;
-
-/* ----------
- * PgStat_MsgSubscriptionError	Sent by the apply worker or the table sync
- *								worker to report an error on the subscription.
- * ----------
- */
-typedef struct PgStat_MsgSubscriptionError
-{
-	PgStat_MsgHdr m_hdr;
-
-	Oid			m_subid;
-	bool		m_is_apply_error;
-} PgStat_MsgSubscriptionError;
-
-/* ----------
- * PgStat_MsgRecoveryConflict	Sent by the backend upon recovery conflict
- * ----------
- */
-typedef struct PgStat_MsgRecoveryConflict
-{
-	PgStat_MsgHdr m_hdr;
-
-	Oid			m_databaseid;
-	int			m_reason;
-} PgStat_MsgRecoveryConflict;
-
-/* ----------
- * PgStat_MsgTempFile	Sent by the backend upon creating a temp file
- * ----------
- */
-typedef struct PgStat_MsgTempFile
-{
-	PgStat_MsgHdr m_hdr;
-
-	Oid			m_databaseid;
-	size_t		m_filesize;
-} PgStat_MsgTempFile;
-
-/* ----------
- * PgStat_FunctionEntry			Per-function info in a MsgFuncstat
- * ----------
- */
-typedef struct PgStat_FunctionEntry
-{
-	Oid			f_id;
-	PgStat_Counter f_numcalls;
-	PgStat_Counter f_total_time;	/* times in microseconds */
-	PgStat_Counter f_self_time;
-} PgStat_FunctionEntry;
-
-/* ----------
- * PgStat_MsgFuncstat			Sent by the backend to report function
- *								usage statistics.
- * ----------
- */
-#define PGSTAT_NUM_FUNCENTRIES	\
-	((PGSTAT_MSG_PAYLOAD - sizeof(Oid) - sizeof(int))  \
-	 / sizeof(PgStat_FunctionEntry))
-
-typedef struct PgStat_MsgFuncstat
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	int			m_nentries;
-	PgStat_FunctionEntry m_entry[PGSTAT_NUM_FUNCENTRIES];
-} PgStat_MsgFuncstat;
-
-/* ----------
- * PgStat_MsgFuncpurge			Sent by the backend to tell the collector
- *								about dead functions.
- * ----------
- */
-#define PGSTAT_NUM_FUNCPURGE  \
-	((PGSTAT_MSG_PAYLOAD - sizeof(Oid) - sizeof(int))  \
-	 / sizeof(Oid))
-
-typedef struct PgStat_MsgFuncpurge
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	int			m_nentries;
-	Oid			m_functionid[PGSTAT_NUM_FUNCPURGE];
-} PgStat_MsgFuncpurge;
-
-/* ----------
- * PgStat_MsgDeadlock			Sent by the backend to tell the collector
- *								about a deadlock that occurred.
- * ----------
- */
-typedef struct PgStat_MsgDeadlock
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-} PgStat_MsgDeadlock;
-
-/* ----------
- * PgStat_MsgChecksumFailure	Sent by the backend to tell the collector
- *								about checksum failures noticed.
- * ----------
- */
-typedef struct PgStat_MsgChecksumFailure
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	int			m_failurecount;
-	TimestampTz m_failure_time;
-} PgStat_MsgChecksumFailure;
-
-/* ----------
- * PgStat_MsgConnect			Sent by the backend upon connection
- *								establishment
- * ----------
- */
-typedef struct PgStat_MsgConnect
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-} PgStat_MsgConnect;
-
-/* ----------
- * PgStat_MsgDisconnect			Sent by the backend when disconnecting
- * ----------
- */
-typedef struct PgStat_MsgDisconnect
-{
-	PgStat_MsgHdr m_hdr;
-	Oid			m_databaseid;
-	SessionEndType m_cause;
-} PgStat_MsgDisconnect;
-
-/* ----------
- * PgStat_Msg					Union over all possible messages.
- * ----------
- */
-typedef union PgStat_Msg
-{
-	PgStat_MsgHdr msg_hdr;
-	PgStat_MsgDummy msg_dummy;
-	PgStat_MsgInquiry msg_inquiry;
-	PgStat_MsgTabstat msg_tabstat;
-	PgStat_MsgTabpurge msg_tabpurge;
-	PgStat_MsgDropdb msg_dropdb;
-	PgStat_MsgResetcounter msg_resetcounter;
-	PgStat_MsgResetsharedcounter msg_resetsharedcounter;
-	PgStat_MsgResetsinglecounter msg_resetsinglecounter;
-	PgStat_MsgResetslrucounter msg_resetslrucounter;
-	PgStat_MsgResetreplslotcounter msg_resetreplslotcounter;
-	PgStat_MsgResetsubcounter msg_resetsubcounter;
-	PgStat_MsgAutovacStart msg_autovacuum_start;
-	PgStat_MsgVacuum msg_vacuum;
-	PgStat_MsgAnalyze msg_analyze;
-	PgStat_MsgArchiver msg_archiver;
-	PgStat_MsgBgWriter msg_bgwriter;
-	PgStat_MsgCheckpointer msg_checkpointer;
-	PgStat_MsgWal msg_wal;
-	PgStat_MsgSLRU msg_slru;
-	PgStat_MsgFuncstat msg_funcstat;
-	PgStat_MsgFuncpurge msg_funcpurge;
-	PgStat_MsgRecoveryConflict msg_recoveryconflict;
-	PgStat_MsgDeadlock msg_deadlock;
-	PgStat_MsgTempFile msg_tempfile;
-	PgStat_MsgChecksumFailure msg_checksumfailure;
-	PgStat_MsgReplSlot msg_replslot;
-	PgStat_MsgConnect msg_connect;
-	PgStat_MsgDisconnect msg_disconnect;
-	PgStat_MsgSubscriptionError msg_subscriptionerror;
-	PgStat_MsgSubscriptionDrop msg_subscriptiondrop;
-} PgStat_Msg;
+	PgStat_Counter apply_error_count;
+	PgStat_Counter sync_error_count;
+} PgStat_BackendSubEntry;
 
 
 /* ------------------------------------------------------------
- * Statistic collector data structures follow
+ * Activity statistics data structures on disk and in shared memory follow
  *
  * PGSTAT_FILE_FORMAT_ID should be changed whenever any of these
  * data structures change.
@@ -784,12 +254,11 @@ typedef union PgStat_Msg
 #define PGSTAT_FILE_FORMAT_ID	0x01A5BCA6
 
 /* ----------
- * PgStat_StatDBEntry			The collector's data per database
+ * PgStat_StatDBEntry			The per database statistics
  * ----------
  */
 typedef struct PgStat_StatDBEntry
 {
-	Oid			databaseid;
 	PgStat_Counter n_xact_commit;
 	PgStat_Counter n_xact_rollback;
 	PgStat_Counter n_blocks_fetched;
@@ -821,24 +290,15 @@ typedef struct PgStat_StatDBEntry
 	PgStat_Counter n_sessions_killed;
 
 	TimestampTz stat_reset_timestamp;
-	TimestampTz stats_timestamp;	/* time of db stats file update */
-
-	/*
-	 * tables and functions must be last in the struct, because we don't write
-	 * the pointers out to the stats file.
-	 */
-	HTAB	   *tables;
-	HTAB	   *functions;
 } PgStat_StatDBEntry;
 
+
 /* ----------
- * PgStat_StatTabEntry			The collector's data per table (or index)
+ * PgStat_StatTabEntry			The per table (or index) statistics
  * ----------
  */
 typedef struct PgStat_StatTabEntry
 {
-	Oid			tableid;
-
 	PgStat_Counter numscans;
 
 	PgStat_Counter tuples_returned;
@@ -867,23 +327,19 @@ typedef struct PgStat_StatTabEntry
 	PgStat_Counter autovac_analyze_count;
 } PgStat_StatTabEntry;
 
+
 /* ----------
- * PgStat_StatFuncEntry			The collector's data per function
+ * PgStat_StatFuncEntry			The per function statistics
  * ----------
  */
 typedef struct PgStat_StatFuncEntry
 {
-	Oid			functionid;
-
 	PgStat_Counter f_numcalls;
 
 	PgStat_Counter f_total_time;	/* times in microseconds */
 	PgStat_Counter f_self_time;
 } PgStat_StatFuncEntry;
 
-/*
- * Archiver statistics kept in the stats collector
- */
 typedef struct PgStat_ArchiverStats
 {
 	PgStat_Counter archived_count;	/* archival successes */
@@ -897,9 +353,6 @@ typedef struct PgStat_ArchiverStats
 	TimestampTz stat_reset_timestamp;
 } PgStat_ArchiverStats;
 
-/*
- * Background writer statistics kept in the stats collector
- */
 typedef struct PgStat_BgWriterStats
 {
 	PgStat_Counter buf_written_clean;
@@ -908,12 +361,8 @@ typedef struct PgStat_BgWriterStats
 	TimestampTz stat_reset_timestamp;
 } PgStat_BgWriterStats;
 
-/*
- * Checkpointer statistics kept in the stats collector
- */
 typedef struct PgStat_CheckpointerStats
 {
-	TimestampTz stats_timestamp;	/* time of stats file update */
 	PgStat_Counter timed_checkpoints;
 	PgStat_Counter requested_checkpoints;
 	PgStat_Counter checkpoint_write_time;	/* times in milliseconds */
@@ -923,20 +372,6 @@ typedef struct PgStat_CheckpointerStats
 	PgStat_Counter buf_fsync_backend;
 } PgStat_CheckpointerStats;
 
-/*
- * Global statistics kept in the stats collector
- */
-typedef struct PgStat_GlobalStats
-{
-	TimestampTz stats_timestamp;	/* time of stats file update */
-
-	PgStat_CheckpointerStats checkpointer;
-	PgStat_BgWriterStats bgwriter;
-} PgStat_GlobalStats;
-
-/*
- * SLRU statistics kept in the stats collector
- */
 typedef struct PgStat_SLRUStats
 {
 	PgStat_Counter blocks_zeroed;
@@ -949,12 +384,13 @@ typedef struct PgStat_SLRUStats
 	TimestampTz stat_reset_timestamp;
 } PgStat_SLRUStats;
 
-/*
- * Replication slot statistics kept in the stats collector
- */
 typedef struct PgStat_StatReplSlotEntry
 {
 	NameData	slotname;
+	/*
+	 * AFIXME: This index needs to be removed. See note in pgstat_read_statsfile()
+	 */
+	uint32		index;
 	PgStat_Counter spill_txns;
 	PgStat_Counter spill_count;
 	PgStat_Counter spill_bytes;
@@ -978,14 +414,10 @@ typedef struct PgStat_StatSubEntry
 	TimestampTz stat_reset_timestamp;
 } PgStat_StatSubEntry;
 
-/*
- * WAL statistics kept in the stats collector
- */
 typedef struct PgStat_WalStats
 {
-	PgStat_Counter wal_records;
-	PgStat_Counter wal_fpi;
-	uint64		wal_bytes;
+	/* AFIXME: I don't think the use of WalUsage here is a good idea */
+	WalUsage	   wal_usage;
 	PgStat_Counter wal_buffers_full;
 	PgStat_Counter wal_write;
 	PgStat_Counter wal_sync;
@@ -1001,14 +433,13 @@ typedef struct PgStat_WalStats
  */
 
 /* functions called from postmaster */
-extern void pgstat_init(void);
-extern void pgstat_reset_all(void);
-extern int	pgstat_start(void);
-extern void allow_immediate_pgstat_restart(void);
+extern Size StatsShmemSize(void);
+extern void StatsShmemInit(void);
 
-#ifdef EXEC_BACKEND
-extern void PgstatCollectorMain(int argc, char *argv[]) pg_attribute_noreturn();
-#endif
+/* Functions called during server startup / shutdown */
+extern void pgstat_restore_stats(void);
+extern void pgstat_discard_stats(void);
+extern void pgstat_before_server_shutdown(int code, Datum arg);
 
 /* Functions for backend initialization */
 extern void pgstat_initialize(void);
@@ -1023,19 +454,25 @@ extern void pgstat_twophase_postcommit(TransactionId xid, uint16 info,
 extern void pgstat_twophase_postabort(TransactionId xid, uint16 info,
 									  void *recdata, uint32 len);
 extern void pgstat_clear_snapshot(void);
+struct PgStat_DroppedStatsItem;
+extern int pgstat_pending_stats_drops(bool isCommit, struct PgStat_DroppedStatsItem **items);
+extern void pgstat_perform_drops(int ndrops, struct PgStat_DroppedStatsItem *items, bool is_redo);
 
 /* Functions called from backends */
-extern void pgstat_report_stat(bool force);
+extern long pgstat_report_stat(bool force);
+extern void pgstat_force_next_flush(void);
 extern void pgstat_vacuum_stat(void);
 extern void pgstat_reset_counters(void);
 extern void pgstat_reset_single_counter(Oid objectid, PgStat_Single_Reset_Type type);
-extern void pgstat_ping(void);
-extern PgStat_GlobalStats *pgstat_fetch_global(void);
+extern TimestampTz pgstat_get_stat_snapshot_timestamp(bool *have_snapshot);
 
 /* GUC parameters */
 extern PGDLLIMPORT bool pgstat_track_counts;
 extern PGDLLIMPORT int pgstat_track_functions;
+extern PGDLLIMPORT int pgstat_fetch_consistency;
 extern char *pgstat_stat_directory;
+
+/* No longer used, but will be removed with GUC */
 extern char *pgstat_stat_tmpname;
 extern char *pgstat_stat_filename;
 
@@ -1087,6 +524,9 @@ extern SessionEndType pgStatSessionEndCause;
  * ----------
  */
 
+extern void pgstat_create_function(Oid proid);
+extern void pgstat_drop_function(Oid proid);
+
 struct FunctionCallInfoBaseData;
 extern void pgstat_init_function_usage(struct FunctionCallInfoBaseData *fcinfo,
 									   PgStat_FunctionCallUsage *fcu);
@@ -1105,22 +545,22 @@ extern PgStat_BackendFunctionEntry *find_funcstat_entry(Oid func_id);
 extern void pgstat_reset_shared_counters(const char *);
 
 /* archiver stats */
-extern void pgstat_send_archiver(const char *xlog, bool failed);
+extern void pgstat_report_archiver(const char *xlog, bool failed);
 extern PgStat_ArchiverStats *pgstat_fetch_stat_archiver(void);
 
 /* bgwriter stats */
-extern void pgstat_send_bgwriter(void);
+extern void pgstat_report_bgwriter(void);
 extern PgStat_BgWriterStats *pgstat_fetch_stat_bgwriter(void);
 
 /* checkpointer stats */
-extern void pgstat_send_checkpointer(void);
+extern void pgstat_report_checkpointer(void);
 extern PgStat_CheckpointerStats *pgstat_fetch_stat_checkpointer(void);
 
 /* replication slot stats */
 extern void pgstat_reset_replslot_counter(const char *name);
-extern void pgstat_report_replslot(const PgStat_StatReplSlotEntry *repSlotStat);
-extern void pgstat_report_replslot_create(const char *slotname);
-extern void pgstat_report_replslot_drop(const char *slotname);
+extern void pgstat_report_replslot(uint32 idx, const PgStat_StatReplSlotEntry *repSlotStat);
+extern void pgstat_report_replslot_create(const char *name, uint32 idx);
+extern void pgstat_report_replslot_drop(uint32 idx);
 extern PgStat_StatReplSlotEntry *pgstat_fetch_replslot(NameData slotname);
 
 /* slru stats */
@@ -1137,24 +577,24 @@ extern int	pgstat_slru_index(const char *name);
 extern PgStat_SLRUStats *pgstat_fetch_slru(void);
 
 /* wal stats */
-extern void pgstat_send_wal(bool force);
+extern void pgstat_report_wal(bool force);
 extern PgStat_WalStats *pgstat_fetch_stat_wal(void);
 
 /*
  * BgWriter statistics counters are updated directly by bgwriter and bufmgr
  */
-extern PgStat_MsgBgWriter PendingBgWriterStats;
+extern PgStat_BgWriterStats PendingBgWriterStats;
 
 /*
  * Checkpointer statistics counters are updated directly by checkpointer and
  * bufmgr.
  */
-extern PgStat_MsgCheckpointer PendingCheckpointerStats;
+extern PgStat_CheckpointerStats PendingCheckpointerStats;
 
 /*
  * WAL statistics counter is updated by backends and background processes
  */
-extern PgStat_MsgWal WalStats;
+extern PgStat_WalStats WalStats;
 
 
 /* ----------
@@ -1162,10 +602,13 @@ extern PgStat_MsgWal WalStats;
  * ----------
  */
 
+extern void pgstat_create_relation(Relation rel);
+extern void pgstat_drop_relation(Relation rel);
 extern void pgstat_copy_relation_stats(Relation dstrel, Relation srcrel);
 
 extern void pgstat_relation_init(Relation rel);
 extern void pgstat_relation_assoc(Relation rel);
+extern void pgstat_relation_delink(Relation rel);
 
 extern void pgstat_report_autovac(Oid dboid);
 extern void pgstat_report_vacuum(Oid tableoid, bool shared,
@@ -1228,6 +671,8 @@ extern void pgstat_count_truncate(Relation rel);
 extern void pgstat_update_heap_dead_tuples(Relation rel, int delta);
 
 extern PgStat_StatTabEntry *pgstat_fetch_stat_tabentry(Oid relid);
+extern PgStat_StatTabEntry *pgstat_fetch_stat_tabentry_extended(bool shared,
+																Oid relid);
 extern PgStat_TableStatus *find_tabstat_entry(Oid rel_id);
 
 
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index c3d5889d7b2..5fd7197d965 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -190,6 +190,7 @@ typedef enum BuiltinTrancheIds
 	LWTRANCHE_SHARED_TIDBITMAP,
 	LWTRANCHE_PARALLEL_APPEND,
 	LWTRANCHE_PER_XACT_PREDICATE_LIST,
+	LWTRANCHE_STATS,
 	LWTRANCHE_FIRST_USER_DEFINED
 }			BuiltinTrancheIds;
 
diff --git a/src/include/utils/pgstat_internal.h b/src/include/utils/pgstat_internal.h
index 51c0d06447b..e77aaf7f376 100644
--- a/src/include/utils/pgstat_internal.h
+++ b/src/include/utils/pgstat_internal.h
@@ -14,22 +14,107 @@
 #define PGSTAT_INTERNAL_H
 
 
+#include "access/xact.h"
+#include "lib/dshash.h"
+#include "lib/ilist.h"
 #include "pgstat.h"
-#include "executor/instrument.h"
+#include "storage/lwlock.h"
+#include "utils/dsa.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.
- * ----------
+/*
+ * Types to define shared statistics structure.
+ *
+ * Per-object statistics are stored in the "shared stats" hashtable. That
+ * table's entries (PgStatShmHashEntry) contain a pointer to the actual stats
+ * data for the object (the size of the stats data varies depending on the
+ * kind of stats). The table is keyed by PgStatHashKey.
+ *
+ * Once a backend has a reference to a shared stats entry, it increments the
+ * entry's refcount. Even after stats data is dropped (e.g. due to a DROP
+ * TABLE), the entry itself can only be deleted once all references have been
+ * released.
+ *
+ * These refcounts, in combination with a backend local hashtable
+ * (pgStatSharedRefHash, with entries pointing to PgStatSharedRef) in front of
+ * the shared hash table, mean that most stats work can happen without
+ * touching the shared hash table, reducing contention.
+ *
+ * Once there are pending stats updates for a table PgStatSharedRef->pending
+ * is allocated to contain a working space for as-of-yet-unapplied stats
+ * updates. Once the stats are flushed, PgStatSharedRef->pending is freed.
+ *
+ * Each stat kind in the shared hash table has a fixed member
+ * PgStat_HashEntryHeader as the first element.
  */
-#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
+
+/* struct for shared statistics hash entry key. */
+typedef struct PgStatHashKey
+{
+	PgStatKind	kind;			/* statistics entry kind */
+	Oid			dboid;		/* database ID. InvalidOid for shared objects. */
+	Oid			objoid;		/* object ID, either table or function. */
+} PgStatHashKey;
+
+/* struct for shared statistics hash entry */
+typedef struct PgStatShmHashEntry
+{
+	PgStatHashKey key;			/* hash key */
+
+	/*
+	 * If dropped is set, backends need to release their references so that
+	 * the memory for the entry can be freed.
+	 */
+	bool		dropped;
+
+	/*
+	 * Refcount managing lifetime of the entry itself (as opposed to the
+	 * dshash entry pointing to it). The stats lifetime has to be separate
+	 * from the hash table entry lifetime because we allow backends to point
+	 * to a stats entry without holding a hash table lock (and some other
+	 * reasons).
+	 *
+	 * As long as the entry is not dropped 1 is added to the refcount
+	 * representing that it should not be dropped. In addition each backend
+	 * that has a reference to the entry needs to increment the refcount as
+	 * long as it does.
+	 *
+	 * When the refcount reaches 0 the entry needs to be freed.
+	 */
+	pg_atomic_uint32  refcount;
+
+	LWLock		lock;
+	dsa_pointer body;			/* pointer to shared stats in
+								 * PgStat_StatEntryHeader */
+} PgStatShmHashEntry;
+
+/*
+ * Common header struct for PgStatShm_Stat*Entry.
+ */
+typedef struct PgStatShm_StatEntryHeader
+{
+	uint32		magic;				/* just a validity cross-check */
+} PgStatShm_StatEntryHeader;
+
+/*
+ * A backend local reference to a shared stats entry. As long as at least one
+ * such reference exists, the shared stats entry will not be released.
+ *
+ * If there are pending stats update to the shared stats, these are stored in
+ * ->pending.
+ */
+typedef struct PgStatSharedRef
+{
+	/*
+	 * Pointers to both the hash table entry pgStatSharedHash, and the stats
+	 * (as a local pointer, to avoid dsa_get_address()).
+	 */
+	PgStatShmHashEntry *shared_entry;
+	PgStatShm_StatEntryHeader *shared_stats;
+
+	dlist_node	pending_node;	/* membership in pgStatPending list */
+	void	   *pending;		/* the pending data itself */
+} PgStatSharedRef;
 
 
 /*
@@ -43,6 +128,16 @@ typedef struct PgStat_SubXactStatus
 
 	struct PgStat_SubXactStatus *prev;	/* higher-level subxact if any */
 
+	/*
+	 * Dropping the statistics for objects that dropped transactionally itself
+	 * needs to be transactional. Therefore we collect the stats dropped in
+	 * the current (sub-)transaction and only execute the stats drop when we
+	 * know if the transaction commits/aborts. To handle replicas and crashes,
+	 * stats drops are included in commit records.
+	 */
+	dlist_head	pending_drops;
+	int			pending_drops_count;
+
 	/*
 	 * Tuple insertion/deletion counts for an open transaction can't be
 	 * propagated into PgStat_TableStatus counters until we know if it is
@@ -55,10 +150,64 @@ typedef struct PgStat_SubXactStatus
 } PgStat_SubXactStatus;
 
 
+/*
+ * Metadata for a specific kinds of statistics.
+ */
+typedef struct pgstat_kind_info
+{
+	/*
+	 * Is this kind of stats global (i.e. a precise number exists)
+	 * or not (e.g. tables).
+	 */
+	bool is_global : 1;
+
+	/*
+	 * Can stats of this kind be accessed from another database? Determines
+	 * whether a stats object gets included in stats snapshots.
+	 */
+	bool accessed_across_databases : 1;
+
+	/*
+	 * The size of an entry in the shared stats hash table (pointed to by
+	 * PgStatShmHashEntry->body).
+	 */
+	uint32 shared_size;
+
+	/*
+	 * The offset/size of the statistics inside the shared stats entry. This
+	 * is used to e.g. avoid touching lwlocks when serializing / restoring
+	 * stats snapshot serialized to / from disk respectively.
+	 */
+	uint32 shared_data_off;
+	uint32 shared_data_len;
+
+	/*
+	 * The size of the pending data for this kind. E.g. how large
+	 * PgStatPendingEntry->pending is. Used for allocations.
+	 *
+	 * -1 signal that an entry of this kind should never have a pending entry.
+	 */
+	uint32 pending_size;
+
+	/*
+	 * For global statistics: Fetch a snapshot of appropriate global stats.
+	 */
+	void (*snapshot_cb)(void);
+
+	/*
+	 * For variable number stats: flush pending stats.
+	 */
+	bool (*flush_pending_cb)(PgStatSharedRef *sr, bool nowait);
+} pgstat_kind_info;
+
+
 /*
  * 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).
+ *
+ * This is only defined here so that SLRU_NUM_ELEMENTS is known for later type
+ * definitions.
  */
 static const char *const slru_names[] = {
 	"CommitTs",
@@ -74,13 +223,177 @@ static const char *const slru_names[] = {
 #define SLRU_NUM_ELEMENTS	lengthof(slru_names)
 
 
+typedef struct PgStatShmemGlobal
+{
+	void   *raw_dsa_area;
+
+	/*
+	 * Stats for objects for which a variable number exists are kept in this
+	 * shared hash table. See comment above PgStatKind for details.
+	 */
+	dshash_table_handle hash_handle;	/* shared dbstat hash */
+
+	/* Has the stats system already been shut down? Just a debugging check. */
+	bool is_shutdown;
+
+	/*
+	 * Whenever the for a dropped stats entry could not be freed (because
+	 * backends still have references), this is incremented, causing backends
+	 * to run pgstat_lookup_cache_gc(), allowing that memory to be reclaimed.
+	 */
+	pg_atomic_uint64 gc_count;
+
+	/*
+	 * Global stats structs.
+	 *
+	 * For the various "changecount" members check the definition of struct
+	 * PgBackendStatus for some explanation.
+	 */
+	struct
+	{
+		PgStat_ArchiverStats stats;
+		uint32 changecount;
+		PgStat_ArchiverStats reset_offset;	/* protected by StatsLock */
+	} archiver;
+
+	struct
+	{
+		PgStat_BgWriterStats stats;
+		uint32 changecount;
+		PgStat_BgWriterStats reset_offset;	/* protected by StatsLock */
+	} bgwriter;
+
+	struct
+	{
+		PgStat_CheckpointerStats stats;
+		uint32 changecount;
+		PgStat_CheckpointerStats reset_offset;	/* protected by StatsLock */
+	} checkpointer;
+
+	struct
+	{
+		LWLock		lock;
+		PgStat_StatReplSlotEntry *stats;
+	} replslot;
+
+	struct
+	{
+		LWLock		lock;
+		PgStat_SLRUStats stats[SLRU_NUM_ELEMENTS];
+#define SizeOfSlruStats sizeof(PgStat_SLRUStats[SLRU_NUM_ELEMENTS])
+	} slru;
+
+	struct
+	{
+		LWLock		lock;
+		PgStat_WalStats stats;
+	} wal;
+} PgStatShmemGlobal;
+
+
+/* ----------
+ * Types and definitions for different kinds of stats
+ *
+ * AFIXME: Can we get rid of these, and instead just have one type that's
+ * cast?
+ * ----------
+ */
+
+typedef struct PgStatShm_StatDBEntry
+{
+	PgStatShm_StatEntryHeader header;
+	PgStat_StatDBEntry stats;
+} PgStatShm_StatDBEntry;
+
+typedef struct PgStatShm_StatTabEntry
+{
+	PgStatShm_StatEntryHeader header;
+	PgStat_StatTabEntry stats;
+} PgStatShm_StatTabEntry;
+
+typedef struct PgStatShm_StatFuncEntry
+{
+	PgStatShm_StatEntryHeader header;
+	PgStat_StatFuncEntry stats;
+} PgStatShm_StatFuncEntry;
+
+typedef struct PgStatShm_StatSubEntry
+{
+	PgStatShm_StatEntryHeader header;
+	PgStat_StatSubEntry stats;
+} PgStatShm_StatSubEntry;
+
+
+/* ----------
+ * Cached statistics snapshot
+ * ----------
+ */
+
+typedef struct PgStatSnapshot
+{
+	PgStatsFetchConsistency mode;
+
+	/* time at which snapshot was taken */
+	TimestampTz snapshot_timestamp;
+
+	bool global_valid[PGSTAT_KIND_LAST + 1];
+
+	PgStat_ArchiverStats archiver;
+
+	PgStat_BgWriterStats bgwriter;
+
+	PgStat_CheckpointerStats checkpointer;
+
+	int replslot_count;
+	PgStat_StatReplSlotEntry *replslot;
+
+	PgStat_SLRUStats slru[SLRU_NUM_ELEMENTS];
+
+	PgStat_WalStats wal;
+
+	struct pgstat_snapshot_hash *stats;
+} PgStatSnapshot;
+
+
+/*
+ * Inline functions defined further below.
+ */
+
+static inline void changecount_before_write(uint32 *cc);
+static inline void changecount_after_write(uint32 *cc);
+static inline uint32 changecount_before_read(uint32 *cc);
+static inline bool changecount_after_read(uint32 *cc, uint32 cc_before);
+
+static inline void pgstat_copy_global_stats(void *dst, void *src, size_t len,
+											uint32 *cc);
+
+
 /*
  * Functions in pgstat.c
  */
 
+extern PgStatSharedRef *pgstat_shared_ref_get(PgStatKind kind,
+											  Oid dboid, Oid objoid,
+											  bool create);
+extern bool pgstat_shared_stat_lock(PgStatSharedRef *shared_ref, bool nowait);
+extern void pgstat_shared_stat_unlock(PgStatSharedRef *shared_ref);
+extern PgStatSharedRef *pgstat_shared_stat_locked(PgStatKind kind,
+												  Oid dboid,
+												  Oid objoid,
+												  bool nowait);
+
+extern PgStatSharedRef *pgstat_pending_prepare(PgStatKind kind, Oid dboid, Oid objoid);
+extern PgStatSharedRef *pgstat_pending_fetch(PgStatKind kind, Oid dboid, Oid objoid);
+
 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);
+
+extern void pgstat_schedule_create(PgStatKind kind, Oid dboid, Oid objoid);
+extern void pgstat_schedule_drop(PgStatKind kind, Oid dboid, Oid objoid);
+extern void pgstat_drop_database_and_contents(Oid dboid);
+
+extern void* pgstat_fetch_entry(PgStatKind kind, Oid dboid, Oid objoid);
+extern void pgstat_snapshot_global(PgStatKind kind);
+
 #ifdef USE_ASSERT_CHECKING
 extern void pgstat_assert_is_up(void);
 #else
@@ -92,42 +405,60 @@ extern void pgstat_assert_is_up(void);
  * Functions in pgstat_database.c
  */
 
+extern bool pgstat_flush_database(PgStatSharedRef *shared_ref, bool nowait);
+extern void pgstat_update_dbstats(TimestampTz now);
 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);
+extern PgStat_StatDBEntry *pgstat_pending_db_prepare(Oid dboid);
 
 
 /*
  * Functions in pgstat_function.c
  */
 
-extern void pgstat_send_funcstats(void);
+extern bool pgstat_flush_function(PgStatSharedRef *shared_ref, bool nowait);
 
 
 /*
  * Functions in pgstat_global.c
  */
 
-extern void pgstat_send_slru(void);
+extern void pgstat_snapshot_archiver(void);
+extern void pgstat_snapshot_bgwriter(void);
+extern void pgstat_snapshot_checkpointer(void);
+extern void pgstat_snapshot_replslot(void);
+extern bool pgstat_flush_slru(bool nowait);
+extern void pgstat_snapshot_slru(void);
+extern bool pgstat_flush_wal(bool nowait);
+extern bool walstats_pending(void);
 extern void pgstat_wal_initialize(void);
+extern void pgstat_snapshot_wal(void);
 
 
 /*
  * Functions in pgstat_relation.c
  */
 
-extern void pgstat_send_tabstats(TimestampTz now, bool disconnect);
+extern bool pgstat_flush_relation(PgStatSharedRef *shared_ref, bool nowait);
 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);
 
 
+/*
+ * Functions in pgstat_subscription.c
+ */
+
+extern bool pgstat_flush_subscription(PgStatSharedRef *shared_ref, bool nowait);
+
+
 /*
  * Variables in pgstat.c
  */
 
-extern pgsocket pgStatSock;
+extern PgStatShmemGlobal *pgStatShmem;
+extern PgStatSnapshot stats_snapshot;
 
 
 /*
@@ -139,10 +470,10 @@ extern int	pgStatXactRollback;
 
 
 /*
- * Variables in pgstat_functions.c
+ * Variables in pgstat_global.c
  */
 
-extern bool have_function_stats;
+extern bool have_slrustats;
 
 
 /*
@@ -153,10 +484,89 @@ extern WalUsage prevWalUsage;
 
 
 /*
- * Variables in pgstat_relation.c
+ * Implementation of inline functions declared above.
  */
 
-extern bool have_relation_stats;
+/*
+ * Helpers for changecount manipulation. See comments around struct
+ * PgBackendStatus for details.
+ */
+
+static inline void
+changecount_before_write(uint32 *cc)
+{
+	Assert((*cc & 1) == 0);
+
+	START_CRIT_SECTION();
+	(*cc)++;
+	pg_write_barrier();
+}
+
+static inline void
+changecount_after_write(uint32 *cc)
+{
+	Assert((*cc & 1) == 1);
+
+	pg_write_barrier();
+
+	(*cc)++;
+
+	END_CRIT_SECTION();
+}
+
+static inline uint32
+changecount_before_read(uint32 *cc)
+{
+	uint32 before_cc = *cc;
+
+	CHECK_FOR_INTERRUPTS();
+
+	pg_read_barrier();
+
+	return before_cc;
+}
+
+/*
+ * Returns true if the read succeeded, false if it needs to be repeated.
+ */
+static inline bool
+changecount_after_read(uint32 *cc, uint32 before_cc)
+{
+	uint32 after_cc;
+
+	pg_read_barrier();
+
+	after_cc = *cc;
+
+	/* was a write in progress when we started? */
+	if (before_cc & 1)
+		return false;
+
+	/* did writes start and complete while we read? */
+	return before_cc == after_cc;
+}
+
+
+/*
+ * pgstat_copy_global_stats - helper function for functions
+ *           pgstat_fetch_stat_*() and pgstat_reset_shared_counters().
+ *
+ * Copies out the specified memory area following change-count protocol.
+ */
+static inline void
+pgstat_copy_global_stats(void *dst, void *src, size_t len,
+						 uint32 *cc)
+{
+	uint32 cc_before;
+
+	do
+	{
+		cc_before = changecount_before_read(cc);
+
+		memcpy(dst, src, len);
+	}
+	while (!changecount_after_read(cc, cc_before));
+}
 
 
 #endif							/* PGSTAT_INTERNAL_H */
diff --git a/src/include/utils/timeout.h b/src/include/utils/timeout.h
index 099f91c61da..c068986d09a 100644
--- a/src/include/utils/timeout.h
+++ b/src/include/utils/timeout.h
@@ -32,6 +32,7 @@ typedef enum TimeoutId
 	STANDBY_LOCK_TIMEOUT,
 	IDLE_IN_TRANSACTION_SESSION_TIMEOUT,
 	IDLE_SESSION_TIMEOUT,
+	IDLE_STATS_UPDATE_TIMEOUT,
 	CLIENT_CONNECTION_CHECK_TIMEOUT,
 	STARTUP_PROGRESS_TIMEOUT,
 	/* First user-definable timeout reason */
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index 395d325c5fe..7cf1a71e353 100644
--- a/src/include/utils/wait_event.h
+++ b/src/include/utils/wait_event.h
@@ -42,7 +42,7 @@ typedef enum
 	WAIT_EVENT_CHECKPOINTER_MAIN,
 	WAIT_EVENT_LOGICAL_APPLY_MAIN,
 	WAIT_EVENT_LOGICAL_LAUNCHER_MAIN,
-	WAIT_EVENT_PGSTAT_MAIN,
+	WAIT_EVENT_READING_STATS_FILE,
 	WAIT_EVENT_RECOVERY_WAL_STREAM,
 	WAIT_EVENT_SYSLOGGER_MAIN,
 	WAIT_EVENT_WAL_RECEIVER_MAIN,
diff --git a/src/backend/access/heap/heapam_handler.c b/src/backend/access/heap/heapam_handler.c
index 39ef8a0b77d..911a76228d5 100644
--- a/src/backend/access/heap/heapam_handler.c
+++ b/src/backend/access/heap/heapam_handler.c
@@ -1089,8 +1089,8 @@ heapam_scan_analyze_next_tuple(TableScanDesc scan, TransactionId OldestXmin,
 				 * our own.  In this case we should count and sample the row,
 				 * to accommodate users who load a table and analyze it in one
 				 * transaction.  (pgstat_report_analyze has to adjust the
-				 * numbers we send to the stats collector to make this come
-				 * out right.)
+				 * numbers we report to the activity stats facility to make
+				 * this come out right.)
 				 */
 				if (TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetXmin(targtuple->t_data)))
 				{
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 40101e0cb82..c54b7ea94e6 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -558,7 +558,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	/*
 	 * Report results to the stats collector, too.
 	 *
-	 * Deliberately avoid telling the stats collector about LP_DEAD items that
+	 * Deliberately avoid telling the stats subsystem about LP_DEAD items that
 	 * remain in the table due to VACUUM bypassing index and heap vacuuming.
 	 * ANALYZE will consider the remaining LP_DEAD items to be dead "tuples".
 	 * It seems like a good idea to err on the side of not vacuuming again too
diff --git a/src/backend/access/rmgrdesc/xactdesc.c b/src/backend/access/rmgrdesc/xactdesc.c
index 025d556f6ce..a0ba66ac034 100644
--- a/src/backend/access/rmgrdesc/xactdesc.c
+++ b/src/backend/access/rmgrdesc/xactdesc.c
@@ -84,6 +84,17 @@ ParseCommitRecord(uint8 info, xl_xact_commit *xlrec, xl_xact_parsed_commit *pars
 		data += xl_relfilenodes->nrels * sizeof(RelFileNode);
 	}
 
+	if (parsed->xinfo & XACT_XINFO_HAS_DROPPED_STATS)
+	{
+		xl_xact_dropped_stats *xl_drops = (xl_xact_dropped_stats *) data;
+
+		parsed->ndroppedstats = xl_drops->ndropped;
+		parsed->droppedstats = xl_drops->dropped_stats;
+
+		data += MinSizeOfXactDroppedStats;
+		data += xl_drops->ndropped * sizeof(PgStat_DroppedStatsItem);
+	}
+
 	if (parsed->xinfo & XACT_XINFO_HAS_INVALS)
 	{
 		xl_xact_invals *xl_invals = (xl_xact_invals *) data;
@@ -179,6 +190,17 @@ ParseAbortRecord(uint8 info, xl_xact_abort *xlrec, xl_xact_parsed_abort *parsed)
 		data += xl_relfilenodes->nrels * sizeof(RelFileNode);
 	}
 
+	if (parsed->xinfo & XACT_XINFO_HAS_DROPPED_STATS)
+	{
+		xl_xact_dropped_stats *xl_drops = (xl_xact_dropped_stats *) data;
+
+		parsed->ndroppedstats = xl_drops->ndropped;
+		parsed->droppedstats = xl_drops->dropped_stats;
+
+		data += MinSizeOfXactDroppedStats;
+		data += xl_drops->ndropped * sizeof(PgStat_DroppedStatsItem);
+	}
+
 	if (parsed->xinfo & XACT_XINFO_HAS_TWOPHASE)
 	{
 		xl_xact_twophase *xl_twophase = (xl_xact_twophase *) data;
@@ -244,6 +266,12 @@ ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parsed_prepare *p
 	parsed->abortnodes = (RelFileNode *) bufptr;
 	bufptr += MAXALIGN(xlrec->nabortrels * sizeof(RelFileNode));
 
+	parsed->droppedstats = (PgStat_DroppedStatsItem *) bufptr;
+	bufptr += MAXALIGN(xlrec->ncommitdroppedstats * sizeof(PgStat_DroppedStatsItem));
+
+	parsed->abortdroppedstats = (PgStat_DroppedStatsItem *) bufptr;
+	bufptr += MAXALIGN(xlrec->nabortdroppedstats * sizeof(PgStat_DroppedStatsItem));
+
 	parsed->msgs = (SharedInvalidationMessage *) bufptr;
 	bufptr += MAXALIGN(xlrec->ninvalmsgs * sizeof(SharedInvalidationMessage));
 }
@@ -280,6 +308,25 @@ xact_desc_subxacts(StringInfo buf, int nsubxacts, TransactionId *subxacts)
 	}
 }
 
+static void
+xact_desc_dropped_stats(StringInfo buf, const char *label,
+						int ndropped, PgStat_DroppedStatsItem *dropped_stats)
+{
+	int			i;
+
+	if (ndropped > 0)
+	{
+		appendStringInfo(buf, "; %sdropped stats:", label);
+		for (i = 0; i < ndropped; i++)
+		{
+			appendStringInfo(buf, " %u/%u/%u",
+							 dropped_stats[i].kind,
+							 dropped_stats[i].dboid,
+							 dropped_stats[i].objoid);
+		}
+	}
+}
+
 static void
 xact_desc_commit(StringInfo buf, uint8 info, xl_xact_commit *xlrec, RepOriginId origin_id)
 {
@@ -295,6 +342,7 @@ xact_desc_commit(StringInfo buf, uint8 info, xl_xact_commit *xlrec, RepOriginId
 
 	xact_desc_relations(buf, "rels", parsed.nrels, parsed.xnodes);
 	xact_desc_subxacts(buf, parsed.nsubxacts, parsed.subxacts);
+	xact_desc_dropped_stats(buf, "", parsed.ndroppedstats, parsed.droppedstats);
 
 	standby_desc_invalidations(buf, parsed.nmsgs, parsed.msgs, parsed.dbId,
 							   parsed.tsId,
@@ -338,6 +386,8 @@ xact_desc_abort(StringInfo buf, uint8 info, xl_xact_abort *xlrec, RepOriginId or
 						 LSN_FORMAT_ARGS(parsed.origin_lsn),
 						 timestamptz_to_str(parsed.origin_timestamp));
 	}
+
+	xact_desc_dropped_stats(buf, "", parsed.ndroppedstats, parsed.droppedstats);
 }
 
 static void
@@ -353,6 +403,8 @@ xact_desc_prepare(StringInfo buf, uint8 info, xl_xact_prepare *xlrec, RepOriginI
 	xact_desc_relations(buf, "rels(commit)", parsed.nrels, parsed.xnodes);
 	xact_desc_relations(buf, "rels(abort)", parsed.nabortrels,
 						parsed.abortnodes);
+	xact_desc_dropped_stats(buf, "commit ", parsed.ndroppedstats, parsed.droppedstats);
+	xact_desc_dropped_stats(buf, "abort ", parsed.nabortdroppedstats, parsed.abortdroppedstats);
 	xact_desc_subxacts(buf, parsed.nsubxacts, parsed.subxacts);
 
 	standby_desc_invalidations(buf, parsed.nmsgs, parsed.msgs, parsed.dbId,
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 874c8ed1256..c2aa88428de 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -205,6 +205,8 @@ static void RecordTransactionCommitPrepared(TransactionId xid,
 											TransactionId *children,
 											int nrels,
 											RelFileNode *rels,
+											int ndroppedstats,
+											PgStat_DroppedStatsItem *droppedstats,
 											int ninvalmsgs,
 											SharedInvalidationMessage *invalmsgs,
 											bool initfileinval,
@@ -214,6 +216,8 @@ static void RecordTransactionAbortPrepared(TransactionId xid,
 										   TransactionId *children,
 										   int nrels,
 										   RelFileNode *rels,
+										   int ndroppedstats,
+										   PgStat_DroppedStatsItem *droppedstats,
 										   const char *gid);
 static void ProcessRecords(char *bufptr, TransactionId xid,
 						   const TwoPhaseCallback callbacks[]);
@@ -1046,6 +1050,8 @@ StartPrepare(GlobalTransaction gxact)
 	TransactionId *children;
 	RelFileNode *commitrels;
 	RelFileNode *abortrels;
+	PgStat_DroppedStatsItem *abortdroppedstats = NULL;
+	PgStat_DroppedStatsItem *commitdroppedstats = NULL;
 	SharedInvalidationMessage *invalmsgs;
 
 	/* Initialize linked list */
@@ -1071,6 +1077,10 @@ StartPrepare(GlobalTransaction gxact)
 	hdr.nsubxacts = xactGetCommittedChildren(&children);
 	hdr.ncommitrels = smgrGetPendingDeletes(true, &commitrels);
 	hdr.nabortrels = smgrGetPendingDeletes(false, &abortrels);
+	hdr.ncommitdroppedstats =
+		pgstat_pending_stats_drops(true, &commitdroppedstats);
+	hdr.nabortdroppedstats =
+		pgstat_pending_stats_drops(false, &abortdroppedstats);
 	hdr.ninvalmsgs = xactGetCommittedInvalidationMessages(&invalmsgs,
 														  &hdr.initfileinval);
 	hdr.gidlen = strlen(gxact->gid) + 1;	/* Include '\0' */
@@ -1101,6 +1111,20 @@ StartPrepare(GlobalTransaction gxact)
 		save_state_data(abortrels, hdr.nabortrels * sizeof(RelFileNode));
 		pfree(abortrels);
 	}
+	if (hdr.ncommitdroppedstats > 0)
+	{
+		save_state_data(commitdroppedstats,
+						hdr.ncommitdroppedstats
+						* sizeof(PgStat_DroppedStatsItem));
+		pfree(commitdroppedstats);
+	}
+	if (hdr.nabortdroppedstats > 0)
+	{
+		save_state_data(abortdroppedstats,
+						hdr.nabortdroppedstats
+						* sizeof(PgStat_DroppedStatsItem));
+		pfree(abortdroppedstats);
+	}
 	if (hdr.ninvalmsgs > 0)
 	{
 		save_state_data(invalmsgs,
@@ -1471,6 +1495,8 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	RelFileNode *abortrels;
 	RelFileNode *delrels;
 	int			ndelrels;
+	PgStat_DroppedStatsItem *commitdroppedstats;
+	PgStat_DroppedStatsItem *abortdroppedstats;
 	SharedInvalidationMessage *invalmsgs;
 
 	/*
@@ -1505,6 +1531,12 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	bufptr += MAXALIGN(hdr->ncommitrels * sizeof(RelFileNode));
 	abortrels = (RelFileNode *) bufptr;
 	bufptr += MAXALIGN(hdr->nabortrels * sizeof(RelFileNode));
+	commitdroppedstats = (PgStat_DroppedStatsItem*) bufptr;
+	bufptr += MAXALIGN(hdr->ncommitdroppedstats
+					   * sizeof(PgStat_DroppedStatsItem));
+	abortdroppedstats = (PgStat_DroppedStatsItem*) bufptr;
+	bufptr += MAXALIGN(hdr->nabortdroppedstats
+					   * sizeof(PgStat_DroppedStatsItem));
 	invalmsgs = (SharedInvalidationMessage *) bufptr;
 	bufptr += MAXALIGN(hdr->ninvalmsgs * sizeof(SharedInvalidationMessage));
 
@@ -1526,12 +1558,16 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 		RecordTransactionCommitPrepared(xid,
 										hdr->nsubxacts, children,
 										hdr->ncommitrels, commitrels,
+										hdr->ncommitdroppedstats,
+										commitdroppedstats,
 										hdr->ninvalmsgs, invalmsgs,
 										hdr->initfileinval, gid);
 	else
 		RecordTransactionAbortPrepared(xid,
 									   hdr->nsubxacts, children,
 									   hdr->nabortrels, abortrels,
+									   hdr->nabortdroppedstats,
+									   abortdroppedstats,
 									   gid);
 
 	ProcArrayRemove(proc, latestXid);
@@ -1567,6 +1603,13 @@ FinishPreparedTransaction(const char *gid, bool isCommit)
 	/* Make sure files supposed to be dropped are dropped */
 	DropRelationFiles(delrels, ndelrels, false);
 
+	if (isCommit)
+		pgstat_perform_drops(hdr->ncommitdroppedstats,
+							 commitdroppedstats, false);
+	else
+		pgstat_perform_drops(hdr->nabortdroppedstats,
+							 abortdroppedstats, false);
+
 	/*
 	 * Handle cache invalidation messages.
 	 *
@@ -2065,6 +2108,10 @@ RecoverPreparedTransactions(void)
 		bufptr += MAXALIGN(hdr->nsubxacts * sizeof(TransactionId));
 		bufptr += MAXALIGN(hdr->ncommitrels * sizeof(RelFileNode));
 		bufptr += MAXALIGN(hdr->nabortrels * sizeof(RelFileNode));
+		bufptr += MAXALIGN(hdr->ncommitdroppedstats
+						   * sizeof(PgStat_DroppedStatsItem));
+		bufptr += MAXALIGN(hdr->nabortdroppedstats
+						   * sizeof(PgStat_DroppedStatsItem));
 		bufptr += MAXALIGN(hdr->ninvalmsgs * sizeof(SharedInvalidationMessage));
 
 		/*
@@ -2247,6 +2294,8 @@ RecordTransactionCommitPrepared(TransactionId xid,
 								TransactionId *children,
 								int nrels,
 								RelFileNode *rels,
+								int ndroppedstats,
+								PgStat_DroppedStatsItem *droppedstats,
 								int ninvalmsgs,
 								SharedInvalidationMessage *invalmsgs,
 								bool initfileinval,
@@ -2275,6 +2324,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
 	 */
 	recptr = XactLogCommitRecord(committs,
 								 nchildren, children, nrels, rels,
+								 ndroppedstats, droppedstats,
 								 ninvalmsgs, invalmsgs,
 								 initfileinval,
 								 MyXactFlags | XACT_FLAGS_ACQUIREDACCESSEXCLUSIVELOCK,
@@ -2341,6 +2391,8 @@ RecordTransactionAbortPrepared(TransactionId xid,
 							   TransactionId *children,
 							   int nrels,
 							   RelFileNode *rels,
+							   int ndroppedstats,
+							   PgStat_DroppedStatsItem *droppedstats,
 							   const char *gid)
 {
 	XLogRecPtr	recptr;
@@ -2371,6 +2423,7 @@ RecordTransactionAbortPrepared(TransactionId xid,
 	recptr = XactLogAbortRecord(GetCurrentTimestamp(),
 								nchildren, children,
 								nrels, rels,
+								ndroppedstats, droppedstats,
 								MyXactFlags | XACT_FLAGS_ACQUIREDACCESSEXCLUSIVELOCK,
 								xid, gid);
 
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index adf763a8ea4..3548794473a 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1285,6 +1285,8 @@ RecordTransactionCommit(void)
 	RelFileNode *rels;
 	int			nchildren;
 	TransactionId *children;
+	int			ndroppedstats = 0;
+	PgStat_DroppedStatsItem *droppedstats = NULL;
 	int			nmsgs = 0;
 	SharedInvalidationMessage *invalMessages = NULL;
 	bool		RelcacheInitFileInval = false;
@@ -1303,6 +1305,7 @@ RecordTransactionCommit(void)
 	/* Get data needed for commit record */
 	nrels = smgrGetPendingDeletes(true, &rels);
 	nchildren = xactGetCommittedChildren(&children);
+	ndroppedstats = pgstat_pending_stats_drops(true, &droppedstats);
 	if (XLogStandbyInfoActive())
 		nmsgs = xactGetCommittedInvalidationMessages(&invalMessages,
 													 &RelcacheInitFileInval);
@@ -1317,10 +1320,12 @@ RecordTransactionCommit(void)
 		/*
 		 * We expect that every RelationDropStorage is followed by a catalog
 		 * update, and hence XID assignment, so we shouldn't get here with any
-		 * pending deletes.  Use a real test not just an Assert to check this,
-		 * since it's a bit fragile.
+		 * pending deletes. Same is true for dropping stats.
+		 *
+		 * Use a real test not just an Assert to check this, since it's a bit
+		 * fragile.
 		 */
-		if (nrels != 0)
+		if (nrels != 0 || ndroppedstats != 0)
 			elog(ERROR, "cannot commit a transaction that deleted files but has no xid");
 
 		/* Can't have child XIDs either; AssignTransactionId enforces this */
@@ -1394,6 +1399,7 @@ RecordTransactionCommit(void)
 
 		XactLogCommitRecord(xactStopTimestamp,
 							nchildren, children, nrels, rels,
+							ndroppedstats, droppedstats,
 							nmsgs, invalMessages,
 							RelcacheInitFileInval,
 							MyXactFlags,
@@ -1697,6 +1703,8 @@ RecordTransactionAbort(bool isSubXact)
 	TransactionId latestXid;
 	int			nrels;
 	RelFileNode *rels;
+	int			ndroppedstats = 0;
+	PgStat_DroppedStatsItem *droppedstats = NULL;
 	int			nchildren;
 	TransactionId *children;
 	TimestampTz xact_time;
@@ -1733,6 +1741,7 @@ RecordTransactionAbort(bool isSubXact)
 	/* Fetch the data we need for the abort record */
 	nrels = smgrGetPendingDeletes(false, &rels);
 	nchildren = xactGetCommittedChildren(&children);
+	ndroppedstats = pgstat_pending_stats_drops(false, &droppedstats);
 
 	/* XXX do we really need a critical section here? */
 	START_CRIT_SECTION();
@@ -1749,6 +1758,7 @@ RecordTransactionAbort(bool isSubXact)
 	XactLogAbortRecord(xact_time,
 					   nchildren, children,
 					   nrels, rels,
+					   ndroppedstats, droppedstats,
 					   MyXactFlags, InvalidTransactionId,
 					   NULL);
 
@@ -5571,6 +5581,7 @@ XLogRecPtr
 XactLogCommitRecord(TimestampTz commit_time,
 					int nsubxacts, TransactionId *subxacts,
 					int nrels, RelFileNode *rels,
+					int ndroppedstats, PgStat_DroppedStatsItem *droppedstats,
 					int nmsgs, SharedInvalidationMessage *msgs,
 					bool relcacheInval,
 					int xactflags, TransactionId twophase_xid,
@@ -5581,6 +5592,7 @@ XactLogCommitRecord(TimestampTz commit_time,
 	xl_xact_dbinfo xl_dbinfo;
 	xl_xact_subxacts xl_subxacts;
 	xl_xact_relfilenodes xl_relfilenodes;
+	xl_xact_dropped_stats xl_dropped_stats;
 	xl_xact_invals xl_invals;
 	xl_xact_twophase xl_twophase;
 	xl_xact_origin xl_origin;
@@ -5638,6 +5650,12 @@ XactLogCommitRecord(TimestampTz commit_time,
 		info |= XLR_SPECIAL_REL_UPDATE;
 	}
 
+	if (ndroppedstats > 0)
+	{
+		xl_xinfo.xinfo |= XACT_XINFO_HAS_DROPPED_STATS;
+		xl_dropped_stats.ndropped = ndroppedstats;
+	}
+
 	if (nmsgs > 0)
 	{
 		xl_xinfo.xinfo |= XACT_XINFO_HAS_INVALS;
@@ -5694,6 +5712,14 @@ XactLogCommitRecord(TimestampTz commit_time,
 						 nrels * sizeof(RelFileNode));
 	}
 
+	if (xl_xinfo.xinfo & XACT_XINFO_HAS_DROPPED_STATS)
+	{
+		XLogRegisterData((char *) (&xl_dropped_stats),
+						 MinSizeOfXactDroppedStats);
+		XLogRegisterData((char *) droppedstats,
+						 ndroppedstats * sizeof(PgStat_DroppedStatsItem));
+	}
+
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_INVALS)
 	{
 		XLogRegisterData((char *) (&xl_invals), MinSizeOfXactInvals);
@@ -5727,6 +5753,7 @@ XLogRecPtr
 XactLogAbortRecord(TimestampTz abort_time,
 				   int nsubxacts, TransactionId *subxacts,
 				   int nrels, RelFileNode *rels,
+				   int ndroppedstats, PgStat_DroppedStatsItem *droppedstats,
 				   int xactflags, TransactionId twophase_xid,
 				   const char *twophase_gid)
 {
@@ -5734,6 +5761,7 @@ XactLogAbortRecord(TimestampTz abort_time,
 	xl_xact_xinfo xl_xinfo;
 	xl_xact_subxacts xl_subxacts;
 	xl_xact_relfilenodes xl_relfilenodes;
+	xl_xact_dropped_stats xl_dropped_stats;
 	xl_xact_twophase xl_twophase;
 	xl_xact_dbinfo xl_dbinfo;
 	xl_xact_origin xl_origin;
@@ -5771,6 +5799,12 @@ XactLogAbortRecord(TimestampTz abort_time,
 		info |= XLR_SPECIAL_REL_UPDATE;
 	}
 
+	if (ndroppedstats > 0)
+	{
+		xl_xinfo.xinfo |= XACT_XINFO_HAS_DROPPED_STATS;
+		xl_dropped_stats.ndropped = ndroppedstats;
+	}
+
 	if (TransactionIdIsValid(twophase_xid))
 	{
 		xl_xinfo.xinfo |= XACT_XINFO_HAS_TWOPHASE;
@@ -5832,6 +5866,14 @@ XactLogAbortRecord(TimestampTz abort_time,
 						 nrels * sizeof(RelFileNode));
 	}
 
+	if (xl_xinfo.xinfo & XACT_XINFO_HAS_DROPPED_STATS)
+	{
+		XLogRegisterData((char *) (&xl_dropped_stats),
+						 MinSizeOfXactDroppedStats);
+		XLogRegisterData((char *) droppedstats,
+						 ndroppedstats * sizeof(PgStat_DroppedStatsItem));
+	}
+
 	if (xl_xinfo.xinfo & XACT_XINFO_HAS_TWOPHASE)
 	{
 		XLogRegisterData((char *) (&xl_twophase), sizeof(xl_xact_twophase));
@@ -5965,6 +6007,13 @@ xact_redo_commit(xl_xact_parsed_commit *parsed,
 		DropRelationFiles(parsed->xnodes, parsed->nrels, true);
 	}
 
+	if (parsed->ndroppedstats > 0)
+	{
+		XLogFlush(lsn);
+
+		pgstat_perform_drops(parsed->ndroppedstats, parsed->droppedstats, true);
+	}
+
 	/*
 	 * We issue an XLogFlush() for the same reason we emit ForceSyncCommit()
 	 * in normal operation. For example, in CREATE DATABASE, we copy all files
@@ -6067,6 +6116,13 @@ xact_redo_abort(xl_xact_parsed_abort *parsed, TransactionId xid,
 
 		DropRelationFiles(parsed->xnodes, parsed->nrels, true);
 	}
+
+	if (parsed->ndroppedstats > 0)
+	{
+		XLogFlush(lsn);
+
+		pgstat_perform_drops(parsed->ndroppedstats, parsed->droppedstats, true);
+	}
 }
 
 void
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 0d2bd7a3576..346440a2798 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -1854,7 +1854,7 @@ AdvanceXLInsertBuffer(XLogRecPtr upto, TimeLineID tli, bool opportunistic)
 					WriteRqst.Flush = 0;
 					XLogWrite(WriteRqst, tli, false);
 					LWLockRelease(WALWriteLock);
-					WalStats.m_wal_buffers_full++;
+					WalStats.wal_buffers_full++;
 					TRACE_POSTGRESQL_WAL_BUFFER_WRITE_DIRTY_DONE();
 				}
 				/* Re-acquire WALBufMappingLock and retry */
@@ -2212,10 +2212,10 @@ XLogWrite(XLogwrtRqst WriteRqst, TimeLineID tli, bool flexible)
 
 					INSTR_TIME_SET_CURRENT(duration);
 					INSTR_TIME_SUBTRACT(duration, start);
-					WalStats.m_wal_write_time += INSTR_TIME_GET_MICROSEC(duration);
+					WalStats.wal_write_time += INSTR_TIME_GET_MICROSEC(duration);
 				}
 
-				WalStats.m_wal_write++;
+				WalStats.wal_write++;
 
 				if (written <= 0)
 				{
@@ -5193,9 +5193,12 @@ StartupXLOG(void)
 		}
 
 		/*
-		 * Reset pgstat data, because it may be invalid after recovery.
+		 * Reset pgstat data, because it may be invalid after recovery. It's
+		 * safe to do this here, because postmaster will not yet have started
+		 * any other processes. NB: This basically just skips loading the data
+		 * from disk, see pgstat_restore_stats() call in clean-startup path.
 		 */
-		pgstat_reset_all();
+		pgstat_discard_stats();
 
 		/* Check that the GUCs used to generate the WAL allow recovery */
 		CheckRequiredParameterValues();
@@ -5287,8 +5290,13 @@ StartupXLOG(void)
 		performedWalRecovery = true;
 	}
 	else
+	{
 		performedWalRecovery = false;
 
+		/* XXX: better location */
+		pgstat_restore_stats();
+	}
+
 	/*
 	 * Finish WAL recovery.
 	 */
@@ -6089,8 +6097,8 @@ LogCheckpointEnd(bool restartpoint)
 												 CheckpointStats.ckpt_sync_end_t);
 
 	/* Accumulate checkpoint timing summary data, in milliseconds. */
-	PendingCheckpointerStats.m_checkpoint_write_time += write_msecs;
-	PendingCheckpointerStats.m_checkpoint_sync_time += sync_msecs;
+	PendingCheckpointerStats.checkpoint_write_time += write_msecs;
+	PendingCheckpointerStats.checkpoint_sync_time += sync_msecs;
 
 	/*
 	 * All of the published timing statistics are accounted for.  Only
@@ -8004,10 +8012,10 @@ issue_xlog_fsync(int fd, XLogSegNo segno, TimeLineID tli)
 
 		INSTR_TIME_SET_CURRENT(duration);
 		INSTR_TIME_SUBTRACT(duration, start);
-		WalStats.m_wal_sync_time += INSTR_TIME_GET_MICROSEC(duration);
+		WalStats.wal_sync_time += INSTR_TIME_GET_MICROSEC(duration);
 	}
 
-	WalStats.m_wal_sync++;
+	WalStats.wal_sync++;
 }
 
 /*
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 7e99de88b34..53b5a0dad99 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -74,6 +74,7 @@
 #include "parser/parse_relation.h"
 #include "parser/parsetree.h"
 #include "partitioning/partdesc.h"
+#include "pgstat.h"
 #include "storage/lmgr.h"
 #include "storage/predicate.h"
 #include "storage/smgr.h"
@@ -1491,6 +1492,9 @@ heap_create_with_catalog(const char *relname,
 	if (oncommit != ONCOMMIT_NOOP)
 		register_on_commit_action(relid, oncommit);
 
+	/* ensure that stats are dropped if transaction aborts */
+	pgstat_create_relation(new_rel_desc);
+
 	/*
 	 * ok, the relation has been cataloged, so close our relations and return
 	 * the OID of the newly created relation.
@@ -1992,6 +1996,9 @@ heap_drop_with_catalog(Oid relid)
 	if (RELKIND_HAS_STORAGE(rel->rd_rel->relkind))
 		RelationDropStorage(rel);
 
+	/* ensure that stats are dropped if transaction commits */
+	pgstat_drop_relation(rel);
+
 	/*
 	 * Close relcache entry, but *keep* AccessExclusiveLock on the relation
 	 * until transaction commit.  This ensures no one else will try to do
diff --git a/src/backend/catalog/pg_proc.c b/src/backend/catalog/pg_proc.c
index 12521c77c3c..2ba872c88e0 100644
--- a/src/backend/catalog/pg_proc.c
+++ b/src/backend/catalog/pg_proc.c
@@ -35,6 +35,7 @@
 #include "parser/analyze.h"
 #include "parser/parse_coerce.h"
 #include "parser/parse_type.h"
+#include "pgstat.h"
 #include "rewrite/rewriteHandler.h"
 #include "tcop/pquery.h"
 #include "tcop/tcopprot.h"
@@ -709,6 +710,10 @@ ProcedureCreate(const char *procedureName,
 			AtEOXact_GUC(true, save_nestlevel);
 	}
 
+	/* ensure that stats are dropped if transaction commits */
+	if (!is_update)
+		pgstat_create_function(retval);
+
 	return myself;
 }
 
diff --git a/src/backend/commands/dbcommands.c b/src/backend/commands/dbcommands.c
index c37e3c9a9a4..bc35f0f80cc 100644
--- a/src/backend/commands/dbcommands.c
+++ b/src/backend/commands/dbcommands.c
@@ -1046,7 +1046,7 @@ dropdb(const char *dbname, bool missing_ok, bool force)
 	DropDatabaseBuffers(db_id);
 
 	/*
-	 * Tell the stats collector to forget it immediately, too.
+	 * Tell the activity stats subsystem to forget it immediately, too.
 	 */
 	pgstat_drop_database(db_id);
 
diff --git a/src/backend/commands/functioncmds.c b/src/backend/commands/functioncmds.c
index 25b75375a8e..91f02a7eb2c 100644
--- a/src/backend/commands/functioncmds.c
+++ b/src/backend/commands/functioncmds.c
@@ -1325,6 +1325,8 @@ RemoveFunctionById(Oid funcOid)
 
 	table_close(relation, RowExclusiveLock);
 
+	pgstat_drop_function(funcOid);
+
 	/*
 	 * If there's a pg_aggregate tuple, delete that too.
 	 */
diff --git a/src/backend/commands/matview.c b/src/backend/commands/matview.c
index 05e7b60059d..db0a4501314 100644
--- a/src/backend/commands/matview.c
+++ b/src/backend/commands/matview.c
@@ -338,10 +338,10 @@ ExecRefreshMatView(RefreshMatViewStmt *stmt, const char *queryString,
 		refresh_by_heap_swap(matviewOid, OIDNewHeap, relpersistence);
 
 		/*
-		 * Inform stats collector about our activity: basically, we truncated
-		 * the matview and inserted some new data.  (The concurrent code path
-		 * above doesn't need to worry about this because the inserts and
-		 * deletes it issues get counted by lower-level code.)
+		 * Inform activity stats facility about our activity: basically, we
+		 * truncated the matview and inserted some new data.  (The concurrent
+		 * code path above doesn't need to worry about this because the inserts
+		 * and deletes it issues get counted by lower-level code.)
 		 */
 		pgstat_count_truncate(matviewRel);
 		if (!stmt->skipData)
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index 50a4a612e58..531b41d9406 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -352,8 +352,8 @@ vacuum(List *relations, VacuumParams *params,
 				 errmsg("PROCESS_TOAST required with VACUUM FULL")));
 
 	/*
-	 * Send info about dead objects to the statistics collector, unless we are
-	 * in autovacuum --- autovacuum.c does this for itself.
+	 * Send info about dead objects to the activity statistics facility, unless
+	 * we are in autovacuum --- autovacuum.c does this for itself.
 	 */
 	if ((params->options & VACOPT_VACUUM) && !IsAutoVacuumWorkerProcess())
 		pgstat_vacuum_stat();
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a96..6b4f742578e 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -28,6 +28,7 @@
 
 #include "access/amapi.h"
 #include "access/table.h"
+#include "access/xact.h"
 #include "catalog/index.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
@@ -35,6 +36,7 @@
 #include "storage/bufmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 
 /*
  * DSM keys for parallel vacuum.  Unlike other parallel execution code, since
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 681ef91b81e..aa51228117c 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -342,9 +342,6 @@ static void autovacuum_do_vac_analyze(autovac_table *tab,
 									  BufferAccessStrategy bstrategy);
 static AutoVacOpts *extract_autovac_opts(HeapTuple tup,
 										 TupleDesc pg_class_desc);
-static PgStat_StatTabEntry *get_pgstat_tabentry_relid(Oid relid, bool isshared,
-													  PgStat_StatDBEntry *shared,
-													  PgStat_StatDBEntry *dbentry);
 static void perform_work_item(AutoVacuumWorkItem *workitem);
 static void autovac_report_activity(autovac_table *tab);
 static void autovac_report_workitem(AutoVacuumWorkItem *workitem,
@@ -1689,7 +1686,7 @@ AutoVacWorkerMain(int argc, char *argv[])
 		char		dbname[NAMEDATALEN];
 
 		/*
-		 * Report autovac startup to the stats collector.  We deliberately do
+		 * Report autovac startup to the stats system.  We deliberately do
 		 * this before InitPostgres, so that the last_autovac_time will get
 		 * updated even if the connection attempt fails.  This is to prevent
 		 * autovac from getting "stuck" repeatedly selecting an unopenable
@@ -1966,8 +1963,6 @@ do_autovacuum(void)
 	HASHCTL		ctl;
 	HTAB	   *table_toast_map;
 	ListCell   *volatile cell;
-	PgStat_StatDBEntry *shared;
-	PgStat_StatDBEntry *dbentry;
 	BufferAccessStrategy bstrategy;
 	ScanKeyData key;
 	TupleDesc	pg_class_desc;
@@ -1986,19 +1981,13 @@ do_autovacuum(void)
 										  ALLOCSET_DEFAULT_SIZES);
 	MemoryContextSwitchTo(AutovacMemCxt);
 
-	/*
-	 * may be NULL if we couldn't find an entry (only happens if we are
-	 * forcing a vacuum for anti-wrap purposes).
-	 */
-	dbentry = pgstat_fetch_stat_dbentry(MyDatabaseId);
-
 	/* Start a transaction so our commands have one to play into. */
 	StartTransactionCommand();
 
 	/*
-	 * Clean up any dead statistics collector entries for this DB. We always
-	 * want to do this exactly once per DB-processing cycle, even if we find
-	 * nothing worth vacuuming in the database.
+	 * Clean up any dead statistics entries for this DB. We always want to do
+	 * this exactly once per DB-processing cycle, even if we find nothing
+	 * worth vacuuming in the database.
 	 */
 	pgstat_vacuum_stat();
 
@@ -2039,9 +2028,6 @@ do_autovacuum(void)
 	/* StartTransactionCommand changed elsewhere */
 	MemoryContextSwitchTo(AutovacMemCxt);
 
-	/* The database hash where pgstat keeps shared relations */
-	shared = pgstat_fetch_stat_dbentry(InvalidOid);
-
 	classRel = table_open(RelationRelationId, AccessShareLock);
 
 	/* create a copy so we can use it after closing pg_class */
@@ -2119,8 +2105,8 @@ do_autovacuum(void)
 
 		/* Fetch reloptions and the pgstat entry for this table */
 		relopts = extract_autovac_opts(tuple, pg_class_desc);
-		tabentry = get_pgstat_tabentry_relid(relid, classForm->relisshared,
-											 shared, dbentry);
+		tabentry = pgstat_fetch_stat_tabentry_extended(classForm->relisshared,
+													   relid);
 
 		/* Check if it needs vacuum or analyze */
 		relation_needs_vacanalyze(relid, relopts, classForm, tabentry,
@@ -2203,8 +2189,8 @@ do_autovacuum(void)
 		}
 
 		/* Fetch the pgstat entry for this table */
-		tabentry = get_pgstat_tabentry_relid(relid, classForm->relisshared,
-											 shared, dbentry);
+		tabentry = pgstat_fetch_stat_tabentry_extended(classForm->relisshared,
+													   relid);
 
 		relation_needs_vacanalyze(relid, relopts, classForm, tabentry,
 								  effective_multixact_freeze_max_age,
@@ -2768,29 +2754,6 @@ extract_autovac_opts(HeapTuple tup, TupleDesc pg_class_desc)
 	return av;
 }
 
-/*
- * get_pgstat_tabentry_relid
- *
- * Fetch the pgstat entry of a table, either local to a database or shared.
- */
-static PgStat_StatTabEntry *
-get_pgstat_tabentry_relid(Oid relid, bool isshared, PgStat_StatDBEntry *shared,
-						  PgStat_StatDBEntry *dbentry)
-{
-	PgStat_StatTabEntry *tabentry = NULL;
-
-	if (isshared)
-	{
-		if (PointerIsValid(shared))
-			tabentry = hash_search(shared->tables, &relid,
-								   HASH_FIND, NULL);
-	}
-	else if (PointerIsValid(dbentry))
-		tabentry = hash_search(dbentry->tables, &relid,
-							   HASH_FIND, NULL);
-
-	return tabentry;
-}
 
 /*
  * table_recheck_autovac
@@ -3001,17 +2964,10 @@ recheck_relation_needs_vacanalyze(Oid relid,
 								  bool *wraparound)
 {
 	PgStat_StatTabEntry *tabentry;
-	PgStat_StatDBEntry *shared = NULL;
-	PgStat_StatDBEntry *dbentry = NULL;
-
-	if (classForm->relisshared)
-		shared = pgstat_fetch_stat_dbentry(InvalidOid);
-	else
-		dbentry = pgstat_fetch_stat_dbentry(MyDatabaseId);
 
 	/* fetch the pgstat table entry */
-	tabentry = get_pgstat_tabentry_relid(relid, classForm->relisshared,
-										 shared, dbentry);
+	tabentry = pgstat_fetch_stat_tabentry_extended(classForm->relisshared,
+												   relid);
 
 	relation_needs_vacanalyze(relid, avopts, classForm, tabentry,
 							  effective_multixact_freeze_max_age,
@@ -3041,7 +2997,7 @@ recheck_relation_needs_vacanalyze(Oid relid,
  *
  * For analyze, the analysis done is that the number of tuples inserted,
  * deleted and updated since the last analyze exceeds a threshold calculated
- * in the same fashion as above.  Note that the collector actually stores
+ * in the same fashion as above.  Note that the stats system stores
  * the number of tuples (both live and dead) that there were as of the last
  * analyze.  This is asymmetric to the VACUUM case.
  *
@@ -3052,7 +3008,7 @@ recheck_relation_needs_vacanalyze(Oid relid,
  * A table whose autovacuum_enabled option is false is
  * automatically skipped (unless we have to vacuum it due to freeze_max_age).
  * Thus autovacuum can be disabled for specific tables. Also, when the stats
- * collector does not have data about a table, it will be skipped.
+ * system does not have data about a table, it will be skipped.
  *
  * A table whose vac_base_thresh value is < 0 takes the base value from the
  * autovacuum_vacuum_threshold GUC variable.  Similarly, a vac_scale_factor
@@ -3484,6 +3440,8 @@ AutoVacuumShmemInit(void)
  *
  * Note: we avoid throttling in the autovac worker, as it would be
  * counterproductive in the recheck logic.
+ *
+ * FIXME: Shouldn't be needed anymore
  */
 static void
 autovac_refresh_stats(void)
diff --git a/src/backend/postmaster/bgwriter.c b/src/backend/postmaster/bgwriter.c
index d1f5d12eff8..9fd09d3196b 100644
--- a/src/backend/postmaster/bgwriter.c
+++ b/src/backend/postmaster/bgwriter.c
@@ -241,9 +241,9 @@ BackgroundWriterMain(void)
 		can_hibernate = BgBufferSync(&wb_context);
 
 		/*
-		 * Send off activity statistics to the stats collector
+		 * Send off activity statistics to the stats system
 		 */
-		pgstat_send_bgwriter();
+		pgstat_report_bgwriter();
 
 		if (FirstCallSinceLastCheckpoint())
 		{
diff --git a/src/backend/postmaster/checkpointer.c b/src/backend/postmaster/checkpointer.c
index 4488e3a4435..60d1a0d527e 100644
--- a/src/backend/postmaster/checkpointer.c
+++ b/src/backend/postmaster/checkpointer.c
@@ -212,6 +212,16 @@ CheckpointerMain(void)
 	 */
 	last_checkpoint_time = last_xlog_switch_time = (pg_time_t) time(NULL);
 
+	/*
+	 * Write out stats after shutdown. This needs to be called by exactly one
+	 * process during a normal shutdown, and since checkpointer is shut down
+	 * very late...
+	 *
+	 * XXX: Are there potential issues with walsenders reporting stats at a
+	 * later time?
+	 */
+	before_shmem_exit(pgstat_before_server_shutdown, 0);
+
 	/*
 	 * Create a memory context that we will do all our work in.  We do this so
 	 * that we can reset the context during error recovery and thereby avoid
@@ -358,7 +368,7 @@ CheckpointerMain(void)
 		if (((volatile CheckpointerShmemStruct *) CheckpointerShmem)->ckpt_flags)
 		{
 			do_checkpoint = true;
-			PendingCheckpointerStats.m_requested_checkpoints++;
+			PendingCheckpointerStats.requested_checkpoints++;
 		}
 
 		/*
@@ -372,7 +382,7 @@ CheckpointerMain(void)
 		if (elapsed_secs >= CheckPointTimeout)
 		{
 			if (!do_checkpoint)
-				PendingCheckpointerStats.m_timed_checkpoints++;
+				PendingCheckpointerStats.timed_checkpoints++;
 			do_checkpoint = true;
 			flags |= CHECKPOINT_CAUSE_TIME;
 		}
@@ -489,13 +499,9 @@ CheckpointerMain(void)
 		/* Check for archive_timeout and switch xlog files if necessary. */
 		CheckArchiveTimeout();
 
-		/*
-		 * Send off activity statistics to the stats collector.
-		 */
-		pgstat_send_checkpointer();
-
-		/* Send WAL statistics to the stats collector. */
-		pgstat_send_wal(true);
+		/* Report pending stats */
+		pgstat_report_checkpointer();
+		pgstat_report_wal(false);
 
 		/*
 		 * If any checkpoint flags have been set, redo the loop to handle the
@@ -570,10 +576,10 @@ HandleCheckpointerInterrupts(void)
 		 * updates the statistics, increment the checkpoint request and send
 		 * the statistics to the stats collector.
 		 */
-		PendingCheckpointerStats.m_requested_checkpoints++;
+		PendingCheckpointerStats.requested_checkpoints++;
 		ShutdownXLOG(0, 0);
-		pgstat_send_checkpointer();
-		pgstat_send_wal(true);
+		pgstat_report_checkpointer();
+		pgstat_report_wal(true);
 
 		/* Normal exit from the checkpointer is here */
 		proc_exit(0);			/* done */
@@ -718,7 +724,7 @@ CheckpointWriteDelay(int flags, double progress)
 		/*
 		 * Report interim activity statistics.
 		 */
-		pgstat_send_checkpointer();
+		pgstat_report_checkpointer();
 
 		/*
 		 * This sleep used to be connected to bgwriter_delay, typically 200ms.
@@ -1262,9 +1268,9 @@ AbsorbSyncRequests(void)
 	LWLockAcquire(CheckpointerCommLock, LW_EXCLUSIVE);
 
 	/* Transfer stats counts into pending pgstats message */
-	PendingCheckpointerStats.m_buf_written_backend
+	PendingCheckpointerStats.buf_written_backend
 		+= CheckpointerShmem->num_backend_writes;
-	PendingCheckpointerStats.m_buf_fsync_backend
+	PendingCheckpointerStats.buf_fsync_backend
 		+= CheckpointerShmem->num_backend_fsync;
 
 	CheckpointerShmem->num_backend_writes = 0;
diff --git a/src/backend/postmaster/pgarch.c b/src/backend/postmaster/pgarch.c
index d916ed39a8c..52f5a0f0e96 100644
--- a/src/backend/postmaster/pgarch.c
+++ b/src/backend/postmaster/pgarch.c
@@ -477,20 +477,20 @@ pgarch_ArchiverCopyLoop(void)
 				pgarch_archiveDone(xlog);
 
 				/*
-				 * Tell the collector about the WAL file that we successfully
-				 * archived
+				 * Tell the activity statistics facility about the WAL file
+				 * that we successfully archived
 				 */
-				pgstat_send_archiver(xlog, false);
+				pgstat_report_archiver(xlog, false);
 
 				break;			/* out of inner retry loop */
 			}
 			else
 			{
 				/*
-				 * Tell the collector about the WAL file that we failed to
-				 * archive
+				 * Tell the activity statistics facility about the WAL file
+				 * that we failed to archive
 				 */
-				pgstat_send_archiver(xlog, true);
+				pgstat_report_archiver(xlog, true);
 
 				if (++failures >= NUM_ARCHIVE_RETRIES)
 				{
diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c
index fc104b52603..fb4e858332a 100644
--- a/src/backend/postmaster/pgstat.c
+++ b/src/backend/postmaster/pgstat.c
@@ -1,74 +1,51 @@
 /* ----------
  * pgstat.c
+ *	  Activity statistics infrastructure.
  *
- *	All the statistics collector stuff hacked up in one big, ugly file.
+ * Provides the infrastructure to collect and access activity statistics,
+ * e.g. per-table access statistics, of all backends in shared memory.
  *
- *	TODO:	- Separate collector, postmaster and backend stuff
- *			  into different files.
+ * For many times of statistics pending stats updates are first accumulated
+ * locally in each process, then later flushed to shared memory (just after
+ * commit, or by idle-timeout)
  *
- *			- Add some automatic call for pgstat vacuuming.
+ * To avoid congestion on the shared memory, shared stats is updated no more
+ * often than once per PGSTAT_MIN_INTERVAL (10000ms). If some local numbers
+ * remain unflushed for lock failure, retry with intervals that is initially
+ * PGSTAT_RETRY_MIN_INTERVAL (1000ms) then doubled at every retry. Finally we
+ * force update after PGSTAT_MAX_INTERVAL (60000ms) since the first trial.
  *
- *			- Add a pgstat config column to pg_database, so this
- *			  entire thing can be enabled/disabled on a per db basis.
+ * AFIXME: Isn't PGSTAT_MIN_INTERVAL way too long? What is the justification
+ * for increasing it?
  *
- *	Copyright (c) 2001-2022, PostgreSQL Global Development Group
+ * NB: Code for individual kinds of statistics belongs into pgstat_*.c
+ * whenever possible, not here.
  *
- *	src/backend/postmaster/pgstat.c
+ * Copyright (c) 2001-2022, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *	  src/backend/postmaster/pgstat.c
  * ----------
  */
 #include "postgres.h"
 
 #include <unistd.h>
-#include <fcntl.h>
-#include <sys/param.h>
-#include <sys/time.h>
-#include <sys/socket.h>
-#include <netdb.h>
-#include <netinet/in.h>
-#include <arpa/inet.h>
-#include <signal.h>
-#include <time.h>
-#ifdef HAVE_SYS_SELECT_H
-#include <sys/select.h>
-#endif
 
-#include "access/heapam.h"
-#include "access/htup_details.h"
-#include "access/tableam.h"
 #include "access/transam.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 "libpq/libpq.h"
-#include "libpq/pqsignal.h"
-#include "mb/pg_wchar.h"
+#include "common/hashfn.h"
+#include "lib/dshash.h"
 #include "miscadmin.h"
 #include "pgstat.h"
-#include "postmaster/autovacuum.h"
-#include "postmaster/fork_process.h"
-#include "postmaster/interrupt.h"
-#include "postmaster/postmaster.h"
+#include "port/atomics.h"
 #include "replication/slot.h"
-#include "replication/walsender.h"
-#include "storage/backendid.h"
-#include "storage/dsm.h"
 #include "storage/fd.h"
 #include "storage/ipc.h"
-#include "storage/latch.h"
-#include "storage/lmgr.h"
+#include "storage/lwlock.h"
 #include "storage/pg_shmem.h"
-#include "storage/proc.h"
-#include "storage/procsignal.h"
-#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"
 #include "utils/timestamp.h"
 
 
@@ -76,25 +53,77 @@
  * Timer definitions.
  * ----------
  */
+#define PGSTAT_MIN_INTERVAL			10000	/* Minimum interval of stats data
+											 * updates; in milliseconds. */
 
-#define PGSTAT_RETRY_DELAY		10	/* How long to wait between checks for a
-									 * new file; in milliseconds. */
+#define PGSTAT_RETRY_MIN_INTERVAL	1000	/* Initial retry interval after
+											 * PGSTAT_MIN_INTERVAL */
 
-#define PGSTAT_MAX_WAIT_TIME	10000	/* Maximum time to wait for a stats
-										 * file update; in milliseconds. */
+#define PGSTAT_MAX_INTERVAL			60000	/* Longest interval of stats data
+											 * updates */
 
-#define PGSTAT_INQ_INTERVAL		640 /* How often to ping the collector for a
-									 * new file; in milliseconds. */
 
-#define PGSTAT_RESTART_INTERVAL 60	/* How often to attempt to restart a
-									 * failed statistics collector; in
-									 * seconds. */
+/* ----------
+ * The initial size hints for the hash tables used in the activity statistics.
+ * ----------
+ */
+#define PGSTAT_TABLE_HASH_SIZE	512
 
-#define PGSTAT_POLL_LOOP_COUNT	(PGSTAT_MAX_WAIT_TIME / PGSTAT_RETRY_DELAY)
-#define PGSTAT_INQ_LOOP_COUNT	(PGSTAT_INQ_INTERVAL / PGSTAT_RETRY_DELAY)
 
-/* Minimum receive buffer size for the collector's socket. */
-#define PGSTAT_MIN_RCVBUF		(100 * 1024)
+/* hash table entry for finding the PgStatSharedRef for a key */
+typedef struct PgStatSharedRefHashEntry
+{
+	PgStatHashKey key;			/* hash key */
+	char		status;			/* for simplehash use */
+	PgStatSharedRef *shared_ref;
+} PgStatSharedRefHashEntry;
+
+/* hash table for statistics snapshots entry */
+typedef struct PgStatSnapshotEntry
+{
+	PgStatHashKey key;
+	char		status;			/* for simplehash use */
+	void	   *data;			/* the stats data itself */
+} PgStatSnapshotEntry;
+
+typedef struct PgStat_PendingDroppedStatsItem
+{
+	PgStat_DroppedStatsItem item;
+	bool is_create;
+	dlist_node	node;
+} PgStat_PendingDroppedStatsItem;
+
+
+/* ----------
+ * Hash Table Types
+ * ----------
+ */
+
+/* for references to shared statistics entries */
+#define SH_PREFIX pgstat_shared_ref_hash
+#define SH_ELEMENT_TYPE PgStatSharedRefHashEntry
+#define SH_KEY_TYPE PgStatHashKey
+#define SH_KEY key
+#define SH_HASH_KEY(tb, key) \
+	hash_bytes((unsigned char *)&key, sizeof(PgStatHashKey))
+#define SH_EQUAL(tb, a, b) (memcmp(&a, &b, sizeof(PgStatHashKey)) == 0)
+#define SH_SCOPE static inline
+#define SH_DEFINE
+#define SH_DECLARE
+#include "lib/simplehash.h"
+
+/* for stats snapshot entries */
+#define SH_PREFIX pgstat_snapshot
+#define SH_ELEMENT_TYPE PgStatSnapshotEntry
+#define SH_KEY_TYPE PgStatHashKey
+#define SH_KEY key
+#define SH_HASH_KEY(tb, key) \
+	hash_bytes((unsigned char *)&key, sizeof(PgStatHashKey))
+#define SH_EQUAL(tb, a, b) (memcmp(&a, &b, sizeof(PgStatHashKey)) == 0)
+#define SH_SCOPE static inline
+#define SH_DEFINE
+#define SH_DECLARE
+#include "lib/simplehash.h"
 
 
 /* ----------
@@ -102,63 +131,28 @@
  * ----------
  */
 
-#ifdef EXEC_BACKEND
-static pid_t pgstat_forkexec(void);
-#endif
-
-NON_EXEC_STATIC void PgstatCollectorMain(int argc, char *argv[]) pg_attribute_noreturn();
-
-static PgStat_StatDBEntry *pgstat_get_db_entry(Oid databaseid, bool create);
-static PgStat_StatTabEntry *pgstat_get_tab_entry(PgStat_StatDBEntry *dbentry,
-												 Oid tableoid, bool create);
-static PgStat_StatSubEntry *pgstat_get_subscription_entry(Oid subid, bool create);
-static void pgstat_reset_subscription(PgStat_StatSubEntry *subentry, TimestampTz ts);
-static void pgstat_write_statsfiles(bool permanent, bool allDbs);
-static void pgstat_write_db_statsfile(PgStat_StatDBEntry *dbentry, bool permanent);
-static HTAB *pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep);
-static void pgstat_read_db_statsfile(Oid databaseid, HTAB *tabhash, HTAB *funchash,
-									 bool permanent);
-static void backend_read_statsfile(void);
-
-static bool pgstat_write_statsfile_needed(void);
-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 HTAB *pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid);
-
 static void pgstat_setup_memcxt(void);
+static void pgstat_write_statsfile(void);
+static void pgstat_read_statsfile(void);
 
-static void pgstat_recv_inquiry(PgStat_MsgInquiry *msg, int len);
-static void pgstat_recv_tabstat(PgStat_MsgTabstat *msg, int len);
-static void pgstat_recv_tabpurge(PgStat_MsgTabpurge *msg, int len);
-static void pgstat_recv_dropdb(PgStat_MsgDropdb *msg, int len);
-static void pgstat_recv_resetcounter(PgStat_MsgResetcounter *msg, int len);
-static void pgstat_recv_resetsharedcounter(PgStat_MsgResetsharedcounter *msg, int len);
-static void pgstat_recv_resetsinglecounter(PgStat_MsgResetsinglecounter *msg, int len);
-static void pgstat_recv_resetslrucounter(PgStat_MsgResetslrucounter *msg, int len);
-static void pgstat_recv_resetreplslotcounter(PgStat_MsgResetreplslotcounter *msg, int len);
-static void pgstat_recv_resetsubcounter(PgStat_MsgResetsubcounter *msg, int len);
-static void pgstat_recv_autovac(PgStat_MsgAutovacStart *msg, int len);
-static void pgstat_recv_vacuum(PgStat_MsgVacuum *msg, int len);
-static void pgstat_recv_analyze(PgStat_MsgAnalyze *msg, int len);
-static void pgstat_recv_archiver(PgStat_MsgArchiver *msg, int len);
-static void pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len);
-static void pgstat_recv_checkpointer(PgStat_MsgCheckpointer *msg, int len);
-static void pgstat_recv_wal(PgStat_MsgWal *msg, int len);
-static void pgstat_recv_slru(PgStat_MsgSLRU *msg, int len);
-static void pgstat_recv_funcstat(PgStat_MsgFuncstat *msg, int len);
-static void pgstat_recv_funcpurge(PgStat_MsgFuncpurge *msg, int len);
-static void pgstat_recv_recoveryconflict(PgStat_MsgRecoveryConflict *msg, int len);
-static void pgstat_recv_deadlock(PgStat_MsgDeadlock *msg, int len);
-static void pgstat_recv_checksum_failure(PgStat_MsgChecksumFailure *msg, int len);
-static void pgstat_recv_connect(PgStat_MsgConnect *msg, int len);
-static void pgstat_recv_disconnect(PgStat_MsgDisconnect *msg, int len);
-static void pgstat_recv_replslot(PgStat_MsgReplSlot *msg, int len);
-static void pgstat_recv_tempfile(PgStat_MsgTempFile *msg, int len);
-static void pgstat_recv_subscription_drop(PgStat_MsgSubscriptionDrop *msg, int len);
-static void pgstat_recv_subscription_error(PgStat_MsgSubscriptionError *msg, int len);
+static PgStatShm_StatEntryHeader *pgstat_shared_stat_entry_init(PgStatKind kind,
+																PgStatShmHashEntry *shhashent,
+																int init_refcount);
+static void pgstat_shared_ref_release(PgStatHashKey key, PgStatSharedRef *shared_ref);
+static inline size_t shared_stat_entry_len(PgStatKind kind);
+static inline void* shared_stat_entry_data(PgStatKind kind, PgStatShm_StatEntryHeader *entry);
+
+static bool pgstat_shared_refs_need_gc(void);
+static void pgstat_shared_refs_gc(void);
+
+static void pgstat_shared_refs_release_all(void);
+
+static void pgstat_perform_drop(PgStat_DroppedStatsItem *drop);
+static bool pgstat_drop_stats_entry(dshash_seq_status *hstat);
+
+static void pgstat_pending_delete(PgStatSharedRef *shared_ref);
+
+static bool pgstat_pending_flush_stats(bool nowait);
 
 
 /* ----------
@@ -167,6 +161,7 @@ static void pgstat_recv_subscription_error(PgStat_MsgSubscriptionError *msg, int
  */
 
 bool		pgstat_track_counts = false;
+int			pgstat_fetch_consistency = STATS_FETCH_CONSISTENCY_NONE;
 
 
 /* ----------
@@ -175,49 +170,68 @@ bool		pgstat_track_counts = false;
  */
 
 char	   *pgstat_stat_directory = NULL;
+/* No longer used, but will be removed with GUC */
 char	   *pgstat_stat_filename = NULL;
 char	   *pgstat_stat_tmpname = NULL;
 
 
 /* ----------
- * Local data
+ * Stats shared memory state
  * ----------
  */
 
-static struct sockaddr_storage pgStatAddr;
+/* backend-lifetime storages */
+PgStatShmemGlobal *pgStatShmem = NULL;
+static dsa_area *pgStatDSA = NULL;
+/* The shared hash to index activity stats entries. */
+static dshash_table *pgStatSharedHash = NULL;
 
-pgsocket pgStatSock = PGINVALID_SOCKET;
 
-static time_t last_pgstat_start_time;
+/* ----------
+ * Local data
+ *
+ * NB: There should be only variables related to stats infrastructure here,
+ * not for specific types of stats.
+ * ----------
+ */
+
+/*
+ * Backend local references to shared stats entries. If there are pending
+ * updates to a stats entry, the PgStatSharedRef is added to the pgStatPending
+ * list.
+ *
+ * When a stats entry is dropped each backend needs to release its reference
+ * to it before the memory can be released. To trigger that
+ * pgStatShmem->gc_count is incremented - which each backend compares to their
+ * copy of pgStatSharedRefAge on a regular basis.
+ */
+static pgstat_shared_ref_hash_hash *pgStatSharedRefHash = NULL;
+static int	pgStatSharedRefAge = 0;	/* cache age of pgStatShmLookupCache */
+
+/*
+ * List of PgStatSharedRefs with unflushed pending stats.
+ *
+ * Newly pending entries should only ever be added to the end of the list,
+ * otherwise pgstat_pending_flush_stats() might not see them immediately.
+ */
+static dlist_head pgStatPending = DLIST_STATIC_INIT(pgStatPending);
+
+/*
+ * Memory context containing the pgStatSharedRefHash table, the
+ * pgStatSharedRef entries, and pending data. Mostly to make it easier to
+ * track memory usage.
+ */
+static MemoryContext pgStatSharedRefContext = NULL;
 
-static bool pgStatRunningInCollector = false;
 
 static PgStat_SubXactStatus *pgStatXactStack = NULL;
 
-/*
- * Info about current "snapshot" of stats file
- */
-static MemoryContext pgStatLocalContext = NULL;
-static HTAB *pgStatDBHash = NULL;
 
 /*
- * Cluster wide statistics, kept in the stats collector.
- * Contains statistics that are not collected per database
- * or per table.
+ * Force the next stats flush to happen regardless of
+ * PGSTAT_MIN_INTERVAL. Useful in test scripts.
  */
-static PgStat_ArchiverStats archiverStats;
-static PgStat_GlobalStats globalStats;
-static PgStat_WalStats walStats;
-static PgStat_SLRUStats slruStats[SLRU_NUM_ELEMENTS];
-static HTAB *replSlotStatHash = NULL;
-static HTAB *subscriptionStatHash = NULL;
-
-/*
- * List of OIDs of databases we need to write out.  If an entry is InvalidOid,
- * it means to write only the shared-catalog stats ("DB 0"); otherwise, we
- * will write both that DB's data and the shared stats.
- */
-static List *pending_write_requests = NIL;
+static bool pgStatForceNextFlush = false;
 
 /*
  * For assertions that check pgstat is not used before initialization / after
@@ -229,492 +243,384 @@ static bool pgstat_is_shutdown = false;
 #endif
 
 
+/*
+ * The current statistics snapshot
+ */
+PgStatSnapshot stats_snapshot;
+/* to free snapshot in bulk */
+static MemoryContext pgStatSnapshotContext = NULL;
+
+
+/* ----------
+ * Constants
+ * ----------
+ */
+
+/*
+ * Define the different kinds of statistics. If reasonably possible, handling
+ * specific to one kind of stats should go through this abstraction, rather
+ * than making more of pgstat.c aware.
+ *
+ * See comments for struct pgstat_kind_info for details about the individual
+ * fields.
+ *
+ * XXX: It'd be nicer to define this outside of this file. But there doesn't
+ * seem to be a great way of doing that, given the split across multiple
+ * files.
+ */
+static const pgstat_kind_info pgstat_kind_infos[PGSTAT_KIND_WAL + 1] = {
+
+	/* stats types with a variable number of stats */
+
+	[PGSTAT_KIND_DB] = {
+		.is_global = false,
+		/* so pg_stat_database entries can be seen in all databases */
+		.accessed_across_databases = true,
+
+		.shared_size = sizeof(PgStatShm_StatDBEntry),
+		.shared_data_off = offsetof(PgStatShm_StatDBEntry, stats),
+		.shared_data_len = sizeof(((PgStatShm_StatDBEntry*) 0)->stats),
+		.pending_size = sizeof(PgStat_StatDBEntry),
+
+		.flush_pending_cb = pgstat_flush_database,
+	},
+
+	[PGSTAT_KIND_TABLE] = {
+		.is_global = false,
+		.shared_size = sizeof(PgStatShm_StatTabEntry),
+		.shared_data_off = offsetof(PgStatShm_StatTabEntry, stats),
+		.shared_data_len = sizeof(((PgStatShm_StatTabEntry*) 0)->stats),
+		.pending_size = sizeof(PgStat_TableStatus),
+
+		.flush_pending_cb = pgstat_flush_relation,
+	},
+
+	[PGSTAT_KIND_FUNCTION] = {
+		.is_global = false,
+		.shared_size = sizeof(PgStatShm_StatFuncEntry),
+		.shared_data_off = offsetof(PgStatShm_StatFuncEntry, stats),
+		.shared_data_len = sizeof(((PgStatShm_StatFuncEntry*) 0)->stats),
+		.pending_size = sizeof(PgStat_BackendFunctionEntry),
+
+		.flush_pending_cb = pgstat_flush_function,
+	},
+
+	[PGSTAT_KIND_SUBSCRIPTION] = {
+		.is_global = false,
+		/* so pg_stat_subscription_counters entries can be seen in all databases */
+		.accessed_across_databases = true,
+		.shared_size = sizeof(PgStatShm_StatSubEntry),
+		.shared_data_off = offsetof(PgStatShm_StatSubEntry, stats),
+		.shared_data_len = sizeof(((PgStatShm_StatSubEntry*) 0)->stats),
+		.pending_size = sizeof(PgStat_BackendSubEntry),
+
+		.flush_pending_cb = pgstat_flush_subscription,
+	},
+
+
+	/* global stats */
+
+	[PGSTAT_KIND_ARCHIVER] = {
+		.is_global = true,
+
+		.snapshot_cb = pgstat_snapshot_archiver,
+	},
+
+	[PGSTAT_KIND_BGWRITER] = {
+		.is_global = true,
+
+		.snapshot_cb = pgstat_snapshot_bgwriter,
+	},
+
+	[PGSTAT_KIND_CHECKPOINTER] = {
+		.is_global = true,
+
+		.snapshot_cb = pgstat_snapshot_checkpointer,
+	},
+
+	[PGSTAT_KIND_REPLSLOT] = {
+		.is_global = true,
+
+		.snapshot_cb = pgstat_snapshot_replslot,
+	},
+
+	[PGSTAT_KIND_SLRU] = {
+		.is_global = true,
+
+		.snapshot_cb = pgstat_snapshot_slru,
+	},
+
+	[PGSTAT_KIND_WAL] = {
+		.is_global = true,
+
+		.snapshot_cb = pgstat_snapshot_wal,
+	},
+};
+
+/* parameter for the shared hash */
+static const dshash_parameters dsh_params = {
+	sizeof(PgStatHashKey),
+	sizeof(PgStatShmHashEntry),
+	dshash_memcmp,
+	dshash_memhash,
+	LWTRANCHE_STATS
+};
+
+
 /* ------------------------------------------------------------
  * Public functions called from postmaster follow
  * ------------------------------------------------------------
  */
 
-/* ----------
- * pgstat_init() -
- *
- *	Called from postmaster at startup. Create the resources required
- *	by the statistics collector process.  If unable to do so, do not
- *	fail --- better to let the postmaster start with stats collection
- *	disabled.
- * ----------
+/*
+ * The size of the shared memory allocation for stats stored in the shared
+ * stats hash table. This allocation will be done as part of the main shared
+ * memory, rather than dynamic shared memory, allowing it to be initialized in
+ * postmaster.
  */
-void
-pgstat_init(void)
+static Size
+stats_dsa_init_size(void)
 {
-	socklen_t	alen;
-	struct addrinfo *addrs = NULL,
-			   *addr,
-				hints;
-	int			ret;
-	fd_set		rset;
-	struct timeval tv;
-	char		test_byte;
-	int			sel_res;
-	int			tries = 0;
-
-#define TESTBYTEVAL ((char) 199)
-
 	/*
-	 * This static assertion verifies that we didn't mess up the calculations
-	 * involved in selecting maximum payload sizes for our UDP messages.
-	 * Because the only consequence of overrunning PGSTAT_MAX_MSG_SIZE would
-	 * be silent performance loss from fragmentation, it seems worth having a
-	 * compile-time cross-check that we didn't.
+	 * AFIXME: What should we choose as an initial size? Should we make this
+	 * configurable?
 	 */
-	StaticAssertStmt(sizeof(PgStat_Msg) <= PGSTAT_MAX_MSG_SIZE,
-					 "maximum stats message size exceeds PGSTAT_MAX_MSG_SIZE");
+	return dsa_minimum_size() + 2 * 1024 * 1024;
+}
 
-	/*
-	 * Create the UDP socket for sending and receiving statistic messages
-	 */
-	hints.ai_flags = AI_PASSIVE;
-	hints.ai_family = AF_UNSPEC;
-	hints.ai_socktype = SOCK_DGRAM;
-	hints.ai_protocol = 0;
-	hints.ai_addrlen = 0;
-	hints.ai_addr = NULL;
-	hints.ai_canonname = NULL;
-	hints.ai_next = NULL;
-	ret = pg_getaddrinfo_all("localhost", NULL, &hints, &addrs);
-	if (ret || !addrs)
-	{
-		ereport(LOG,
-				(errmsg("could not resolve \"localhost\": %s",
-						gai_strerror(ret))));
-		goto startup_failed;
-	}
-
-	/*
-	 * On some platforms, pg_getaddrinfo_all() may return multiple addresses
-	 * only one of which will actually work (eg, both IPv6 and IPv4 addresses
-	 * when kernel will reject IPv6).  Worse, the failure may occur at the
-	 * bind() or perhaps even connect() stage.  So we must loop through the
-	 * results till we find a working combination. We will generate LOG
-	 * messages, but no error, for bogus combinations.
-	 */
-	for (addr = addrs; addr; addr = addr->ai_next)
-	{
-#ifdef HAVE_UNIX_SOCKETS
-		/* Ignore AF_UNIX sockets, if any are returned. */
-		if (addr->ai_family == AF_UNIX)
-			continue;
-#endif
-
-		if (++tries > 1)
-			ereport(LOG,
-					(errmsg("trying another address for the statistics collector")));
-
-		/*
-		 * Create the socket.
-		 */
-		if ((pgStatSock = socket(addr->ai_family, SOCK_DGRAM, 0)) == PGINVALID_SOCKET)
-		{
-			ereport(LOG,
-					(errcode_for_socket_access(),
-					 errmsg("could not create socket for statistics collector: %m")));
-			continue;
-		}
-
-		/*
-		 * Bind it to a kernel assigned port on localhost and get the assigned
-		 * port via getsockname().
-		 */
-		if (bind(pgStatSock, addr->ai_addr, addr->ai_addrlen) < 0)
-		{
-			ereport(LOG,
-					(errcode_for_socket_access(),
-					 errmsg("could not bind socket for statistics collector: %m")));
-			closesocket(pgStatSock);
-			pgStatSock = PGINVALID_SOCKET;
-			continue;
-		}
-
-		alen = sizeof(pgStatAddr);
-		if (getsockname(pgStatSock, (struct sockaddr *) &pgStatAddr, &alen) < 0)
-		{
-			ereport(LOG,
-					(errcode_for_socket_access(),
-					 errmsg("could not get address of socket for statistics collector: %m")));
-			closesocket(pgStatSock);
-			pgStatSock = PGINVALID_SOCKET;
-			continue;
-		}
-
-		/*
-		 * Connect the socket to its own address.  This saves a few cycles by
-		 * not having to respecify the target address on every send. This also
-		 * provides a kernel-level check that only packets from this same
-		 * address will be received.
-		 */
-		if (connect(pgStatSock, (struct sockaddr *) &pgStatAddr, alen) < 0)
-		{
-			ereport(LOG,
-					(errcode_for_socket_access(),
-					 errmsg("could not connect socket for statistics collector: %m")));
-			closesocket(pgStatSock);
-			pgStatSock = PGINVALID_SOCKET;
-			continue;
-		}
-
-		/*
-		 * Try to send and receive a one-byte test message on the socket. This
-		 * is to catch situations where the socket can be created but will not
-		 * actually pass data (for instance, because kernel packet filtering
-		 * rules prevent it).
-		 */
-		test_byte = TESTBYTEVAL;
-
-retry1:
-		if (send(pgStatSock, &test_byte, 1, 0) != 1)
-		{
-			if (errno == EINTR)
-				goto retry1;	/* if interrupted, just retry */
-			ereport(LOG,
-					(errcode_for_socket_access(),
-					 errmsg("could not send test message on socket for statistics collector: %m")));
-			closesocket(pgStatSock);
-			pgStatSock = PGINVALID_SOCKET;
-			continue;
-		}
-
-		/*
-		 * There could possibly be a little delay before the message can be
-		 * received.  We arbitrarily allow up to half a second before deciding
-		 * it's broken.
-		 */
-		for (;;)				/* need a loop to handle EINTR */
-		{
-			FD_ZERO(&rset);
-			FD_SET(pgStatSock, &rset);
-
-			tv.tv_sec = 0;
-			tv.tv_usec = 500000;
-			sel_res = select(pgStatSock + 1, &rset, NULL, NULL, &tv);
-			if (sel_res >= 0 || errno != EINTR)
-				break;
-		}
-		if (sel_res < 0)
-		{
-			ereport(LOG,
-					(errcode_for_socket_access(),
-					 errmsg("select() failed in statistics collector: %m")));
-			closesocket(pgStatSock);
-			pgStatSock = PGINVALID_SOCKET;
-			continue;
-		}
-		if (sel_res == 0 || !FD_ISSET(pgStatSock, &rset))
-		{
-			/*
-			 * This is the case we actually think is likely, so take pains to
-			 * give a specific message for it.
-			 *
-			 * errno will not be set meaningfully here, so don't use it.
-			 */
-			ereport(LOG,
-					(errcode(ERRCODE_CONNECTION_FAILURE),
-					 errmsg("test message did not get through on socket for statistics collector")));
-			closesocket(pgStatSock);
-			pgStatSock = PGINVALID_SOCKET;
-			continue;
-		}
-
-		test_byte++;			/* just make sure variable is changed */
-
-retry2:
-		if (recv(pgStatSock, &test_byte, 1, 0) != 1)
-		{
-			if (errno == EINTR)
-				goto retry2;	/* if interrupted, just retry */
-			ereport(LOG,
-					(errcode_for_socket_access(),
-					 errmsg("could not receive test message on socket for statistics collector: %m")));
-			closesocket(pgStatSock);
-			pgStatSock = PGINVALID_SOCKET;
-			continue;
-		}
-
-		if (test_byte != TESTBYTEVAL)	/* strictly paranoia ... */
-		{
-			ereport(LOG,
-					(errcode(ERRCODE_INTERNAL_ERROR),
-					 errmsg("incorrect test message transmission on socket for statistics collector")));
-			closesocket(pgStatSock);
-			pgStatSock = PGINVALID_SOCKET;
-			continue;
-		}
-
-		/* If we get here, we have a working socket */
-		break;
-	}
-
-	/* Did we find a working address? */
-	if (!addr || pgStatSock == PGINVALID_SOCKET)
-		goto startup_failed;
-
-	/*
-	 * Set the socket to non-blocking IO.  This ensures that if the collector
-	 * falls behind, statistics messages will be discarded; backends won't
-	 * block waiting to send messages to the collector.
-	 */
-	if (!pg_set_noblock(pgStatSock))
-	{
-		ereport(LOG,
-				(errcode_for_socket_access(),
-				 errmsg("could not set statistics collector socket to nonblocking mode: %m")));
-		goto startup_failed;
-	}
-
-	/*
-	 * Try to ensure that the socket's receive buffer is at least
-	 * PGSTAT_MIN_RCVBUF bytes, so that it won't easily overflow and lose
-	 * data.  Use of UDP protocol means that we are willing to lose data under
-	 * heavy load, but we don't want it to happen just because of ridiculously
-	 * small default buffer sizes (such as 8KB on older Windows versions).
-	 */
-	{
-		int			old_rcvbuf;
-		int			new_rcvbuf;
-		socklen_t	rcvbufsize = sizeof(old_rcvbuf);
-
-		if (getsockopt(pgStatSock, SOL_SOCKET, SO_RCVBUF,
-					   (char *) &old_rcvbuf, &rcvbufsize) < 0)
-		{
-			ereport(LOG,
-					(errmsg("%s(%s) failed: %m", "getsockopt", "SO_RCVBUF")));
-			/* if we can't get existing size, always try to set it */
-			old_rcvbuf = 0;
-		}
-
-		new_rcvbuf = PGSTAT_MIN_RCVBUF;
-		if (old_rcvbuf < new_rcvbuf)
-		{
-			if (setsockopt(pgStatSock, SOL_SOCKET, SO_RCVBUF,
-						   (char *) &new_rcvbuf, sizeof(new_rcvbuf)) < 0)
-				ereport(LOG,
-						(errmsg("%s(%s) failed: %m", "setsockopt", "SO_RCVBUF")));
-		}
-	}
-
-	pg_freeaddrinfo_all(hints.ai_family, addrs);
-
-	/* Now that we have a long-lived socket, tell fd.c about it. */
-	ReserveExternalFD();
-
-	return;
-
-startup_failed:
-	ereport(LOG,
-			(errmsg("disabling statistics collector for lack of working socket")));
-
-	if (addrs)
-		pg_freeaddrinfo_all(hints.ai_family, addrs);
-
-	if (pgStatSock != PGINVALID_SOCKET)
-		closesocket(pgStatSock);
-	pgStatSock = PGINVALID_SOCKET;
-
-	/*
-	 * Adjust GUC variables to suppress useless activity, and for debugging
-	 * purposes (seeing track_counts off is a clue that we failed here). We
-	 * use PGC_S_OVERRIDE because there is no point in trying to turn it back
-	 * on from postgresql.conf without a restart.
-	 */
-	SetConfigOption("track_counts", "off", PGC_INTERNAL, PGC_S_OVERRIDE);
+static Size
+stats_replslot_size(void)
+{
+	return sizeof(PgStat_StatReplSlotEntry) * max_replication_slots;
 }
 
 /*
- * subroutine for pgstat_reset_all
+ * Compute shared memory space needed for activity statistics
  */
-static void
-pgstat_reset_remove_files(const char *directory)
+Size
+StatsShmemSize(void)
 {
-	DIR		   *dir;
-	struct dirent *entry;
-	char		fname[MAXPGPATH * 2];
+	Size		sz;
 
-	dir = AllocateDir(directory);
-	while ((entry = ReadDir(dir, directory)) != NULL)
+	sz = MAXALIGN(sizeof(PgStatShmemGlobal));
+	sz = add_size(sz, MAXALIGN(stats_dsa_init_size()));
+	sz = add_size(sz, MAXALIGN(stats_replslot_size()));
+
+	return sz;
+}
+
+/*
+ * Initialize activity statistics initialize during startup
+ */
+void
+StatsShmemInit(void)
+{
+	bool		found;
+	Size		sz;
+
+	sz = StatsShmemSize();
+	pgStatShmem = (PgStatShmemGlobal *)
+		ShmemInitStruct("Stats area", sz, &found);
+
+	if (!IsUnderPostmaster)
 	{
-		int			nchars;
-		Oid			tmp_oid;
+		dsa_area *dsa;
+		dshash_table *dsh;
+		char *p = (char *) pgStatShmem;
+
+		Assert(!found);
+
+		/* the allocation of pgStatShmem itself */
+		p += MAXALIGN(sizeof(PgStatShmemGlobal));
 
 		/*
-		 * Skip directory entries that don't match the file names we write.
-		 * See get_dbstat_filename for the database-specific pattern.
+		 * Create a small dsa allocation in plain shared memory. Doing so
+		 * initially makes it easier to manage server startup, and it also is
+		 * a small efficiency win.
 		 */
-		if (strncmp(entry->d_name, "global.", 7) == 0)
-			nchars = 7;
+		pgStatShmem->raw_dsa_area = p;
+		p += MAXALIGN(stats_dsa_init_size());
+		dsa = dsa_create_in_place(pgStatShmem->raw_dsa_area,
+								  stats_dsa_init_size(),
+								  LWTRANCHE_STATS, 0);
+		dsa_pin(dsa);
+
+		/*
+		 * Same with the dshash table.
+		 *
+		 * FIXME: we need to guarantee this can be allocated in plain shared
+		 * memory, rather than allocating dsm segments.
+		 */
+		dsh = dshash_create(dsa, &dsh_params, 0);
+		pgStatShmem->hash_handle = dshash_get_hash_table_handle(dsh);
+
+
+		/*
+		 * Postmaster will never access these again, thus free the local
+		 * dsa/dshash references.
+		 */
+		dshash_detach(dsh);
+		dsa_detach(dsa);
+
+		pg_atomic_init_u64(&pgStatShmem->gc_count, 0);
+
+
+		/*
+		 * Initialize global statistics.
+		 */
+
+		pgStatShmem->replslot.stats = (PgStat_StatReplSlotEntry *) p;
+		p += MAXALIGN(stats_replslot_size());
+		LWLockInitialize(&pgStatShmem->replslot.lock, LWTRANCHE_STATS);
+		for (int i = 0; i < max_replication_slots; i++)
+		{
+			pgStatShmem->replslot.stats[i].index = -1;
+		}
+
+		LWLockInitialize(&pgStatShmem->slru.lock, LWTRANCHE_STATS);
+
+		LWLockInitialize(&pgStatShmem->wal.lock, LWTRANCHE_STATS);
+	}
+	else
+	{
+		Assert(found);
+	}
+}
+
+
+/* ------------------------------------------------------------
+ * Functions manging the state of the stats system for all backends.
+ * ------------------------------------------------------------
+ */
+
+/*
+ * pgstat_restore_stats() - read on-disk stats into memory at server start.
+ *
+ * Should only be called by the startup process or in single user mode.
+ */
+void
+pgstat_restore_stats(void)
+{
+	pgstat_read_statsfile();
+}
+
+/*
+ * pgstat_discard_stats() -
+ *
+ * Remove the stats file.  This is currently used only if WAL recovery is
+ * needed after a crash.
+ *
+ * Should only be called by the startup process or in single user mode.
+ */
+void
+pgstat_discard_stats(void)
+{
+	int ret;
+
+	/* NB: this needs to be done even in single user mode */
+
+	ret = unlink(PGSTAT_STAT_PERMANENT_FILENAME);
+	if (ret != 0)
+	{
+		if (errno == ENOENT)
+			elog(DEBUG2,
+				 "didn't need to unlink permanent stats file \"%s\" - didn't exist",
+				 PGSTAT_STAT_PERMANENT_FILENAME);
 		else
-		{
-			nchars = 0;
-			(void) sscanf(entry->d_name, "db_%u.%n",
-						  &tmp_oid, &nchars);
-			if (nchars <= 0)
-				continue;
-			/* %u allows leading whitespace, so reject that */
-			if (strchr("0123456789", entry->d_name[3]) == NULL)
-				continue;
-		}
-
-		if (strcmp(entry->d_name + nchars, "tmp") != 0 &&
-			strcmp(entry->d_name + nchars, "stat") != 0)
-			continue;
-
-		snprintf(fname, sizeof(fname), "%s/%s", directory,
-				 entry->d_name);
-		unlink(fname);
-	}
-	FreeDir(dir);
-}
-
-/*
- * pgstat_reset_all() -
- *
- * Remove the stats files.  This is currently used only if WAL
- * recovery is needed after a crash.
- */
-void
-pgstat_reset_all(void)
-{
-	pgstat_reset_remove_files(pgstat_stat_directory);
-	pgstat_reset_remove_files(PGSTAT_STAT_PERMANENT_DIRECTORY);
-}
-
-#ifdef EXEC_BACKEND
-
-/*
- * pgstat_forkexec() -
- *
- * Format up the arglist for, then fork and exec, statistics collector process
- */
-static pid_t
-pgstat_forkexec(void)
-{
-	char	   *av[10];
-	int			ac = 0;
-
-	av[ac++] = "postgres";
-	av[ac++] = "--forkcol";
-	av[ac++] = NULL;			/* filled in by postmaster_forkexec */
-
-	av[ac] = NULL;
-	Assert(ac < lengthof(av));
-
-	return postmaster_forkexec(ac, av);
-}
-#endif							/* EXEC_BACKEND */
-
-
-/*
- * pgstat_start() -
- *
- *	Called from postmaster at startup or after an existing collector
- *	died.  Attempt to fire up a fresh statistics collector.
- *
- *	Returns PID of child process, or 0 if fail.
- *
- *	Note: if fail, we will be called again from the postmaster main loop.
- */
-int
-pgstat_start(void)
-{
-	time_t		curtime;
-	pid_t		pgStatPid;
-
-	/*
-	 * Check that the socket is there, else pgstat_init failed and we can do
-	 * nothing useful.
-	 */
-	if (pgStatSock == PGINVALID_SOCKET)
-		return 0;
-
-	/*
-	 * Do nothing if too soon since last collector start.  This is a safety
-	 * valve to protect against continuous respawn attempts if the collector
-	 * is dying immediately at launch.  Note that since we will be re-called
-	 * from the postmaster main loop, we will get another chance later.
-	 */
-	curtime = time(NULL);
-	if ((unsigned int) (curtime - last_pgstat_start_time) <
-		(unsigned int) PGSTAT_RESTART_INTERVAL)
-		return 0;
-	last_pgstat_start_time = curtime;
-
-	/*
-	 * Okay, fork off the collector.
-	 */
-#ifdef EXEC_BACKEND
-	switch ((pgStatPid = pgstat_forkexec()))
-#else
-	switch ((pgStatPid = fork_process()))
-#endif
-	{
-		case -1:
 			ereport(LOG,
-					(errmsg("could not fork statistics collector: %m")));
-			return 0;
-
-#ifndef EXEC_BACKEND
-		case 0:
-			/* in postmaster child ... */
-			InitPostmasterChild();
-
-			/* Close the postmaster's sockets */
-			ClosePostmasterPorts(false);
-
-			/* Drop our connection to postmaster's shared memory, as well */
-			dsm_detach_all();
-			PGSharedMemoryDetach();
-
-			PgstatCollectorMain(0, NULL);
-			break;
-#endif
-
-		default:
-			return (int) pgStatPid;
+					(errcode_for_file_access(),
+					 errmsg("could not unlink permanent statistics file \"%s\": %m",
+							PGSTAT_STAT_PERMANENT_FILENAME)));
+	}
+	else
+	{
+		ereport(DEBUG2,
+				(errcode_for_file_access(),
+				 errmsg("unlinked permanent statistics file \"%s\": %m",
+						PGSTAT_STAT_PERMANENT_FILENAME)));
 	}
-
-	/* shouldn't get here */
-	return 0;
 }
 
+/*
+ * pgstat_before_server_shutdown() needs to be called by exactly one process
+ * during regular server shutdowns. Otherwise all stats will be lost.
+ *
+ * We currently only write out stats for proc_exit(0). We might want to change
+ * that at some point... But right now pgstat_discard_stats() would be called
+ * during the start after a disorderly shutdown, anyway.
+ */
 void
-allow_immediate_pgstat_restart(void)
+pgstat_before_server_shutdown(int code, Datum arg)
 {
-	last_pgstat_start_time = 0;
+	Assert(pgStatShmem != 0);
+	Assert(!pgStatShmem->is_shutdown);
+
+	/*
+	 * Stats should only be reported after pgstat_initialize() and before
+	 * pgstat_shutdown(). This is a convenient point to catch most violations
+	 * of this rule.
+	 */
+	Assert(pgstat_is_initialized && !pgstat_is_shutdown);
+
+	/* flush out our own pending changes before writing out */
+	pgstat_report_stat(true);
+
+	/*
+	 * Only write out file during normal shutdown. Don't even signal that
+	 * we've shutdown during irregular shutdowns, because the shutdown
+	 * sequence isn't coordinated to ensure this backend shuts down last.
+	 */
+	if (code == 0)
+	{
+		pgStatShmem->is_shutdown = true;
+		pgstat_write_statsfile();
+	}
 }
 
 
 /* ------------------------------------------------------------
  * Backend initialization / shutdown functions
- * ------------------------------------------------------------
+ *------------------------------------------------------------
  */
 
 /*
  * Shut down a single backend's statistics reporting at process exit.
  *
- * Flush any remaining statistics counts out to the collector.
- * Without this, operations triggered during backend exit (such as
- * temp table deletions) won't be counted.
+ * Flush any remaining statistics counts out to shared stats.  Without this,
+ * operations triggered during backend exit (such as temp table deletions)
+ * won't be counted.
  */
 static void
 pgstat_shutdown_hook(int code, Datum arg)
 {
 	Assert(!pgstat_is_shutdown);
+	Assert(IsUnderPostmaster || !IsPostmasterEnvironment);
 
 	/*
 	 * If we got as far as discovering our own database ID, we can report what
-	 * we did to the collector.  Otherwise, we'd be sending an invalid
+	 * we did to the shared stats.  Otherwise, we'd be dealing with an invalid
 	 * database ID, so forget it.  (This means that accesses to pg_database
 	 * during failed backend starts might never get counted.)
 	 */
 	if (OidIsValid(MyDatabaseId))
-		pgstat_report_stat(true);
+		pgstat_report_disconnect(MyDatabaseId);
+
+	pgstat_report_stat(true);
+
+	Assert(pgStatDSA);
+
+	/* We shouldn't leave a reference to shared stats. */
+	pgstat_shared_refs_release_all();
+
+	dshash_detach(pgStatSharedHash);
+	pgStatSharedHash = NULL;
+
+	/* there shouldn't be any pending changes left */
+	Assert(dlist_is_empty(&pgStatPending));
+	dlist_init(&pgStatPending);
+
+	dsa_detach(pgStatDSA);
+	pgStatDSA = NULL;
 
 #ifdef USE_ASSERT_CHECKING
 	pgstat_is_shutdown = true;
@@ -733,11 +639,25 @@ pgstat_shutdown_hook(int code, Datum arg)
 void
 pgstat_initialize(void)
 {
+	MemoryContext oldcontext;
+
 	Assert(!pgstat_is_initialized);
+	Assert(pgStatDSA == NULL);
+
+	/* stats shared memory persists for the backend lifetime */
+	oldcontext = MemoryContextSwitchTo(TopMemoryContext);
+
+	pgStatDSA = dsa_attach_in_place(pgStatShmem->raw_dsa_area, NULL);
+	dsa_pin_mapping(pgStatDSA);
+
+	pgStatSharedHash = dshash_attach(pgStatDSA, &dsh_params,
+									 pgStatShmem->hash_handle, 0);
+
+	MemoryContextSwitchTo(oldcontext);
 
 	pgstat_wal_initialize();
 
-	/* Set up a process-exit hook to clean up */
+	/* need to be called before dsm shutdown */
 	before_shmem_exit(pgstat_shutdown_hook, 0);
 
 #ifdef USE_ASSERT_CHECKING
@@ -751,6 +671,57 @@ pgstat_initialize(void)
  *------------------------------------------------------------
  */
 
+/*
+ * AFIXME: Should all the stats drop code be moved into pgstat_drop.c?
+ */
+static void
+AtEOXact_PgStat_DroppedStats(PgStat_SubXactStatus *xact_state, bool isCommit)
+{
+	dlist_mutable_iter iter;
+
+	if (xact_state->pending_drops_count == 0)
+	{
+		Assert(dlist_is_empty(&xact_state->pending_drops));
+		return;
+	}
+
+	dlist_foreach_modify(iter, &xact_state->pending_drops)
+	{
+		PgStat_PendingDroppedStatsItem *pending =
+			dlist_container(PgStat_PendingDroppedStatsItem, node, iter.cur);
+
+		elog(DEBUG2, "on %s checking if to to dop %s stats %u/%u/%u",
+			 isCommit ? "commit" : "abort",
+			 pending->is_create ? "create" : "drop",
+			 pending->item.kind,
+			 pending->item.dboid,
+			 pending->item.objoid);
+
+		if (isCommit && !pending->is_create)
+		{
+			/*
+			 * Transaction that dropped an object committed. Drop the stats
+			 * too.
+			 */
+			pgstat_perform_drop(&pending->item);
+		}
+		else if (!isCommit && pending->is_create)
+		{
+			/*
+			 * Transaction that created an object aborted. Drop the stats
+			 * associated with the object.
+			 */
+			pgstat_perform_drop(&pending->item);
+		}
+
+		dlist_delete(&pending->node);
+		xact_state->pending_drops_count--;
+		pfree(pending);
+	}
+
+	pg_atomic_fetch_add_u64(&pgStatShmem->gc_count, 1);
+}
+
 /* ----------
  * AtEOXact_PgStat
  *
@@ -772,6 +743,7 @@ AtEOXact_PgStat(bool isCommit, bool parallel)
 		Assert(xact_state->prev == NULL);
 
 		AtEOXact_PgStat_Relations(xact_state, isCommit);
+		AtEOXact_PgStat_DroppedStats(xact_state, isCommit);
 	}
 	pgStatXactStack = NULL;
 
@@ -779,6 +751,54 @@ AtEOXact_PgStat(bool isCommit, bool parallel)
 	pgstat_clear_snapshot();
 }
 
+static void
+AtEOSubXact_PgStat_DroppedStats(PgStat_SubXactStatus *xact_state,
+								bool isCommit, int nestDepth)
+{
+	PgStat_SubXactStatus *parent_xact_state;
+	dlist_mutable_iter iter;
+
+	if (xact_state->pending_drops_count == 0)
+		return;
+
+	parent_xact_state = pgstat_xact_stack_level_get(nestDepth - 1);
+
+	dlist_foreach_modify(iter, &xact_state->pending_drops)
+	{
+		PgStat_PendingDroppedStatsItem *pending =
+			dlist_container(PgStat_PendingDroppedStatsItem, node, iter.cur);
+
+		dlist_delete(&pending->node);
+		xact_state->pending_drops_count--;
+
+		if (!isCommit && pending->is_create)
+		{
+			/*
+			 * Subtransaction creating a new stats object aborted. Drop the
+			 * stats object.
+			 */
+			pgstat_perform_drop(&pending->item);
+			pfree(pending);
+		}
+		else if (isCommit)
+		{
+			/*
+			 * Subtransaction dropping a stats object committed. Can't yet
+			 * remove the stats object, the surrounding transaction might still
+			 * abort. Pass it on to the parent.
+			 */
+			dlist_push_tail(&parent_xact_state->pending_drops, &pending->node);
+			parent_xact_state->pending_drops_count++;
+		}
+		else
+		{
+			pfree(pending);
+		}
+	}
+
+	Assert(xact_state->pending_drops_count == 0);
+}
+
 /* ----------
  * AtEOSubXact_PgStat
  *
@@ -799,6 +819,7 @@ AtEOSubXact_PgStat(bool isCommit, int nestDepth)
 		pgStatXactStack = xact_state->prev;
 
 		AtEOSubXact_PgStat_Relations(xact_state, isCommit, nestDepth);
+		AtEOSubXact_PgStat_DroppedStats(xact_state, isCommit, nestDepth);
 
 		pfree(xact_state);
 	}
@@ -867,15 +888,18 @@ pgstat_clear_snapshot(void)
 {
 	pgstat_assert_is_up();
 
-	/* Release memory, if any was allocated */
-	if (pgStatLocalContext)
-		MemoryContextDelete(pgStatLocalContext);
+	memset(&stats_snapshot.global_valid, 0, sizeof(stats_snapshot.global_valid));
+	stats_snapshot.stats = NULL;
+	stats_snapshot.mode = STATS_FETCH_CONSISTENCY_NONE;
 
-	/* Reset variables */
-	pgStatLocalContext = NULL;
-	pgStatDBHash = NULL;
-	replSlotStatHash = NULL;
-	subscriptionStatHash = NULL;
+	/* Release memory, if any was allocated */
+	if (pgStatSnapshotContext)
+	{
+		MemoryContextDelete(pgStatSnapshotContext);
+
+		/* Reset variables */
+		pgStatSnapshotContext = NULL;
+	}
 
 	/*
 	 * Historically the backend_status.c facilities lived in this file, and
@@ -885,6 +909,60 @@ pgstat_clear_snapshot(void)
 	pgstat_clear_backend_activity_snapshot();
 }
 
+int
+pgstat_pending_stats_drops(bool isCommit, PgStat_DroppedStatsItem **items)
+{
+	PgStat_SubXactStatus *xact_state = pgStatXactStack;
+	int nitems = 0;
+	dlist_iter iter;
+
+	if (xact_state == NULL)
+		return 0;
+
+	/*
+	 * We expect to be called for subtransaction abort (which logs a WAL
+	 * record), but not for subtransaction commit (which doesn't).
+	 */
+	Assert(!isCommit || xact_state->nest_level == 1);
+	Assert(!isCommit || xact_state->prev == NULL);
+
+	*items = palloc(xact_state->pending_drops_count
+					* sizeof(PgStat_PendingDroppedStatsItem));
+
+	dlist_foreach(iter, &xact_state->pending_drops)
+	{
+		PgStat_PendingDroppedStatsItem *pending =
+			dlist_container(PgStat_PendingDroppedStatsItem, node, iter.cur);
+
+		if (isCommit && pending->is_create)
+			continue;
+		if (!isCommit && !pending->is_create)
+			continue;
+
+		Assert(nitems < xact_state->pending_drops_count);
+		(*items)[nitems++] = pending->item;
+	}
+
+	return nitems;
+}
+
+/*
+ * Execute scheduled drops post-commit. Called from xact_redo_commit() during
+ * recovery, and from FinishPreparedTransaction() during normal commit
+ * processing.
+ */
+void
+pgstat_perform_drops(int ndrops, struct PgStat_DroppedStatsItem *items, bool is_redo)
+{
+	if (ndrops == 0)
+		return;
+
+	for (int i = 0; i < ndrops; i++)
+		pgstat_perform_drop(&items[i]);
+
+	pg_atomic_fetch_add_u64(&pgStatShmem->gc_count, 1);
+}
+
 /*
  * Ensure (sub)transaction stack entry for the given nest_level exists, adding
  * it if needed.
@@ -900,6 +978,8 @@ pgstat_xact_stack_level_get(int nest_level)
 		xact_state = (PgStat_SubXactStatus *)
 			MemoryContextAlloc(TopTransactionContext,
 							   sizeof(PgStat_SubXactStatus));
+		dlist_init(&xact_state->pending_drops);
+		xact_state->pending_drops_count = 0;
 		xact_state->nest_level = nest_level;
 		xact_state->prev = pgStatXactStack;
 		xact_state->first = NULL;
@@ -911,350 +991,168 @@ pgstat_xact_stack_level_get(int nest_level)
 
 /* ------------------------------------------------------------
  * Public functions used by backends follow
- *------------------------------------------------------------
+ * ------------------------------------------------------------
  */
 
-/* ----------
- * pgstat_report_stat() -
+/*
+ * Must be called by processes that performs DML: tcop/postgres.c, logical
+ * receiver processes, SPI worker, etc. to apply the so far collected
+ * per-table and function usage statistics to the shared statistics hashes.
  *
- *	Must be called by processes that performs DML: tcop/postgres.c, logical
- *	receiver processes, SPI worker, etc. to send the so far collected
- *	per-table and function usage statistics to the collector.  Note that this
- *	is called only when not within a transaction, so it is fair to use
- *	transaction stop time as an approximation of current time.
+ * Updates are applied not more frequent than the interval of
+ * PGSTAT_MIN_INTERVAL milliseconds. They are also postponed on lock
+ * failure if force is false and there's no pending updates longer than
+ * PGSTAT_MAX_INTERVAL milliseconds. Postponed updates are retried in
+ * succeeding calls of this function.
  *
- *	"disconnect" is "true" only for the last call before the backend
- *	exits.  This makes sure that no data is lost and that interrupted
- *	sessions are reported correctly.
- * ----------
+ * Returns the time until the next timing when updates are applied in
+ * milliseconds if there are no updates held for more than
+ * PGSTAT_MIN_INTERVAL milliseconds.
+ *
+ * Note that this is called only when not within a transaction, so it is fair
+ * to use transaction stop time as an approximation of current time.
  */
-void
-pgstat_report_stat(bool disconnect)
+long
+pgstat_report_stat(bool force)
 {
-	static TimestampTz last_report = 0;
-
+	static TimestampTz next_flush = 0;
+	static TimestampTz pending_since = 0;
+	static long retry_interval = 0;
+	bool		partial_flush;
 	TimestampTz now;
+	bool		nowait;
 
 	pgstat_assert_is_up();
 
-	/*
-	 * Don't expend a clock check if nothing to do.
-	 *
-	 * To determine whether any WAL activity has occurred since last time, not
-	 * only the number of generated WAL records but also the numbers of WAL
-	 * writes and syncs need to be checked. Because even transaction that
-	 * generates no WAL records can write or sync WAL data when flushing the
-	 * data pages.
-	 */
-	if (!have_relation_stats &&
-		pgStatXactCommit == 0 && pgStatXactRollback == 0 &&
-		pgWalUsage.wal_records == prevWalUsage.wal_records &&
-		WalStats.m_wal_write == 0 && WalStats.m_wal_sync == 0 &&
-		!have_function_stats && !disconnect)
-		return;
+	/* "absorb" the forced flush even if there's nothing to flush */
+	if (pgStatForceNextFlush)
+	{
+		force = true;
+		pgStatForceNextFlush = false;
+	}
+
+	/* Don't expend a clock check if nothing to do */
+	if (dlist_is_empty(&pgStatPending)
+		&& !have_slrustats
+		&& !walstats_pending())
+	{
+		return 0;
+	}
 
 	/*
-	 * Don't send a message unless it's been at least PGSTAT_STAT_INTERVAL
-	 * msec since we last sent one, or the backend is about to exit.
+	 * There should never be stats to report once stats are shut down. Can't
+	 * assert that before the checks above, as there is an unconditional
+	 * pgstat_report_stat() call in pgstat_shutdown_hook() - which at least
+	 * the process that ran pgstat_before_server_shutdown() will still call.
 	 */
+	Assert(!pgStatShmem->is_shutdown);
+
 	now = GetCurrentTransactionStopTimestamp();
-	if (!disconnect &&
-		!TimestampDifferenceExceeds(last_report, now, PGSTAT_STAT_INTERVAL))
-		return;
 
-	last_report = now;
+	if (!force)
+	{
+		/*
+		 * Don't flush stats too frequently.  Return the time to the next
+		 * flush.
+		 */
+		if (now < next_flush)
+		{
+			/* Record the epoch time if retrying. */
+			if (pending_since == 0)
+				pending_since = now;
 
-	if (disconnect)
-		pgstat_report_disconnect(MyDatabaseId);
+			return (next_flush - now) / 1000;
+		}
 
-	/* First, send relation statistics */
-	pgstat_send_tabstats(now, disconnect);
+		/* But, don't keep pending updates longer than PGSTAT_MAX_INTERVAL. */
 
-	/* Now, send function statistics */
-	pgstat_send_funcstats();
+		if (pending_since > 0 &&
+			TimestampDifferenceExceeds(pending_since, now, PGSTAT_MAX_INTERVAL))
+			force = true;
+	}
 
-	/* Send WAL statistics */
-	pgstat_send_wal(true);
+	pgstat_update_dbstats(now);
 
-	/* Finally send SLRU statistics */
-	pgstat_send_slru();
+	/* don't wait for lock acquisition when !force */
+	nowait = !force;
+
+	partial_flush = false;
+
+	/* flush database / relation / function / ... stats */
+	partial_flush |= pgstat_pending_flush_stats(nowait);
+
+	/* flush wal stats */
+	partial_flush |= pgstat_flush_wal(nowait);
+
+	/* flush SLRU stats */
+	partial_flush |= pgstat_flush_slru(nowait);
+
+	/*
+	 * Some of the pending stats may have not been flushed due to lock
+	 * contention.  If we have such pending stats here, let the caller know
+	 * the retry interval.
+	 */
+	if (partial_flush)
+	{
+		/* Retain the epoch time */
+		if (pending_since == 0)
+			pending_since = now;
+
+		/* The interval is doubled at every retry. */
+		if (retry_interval == 0)
+			retry_interval = PGSTAT_RETRY_MIN_INTERVAL * 1000;
+		else
+			retry_interval = retry_interval * 2;
+
+		/*
+		 * Determine the next retry interval so as not to get shorter than the
+		 * previous interval.
+		 */
+		if (!TimestampDifferenceExceeds(pending_since,
+										now + 2 * retry_interval,
+										PGSTAT_MAX_INTERVAL))
+			next_flush = now + retry_interval;
+		else
+		{
+			next_flush = pending_since + PGSTAT_MAX_INTERVAL * 1000;
+			retry_interval = next_flush - now;
+		}
+
+		return retry_interval / 1000;
+	}
+
+	/* Set the next time to update stats */
+	next_flush = now + PGSTAT_MIN_INTERVAL * 1000;
+	retry_interval = 0;
+	pending_since = 0;
+
+	return 0;
 }
 
-/* ----------
- * pgstat_vacuum_stat() -
- *
- *	Will tell the collector about objects he can get rid of.
- * ----------
+/*
+ * Force locally pending stats to be flushed during the next
+ * pgstat_report_stat() call. This is useful for writing tests.
+ */
+void
+pgstat_force_next_flush(void)
+{
+	pgStatForceNextFlush = true;
+}
+
+/*
+ * AFIXME: Function stats can escape function drops in some rare
+ * situations. It's probably not worth closing that window, but if we wanted,
+ * we'd do it here.
  */
 void
 pgstat_vacuum_stat(void)
 {
-	HTAB	   *htab;
-	PgStat_MsgTabpurge msg;
-	PgStat_MsgFuncpurge f_msg;
-	HASH_SEQ_STATUS hstat;
-	PgStat_StatDBEntry *dbentry;
-	PgStat_StatTabEntry *tabentry;
-	PgStat_StatFuncEntry *funcentry;
-	int			len;
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	/*
-	 * If not done for this transaction, read the statistics collector stats
-	 * file into some hash tables.
-	 */
-	backend_read_statsfile();
-
-	/*
-	 * Read pg_database and make a list of OIDs of all existing databases
-	 */
-	htab = pgstat_collect_oids(DatabaseRelationId, Anum_pg_database_oid);
-
-	/*
-	 * Search the database hash table for dead databases and tell the
-	 * collector to drop them.
-	 */
-	hash_seq_init(&hstat, pgStatDBHash);
-	while ((dbentry = (PgStat_StatDBEntry *) hash_seq_search(&hstat)) != NULL)
-	{
-		Oid			dbid = dbentry->databaseid;
-
-		CHECK_FOR_INTERRUPTS();
-
-		/* the DB entry for shared tables (with InvalidOid) is never dropped */
-		if (OidIsValid(dbid) &&
-			hash_search(htab, (void *) &dbid, HASH_FIND, NULL) == NULL)
-			pgstat_drop_database(dbid);
-	}
-
-	/* Clean up */
-	hash_destroy(htab);
-
-	/*
-	 * Search for all the dead replication slots in stats hashtable and tell
-	 * the stats collector to drop them.
-	 */
-	if (replSlotStatHash)
-	{
-		PgStat_StatReplSlotEntry *slotentry;
-
-		hash_seq_init(&hstat, replSlotStatHash);
-		while ((slotentry = (PgStat_StatReplSlotEntry *) hash_seq_search(&hstat)) != NULL)
-		{
-			CHECK_FOR_INTERRUPTS();
-
-			if (SearchNamedReplicationSlot(NameStr(slotentry->slotname), true) == NULL)
-				pgstat_report_replslot_drop(NameStr(slotentry->slotname));
-		}
-	}
-
-	/*
-	 * Repeat the above steps for subscriptions, if subscription stats are
-	 * being collected.
-	 */
-	if (subscriptionStatHash)
-	{
-		PgStat_StatSubEntry *subentry;
-
-		/*
-		 * Read pg_subscription and make a list of OIDs of all existing
-		 * subscriptions.
-		 */
-		htab = pgstat_collect_oids(SubscriptionRelationId, Anum_pg_subscription_oid);
-
-		hash_seq_init(&hstat, subscriptionStatHash);
-		while ((subentry = (PgStat_StatSubEntry *) hash_seq_search(&hstat)) != NULL)
-		{
-			CHECK_FOR_INTERRUPTS();
-
-			if (hash_search(htab, (void *) &(subentry->subid), HASH_FIND, NULL) == NULL)
-				pgstat_report_subscription_drop(subentry->subid);
-		}
-
-		hash_destroy(htab);
-	}
-
-	/*
-	 * Lookup our own database entry; if not found, nothing more to do.
-	 */
-	dbentry = (PgStat_StatDBEntry *) hash_search(pgStatDBHash,
-												 (void *) &MyDatabaseId,
-												 HASH_FIND, NULL);
-	if (dbentry == NULL || dbentry->tables == NULL)
-		return;
-
-	/*
-	 * Similarly to above, make a list of all known relations in this DB.
-	 */
-	htab = pgstat_collect_oids(RelationRelationId, Anum_pg_class_oid);
-
-	/*
-	 * Initialize our messages table counter to zero
-	 */
-	msg.m_nentries = 0;
-
-	/*
-	 * Check for all tables listed in stats hashtable if they still exist.
-	 */
-	hash_seq_init(&hstat, dbentry->tables);
-	while ((tabentry = (PgStat_StatTabEntry *) hash_seq_search(&hstat)) != NULL)
-	{
-		Oid			tabid = tabentry->tableid;
-
-		CHECK_FOR_INTERRUPTS();
-
-		if (hash_search(htab, (void *) &tabid, HASH_FIND, NULL) != NULL)
-			continue;
-
-		/*
-		 * Not there, so add this table's Oid to the message
-		 */
-		msg.m_tableid[msg.m_nentries++] = tabid;
-
-		/*
-		 * If the message is full, send it out and reinitialize to empty
-		 */
-		if (msg.m_nentries >= PGSTAT_NUM_TABPURGE)
-		{
-			len = offsetof(PgStat_MsgTabpurge, m_tableid[0])
-				+ msg.m_nentries * sizeof(Oid);
-
-			pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_TABPURGE);
-			msg.m_databaseid = MyDatabaseId;
-			pgstat_send(&msg, len);
-
-			msg.m_nentries = 0;
-		}
-	}
-
-	/*
-	 * Send the rest
-	 */
-	if (msg.m_nentries > 0)
-	{
-		len = offsetof(PgStat_MsgTabpurge, m_tableid[0])
-			+ msg.m_nentries * sizeof(Oid);
-
-		pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_TABPURGE);
-		msg.m_databaseid = MyDatabaseId;
-		pgstat_send(&msg, len);
-	}
-
-	/* Clean up */
-	hash_destroy(htab);
-
-	/*
-	 * Now repeat the above steps for functions.  However, we needn't bother
-	 * in the common case where no function stats are being collected.
-	 */
-	if (dbentry->functions != NULL &&
-		hash_get_num_entries(dbentry->functions) > 0)
-	{
-		htab = pgstat_collect_oids(ProcedureRelationId, Anum_pg_proc_oid);
-
-		pgstat_setheader(&f_msg.m_hdr, PGSTAT_MTYPE_FUNCPURGE);
-		f_msg.m_databaseid = MyDatabaseId;
-		f_msg.m_nentries = 0;
-
-		hash_seq_init(&hstat, dbentry->functions);
-		while ((funcentry = (PgStat_StatFuncEntry *) hash_seq_search(&hstat)) != NULL)
-		{
-			Oid			funcid = funcentry->functionid;
-
-			CHECK_FOR_INTERRUPTS();
-
-			if (hash_search(htab, (void *) &funcid, HASH_FIND, NULL) != NULL)
-				continue;
-
-			/*
-			 * Not there, so add this function's Oid to the message
-			 */
-			f_msg.m_functionid[f_msg.m_nentries++] = funcid;
-
-			/*
-			 * If the message is full, send it out and reinitialize to empty
-			 */
-			if (f_msg.m_nentries >= PGSTAT_NUM_FUNCPURGE)
-			{
-				len = offsetof(PgStat_MsgFuncpurge, m_functionid[0])
-					+ f_msg.m_nentries * sizeof(Oid);
-
-				pgstat_send(&f_msg, len);
-
-				f_msg.m_nentries = 0;
-			}
-		}
-
-		/*
-		 * Send the rest
-		 */
-		if (f_msg.m_nentries > 0)
-		{
-			len = offsetof(PgStat_MsgFuncpurge, m_functionid[0])
-				+ f_msg.m_nentries * sizeof(Oid);
-
-			pgstat_send(&f_msg, len);
-		}
-
-		hash_destroy(htab);
-	}
-}
-
-/* ----------
- * pgstat_collect_oids() -
- *
- *	Collect the OIDs of all objects listed in the specified system catalog
- *	into a temporary hash table.  Caller should hash_destroy the result
- *	when done with it.  (However, we make the table in CurrentMemoryContext
- *	so that it will be freed properly in event of an error.)
- * ----------
- */
-static HTAB *
-pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid)
-{
-	HTAB	   *htab;
-	HASHCTL		hash_ctl;
-	Relation	rel;
-	TableScanDesc scan;
-	HeapTuple	tup;
-	Snapshot	snapshot;
-
-	hash_ctl.keysize = sizeof(Oid);
-	hash_ctl.entrysize = sizeof(Oid);
-	hash_ctl.hcxt = CurrentMemoryContext;
-	htab = hash_create("Temporary table of OIDs",
-					   PGSTAT_TAB_HASH_SIZE,
-					   &hash_ctl,
-					   HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
-
-	rel = table_open(catalogid, AccessShareLock);
-	snapshot = RegisterSnapshot(GetLatestSnapshot());
-	scan = table_beginscan(rel, snapshot, 0, NULL);
-	while ((tup = heap_getnext(scan, ForwardScanDirection)) != NULL)
-	{
-		Oid			thisoid;
-		bool		isnull;
-
-		thisoid = heap_getattr(tup, anum_oid, RelationGetDescr(rel), &isnull);
-		Assert(!isnull);
-
-		CHECK_FOR_INTERRUPTS();
-
-		(void) hash_search(htab, (void *) &thisoid, HASH_ENTER, NULL);
-	}
-	table_endscan(scan);
-	UnregisterSnapshot(snapshot);
-	table_close(rel, AccessShareLock);
-
-	return htab;
 }
 
 /* ----------
  * pgstat_reset_counters() -
  *
- *	Tell the statistics collector to reset counters for our database.
+ *	Reset counters for our database.
  *
  *	Permission checking for this function is managed through the normal
  *	GRANT system.
@@ -1263,20 +1161,46 @@ pgstat_collect_oids(Oid catalogid, AttrNumber anum_oid)
 void
 pgstat_reset_counters(void)
 {
-	PgStat_MsgResetcounter msg;
+	dshash_seq_status hstat;
+	PgStatShmHashEntry *p;
 
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
+	/* dshash entry is not modified, take shared lock */
+	dshash_seq_init(&hstat, pgStatSharedHash, false);
+	while ((p = dshash_seq_next(&hstat)) != NULL)
+	{
+		PgStatShm_StatEntryHeader *header;
 
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETCOUNTER);
-	msg.m_databaseid = MyDatabaseId;
-	pgstat_send(&msg, sizeof(msg));
+		if (p->key.dboid != MyDatabaseId)
+			continue;
+
+		header = dsa_get_address(pgStatDSA, p->body);
+
+		if (p->dropped)
+			continue;
+
+		LWLockAcquire(&p->lock, LW_EXCLUSIVE);
+		memset(shared_stat_entry_data(p->key.kind, header), 0,
+			   shared_stat_entry_len(p->key.kind));
+
+		/*
+		 * FIXME: call into a dedicated function instead. Including creating
+		 * the stats entry if it doesn't eyt exist.
+		 */
+		if (p->key.kind == PGSTAT_KIND_DB)
+		{
+			PgStatShm_StatDBEntry *dbstat = (PgStatShm_StatDBEntry *) header;
+
+			dbstat->stats.stat_reset_timestamp = GetCurrentTimestamp();
+		}
+		LWLockRelease(&p->lock);
+	}
+	dshash_seq_term(&hstat);
 }
 
 /* ----------
  * pgstat_reset_single_counter() -
  *
- *	Tell the statistics collector to reset a single counter.
+ *	Reset a single counter.
  *
  *	Permission checking for this function is managed through the normal
  *	GRANT system.
@@ -1285,315 +1209,134 @@ pgstat_reset_counters(void)
 void
 pgstat_reset_single_counter(Oid objoid, PgStat_Single_Reset_Type type)
 {
-	PgStat_MsgResetsinglecounter msg;
+	PgStatSharedRef *db_ref;
+	PgStatSharedRef *counter_ref;
 
-	if (pgStatSock == PGINVALID_SOCKET)
+	PgStatShm_StatEntryHeader *header;
+	PgStatShm_StatDBEntry *dbentry;
+	PgStatKind kind;
+	TimestampTz ts = GetCurrentTimestamp();
+
+	/* FIXME: move into a dedicated function */
+	db_ref = pgstat_shared_ref_get(PGSTAT_KIND_DB, MyDatabaseId, InvalidOid,
+								   false);
+	if (db_ref == NULL)
 		return;
 
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETSINGLECOUNTER);
-	msg.m_databaseid = MyDatabaseId;
-	msg.m_resettype = type;
-	msg.m_objectid = objoid;
+	dbentry = (PgStatShm_StatDBEntry *) db_ref->shared_stats;
 
-	pgstat_send(&msg, sizeof(msg));
+	/* XXX: isn't this pretty grotty behaviour? */
+	/* Set the reset timestamp for the whole database */
+	pgstat_shared_stat_lock(db_ref, false);
+	dbentry->stats.stat_reset_timestamp = ts;
+	pgstat_shared_stat_unlock(db_ref);
+
+	/* Remove object if it exists, ignore if not */
+	switch (type)
+	{
+		case RESET_TABLE:
+			kind = PGSTAT_KIND_TABLE;
+			break;
+		case RESET_FUNCTION:
+			kind = PGSTAT_KIND_FUNCTION;
+			break;
+		default:
+			return;
+	}
+
+	counter_ref = pgstat_shared_ref_get(kind, MyDatabaseId, objoid, false);
+	if (!counter_ref || counter_ref->shared_entry->dropped)
+		return;
+
+	pgstat_shared_stat_lock(counter_ref, false);
+
+	header = counter_ref->shared_stats;
+	memset(shared_stat_entry_data(kind, header), 0,
+		   shared_stat_entry_len(kind));
+
+	pgstat_shared_stat_unlock(counter_ref);
 }
 
 /* ----------
- * pgstat_ping() -
+ * pgstat_reset_subscription_counter() -
  *
- *	Send some junk data to the collector to increase traffic.
+ *	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.
+ *
+ * AFIXME: move to pgstat_subscription.c - right now it uses
+ * shared_stat_entry_len(), which isn't exported.
+ *
+ * AFIXME: this doesn't yet support the behaviour of subid = InvalidOid =>
+ * drop all subscription stats.
  * ----------
  */
 void
-pgstat_ping(void)
+pgstat_reset_subscription_counter(Oid subid)
 {
-	PgStat_MsgDummy msg;
+	PgStatSharedRef *counter_ref;
+	PgStatShm_StatEntryHeader *header;
 
-	if (pgStatSock == PGINVALID_SOCKET)
+	counter_ref = pgstat_shared_ref_get(PGSTAT_KIND_SUBSCRIPTION, subid, InvalidOid, false);
+	if (!counter_ref || counter_ref->shared_entry->dropped)
 		return;
 
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_DUMMY);
-	pgstat_send(&msg, sizeof(msg));
-}
+	pgstat_shared_stat_lock(counter_ref, false);
 
-/* ----------
- * pgstat_send_inquiry() -
- *
- *	Notify collector that we need fresh data.
- * ----------
- */
-static void
-pgstat_send_inquiry(TimestampTz clock_time, TimestampTz cutoff_time, Oid databaseid)
-{
-	PgStat_MsgInquiry msg;
+	header = counter_ref->shared_stats;
+	memset(shared_stat_entry_data(PGSTAT_KIND_SUBSCRIPTION, header), 0,
+		   shared_stat_entry_len(PGSTAT_KIND_SUBSCRIPTION));
 
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_INQUIRY);
-	msg.clock_time = clock_time;
-	msg.cutoff_time = cutoff_time;
-	msg.databaseid = databaseid;
-	pgstat_send(&msg, sizeof(msg));
-}
-
-/* ----------
- * pgstat_fetch_stat_dbentry() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	the collected statistics for one database or NULL. NULL doesn't mean
- *	that the database doesn't exist, it is just not yet known by the
- *	collector, so the caller is better off to report ZERO instead.
- * ----------
- */
-PgStat_StatDBEntry *
-pgstat_fetch_stat_dbentry(Oid dbid)
-{
-	/*
-	 * If not done for this transaction, read the statistics collector stats
-	 * file into some hash tables.
-	 */
-	backend_read_statsfile();
-
-	/*
-	 * Lookup the requested database; return NULL if not found
-	 */
-	return (PgStat_StatDBEntry *) hash_search(pgStatDBHash,
-											  (void *) &dbid,
-											  HASH_FIND, NULL);
+	pgstat_shared_stat_unlock(counter_ref);
 }
 
 /*
- * ---------
- * pgstat_fetch_global() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	a pointer to the global statistics struct.
- * ---------
+ * If a stats snapshot has been taken, return the timestamp at which that was
+ * done, and set *have_snapshot to true. Otherwise *have_snapshot is set to
+ * false.
  */
-PgStat_GlobalStats *
-pgstat_fetch_global(void)
+TimestampTz
+pgstat_get_stat_snapshot_timestamp(bool *have_snapshot)
 {
-	backend_read_statsfile();
-
-	return &globalStats;
-}
-
-/* ----------
- * pgstat_fetch_stat_tabentry() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	the collected statistics for one table or NULL. NULL doesn't mean
- *	that the table doesn't exist, it is just not yet known by the
- *	collector, so the caller is better off to report ZERO instead.
- * ----------
- */
-PgStat_StatTabEntry *
-pgstat_fetch_stat_tabentry(Oid relid)
-{
-	Oid			dbid;
-	PgStat_StatDBEntry *dbentry;
-	PgStat_StatTabEntry *tabentry;
-
-	/*
-	 * If not done for this transaction, read the statistics collector stats
-	 * file into some hash tables.
-	 */
-	backend_read_statsfile();
-
-	/*
-	 * Lookup our database, then look in its table hash table.
-	 */
-	dbid = MyDatabaseId;
-	dbentry = (PgStat_StatDBEntry *) hash_search(pgStatDBHash,
-												 (void *) &dbid,
-												 HASH_FIND, NULL);
-	if (dbentry != NULL && dbentry->tables != NULL)
+	if (stats_snapshot.mode == STATS_FETCH_CONSISTENCY_SNAPSHOT)
 	{
-		tabentry = (PgStat_StatTabEntry *) hash_search(dbentry->tables,
-													   (void *) &relid,
-													   HASH_FIND, NULL);
-		if (tabentry)
-			return tabentry;
+		*have_snapshot = true;
+		return stats_snapshot.snapshot_timestamp;
 	}
 
-	/*
-	 * If we didn't find it, maybe it's a shared table.
-	 */
-	dbid = InvalidOid;
-	dbentry = (PgStat_StatDBEntry *) hash_search(pgStatDBHash,
-												 (void *) &dbid,
-												 HASH_FIND, NULL);
-	if (dbentry != NULL && dbentry->tables != NULL)
-	{
-		tabentry = (PgStat_StatTabEntry *) hash_search(dbentry->tables,
-													   (void *) &relid,
-													   HASH_FIND, NULL);
-		if (tabentry)
-			return tabentry;
-	}
+	*have_snapshot = false;
 
-	return NULL;
-}
-
-
-/* ----------
- * pgstat_fetch_stat_funcentry() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	the collected statistics for one function or NULL.
- * ----------
- */
-PgStat_StatFuncEntry *
-pgstat_fetch_stat_funcentry(Oid func_id)
-{
-	PgStat_StatDBEntry *dbentry;
-	PgStat_StatFuncEntry *funcentry = NULL;
-
-	/* load the stats file if needed */
-	backend_read_statsfile();
-
-	/* Lookup our database, then find the requested function.  */
-	dbentry = pgstat_fetch_stat_dbentry(MyDatabaseId);
-	if (dbentry != NULL && dbentry->functions != NULL)
-	{
-		funcentry = (PgStat_StatFuncEntry *) hash_search(dbentry->functions,
-														 (void *) &func_id,
-														 HASH_FIND, NULL);
-	}
-
-	return funcentry;
-}
-
-/*
- * ---------
- * pgstat_fetch_stat_archiver() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	a pointer to the archiver statistics struct.
- * ---------
- */
-PgStat_ArchiverStats *
-pgstat_fetch_stat_archiver(void)
-{
-	backend_read_statsfile();
-
-	return &archiverStats;
-}
-
-/*
- * ---------
- * pgstat_fetch_stat_bgwriter() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	a pointer to the bgwriter statistics struct.
- * ---------
- */
-PgStat_BgWriterStats *
-pgstat_fetch_stat_bgwriter(void)
-{
-	backend_read_statsfile();
-
-	return &globalStats.bgwriter;
-}
-
-/*
- * ---------
- * pgstat_fetch_stat_checkpointer() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	a pointer to the checkpointer statistics struct.
- * ---------
- */
-PgStat_CheckpointerStats *
-pgstat_fetch_stat_checkpointer(void)
-{
-	backend_read_statsfile();
-
-	return &globalStats.checkpointer;
-}
-
-/*
- * ---------
- * pgstat_fetch_stat_wal() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	a pointer to the WAL statistics struct.
- * ---------
- */
-PgStat_WalStats *
-pgstat_fetch_stat_wal(void)
-{
-	backend_read_statsfile();
-
-	return &walStats;
-}
-
-/*
- * ---------
- * pgstat_fetch_slru() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	a pointer to the slru statistics struct.
- * ---------
- */
-PgStat_SLRUStats *
-pgstat_fetch_slru(void)
-{
-	backend_read_statsfile();
-
-	return slruStats;
-}
-
-/*
- * ---------
- * pgstat_fetch_replslot() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	a pointer to the replication slot statistics struct.
- * ---------
- */
-PgStat_StatReplSlotEntry *
-pgstat_fetch_replslot(NameData slotname)
-{
-	backend_read_statsfile();
-
-	return pgstat_get_replslot_entry(slotname, false);
-}
-
-/*
- * ---------
- * pgstat_fetch_stat_subscription() -
- *
- *	Support function for the SQL-callable pgstat* functions. Returns
- *	the collected statistics for one subscription or NULL.
- * ---------
- */
-PgStat_StatSubEntry *
-pgstat_fetch_stat_subscription(Oid subid)
-{
-	/* Load the stats file if needed */
-	backend_read_statsfile();
-
-	return pgstat_get_subscription_entry(subid, false);
+	return 0;
 }
 
 
 /* ------------------------------------------------------------
- * Helper / infrastructure functions
- * ------------------------------------------------------------
+ * Helper functions
+ *------------------------------------------------------------
  */
 
 /* ----------
  * pgstat_setup_memcxt() -
  *
- *	Create pgStatLocalContext, if not already done.
+ *	Create pgStatSnapshotContext if not already done.
  * ----------
  */
 static void
 pgstat_setup_memcxt(void)
 {
-	if (!pgStatLocalContext)
-		pgStatLocalContext = AllocSetContextCreate(TopMemoryContext,
-												   "Statistics snapshot",
-												   ALLOCSET_SMALL_SIZES);
+	if (unlikely(!pgStatSharedRefContext))
+		pgStatSharedRefContext =
+			AllocSetContextCreate(CacheMemoryContext,
+								  "Backend statistics data",
+								  ALLOCSET_SMALL_SIZES);
+
+	if (unlikely(!pgStatSnapshotContext))
+		pgStatSnapshotContext =
+			AllocSetContextCreate(TopMemoryContext,
+								  "Backend statistics snapshot",
+								  ALLOCSET_SMALL_SIZES);
 }
 
 /*
@@ -1609,521 +1352,25 @@ pgstat_assert_is_up(void)
 }
 #endif
 
-/* ----------
- * pgstat_setheader() -
- *
- *		Set common header fields in a statistics message
- * ----------
- */
-void
-pgstat_setheader(PgStat_MsgHdr *hdr, StatMsgType mtype)
-{
-	hdr->m_type = mtype;
-}
-
-
-/* ----------
- * pgstat_send() -
- *
- *		Send out one statistics message to the collector
- * ----------
- */
-void
-pgstat_send(void *msg, int len)
-{
-	int			rc;
-
-	pgstat_assert_is_up();
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
-
-	((PgStat_MsgHdr *) msg)->m_size = len;
-
-	/* We'll retry after EINTR, but ignore all other failures */
-	do
-	{
-		rc = send(pgStatSock, msg, len, 0);
-	} while (rc < 0 && errno == EINTR);
-
-#ifdef USE_ASSERT_CHECKING
-	/* In debug builds, log send failures ... */
-	if (rc < 0)
-		elog(LOG, "could not send to statistics collector: %m");
-#endif
-}
-
-/* ----------
- * PgstatCollectorMain() -
- *
- *	Start up the statistics collector process.  This is the body of the
- *	postmaster child process.
- *
- *	The argc/argv parameters are valid only in EXEC_BACKEND case.
- * ----------
- */
-NON_EXEC_STATIC void
-PgstatCollectorMain(int argc, char *argv[])
-{
-	int			len;
-	PgStat_Msg	msg;
-	int			wr;
-	WaitEvent	event;
-	WaitEventSet *wes;
-
-	/*
-	 * Ignore all signals usually bound to some action in the postmaster,
-	 * except SIGHUP and SIGQUIT.  Note we don't need a SIGUSR1 handler to
-	 * support latch operations, because we only use a local latch.
-	 */
-	pqsignal(SIGHUP, SignalHandlerForConfigReload);
-	pqsignal(SIGINT, SIG_IGN);
-	pqsignal(SIGTERM, SIG_IGN);
-	pqsignal(SIGQUIT, SignalHandlerForShutdownRequest);
-	pqsignal(SIGALRM, SIG_IGN);
-	pqsignal(SIGPIPE, SIG_IGN);
-	pqsignal(SIGUSR1, SIG_IGN);
-	pqsignal(SIGUSR2, SIG_IGN);
-	/* Reset some signals that are accepted by postmaster but not here */
-	pqsignal(SIGCHLD, SIG_DFL);
-	PG_SETMASK(&UnBlockSig);
-
-	MyBackendType = B_STATS_COLLECTOR;
-	init_ps_display(NULL);
-
-	/*
-	 * Read in existing stats files or initialize the stats to zero.
-	 */
-	pgStatRunningInCollector = true;
-	pgStatDBHash = pgstat_read_statsfiles(InvalidOid, true, true);
-
-	/* Prepare to wait for our latch or data in our socket. */
-	wes = CreateWaitEventSet(CurrentMemoryContext, 3);
-	AddWaitEventToSet(wes, WL_LATCH_SET, PGINVALID_SOCKET, MyLatch, NULL);
-	AddWaitEventToSet(wes, WL_POSTMASTER_DEATH, PGINVALID_SOCKET, NULL, NULL);
-	AddWaitEventToSet(wes, WL_SOCKET_READABLE, pgStatSock, NULL, NULL);
-
-	/*
-	 * Loop to process messages until we get SIGQUIT or detect ungraceful
-	 * death of our parent postmaster.
-	 *
-	 * For performance reasons, we don't want to do ResetLatch/WaitLatch after
-	 * every message; instead, do that only after a recv() fails to obtain a
-	 * message.  (This effectively means that if backends are sending us stuff
-	 * like mad, we won't notice postmaster death until things slack off a
-	 * bit; which seems fine.)	To do that, we have an inner loop that
-	 * iterates as long as recv() succeeds.  We do check ConfigReloadPending
-	 * inside the inner loop, which means that such interrupts will get
-	 * serviced but the latch won't get cleared until next time there is a
-	 * break in the action.
-	 */
-	for (;;)
-	{
-		/* Clear any already-pending wakeups */
-		ResetLatch(MyLatch);
-
-		/*
-		 * Quit if we get SIGQUIT from the postmaster.
-		 */
-		if (ShutdownRequestPending)
-			break;
-
-		/*
-		 * Inner loop iterates as long as we keep getting messages, or until
-		 * ShutdownRequestPending becomes set.
-		 */
-		while (!ShutdownRequestPending)
-		{
-			/*
-			 * Reload configuration if we got SIGHUP from the postmaster.
-			 */
-			if (ConfigReloadPending)
-			{
-				ConfigReloadPending = false;
-				ProcessConfigFile(PGC_SIGHUP);
-			}
-
-			/*
-			 * Write the stats file(s) if a new request has arrived that is
-			 * not satisfied by existing file(s).
-			 */
-			if (pgstat_write_statsfile_needed())
-				pgstat_write_statsfiles(false, false);
-
-			/*
-			 * Try to receive and process a message.  This will not block,
-			 * since the socket is set to non-blocking mode.
-			 *
-			 * XXX On Windows, we have to force pgwin32_recv to cooperate,
-			 * despite the previous use of pg_set_noblock() on the socket.
-			 * This is extremely broken and should be fixed someday.
-			 */
-#ifdef WIN32
-			pgwin32_noblock = 1;
-#endif
-
-			len = recv(pgStatSock, (char *) &msg,
-					   sizeof(PgStat_Msg), 0);
-
-#ifdef WIN32
-			pgwin32_noblock = 0;
-#endif
-
-			if (len < 0)
-			{
-				if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR)
-					break;		/* out of inner loop */
-				ereport(ERROR,
-						(errcode_for_socket_access(),
-						 errmsg("could not read statistics message: %m")));
-			}
-
-			/*
-			 * We ignore messages that are smaller than our common header
-			 */
-			if (len < sizeof(PgStat_MsgHdr))
-				continue;
-
-			/*
-			 * The received length must match the length in the header
-			 */
-			if (msg.msg_hdr.m_size != len)
-				continue;
-
-			/*
-			 * O.K. - we accept this message.  Process it.
-			 */
-			switch (msg.msg_hdr.m_type)
-			{
-				case PGSTAT_MTYPE_DUMMY:
-					break;
-
-				case PGSTAT_MTYPE_INQUIRY:
-					pgstat_recv_inquiry(&msg.msg_inquiry, len);
-					break;
-
-				case PGSTAT_MTYPE_TABSTAT:
-					pgstat_recv_tabstat(&msg.msg_tabstat, len);
-					break;
-
-				case PGSTAT_MTYPE_TABPURGE:
-					pgstat_recv_tabpurge(&msg.msg_tabpurge, len);
-					break;
-
-				case PGSTAT_MTYPE_DROPDB:
-					pgstat_recv_dropdb(&msg.msg_dropdb, len);
-					break;
-
-				case PGSTAT_MTYPE_RESETCOUNTER:
-					pgstat_recv_resetcounter(&msg.msg_resetcounter, len);
-					break;
-
-				case PGSTAT_MTYPE_RESETSHAREDCOUNTER:
-					pgstat_recv_resetsharedcounter(&msg.msg_resetsharedcounter,
-												   len);
-					break;
-
-				case PGSTAT_MTYPE_RESETSINGLECOUNTER:
-					pgstat_recv_resetsinglecounter(&msg.msg_resetsinglecounter,
-												   len);
-					break;
-
-				case PGSTAT_MTYPE_RESETSLRUCOUNTER:
-					pgstat_recv_resetslrucounter(&msg.msg_resetslrucounter,
-												 len);
-					break;
-
-				case PGSTAT_MTYPE_RESETREPLSLOTCOUNTER:
-					pgstat_recv_resetreplslotcounter(&msg.msg_resetreplslotcounter,
-													 len);
-					break;
-
-				case PGSTAT_MTYPE_RESETSUBCOUNTER:
-					pgstat_recv_resetsubcounter(&msg.msg_resetsubcounter, len);
-					break;
-
-				case PGSTAT_MTYPE_AUTOVAC_START:
-					pgstat_recv_autovac(&msg.msg_autovacuum_start, len);
-					break;
-
-				case PGSTAT_MTYPE_VACUUM:
-					pgstat_recv_vacuum(&msg.msg_vacuum, len);
-					break;
-
-				case PGSTAT_MTYPE_ANALYZE:
-					pgstat_recv_analyze(&msg.msg_analyze, len);
-					break;
-
-				case PGSTAT_MTYPE_ARCHIVER:
-					pgstat_recv_archiver(&msg.msg_archiver, len);
-					break;
-
-				case PGSTAT_MTYPE_BGWRITER:
-					pgstat_recv_bgwriter(&msg.msg_bgwriter, len);
-					break;
-
-				case PGSTAT_MTYPE_CHECKPOINTER:
-					pgstat_recv_checkpointer(&msg.msg_checkpointer, len);
-					break;
-
-				case PGSTAT_MTYPE_WAL:
-					pgstat_recv_wal(&msg.msg_wal, len);
-					break;
-
-				case PGSTAT_MTYPE_SLRU:
-					pgstat_recv_slru(&msg.msg_slru, len);
-					break;
-
-				case PGSTAT_MTYPE_FUNCSTAT:
-					pgstat_recv_funcstat(&msg.msg_funcstat, len);
-					break;
-
-				case PGSTAT_MTYPE_FUNCPURGE:
-					pgstat_recv_funcpurge(&msg.msg_funcpurge, len);
-					break;
-
-				case PGSTAT_MTYPE_RECOVERYCONFLICT:
-					pgstat_recv_recoveryconflict(&msg.msg_recoveryconflict,
-												 len);
-					break;
-
-				case PGSTAT_MTYPE_DEADLOCK:
-					pgstat_recv_deadlock(&msg.msg_deadlock, len);
-					break;
-
-				case PGSTAT_MTYPE_TEMPFILE:
-					pgstat_recv_tempfile(&msg.msg_tempfile, len);
-					break;
-
-				case PGSTAT_MTYPE_CHECKSUMFAILURE:
-					pgstat_recv_checksum_failure(&msg.msg_checksumfailure,
-												 len);
-					break;
-
-				case PGSTAT_MTYPE_REPLSLOT:
-					pgstat_recv_replslot(&msg.msg_replslot, len);
-					break;
-
-				case PGSTAT_MTYPE_CONNECT:
-					pgstat_recv_connect(&msg.msg_connect, len);
-					break;
-
-				case PGSTAT_MTYPE_DISCONNECT:
-					pgstat_recv_disconnect(&msg.msg_disconnect, len);
-					break;
-
-				case PGSTAT_MTYPE_SUBSCRIPTIONDROP:
-					pgstat_recv_subscription_drop(&msg.msg_subscriptiondrop, len);
-					break;
-
-				case PGSTAT_MTYPE_SUBSCRIPTIONERROR:
-					pgstat_recv_subscription_error(&msg.msg_subscriptionerror, len);
-					break;
-
-				default:
-					break;
-			}
-		}						/* end of inner message-processing loop */
-
-		/* Sleep until there's something to do */
-#ifndef WIN32
-		wr = WaitEventSetWait(wes, -1L, &event, 1, WAIT_EVENT_PGSTAT_MAIN);
-#else
-
-		/*
-		 * Windows, at least in its Windows Server 2003 R2 incarnation,
-		 * sometimes loses FD_READ events.  Waking up and retrying the recv()
-		 * fixes that, so don't sleep indefinitely.  This is a crock of the
-		 * first water, but until somebody wants to debug exactly what's
-		 * happening there, this is the best we can do.  The two-second
-		 * timeout matches our pre-9.2 behavior, and needs to be short enough
-		 * to not provoke "using stale statistics" complaints from
-		 * backend_read_statsfile.
-		 */
-		wr = WaitEventSetWait(wes, 2 * 1000L /* msec */ , &event, 1,
-							  WAIT_EVENT_PGSTAT_MAIN);
-#endif
-
-		/*
-		 * Emergency bailout if postmaster has died.  This is to avoid the
-		 * necessity for manual cleanup of all postmaster children.
-		 */
-		if (wr == 1 && event.events == WL_POSTMASTER_DEATH)
-			break;
-	}							/* end of outer loop */
-
-	/*
-	 * Save the final stats to reuse at next startup.
-	 */
-	pgstat_write_statsfiles(true, true);
-
-	FreeWaitEventSet(wes);
-
-	exit(0);
-}
-
-/*
- * Subroutine to clear stats in a database entry
- *
- * Tables and functions hashes are initialized to empty.
- */
-static void
-reset_dbentry_counters(PgStat_StatDBEntry *dbentry)
-{
-	HASHCTL		hash_ctl;
-
-	dbentry->n_xact_commit = 0;
-	dbentry->n_xact_rollback = 0;
-	dbentry->n_blocks_fetched = 0;
-	dbentry->n_blocks_hit = 0;
-	dbentry->n_tuples_returned = 0;
-	dbentry->n_tuples_fetched = 0;
-	dbentry->n_tuples_inserted = 0;
-	dbentry->n_tuples_updated = 0;
-	dbentry->n_tuples_deleted = 0;
-	dbentry->last_autovac_time = 0;
-	dbentry->n_conflict_tablespace = 0;
-	dbentry->n_conflict_lock = 0;
-	dbentry->n_conflict_snapshot = 0;
-	dbentry->n_conflict_bufferpin = 0;
-	dbentry->n_conflict_startup_deadlock = 0;
-	dbentry->n_temp_files = 0;
-	dbentry->n_temp_bytes = 0;
-	dbentry->n_deadlocks = 0;
-	dbentry->n_checksum_failures = 0;
-	dbentry->last_checksum_failure = 0;
-	dbentry->n_block_read_time = 0;
-	dbentry->n_block_write_time = 0;
-	dbentry->n_sessions = 0;
-	dbentry->total_session_time = 0;
-	dbentry->total_active_time = 0;
-	dbentry->total_idle_in_xact_time = 0;
-	dbentry->n_sessions_abandoned = 0;
-	dbentry->n_sessions_fatal = 0;
-	dbentry->n_sessions_killed = 0;
-
-	dbentry->stat_reset_timestamp = GetCurrentTimestamp();
-	dbentry->stats_timestamp = 0;
-
-	hash_ctl.keysize = sizeof(Oid);
-	hash_ctl.entrysize = sizeof(PgStat_StatTabEntry);
-	dbentry->tables = hash_create("Per-database table",
-								  PGSTAT_TAB_HASH_SIZE,
-								  &hash_ctl,
-								  HASH_ELEM | HASH_BLOBS);
-
-	hash_ctl.keysize = sizeof(Oid);
-	hash_ctl.entrysize = sizeof(PgStat_StatFuncEntry);
-	dbentry->functions = hash_create("Per-database function",
-									 PGSTAT_FUNCTION_HASH_SIZE,
-									 &hash_ctl,
-									 HASH_ELEM | HASH_BLOBS);
-}
-
-/*
- * Lookup the hash table entry for the specified database. If no hash
- * table entry exists, initialize it, if the create parameter is true.
- * Else, return NULL.
- */
-static PgStat_StatDBEntry *
-pgstat_get_db_entry(Oid databaseid, bool create)
-{
-	PgStat_StatDBEntry *result;
-	bool		found;
-	HASHACTION	action = (create ? HASH_ENTER : HASH_FIND);
-
-	/* Lookup or create the hash table entry for this database */
-	result = (PgStat_StatDBEntry *) hash_search(pgStatDBHash,
-												&databaseid,
-												action, &found);
-
-	if (!create && !found)
-		return NULL;
-
-	/*
-	 * If not found, initialize the new one.  This creates empty hash tables
-	 * for tables and functions, too.
-	 */
-	if (!found)
-		reset_dbentry_counters(result);
-
-	return result;
-}
-
-
-/*
- * Lookup the hash table entry for the specified table. If no hash
- * table entry exists, initialize it, if the create parameter is true.
- * Else, return NULL.
- */
-static PgStat_StatTabEntry *
-pgstat_get_tab_entry(PgStat_StatDBEntry *dbentry, Oid tableoid, bool create)
-{
-	PgStat_StatTabEntry *result;
-	bool		found;
-	HASHACTION	action = (create ? HASH_ENTER : HASH_FIND);
-
-	/* Lookup or create the hash table entry for this table */
-	result = (PgStat_StatTabEntry *) hash_search(dbentry->tables,
-												 &tableoid,
-												 action, &found);
-
-	if (!create && !found)
-		return NULL;
-
-	/* If not found, initialize the new one. */
-	if (!found)
-	{
-		result->numscans = 0;
-		result->tuples_returned = 0;
-		result->tuples_fetched = 0;
-		result->tuples_inserted = 0;
-		result->tuples_updated = 0;
-		result->tuples_deleted = 0;
-		result->tuples_hot_updated = 0;
-		result->n_live_tuples = 0;
-		result->n_dead_tuples = 0;
-		result->changes_since_analyze = 0;
-		result->inserts_since_vacuum = 0;
-		result->blocks_fetched = 0;
-		result->blocks_hit = 0;
-		result->vacuum_timestamp = 0;
-		result->vacuum_count = 0;
-		result->autovac_vacuum_timestamp = 0;
-		result->autovac_vacuum_count = 0;
-		result->analyze_timestamp = 0;
-		result->analyze_count = 0;
-		result->autovac_analyze_timestamp = 0;
-		result->autovac_analyze_count = 0;
-	}
-
-	return result;
-}
-
 /* ----------
  * pgstat_write_statsfiles() -
- *		Write the global statistics file, as well as requested DB files.
  *
- *	'permanent' specifies writing to the permanent files not temporary ones.
- *	When true (happens only when the collector is shutting down), also remove
- *	the temporary files so that backends starting up under a new postmaster
- *	can't read old data before the new collector is ready.
- *
- *	When 'allDbs' is false, only the requested databases (listed in
- *	pending_write_requests) will be written; otherwise, all databases
- *	will be written.
+ * This function is called in the last process that is accessing the shared
+ * stats so locking is not required.
  * ----------
  */
 static void
-pgstat_write_statsfiles(bool permanent, bool allDbs)
+pgstat_write_statsfile(void)
 {
-	HASH_SEQ_STATUS hstat;
-	PgStat_StatDBEntry *dbentry;
 	FILE	   *fpout;
 	int32		format_id;
-	const char *tmpfile = permanent ? PGSTAT_STAT_PERMANENT_TMPFILE : pgstat_stat_tmpname;
-	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	const char *tmpfile = PGSTAT_STAT_PERMANENT_TMPFILE;
+	const char *statfile = PGSTAT_STAT_PERMANENT_FILENAME;
 	int			rc;
+	dshash_seq_status hstat;
+	PgStatShmHashEntry *ps;
+
+	Assert(pgStatDSA);
 
 	elog(DEBUG2, "writing stats file \"%s\"", statfile);
 
@@ -2140,11 +1387,6 @@ pgstat_write_statsfiles(bool permanent, bool allDbs)
 		return;
 	}
 
-	/*
-	 * Set the timestamp of the stats file.
-	 */
-	globalStats.stats_timestamp = GetCurrentTimestamp();
-
 	/*
 	 * Write the file header --- currently just a format ID.
 	 */
@@ -2153,220 +1395,86 @@ pgstat_write_statsfiles(bool permanent, bool allDbs)
 	(void) rc;					/* we'll check for error with ferror */
 
 	/*
-	 * Write global stats struct
+	 * XXX: The following could now be generalized to just iterate over
+	 * pgstat_kind_infos instead of knowing about the different kinds of
+	 * stats.
 	 */
-	rc = fwrite(&globalStats, sizeof(globalStats), 1, fpout);
+
+	/*
+	 * Write bgwriter global stats struct
+	 */
+	rc = fwrite(&pgStatShmem->bgwriter.stats, sizeof(PgStat_BgWriterStats), 1, fpout);
 	(void) rc;					/* we'll check for error with ferror */
 
 	/*
-	 * Write archiver stats struct
+	 * Write checkpointer global stats struct
 	 */
-	rc = fwrite(&archiverStats, sizeof(archiverStats), 1, fpout);
+	rc = fwrite(&pgStatShmem->checkpointer.stats, sizeof(PgStat_CheckpointerStats), 1, fpout);
 	(void) rc;					/* we'll check for error with ferror */
 
 	/*
-	 * Write WAL stats struct
+	 * Write archiver global stats struct
 	 */
-	rc = fwrite(&walStats, sizeof(walStats), 1, fpout);
+	rc = fwrite(&pgStatShmem->archiver.stats, sizeof(PgStat_ArchiverStats), 1,
+				fpout);
+	(void) rc;					/* we'll check for error with ferror */
+
+	/*
+	 * Write WAL global stats struct
+	 */
+	rc = fwrite(&pgStatShmem->wal.stats, sizeof(PgStat_WalStats), 1, fpout);
 	(void) rc;					/* we'll check for error with ferror */
 
 	/*
 	 * Write SLRU stats struct
 	 */
-	rc = fwrite(slruStats, sizeof(slruStats), 1, fpout);
+	rc = fwrite(&pgStatShmem->slru.stats,
+				sizeof(PgStat_SLRUStats[SLRU_NUM_ELEMENTS]),
+				1, fpout);
 	(void) rc;					/* we'll check for error with ferror */
 
 	/*
-	 * Walk through the database table.
+	 * Walk through the stats entries
 	 */
-	hash_seq_init(&hstat, pgStatDBHash);
-	while ((dbentry = (PgStat_StatDBEntry *) hash_seq_search(&hstat)) != NULL)
+	dshash_seq_init(&hstat, pgStatSharedHash, false);
+	while ((ps = dshash_seq_next(&hstat)) != NULL)
 	{
-		/*
-		 * Write out the table and function stats for this DB into the
-		 * appropriate per-DB stat file, if required.
-		 */
-		if (allDbs || pgstat_db_requested(dbentry->databaseid))
-		{
-			/* Make DB's timestamp consistent with the global stats */
-			dbentry->stats_timestamp = globalStats.stats_timestamp;
+		PgStatShm_StatEntryHeader *shstats;
+		size_t		len;
 
-			pgstat_write_db_statsfile(dbentry, permanent);
-		}
+		CHECK_FOR_INTERRUPTS();
 
-		/*
-		 * Write out the DB entry. We don't write the tables or functions
-		 * pointers, since they're of no use to any other process.
-		 */
-		fputc('D', fpout);
-		rc = fwrite(dbentry, offsetof(PgStat_StatDBEntry, tables), 1, fpout);
+		/* we may have some "dropped" entries not yet removed, skip them */
+		if (ps->dropped)
+			continue;
+
+		shstats = (PgStatShm_StatEntryHeader *) dsa_get_address(pgStatDSA, ps->body);
+
+		/* if not dropped the valid-entry refcount should exist */
+		Assert(pg_atomic_read_u32(&ps->refcount) > 0);
+
+		fputc('S', fpout);
+		rc = fwrite(&ps->key, sizeof(PgStatHashKey), 1, fpout);
+
+		/* Write except the header part of the etnry */
+		len = shared_stat_entry_len(ps->key.kind);
+		rc = fwrite(shared_stat_entry_data(ps->key.kind, shstats), len, 1, fpout);
 		(void) rc;				/* we'll check for error with ferror */
 	}
+	dshash_seq_term(&hstat);
 
 	/*
 	 * Write replication slot stats struct
 	 */
-	if (replSlotStatHash)
+	for (int i = 0; i < max_replication_slots; i++)
 	{
-		PgStat_StatReplSlotEntry *slotent;
+		PgStat_StatReplSlotEntry *statent = &pgStatShmem->replslot.stats[i];
 
-		hash_seq_init(&hstat, replSlotStatHash);
-		while ((slotent = (PgStat_StatReplSlotEntry *) hash_seq_search(&hstat)) != NULL)
-		{
-			fputc('R', fpout);
-			rc = fwrite(slotent, sizeof(PgStat_StatReplSlotEntry), 1, fpout);
-			(void) rc;			/* we'll check for error with ferror */
-		}
-	}
+		if (statent->index == -1)
+			continue;
 
-	/*
-	 * Write subscription stats struct
-	 */
-	if (subscriptionStatHash)
-	{
-		PgStat_StatSubEntry *subentry;
-
-		hash_seq_init(&hstat, subscriptionStatHash);
-		while ((subentry = (PgStat_StatSubEntry *) hash_seq_search(&hstat)) != NULL)
-		{
-			fputc('S', fpout);
-			rc = fwrite(subentry, sizeof(PgStat_StatSubEntry), 1, fpout);
-			(void) rc;			/* we'll check for error with ferror */
-		}
-	}
-
-	/*
-	 * No more output to be done. Close the temp file and replace the old
-	 * pgstat.stat with it.  The ferror() check replaces testing for error
-	 * after each individual fputc or fwrite above.
-	 */
-	fputc('E', fpout);
-
-	if (ferror(fpout))
-	{
-		ereport(LOG,
-				(errcode_for_file_access(),
-				 errmsg("could not write temporary statistics file \"%s\": %m",
-						tmpfile)));
-		FreeFile(fpout);
-		unlink(tmpfile);
-	}
-	else if (FreeFile(fpout) < 0)
-	{
-		ereport(LOG,
-				(errcode_for_file_access(),
-				 errmsg("could not close temporary statistics file \"%s\": %m",
-						tmpfile)));
-		unlink(tmpfile);
-	}
-	else if (rename(tmpfile, statfile) < 0)
-	{
-		ereport(LOG,
-				(errcode_for_file_access(),
-				 errmsg("could not rename temporary statistics file \"%s\" to \"%s\": %m",
-						tmpfile, statfile)));
-		unlink(tmpfile);
-	}
-
-	if (permanent)
-		unlink(pgstat_stat_filename);
-
-	/*
-	 * Now throw away the list of requests.  Note that requests sent after we
-	 * started the write are still waiting on the network socket.
-	 */
-	list_free(pending_write_requests);
-	pending_write_requests = NIL;
-}
-
-/*
- * return the filename for a DB stat file; filename is the output buffer,
- * of length len.
- */
-static void
-get_dbstat_filename(bool permanent, bool tempname, Oid databaseid,
-					char *filename, int len)
-{
-	int			printed;
-
-	/* NB -- pgstat_reset_remove_files knows about the pattern this uses */
-	printed = snprintf(filename, len, "%s/db_%u.%s",
-					   permanent ? PGSTAT_STAT_PERMANENT_DIRECTORY :
-					   pgstat_stat_directory,
-					   databaseid,
-					   tempname ? "tmp" : "stat");
-	if (printed >= len)
-		elog(ERROR, "overlength pgstat path");
-}
-
-/* ----------
- * pgstat_write_db_statsfile() -
- *		Write the stat file for a single database.
- *
- *	If writing to the permanent file (happens when the collector is
- *	shutting down only), remove the temporary file so that backends
- *	starting up under a new postmaster can't read the old data before
- *	the new collector is ready.
- * ----------
- */
-static void
-pgstat_write_db_statsfile(PgStat_StatDBEntry *dbentry, bool permanent)
-{
-	HASH_SEQ_STATUS tstat;
-	HASH_SEQ_STATUS fstat;
-	PgStat_StatTabEntry *tabentry;
-	PgStat_StatFuncEntry *funcentry;
-	FILE	   *fpout;
-	int32		format_id;
-	Oid			dbid = dbentry->databaseid;
-	int			rc;
-	char		tmpfile[MAXPGPATH];
-	char		statfile[MAXPGPATH];
-
-	get_dbstat_filename(permanent, true, dbid, tmpfile, MAXPGPATH);
-	get_dbstat_filename(permanent, false, dbid, statfile, MAXPGPATH);
-
-	elog(DEBUG2, "writing stats file \"%s\"", statfile);
-
-	/*
-	 * Open the statistics temp file to write out the current values.
-	 */
-	fpout = AllocateFile(tmpfile, PG_BINARY_W);
-	if (fpout == NULL)
-	{
-		ereport(LOG,
-				(errcode_for_file_access(),
-				 errmsg("could not open temporary statistics file \"%s\": %m",
-						tmpfile)));
-		return;
-	}
-
-	/*
-	 * Write the file header --- currently just a format ID.
-	 */
-	format_id = PGSTAT_FILE_FORMAT_ID;
-	rc = fwrite(&format_id, sizeof(format_id), 1, fpout);
-	(void) rc;					/* we'll check for error with ferror */
-
-	/*
-	 * Walk through the database's access stats per table.
-	 */
-	hash_seq_init(&tstat, dbentry->tables);
-	while ((tabentry = (PgStat_StatTabEntry *) hash_seq_search(&tstat)) != NULL)
-	{
-		fputc('T', fpout);
-		rc = fwrite(tabentry, sizeof(PgStat_StatTabEntry), 1, fpout);
-		(void) rc;				/* we'll check for error with ferror */
-	}
-
-	/*
-	 * Walk through the database's function stats table.
-	 */
-	hash_seq_init(&fstat, dbentry->functions);
-	while ((funcentry = (PgStat_StatFuncEntry *) hash_seq_search(&fstat)) != NULL)
-	{
-		fputc('F', fpout);
-		rc = fwrite(funcentry, sizeof(PgStat_StatFuncEntry), 1, fpout);
+		fputc('R', fpout);
+		rc = fwrite(statent, sizeof(*statent), 1, fpout);
 		(void) rc;				/* we'll check for error with ferror */
 	}
 
@@ -2402,406 +1510,53 @@ pgstat_write_db_statsfile(PgStat_StatDBEntry *dbentry, bool permanent)
 						tmpfile, statfile)));
 		unlink(tmpfile);
 	}
-
-	if (permanent)
-	{
-		get_dbstat_filename(false, false, dbid, statfile, MAXPGPATH);
-
-		elog(DEBUG2, "removing temporary stats file \"%s\"", statfile);
-		unlink(statfile);
-	}
 }
 
 /* ----------
- * pgstat_read_statsfiles() -
+ * pgstat_read_statsfile() -
  *
- *	Reads in some existing statistics collector files and returns the
- *	databases hash table that is the top level of the data.
+ *	Reads in existing activity statistics file into the shared stats hash.
  *
- *	If 'onlydb' is not InvalidOid, it means we only want data for that DB
- *	plus the shared catalogs ("DB 0").  We'll still populate the DB hash
- *	table for all databases, but we don't bother even creating table/function
- *	hash tables for other databases.
- *
- *	'permanent' specifies reading from the permanent files not temporary ones.
- *	When true (happens only when the collector is starting up), remove the
- *	files after reading; the in-memory status is now authoritative, and the
- *	files would be out of date in case somebody else reads them.
- *
- *	If a 'deep' read is requested, table/function stats are read, otherwise
- *	the table/function hash tables remain empty.
+ * This function is called in the only process that is accessing the shared
+ * stats so locking is not required.
  * ----------
  */
-static HTAB *
-pgstat_read_statsfiles(Oid onlydb, bool permanent, bool deep)
+static void
+pgstat_read_statsfile(void)
 {
-	PgStat_StatDBEntry *dbentry;
-	PgStat_StatDBEntry dbbuf;
-	HASHCTL		hash_ctl;
-	HTAB	   *dbhash;
 	FILE	   *fpin;
 	int32		format_id;
 	bool		found;
-	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
-	int			i;
-	TimestampTz	ts;
+	const char *statfile = PGSTAT_STAT_PERMANENT_FILENAME;
 
-	/*
-	 * The tables will live in pgStatLocalContext.
-	 */
-	pgstat_setup_memcxt();
+	/* shouldn't be called from postmaster */
+	Assert(IsUnderPostmaster || !IsPostmasterEnvironment);
 
-	/*
-	 * Create the DB hashtable
-	 */
-	hash_ctl.keysize = sizeof(Oid);
-	hash_ctl.entrysize = sizeof(PgStat_StatDBEntry);
-	hash_ctl.hcxt = pgStatLocalContext;
-	dbhash = hash_create("Databases hash", PGSTAT_DB_HASH_SIZE, &hash_ctl,
-						 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
-
-	/*
-	 * Clear out global, archiver, WAL and SLRU statistics so they start from
-	 * zero in case we can't load an existing statsfile.
-	 */
-	memset(&globalStats, 0, sizeof(globalStats));
-	memset(&archiverStats, 0, sizeof(archiverStats));
-	memset(&walStats, 0, sizeof(walStats));
-	memset(&slruStats, 0, sizeof(slruStats));
+	elog(DEBUG2, "reading stats file \"%s\"", statfile);
 
 	/*
 	 * Set the current timestamp (will be kept only in case we can't load an
 	 * existing statsfile).
 	 */
-	ts = GetCurrentTimestamp();
-	globalStats.bgwriter.stat_reset_timestamp = ts;
-	archiverStats.stat_reset_timestamp = ts;
-	walStats.stat_reset_timestamp = ts;
-
-	/*
-	 * Set the same reset timestamp for all SLRU items too.
-	 */
-	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
-		slruStats[i].stat_reset_timestamp = ts;
+	pgStatShmem->bgwriter.stats.stat_reset_timestamp = GetCurrentTimestamp();
+	pgStatShmem->archiver.stats.stat_reset_timestamp =
+		pgStatShmem->bgwriter.stats.stat_reset_timestamp;
+	pgStatShmem->wal.stats.stat_reset_timestamp =
+		pgStatShmem->bgwriter.stats.stat_reset_timestamp;
 
 	/*
 	 * Try to open the stats file. If it doesn't exist, the backends simply
-	 * return zero for anything and the collector simply starts from scratch
-	 * with empty counters.
+	 * returns zero for anything and the activity statistics simply starts
+	 * from scratch with empty counters.
 	 *
-	 * ENOENT is a possibility if the stats collector is not running or has
-	 * not yet written the stats file the first time.  Any other failure
+	 * ENOENT is a possibility if the activity statistics is not running or
+	 * has not yet written the stats file the first time.  Any other failure
 	 * condition is suspicious.
 	 */
 	if ((fpin = AllocateFile(statfile, PG_BINARY_R)) == NULL)
 	{
 		if (errno != ENOENT)
-			ereport(pgStatRunningInCollector ? LOG : WARNING,
-					(errcode_for_file_access(),
-					 errmsg("could not open statistics file \"%s\": %m",
-							statfile)));
-		return dbhash;
-	}
-
-	/*
-	 * Verify it's of the expected format.
-	 */
-	if (fread(&format_id, 1, sizeof(format_id), fpin) != sizeof(format_id) ||
-		format_id != PGSTAT_FILE_FORMAT_ID)
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		goto done;
-	}
-
-	/*
-	 * Read global stats struct
-	 */
-	if (fread(&globalStats, 1, sizeof(globalStats), fpin) != sizeof(globalStats))
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		memset(&globalStats, 0, sizeof(globalStats));
-		goto done;
-	}
-
-	/*
-	 * In the collector, disregard the timestamp we read from the permanent
-	 * stats file; we should be willing to write a temp stats file immediately
-	 * upon the first request from any backend.  This only matters if the old
-	 * file's timestamp is less than PGSTAT_STAT_INTERVAL ago, but that's not
-	 * an unusual scenario.
-	 */
-	if (pgStatRunningInCollector)
-		globalStats.stats_timestamp = 0;
-
-	/*
-	 * Read archiver stats struct
-	 */
-	if (fread(&archiverStats, 1, sizeof(archiverStats), fpin) != sizeof(archiverStats))
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		memset(&archiverStats, 0, sizeof(archiverStats));
-		goto done;
-	}
-
-	/*
-	 * Read WAL stats struct
-	 */
-	if (fread(&walStats, 1, sizeof(walStats), fpin) != sizeof(walStats))
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		memset(&walStats, 0, sizeof(walStats));
-		goto done;
-	}
-
-	/*
-	 * Read SLRU stats struct
-	 */
-	if (fread(slruStats, 1, sizeof(slruStats), fpin) != sizeof(slruStats))
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		memset(&slruStats, 0, sizeof(slruStats));
-		goto done;
-	}
-
-	/*
-	 * We found an existing collector stats file. Read it and put all the
-	 * hashtable entries into place.
-	 */
-	for (;;)
-	{
-		switch (fgetc(fpin))
-		{
-				/*
-				 * 'D'	A PgStat_StatDBEntry struct describing a database
-				 * follows.
-				 */
-			case 'D':
-				if (fread(&dbbuf, 1, offsetof(PgStat_StatDBEntry, tables),
-						  fpin) != offsetof(PgStat_StatDBEntry, tables))
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					goto done;
-				}
-
-				/*
-				 * Add to the DB hash
-				 */
-				dbentry = (PgStat_StatDBEntry *) hash_search(dbhash,
-															 (void *) &dbbuf.databaseid,
-															 HASH_ENTER,
-															 &found);
-				if (found)
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					goto done;
-				}
-
-				memcpy(dbentry, &dbbuf, sizeof(PgStat_StatDBEntry));
-				dbentry->tables = NULL;
-				dbentry->functions = NULL;
-
-				/*
-				 * In the collector, disregard the timestamp we read from the
-				 * permanent stats file; we should be willing to write a temp
-				 * stats file immediately upon the first request from any
-				 * backend.
-				 */
-				if (pgStatRunningInCollector)
-					dbentry->stats_timestamp = 0;
-
-				/*
-				 * Don't create tables/functions hashtables for uninteresting
-				 * databases.
-				 */
-				if (onlydb != InvalidOid)
-				{
-					if (dbbuf.databaseid != onlydb &&
-						dbbuf.databaseid != InvalidOid)
-						break;
-				}
-
-				hash_ctl.keysize = sizeof(Oid);
-				hash_ctl.entrysize = sizeof(PgStat_StatTabEntry);
-				hash_ctl.hcxt = pgStatLocalContext;
-				dbentry->tables = hash_create("Per-database table",
-											  PGSTAT_TAB_HASH_SIZE,
-											  &hash_ctl,
-											  HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
-
-				hash_ctl.keysize = sizeof(Oid);
-				hash_ctl.entrysize = sizeof(PgStat_StatFuncEntry);
-				hash_ctl.hcxt = pgStatLocalContext;
-				dbentry->functions = hash_create("Per-database function",
-												 PGSTAT_FUNCTION_HASH_SIZE,
-												 &hash_ctl,
-												 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
-
-				/*
-				 * If requested, read the data from the database-specific
-				 * file.  Otherwise we just leave the hashtables empty.
-				 */
-				if (deep)
-					pgstat_read_db_statsfile(dbentry->databaseid,
-											 dbentry->tables,
-											 dbentry->functions,
-											 permanent);
-
-				break;
-
-				/*
-				 * 'R'	A PgStat_StatReplSlotEntry struct describing a
-				 * replication slot follows.
-				 */
-			case 'R':
-				{
-					PgStat_StatReplSlotEntry slotbuf;
-					PgStat_StatReplSlotEntry *slotent;
-
-					if (fread(&slotbuf, 1, sizeof(PgStat_StatReplSlotEntry), fpin)
-						!= sizeof(PgStat_StatReplSlotEntry))
-					{
-						ereport(pgStatRunningInCollector ? LOG : WARNING,
-								(errmsg("corrupted statistics file \"%s\"",
-										statfile)));
-						goto done;
-					}
-
-					/* Create hash table if we don't have it already. */
-					if (replSlotStatHash == NULL)
-					{
-						HASHCTL		hash_ctl;
-
-						hash_ctl.keysize = sizeof(NameData);
-						hash_ctl.entrysize = sizeof(PgStat_StatReplSlotEntry);
-						hash_ctl.hcxt = pgStatLocalContext;
-						replSlotStatHash = hash_create("Replication slots hash",
-													   PGSTAT_REPLSLOT_HASH_SIZE,
-													   &hash_ctl,
-													   HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
-					}
-
-					slotent = (PgStat_StatReplSlotEntry *) hash_search(replSlotStatHash,
-																	   (void *) &slotbuf.slotname,
-																	   HASH_ENTER, NULL);
-					memcpy(slotent, &slotbuf, sizeof(PgStat_StatReplSlotEntry));
-					break;
-				}
-
-				/*
-				 * 'S'	A PgStat_StatSubEntry struct describing subscription
-				 * statistics.
-				 */
-			case 'S':
-				{
-					PgStat_StatSubEntry subbuf;
-					PgStat_StatSubEntry *subentry;
-
-					if (fread(&subbuf, 1, sizeof(PgStat_StatSubEntry), fpin)
-							!= sizeof(PgStat_StatSubEntry))
-					{
-						ereport(pgStatRunningInCollector ? LOG : WARNING,
-								(errmsg("corrupted statistics file \"%s\"",
-										statfile)));
-						goto done;
-					}
-
-					if (subscriptionStatHash == NULL)
-					{
-						HASHCTL		hash_ctl;
-
-						hash_ctl.keysize = sizeof(Oid);
-						hash_ctl.entrysize = sizeof(PgStat_StatSubEntry);
-						hash_ctl.hcxt = pgStatLocalContext;
-						subscriptionStatHash = hash_create("Subscription hash",
-														   PGSTAT_SUBSCRIPTION_HASH_SIZE,
-														   &hash_ctl,
-														   HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
-					}
-
-					subentry = (PgStat_StatSubEntry *) hash_search(subscriptionStatHash,
-																   (void *) &subbuf.subid,
-																   HASH_ENTER, NULL);
-
-					memcpy(subentry, &subbuf, sizeof(subbuf));
-					break;
-				}
-
-			case 'E':
-				goto done;
-
-			default:
-				ereport(pgStatRunningInCollector ? LOG : WARNING,
-						(errmsg("corrupted statistics file \"%s\"",
-								statfile)));
-				goto done;
-		}
-	}
-
-done:
-	FreeFile(fpin);
-
-	/* If requested to read the permanent file, also get rid of it. */
-	if (permanent)
-	{
-		elog(DEBUG2, "removing permanent stats file \"%s\"", statfile);
-		unlink(statfile);
-	}
-
-	return dbhash;
-}
-
-
-/* ----------
- * pgstat_read_db_statsfile() -
- *
- *	Reads in the existing statistics collector file for the given database,
- *	filling the passed-in tables and functions hash tables.
- *
- *	As in pgstat_read_statsfiles, if the permanent file is requested, it is
- *	removed after reading.
- *
- *	Note: this code has the ability to skip storing per-table or per-function
- *	data, if NULL is passed for the corresponding hashtable.  That's not used
- *	at the moment though.
- * ----------
- */
-static void
-pgstat_read_db_statsfile(Oid databaseid, HTAB *tabhash, HTAB *funchash,
-						 bool permanent)
-{
-	PgStat_StatTabEntry *tabentry;
-	PgStat_StatTabEntry tabbuf;
-	PgStat_StatFuncEntry funcbuf;
-	PgStat_StatFuncEntry *funcentry;
-	FILE	   *fpin;
-	int32		format_id;
-	bool		found;
-	char		statfile[MAXPGPATH];
-
-	get_dbstat_filename(permanent, false, databaseid, statfile, MAXPGPATH);
-
-	/*
-	 * Try to open the stats file. If it doesn't exist, the backends simply
-	 * return zero for anything and the collector simply starts from scratch
-	 * with empty counters.
-	 *
-	 * ENOENT is a possibility if the stats collector is not running or has
-	 * not yet written the stats file the first time.  Any other failure
-	 * condition is suspicious.
-	 */
-	if ((fpin = AllocateFile(statfile, PG_BINARY_R)) == NULL)
-	{
-		if (errno != ENOENT)
-			ereport(pgStatRunningInCollector ? LOG : WARNING,
+			ereport(LOG,
 					(errcode_for_file_access(),
 					 errmsg("could not open statistics file \"%s\": %m",
 							statfile)));
@@ -2814,95 +1569,163 @@ pgstat_read_db_statsfile(Oid databaseid, HTAB *tabhash, HTAB *funchash,
 	if (fread(&format_id, 1, sizeof(format_id), fpin) != sizeof(format_id) ||
 		format_id != PGSTAT_FILE_FORMAT_ID)
 	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
+		ereport(LOG,
 				(errmsg("corrupted statistics file \"%s\"", statfile)));
 		goto done;
 	}
 
 	/*
-	 * We found an existing collector stats file. Read it and put all the
-	 * hashtable entries into place.
+	 * XXX: The following could now be generalized to just iterate over
+	 * pgstat_kind_infos instead of knowing about the different kinds of
+	 * stats.
+	 */
+
+	/*
+	 * Read bgwiter stats struct
+	 */
+	if (fread(&pgStatShmem->bgwriter.stats, 1, sizeof(PgStat_BgWriterStats), fpin) !=
+		sizeof(PgStat_BgWriterStats))
+	{
+		ereport(LOG,
+				(errmsg("corrupted statistics file \"%s\"", statfile)));
+		MemSet(&pgStatShmem->bgwriter.stats, 0, sizeof(PgStat_BgWriterStats));
+		goto done;
+	}
+
+	/*
+	 * Read checkpointer stats struct
+	 */
+	if (fread(&pgStatShmem->checkpointer.stats, 1, sizeof(PgStat_CheckpointerStats), fpin) !=
+		sizeof(PgStat_CheckpointerStats))
+	{
+		ereport(LOG,
+				(errmsg("corrupted statistics file \"%s\"", statfile)));
+		MemSet(&pgStatShmem->checkpointer.stats, 0, sizeof(PgStat_CheckpointerStats));
+		goto done;
+	}
+
+	/*
+	 * Read archiver stats struct
+	 */
+	if (fread(&pgStatShmem->archiver.stats, 1, sizeof(PgStat_ArchiverStats),
+			  fpin) != sizeof(PgStat_ArchiverStats))
+	{
+		ereport(LOG,
+				(errmsg("corrupted statistics file \"%s\"", statfile)));
+		MemSet(&pgStatShmem->archiver.stats, 0, sizeof(PgStat_ArchiverStats));
+		goto done;
+	}
+
+	/*
+	 * Read WAL stats struct
+	 */
+	if (fread(&pgStatShmem->wal.stats, 1, sizeof(PgStat_WalStats), fpin)
+		!= sizeof(PgStat_WalStats))
+	{
+		ereport(LOG,
+				(errmsg("corrupted statistics file \"%s\"", statfile)));
+		MemSet(&pgStatShmem->wal.stats, 0, sizeof(PgStat_WalStats));
+		goto done;
+	}
+
+	/*
+	 * Read SLRU stats struct
+	 */
+	if (fread(&pgStatShmem->slru.stats, 1, SizeOfSlruStats, fpin) != SizeOfSlruStats)
+	{
+		ereport(LOG,
+				(errmsg("corrupted statistics file \"%s\"", statfile)));
+		goto done;
+	}
+
+	/*
+	 * We found an existing activity statistics file. Read it and put all the
+	 * hash table entries into place.
 	 */
 	for (;;)
 	{
 		switch (fgetc(fpin))
 		{
-				/*
-				 * 'T'	A PgStat_StatTabEntry follows.
-				 */
-			case 'T':
-				if (fread(&tabbuf, 1, sizeof(PgStat_StatTabEntry),
-						  fpin) != sizeof(PgStat_StatTabEntry))
+			case 'S':
 				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					goto done;
-				}
+					PgStatHashKey key;
+					PgStatShmHashEntry *p;
+					PgStatShm_StatEntryHeader *header;
+					size_t		len;
+
+					CHECK_FOR_INTERRUPTS();
+
+					if (fread(&key, 1, sizeof(key), fpin) != sizeof(key))
+					{
+						ereport(LOG,
+								(errmsg("corrupted statistics file \"%s\"", statfile)));
+						goto done;
+					}
+
+					/*
+					 * This intentionally doesn't use pgstat_shared_ref_get()
+					 * - putting all stats into checkpointer's
+					 * pgStatSharedRefHash would be wasted effort and memory.
+					 */
+					p = dshash_find_or_insert(pgStatSharedHash, &key, &found);
+
+					/* don't allow duplicate entries */
+					if (found)
+					{
+						dshash_release_lock(pgStatSharedHash, p);
+						ereport(LOG,
+								(errmsg("corrupted statistics file \"%s\"",
+										statfile)));
+						goto done;
+					}
+
+					header = pgstat_shared_stat_entry_init(key.kind, p, 1);
+					dshash_release_lock(pgStatSharedHash, p);
+
+					/* Avoid overwriting header part */
+					len = shared_stat_entry_len(key.kind);
+
+					if (fread(shared_stat_entry_data(key.kind, header), 1, len, fpin) != len)
+					{
+						ereport(LOG,
+								(errmsg("corrupted statistics file \"%s\"", statfile)));
+						goto done;
+					}
 
-				/*
-				 * Skip if table data not wanted.
-				 */
-				if (tabhash == NULL)
 					break;
-
-				tabentry = (PgStat_StatTabEntry *) hash_search(tabhash,
-															   (void *) &tabbuf.tableid,
-															   HASH_ENTER, &found);
-
-				if (found)
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					goto done;
 				}
 
-				memcpy(tabentry, &tabbuf, sizeof(tabbuf));
+				/*
+				 * 'R'	A PgStat_StatReplSlotEntry struct describing a
+				 * replication slot follows.
+				 */
+			case 'R':
+				{
+					PgStat_StatReplSlotEntry tmp;
+
+					if (fread(&tmp, 1, sizeof(tmp), fpin) != sizeof(tmp))
+					{
+						ereport(LOG,
+								(errmsg("corrupted statistics file \"%s\"", statfile)));
+						goto done;
+					}
+
+					/*
+					 * AFIXME: ask slot.c for the proper index - it may be
+					 * different from before. Then we can remove
+					 * PgStat_ReplSlotStats.index.
+					 */
+
+					if (tmp.index < max_replication_slots)
+						pgStatShmem->replslot.stats[tmp.index] = tmp;
+				}
 				break;
 
-				/*
-				 * 'F'	A PgStat_StatFuncEntry follows.
-				 */
-			case 'F':
-				if (fread(&funcbuf, 1, sizeof(PgStat_StatFuncEntry),
-						  fpin) != sizeof(PgStat_StatFuncEntry))
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					goto done;
-				}
-
-				/*
-				 * Skip if function data not wanted.
-				 */
-				if (funchash == NULL)
-					break;
-
-				funcentry = (PgStat_StatFuncEntry *) hash_search(funchash,
-																 (void *) &funcbuf.functionid,
-																 HASH_ENTER, &found);
-
-				if (found)
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					goto done;
-				}
-
-				memcpy(funcentry, &funcbuf, sizeof(funcbuf));
-				break;
-
-				/*
-				 * 'E'	The EOF marker of a complete stats file.
-				 */
 			case 'E':
 				goto done;
 
 			default:
-				ereport(pgStatRunningInCollector ? LOG : WARNING,
+				ereport(LOG,
 						(errmsg("corrupted statistics file \"%s\"",
 								statfile)));
 				goto done;
@@ -2912,1533 +1735,958 @@ pgstat_read_db_statsfile(Oid databaseid, HTAB *tabhash, HTAB *funchash,
 done:
 	FreeFile(fpin);
 
-	if (permanent)
-	{
-		elog(DEBUG2, "removing permanent stats file \"%s\"", statfile);
-		unlink(statfile);
-	}
+	elog(DEBUG2, "removing permanent stats file \"%s\"", statfile);
+	unlink(statfile);
+
+	return;
 }
 
-/* ----------
- * pgstat_read_db_statsfile_timestamp() -
- *
- *	Attempt to determine the timestamp of the last db statfile write.
- *	Returns true if successful; the timestamp is stored in *ts. The caller must
- *	rely on timestamp stored in *ts iff the function returns true.
- *
- *	This needs to be careful about handling databases for which no stats file
- *	exists, such as databases without a stat entry or those not yet written:
- *
- *	- if there's a database entry in the global file, return the corresponding
- *	stats_timestamp value.
- *
- *	- if there's no db stat entry (e.g. for a new or inactive database),
- *	there's no stats_timestamp value, but also nothing to write so we return
- *	the timestamp of the global statfile.
- * ----------
- */
-static bool
-pgstat_read_db_statsfile_timestamp(Oid databaseid, bool permanent,
-								   TimestampTz *ts)
+static PgStatShm_StatEntryHeader *
+pgstat_shared_stat_entry_init(PgStatKind kind,
+							  PgStatShmHashEntry *shhashent,
+							  int init_refcount)
 {
-	PgStat_StatDBEntry dbentry;
-	PgStat_GlobalStats myGlobalStats;
-	PgStat_ArchiverStats myArchiverStats;
-	PgStat_WalStats myWalStats;
-	PgStat_SLRUStats mySLRUStats[SLRU_NUM_ELEMENTS];
-	PgStat_StatReplSlotEntry myReplSlotStats;
-	PgStat_StatSubEntry mySubStats;
-	FILE	   *fpin;
-	int32		format_id;
-	const char *statfile = permanent ? PGSTAT_STAT_PERMANENT_FILENAME : pgstat_stat_filename;
+	/* Create new stats entry. */
+	dsa_pointer chunk;
+	PgStatShm_StatEntryHeader *shheader;
 
-	/*
-	 * Try to open the stats file.  As above, anything but ENOENT is worthy of
-	 * complaining about.
-	 */
-	if ((fpin = AllocateFile(statfile, PG_BINARY_R)) == NULL)
-	{
-		if (errno != ENOENT)
-			ereport(pgStatRunningInCollector ? LOG : WARNING,
-					(errcode_for_file_access(),
-					 errmsg("could not open statistics file \"%s\": %m",
-							statfile)));
-		return false;
-	}
+	LWLockInitialize(&shhashent->lock, LWTRANCHE_STATS);
+	pg_atomic_init_u32(&shhashent->refcount, init_refcount);
+	shhashent->dropped = false;
 
-	/*
-	 * Verify it's of the expected format.
-	 */
-	if (fread(&format_id, 1, sizeof(format_id), fpin) != sizeof(format_id) ||
-		format_id != PGSTAT_FILE_FORMAT_ID)
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		FreeFile(fpin);
-		return false;
-	}
+	chunk = dsa_allocate0(pgStatDSA, pgstat_kind_infos[kind].shared_size);
+	shheader = dsa_get_address(pgStatDSA, chunk);
+	shheader->magic = 0xdeadbeef;
 
-	/*
-	 * Read global stats struct
-	 */
-	if (fread(&myGlobalStats, 1, sizeof(myGlobalStats),
-			  fpin) != sizeof(myGlobalStats))
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		FreeFile(fpin);
-		return false;
-	}
+	/* Link the new entry from the hash entry. */
+	shhashent->body = chunk;
 
-	/*
-	 * Read archiver stats struct
-	 */
-	if (fread(&myArchiverStats, 1, sizeof(myArchiverStats),
-			  fpin) != sizeof(myArchiverStats))
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		FreeFile(fpin);
-		return false;
-	}
-
-	/*
-	 * Read WAL stats struct
-	 */
-	if (fread(&myWalStats, 1, sizeof(myWalStats), fpin) != sizeof(myWalStats))
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		FreeFile(fpin);
-		return false;
-	}
-
-	/*
-	 * Read SLRU stats struct
-	 */
-	if (fread(mySLRUStats, 1, sizeof(mySLRUStats), fpin) != sizeof(mySLRUStats))
-	{
-		ereport(pgStatRunningInCollector ? LOG : WARNING,
-				(errmsg("corrupted statistics file \"%s\"", statfile)));
-		FreeFile(fpin);
-		return false;
-	}
-
-	/* By default, we're going to return the timestamp of the global file. */
-	*ts = myGlobalStats.stats_timestamp;
-
-	/*
-	 * We found an existing collector stats file.  Read it and look for a
-	 * record for the requested database.  If found, use its timestamp.
-	 */
-	for (;;)
-	{
-		switch (fgetc(fpin))
-		{
-				/*
-				 * 'D'	A PgStat_StatDBEntry struct describing a database
-				 * follows.
-				 */
-			case 'D':
-				if (fread(&dbentry, 1, offsetof(PgStat_StatDBEntry, tables),
-						  fpin) != offsetof(PgStat_StatDBEntry, tables))
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					FreeFile(fpin);
-					return false;
-				}
-
-				/*
-				 * If this is the DB we're looking for, save its timestamp and
-				 * we're done.
-				 */
-				if (dbentry.databaseid == databaseid)
-				{
-					*ts = dbentry.stats_timestamp;
-					goto done;
-				}
-
-				break;
-
-				/*
-				 * 'R'	A PgStat_StatReplSlotEntry struct describing a
-				 * replication slot follows.
-				 */
-			case 'R':
-				if (fread(&myReplSlotStats, 1, sizeof(PgStat_StatReplSlotEntry), fpin)
-					!= sizeof(PgStat_StatReplSlotEntry))
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					FreeFile(fpin);
-					return false;
-				}
-				break;
-
-				/*
-				 * 'S'	A PgStat_StatSubEntry struct describing subscription
-				 * statistics follows.
-				 */
-			case 'S':
-				if (fread(&mySubStats, 1, sizeof(PgStat_StatSubEntry), fpin)
-					!= sizeof(PgStat_StatSubEntry))
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					FreeFile(fpin);
-					return false;
-				}
-				break;
-
-			case 'E':
-				goto done;
-
-			default:
-				{
-					ereport(pgStatRunningInCollector ? LOG : WARNING,
-							(errmsg("corrupted statistics file \"%s\"",
-									statfile)));
-					FreeFile(fpin);
-					return false;
-				}
-		}
-	}
-
-done:
-	FreeFile(fpin);
-	return true;
+	return shheader;
 }
 
 /*
- * If not already done, read the statistics collector stats file into
- * some hash tables.  The results will be kept until pgstat_clear_snapshot()
- * is called (typically, at end of transaction).
+ * Helper function for pgstat_shared_ref_get().
  */
-static void
-backend_read_statsfile(void)
+static bool
+pgstat_shared_ref_get_cached(PgStatHashKey key, PgStatSharedRef **shared_ref_p)
 {
-	TimestampTz min_ts = 0;
-	TimestampTz ref_ts = 0;
-	Oid			inquiry_db;
-	int			count;
+	bool found;
+	PgStatSharedRefHashEntry *cache_entry;
 
 	pgstat_assert_is_up();
 
-	/* already read it? */
-	if (pgStatDBHash)
-		return;
-	Assert(!pgStatRunningInCollector);
+	pgstat_setup_memcxt();
 
-	/*
-	 * In a normal backend, we check staleness of the data for our own DB, and
-	 * so we send MyDatabaseId in inquiry messages.  In the autovac launcher,
-	 * check staleness of the shared-catalog data, and send InvalidOid in
-	 * inquiry messages so as not to force writing unnecessary data.
-	 */
-	if (IsAutoVacuumLauncherProcess())
-		inquiry_db = InvalidOid;
-	else
-		inquiry_db = MyDatabaseId;
-
-	/*
-	 * Loop until fresh enough stats file is available or we ran out of time.
-	 * The stats inquiry message is sent repeatedly in case collector drops
-	 * it; but not every single time, as that just swamps the collector.
-	 */
-	for (count = 0; count < PGSTAT_POLL_LOOP_COUNT; count++)
+	if (!pgStatSharedRefHash)
 	{
-		bool		ok;
-		TimestampTz file_ts = 0;
-		TimestampTz cur_ts;
-
-		CHECK_FOR_INTERRUPTS();
-
-		ok = pgstat_read_db_statsfile_timestamp(inquiry_db, false, &file_ts);
-
-		cur_ts = GetCurrentTimestamp();
-		/* Calculate min acceptable timestamp, if we didn't already */
-		if (count == 0 || cur_ts < ref_ts)
-		{
-			/*
-			 * We set the minimum acceptable timestamp to PGSTAT_STAT_INTERVAL
-			 * msec before now.  This indirectly ensures that the collector
-			 * needn't write the file more often than PGSTAT_STAT_INTERVAL. In
-			 * an autovacuum worker, however, we want a lower delay to avoid
-			 * using stale data, so we use PGSTAT_RETRY_DELAY (since the
-			 * number of workers is low, this shouldn't be a problem).
-			 *
-			 * We don't recompute min_ts after sleeping, except in the
-			 * unlikely case that cur_ts went backwards.  So we might end up
-			 * accepting a file a bit older than PGSTAT_STAT_INTERVAL.  In
-			 * practice that shouldn't happen, though, as long as the sleep
-			 * time is less than PGSTAT_STAT_INTERVAL; and we don't want to
-			 * tell the collector that our cutoff time is less than what we'd
-			 * actually accept.
-			 */
-			ref_ts = cur_ts;
-			if (IsAutoVacuumWorkerProcess())
-				min_ts = TimestampTzPlusMilliseconds(ref_ts,
-													 -PGSTAT_RETRY_DELAY);
-			else
-				min_ts = TimestampTzPlusMilliseconds(ref_ts,
-													 -PGSTAT_STAT_INTERVAL);
-		}
-
-		/*
-		 * If the file timestamp is actually newer than cur_ts, we must have
-		 * had a clock glitch (system time went backwards) or there is clock
-		 * skew between our processor and the stats collector's processor.
-		 * Accept the file, but send an inquiry message anyway to make
-		 * pgstat_recv_inquiry do a sanity check on the collector's time.
-		 */
-		if (ok && file_ts > cur_ts)
-		{
-			/*
-			 * A small amount of clock skew between processors isn't terribly
-			 * surprising, but a large difference is worth logging.  We
-			 * arbitrarily define "large" as 1000 msec.
-			 */
-			if (file_ts >= TimestampTzPlusMilliseconds(cur_ts, 1000))
-			{
-				char	   *filetime;
-				char	   *mytime;
-
-				/* Copy because timestamptz_to_str returns a static buffer */
-				filetime = pstrdup(timestamptz_to_str(file_ts));
-				mytime = pstrdup(timestamptz_to_str(cur_ts));
-				ereport(LOG,
-						(errmsg("statistics collector's time %s is later than backend local time %s",
-								filetime, mytime)));
-				pfree(filetime);
-				pfree(mytime);
-			}
-
-			pgstat_send_inquiry(cur_ts, min_ts, inquiry_db);
-			break;
-		}
-
-		/* Normal acceptance case: file is not older than cutoff time */
-		if (ok && file_ts >= min_ts)
-			break;
-
-		/* Not there or too old, so kick the collector and wait a bit */
-		if ((count % PGSTAT_INQ_LOOP_COUNT) == 0)
-			pgstat_send_inquiry(cur_ts, min_ts, inquiry_db);
-
-		pg_usleep(PGSTAT_RETRY_DELAY * 1000L);
+		pgStatSharedRefHash =
+			pgstat_shared_ref_hash_create(pgStatSharedRefContext,
+										  PGSTAT_TABLE_HASH_SIZE, NULL);
+		pgStatSharedRefAge =
+			pg_atomic_read_u64(&pgStatShmem->gc_count);
 	}
 
-	if (count >= PGSTAT_POLL_LOOP_COUNT)
-		ereport(LOG,
-				(errmsg("using stale statistics instead of current ones "
-						"because stats collector is not responding")));
-
 	/*
-	 * Autovacuum launcher wants stats about all databases, but a shallow read
-	 * is sufficient.  Regular backends want a deep read for just the tables
-	 * they can see (MyDatabaseId + shared catalogs).
-	 */
-	if (IsAutoVacuumLauncherProcess())
-		pgStatDBHash = pgstat_read_statsfiles(InvalidOid, false, false);
-	else
-		pgStatDBHash = pgstat_read_statsfiles(MyDatabaseId, false, true);
-}
-
-
-/* ----------
- * pgstat_recv_inquiry() -
- *
- *	Process stat inquiry requests.
- * ----------
- */
-static void
-pgstat_recv_inquiry(PgStat_MsgInquiry *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	elog(DEBUG2, "received inquiry for database %u", msg->databaseid);
-
-	/*
-	 * If there's already a write request for this DB, there's nothing to do.
+	 * pgStatSharedRefAge increments quite slowly than the time the
+	 * following loop takes so this is expected to iterate no more than
+	 * twice.
 	 *
-	 * Note that if a request is found, we return early and skip the below
-	 * check for clock skew.  This is okay, since the only way for a DB
-	 * request to be present in the list is that we have been here since the
-	 * last write round.  It seems sufficient to check for clock skew once per
-	 * write round.
+	 * XXX: Why is this a good place to do this?
 	 */
-	if (list_member_oid(pending_write_requests, msg->databaseid))
-		return;
+	while (pgstat_shared_refs_need_gc())
+		pgstat_shared_refs_gc();
 
 	/*
-	 * Check to see if we last wrote this database at a time >= the requested
-	 * cutoff time.  If so, this is a stale request that was generated before
-	 * we updated the DB file, and we don't need to do so again.
-	 *
-	 * If the requestor's local clock time is older than stats_timestamp, we
-	 * should suspect a clock glitch, ie system time going backwards; though
-	 * the more likely explanation is just delayed message receipt.  It is
-	 * worth expending a GetCurrentTimestamp call to be sure, since a large
-	 * retreat in the system clock reading could otherwise cause us to neglect
-	 * to update the stats file for a long time.
+	 * We immediately insert a cache entry, because it avoids 1) multiple
+	 * hashtable lookups in case of a cache miss 2) having to deal with
+	 * out-of-memory errors after incrementing
+	 * PgStatShm_StatEntryHeader->refcount.
 	 */
-	dbentry = pgstat_get_db_entry(msg->databaseid, false);
-	if (dbentry == NULL)
+
+	cache_entry = pgstat_shared_ref_hash_insert(pgStatSharedRefHash, key, &found);
+
+	if (!found || !cache_entry->shared_ref)
+	{
+		PgStatSharedRef *shared_ref;
+
+		cache_entry->shared_ref = shared_ref =
+			MemoryContextAlloc(pgStatSharedRefContext,
+							   sizeof(PgStatSharedRef));
+		shared_ref->shared_stats = NULL;
+		shared_ref->shared_entry = NULL;
+		shared_ref->pending = NULL;
+
+		found = false;
+	}
+	else if (cache_entry->shared_ref->shared_stats == NULL)
+	{
+		Assert(cache_entry->shared_ref->shared_entry == NULL);
+		found = false;
+	}
+	else
+	{
+		PgStatSharedRef *shared_ref PG_USED_FOR_ASSERTS_ONLY;
+
+		shared_ref = cache_entry->shared_ref;
+		Assert(shared_ref->shared_entry != NULL);
+		Assert(shared_ref->shared_stats != NULL);
+
+		Assert(shared_ref->shared_stats->magic == 0xdeadbeef);
+		/* should have at least our reference */
+		Assert(pg_atomic_read_u32(&shared_ref->shared_entry->refcount) > 0);
+	}
+
+	*shared_ref_p = cache_entry->shared_ref;
+	return found;
+}
+
+/*
+ * Get a shared stats reference. If create is true, the shared stats object is
+ * created if it does not exist.
+ */
+PgStatSharedRef *
+pgstat_shared_ref_get(PgStatKind type, Oid dboid, Oid objoid, bool create)
+{
+	PgStatHashKey key;
+	PgStatShmHashEntry *shhashent;
+	PgStatShm_StatEntryHeader *shheader = NULL;
+	PgStatSharedRef *shared_ref;
+	bool		shfound;
+
+	pgstat_assert_is_up();
+
+	key.kind = type;
+	key.dboid = dboid;
+	key.objoid = objoid;
+
+	Assert(pgStatSharedHash != NULL);
+	Assert(!pgStatShmem->is_shutdown);
+
+	/*
+	 * First check the lookup cache hashtable in local memory. If we find a
+	 * match here we can avoid taking locks / contention.
+	 */
+	if (pgstat_shared_ref_get_cached(key, &shared_ref))
+		return shared_ref;
+
+	Assert(shared_ref != NULL);
+
+	/*
+	 * Do a lookup in the hash table first - it's quite likely that the entry
+	 * already exists, and that way we only need a shared lock.
+	 */
+	shhashent = dshash_find(pgStatSharedHash, &key, false);
+
+	if (shhashent)
+		shfound = true;
+	else if (create)
 	{
 		/*
-		 * We have no data for this DB.  Enter a write request anyway so that
-		 * the global stats will get updated.  This is needed to prevent
-		 * backend_read_statsfile from waiting for data that we cannot supply,
-		 * in the case of a new DB that nobody has yet reported any stats for.
-		 * See the behavior of pgstat_read_db_statsfile_timestamp.
+		 * It's possible that somebody created the entry since the above
+		 * lookup, fall through to the same path as before if so.
 		 */
-	}
-	else if (msg->clock_time < dbentry->stats_timestamp)
-	{
-		TimestampTz cur_ts = GetCurrentTimestamp();
-
-		if (cur_ts < dbentry->stats_timestamp)
+		shhashent = dshash_find_or_insert(pgStatSharedHash, &key, &shfound);
+		if (!shfound)
 		{
 			/*
-			 * Sure enough, time went backwards.  Force a new stats file write
-			 * to get back in sync; but first, log a complaint.
+			 * Initialize refcount to 2, (1 marking it as valid, one for the
+			 * local reference). That prevents another backend from freeing
+			 * the entry once we release the lock below. The entry can't be
+			 * freed before the initialization because it can't be found as
+			 * long as we hold the dshash partition lock.
 			 */
-			char	   *writetime;
-			char	   *mytime;
+			shheader = pgstat_shared_stat_entry_init(type, shhashent, 2);
 
-			/* Copy because timestamptz_to_str returns a static buffer */
-			writetime = pstrdup(timestamptz_to_str(dbentry->stats_timestamp));
-			mytime = pstrdup(timestamptz_to_str(cur_ts));
-			ereport(LOG,
-					(errmsg("stats_timestamp %s is later than collector's time %s for database %u",
-							writetime, mytime, dbentry->databaseid)));
-			pfree(writetime);
-			pfree(mytime);
-		}
-		else
-		{
-			/*
-			 * Nope, it's just an old request.  Assuming msg's clock_time is
-			 * >= its cutoff_time, it must be stale, so we can ignore it.
-			 */
-			return;
+			dshash_release_lock(pgStatSharedHash, shhashent);
+
+			shared_ref->shared_stats = shheader;
+			shared_ref->shared_entry = shhashent;
 		}
 	}
-	else if (msg->cutoff_time <= dbentry->stats_timestamp)
+	else
 	{
-		/* Stale request, ignore it */
-		return;
-	}
-
-	/*
-	 * We need to write this DB, so create a request.
-	 */
-	pending_write_requests = lappend_oid(pending_write_requests,
-										 msg->databaseid);
-}
-
-
-/* ----------
- * pgstat_recv_tabstat() -
- *
- *	Count what the backend has done.
- * ----------
- */
-static void
-pgstat_recv_tabstat(PgStat_MsgTabstat *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-	PgStat_StatTabEntry *tabentry;
-	int			i;
-	bool		found;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	/*
-	 * Update database-wide stats.
-	 */
-	dbentry->n_xact_commit += (PgStat_Counter) (msg->m_xact_commit);
-	dbentry->n_xact_rollback += (PgStat_Counter) (msg->m_xact_rollback);
-	dbentry->n_block_read_time += msg->m_block_read_time;
-	dbentry->n_block_write_time += msg->m_block_write_time;
-
-	dbentry->total_session_time += msg->m_session_time;
-	dbentry->total_active_time += msg->m_active_time;
-	dbentry->total_idle_in_xact_time += msg->m_idle_in_xact_time;
-
-	/*
-	 * Process all table entries in the message.
-	 */
-	for (i = 0; i < msg->m_nentries; i++)
-	{
-		PgStat_TableEntry *tabmsg = &(msg->m_entry[i]);
-
-		tabentry = (PgStat_StatTabEntry *) hash_search(dbentry->tables,
-													   (void *) &(tabmsg->t_id),
-													   HASH_ENTER, &found);
-
-		if (!found)
-		{
-			/*
-			 * If it's a new table entry, initialize counters to the values we
-			 * just got.
-			 */
-			tabentry->numscans = tabmsg->t_counts.t_numscans;
-			tabentry->tuples_returned = tabmsg->t_counts.t_tuples_returned;
-			tabentry->tuples_fetched = tabmsg->t_counts.t_tuples_fetched;
-			tabentry->tuples_inserted = tabmsg->t_counts.t_tuples_inserted;
-			tabentry->tuples_updated = tabmsg->t_counts.t_tuples_updated;
-			tabentry->tuples_deleted = tabmsg->t_counts.t_tuples_deleted;
-			tabentry->tuples_hot_updated = tabmsg->t_counts.t_tuples_hot_updated;
-			tabentry->n_live_tuples = tabmsg->t_counts.t_delta_live_tuples;
-			tabentry->n_dead_tuples = tabmsg->t_counts.t_delta_dead_tuples;
-			tabentry->changes_since_analyze = tabmsg->t_counts.t_changed_tuples;
-			tabentry->inserts_since_vacuum = tabmsg->t_counts.t_tuples_inserted;
-			tabentry->blocks_fetched = tabmsg->t_counts.t_blocks_fetched;
-			tabentry->blocks_hit = tabmsg->t_counts.t_blocks_hit;
-
-			tabentry->vacuum_timestamp = 0;
-			tabentry->vacuum_count = 0;
-			tabentry->autovac_vacuum_timestamp = 0;
-			tabentry->autovac_vacuum_count = 0;
-			tabentry->analyze_timestamp = 0;
-			tabentry->analyze_count = 0;
-			tabentry->autovac_analyze_timestamp = 0;
-			tabentry->autovac_analyze_count = 0;
-		}
-		else
-		{
-			/*
-			 * Otherwise add the values to the existing entry.
-			 */
-			tabentry->numscans += tabmsg->t_counts.t_numscans;
-			tabentry->tuples_returned += tabmsg->t_counts.t_tuples_returned;
-			tabentry->tuples_fetched += tabmsg->t_counts.t_tuples_fetched;
-			tabentry->tuples_inserted += tabmsg->t_counts.t_tuples_inserted;
-			tabentry->tuples_updated += tabmsg->t_counts.t_tuples_updated;
-			tabentry->tuples_deleted += tabmsg->t_counts.t_tuples_deleted;
-			tabentry->tuples_hot_updated += tabmsg->t_counts.t_tuples_hot_updated;
-			/*
-			 * If table was truncated/dropped, first reset the live/dead
-			 * counters.
-			 */
-			if (tabmsg->t_counts.t_truncdropped)
-			{
-				tabentry->n_live_tuples = 0;
-				tabentry->n_dead_tuples = 0;
-				tabentry->inserts_since_vacuum = 0;
-			}
-			tabentry->n_live_tuples += tabmsg->t_counts.t_delta_live_tuples;
-			tabentry->n_dead_tuples += tabmsg->t_counts.t_delta_dead_tuples;
-			tabentry->changes_since_analyze += tabmsg->t_counts.t_changed_tuples;
-			tabentry->inserts_since_vacuum += tabmsg->t_counts.t_tuples_inserted;
-			tabentry->blocks_fetched += tabmsg->t_counts.t_blocks_fetched;
-			tabentry->blocks_hit += tabmsg->t_counts.t_blocks_hit;
-		}
-
-		/* Clamp n_live_tuples in case of negative delta_live_tuples */
-		tabentry->n_live_tuples = Max(tabentry->n_live_tuples, 0);
-		/* Likewise for n_dead_tuples */
-		tabentry->n_dead_tuples = Max(tabentry->n_dead_tuples, 0);
+		shfound = false;
 
 		/*
-		 * Add per-table stats to the per-database entry, too.
+		 * If we're not creating, delete the reference again. In all
+		 * likelihood it's just a stats lookup - no point wasting memory for a
+		 * shared ref to nothing...
 		 */
-		dbentry->n_tuples_returned += tabmsg->t_counts.t_tuples_returned;
-		dbentry->n_tuples_fetched += tabmsg->t_counts.t_tuples_fetched;
-		dbentry->n_tuples_inserted += tabmsg->t_counts.t_tuples_inserted;
-		dbentry->n_tuples_updated += tabmsg->t_counts.t_tuples_updated;
-		dbentry->n_tuples_deleted += tabmsg->t_counts.t_tuples_deleted;
-		dbentry->n_blocks_fetched += tabmsg->t_counts.t_blocks_fetched;
-		dbentry->n_blocks_hit += tabmsg->t_counts.t_blocks_hit;
-	}
-}
+		pgstat_shared_ref_release(key, shared_ref);
 
-
-/* ----------
- * pgstat_recv_tabpurge() -
- *
- *	Arrange for dead table removal.
- * ----------
- */
-static void
-pgstat_recv_tabpurge(PgStat_MsgTabpurge *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-	int			i;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, false);
-
-	/*
-	 * No need to purge if we don't even know the database.
-	 */
-	if (!dbentry || !dbentry->tables)
-		return;
-
-	/*
-	 * Process all table entries in the message.
-	 */
-	for (i = 0; i < msg->m_nentries; i++)
-	{
-		/* Remove from hashtable if present; we don't care if it's not. */
-		(void) hash_search(dbentry->tables,
-						   (void *) &(msg->m_tableid[i]),
-						   HASH_REMOVE, NULL);
-	}
-}
-
-
-/* ----------
- * pgstat_recv_dropdb() -
- *
- *	Arrange for dead database removal
- * ----------
- */
-static void
-pgstat_recv_dropdb(PgStat_MsgDropdb *msg, int len)
-{
-	Oid			dbid = msg->m_databaseid;
-	PgStat_StatDBEntry *dbentry;
-
-	/*
-	 * Lookup the database in the hashtable.
-	 */
-	dbentry = pgstat_get_db_entry(dbid, false);
-
-	/*
-	 * If found, remove it (along with the db statfile).
-	 */
-	if (dbentry)
-	{
-		char		statfile[MAXPGPATH];
-
-		get_dbstat_filename(false, false, dbid, statfile, MAXPGPATH);
-
-		elog(DEBUG2, "removing stats file \"%s\"", statfile);
-		unlink(statfile);
-
-		if (dbentry->tables != NULL)
-			hash_destroy(dbentry->tables);
-		if (dbentry->functions != NULL)
-			hash_destroy(dbentry->functions);
-
-		if (hash_search(pgStatDBHash,
-						(void *) &dbid,
-						HASH_REMOVE, NULL) == NULL)
-			ereport(ERROR,
-					(errmsg("database hash table corrupted during cleanup --- abort")));
-	}
-}
-
-
-/* ----------
- * pgstat_recv_resetcounter() -
- *
- *	Reset the statistics for the specified database.
- * ----------
- */
-static void
-pgstat_recv_resetcounter(PgStat_MsgResetcounter *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	/*
-	 * Lookup the database in the hashtable.  Nothing to do if not there.
-	 */
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, false);
-
-	if (!dbentry)
-		return;
-
-	/*
-	 * We simply throw away all the database's table entries by recreating a
-	 * new hash table for them.
-	 */
-	if (dbentry->tables != NULL)
-		hash_destroy(dbentry->tables);
-	if (dbentry->functions != NULL)
-		hash_destroy(dbentry->functions);
-
-	dbentry->tables = NULL;
-	dbentry->functions = NULL;
-
-	/*
-	 * Reset database-level stats, too.  This creates empty hash tables for
-	 * tables and functions.
-	 */
-	reset_dbentry_counters(dbentry);
-}
-
-/* ----------
- * pgstat_recv_resetsharedcounter() -
- *
- *	Reset some shared statistics of the cluster.
- * ----------
- */
-static void
-pgstat_recv_resetsharedcounter(PgStat_MsgResetsharedcounter *msg, int len)
-{
-	if (msg->m_resettarget == RESET_BGWRITER)
-	{
-		/* Reset the global, bgwriter and checkpointer statistics for the cluster. */
-		memset(&globalStats, 0, sizeof(globalStats));
-		globalStats.bgwriter.stat_reset_timestamp = GetCurrentTimestamp();
-	}
-	else if (msg->m_resettarget == RESET_ARCHIVER)
-	{
-		/* Reset the archiver statistics for the cluster. */
-		memset(&archiverStats, 0, sizeof(archiverStats));
-		archiverStats.stat_reset_timestamp = GetCurrentTimestamp();
-	}
-	else if (msg->m_resettarget == RESET_WAL)
-	{
-		/* Reset the WAL statistics for the cluster. */
-		memset(&walStats, 0, sizeof(walStats));
-		walStats.stat_reset_timestamp = GetCurrentTimestamp();
-	}
-
-	/*
-	 * Presumably the sender of this message validated the target, don't
-	 * complain here if it's not valid
-	 */
-}
-
-/* ----------
- * pgstat_recv_resetsinglecounter() -
- *
- *	Reset a statistics for a single object, which may be of current
- *	database or shared across all databases in the cluster.
- * ----------
- */
-static void
-pgstat_recv_resetsinglecounter(PgStat_MsgResetsinglecounter *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	if (IsSharedRelation(msg->m_objectid))
-		dbentry = pgstat_get_db_entry(InvalidOid, false);
-	else
-		dbentry = pgstat_get_db_entry(msg->m_databaseid, false);
-
-	if (!dbentry)
-		return;
-
-	/* Set the reset timestamp for the whole database */
-	dbentry->stat_reset_timestamp = GetCurrentTimestamp();
-
-	/* Remove object if it exists, ignore it if not */
-	if (msg->m_resettype == RESET_TABLE)
-		(void) hash_search(dbentry->tables, (void *) &(msg->m_objectid),
-						   HASH_REMOVE, NULL);
-	else if (msg->m_resettype == RESET_FUNCTION)
-		(void) hash_search(dbentry->functions, (void *) &(msg->m_objectid),
-						   HASH_REMOVE, NULL);
-}
-
-/* ----------
- * pgstat_recv_resetslrucounter() -
- *
- *	Reset some SLRU statistics of the cluster.
- * ----------
- */
-static void
-pgstat_recv_resetslrucounter(PgStat_MsgResetslrucounter *msg, int len)
-{
-	int			i;
-	TimestampTz ts = GetCurrentTimestamp();
-
-	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
-	{
-		/* reset entry with the given index, or all entries (index is -1) */
-		if ((msg->m_index == -1) || (msg->m_index == i))
-		{
-			memset(&slruStats[i], 0, sizeof(slruStats[i]));
-			slruStats[i].stat_reset_timestamp = ts;
-		}
-	}
-}
-
-/* ----------
- * pgstat_recv_resetreplslotcounter() -
- *
- *	Reset some replication slot statistics of the cluster.
- * ----------
- */
-static void
-pgstat_recv_resetreplslotcounter(PgStat_MsgResetreplslotcounter *msg,
-								 int len)
-{
-	PgStat_StatReplSlotEntry *slotent;
-	TimestampTz ts;
-
-	/* Return if we don't have replication slot statistics */
-	if (replSlotStatHash == NULL)
-		return;
-
-	ts = GetCurrentTimestamp();
-	if (msg->clearall)
-	{
-		HASH_SEQ_STATUS sstat;
-
-		hash_seq_init(&sstat, replSlotStatHash);
-		while ((slotent = (PgStat_StatReplSlotEntry *) hash_seq_search(&sstat)) != NULL)
-			pgstat_reset_replslot(slotent, ts);
-	}
-	else
-	{
-		/* Get the slot statistics to reset */
-		slotent = pgstat_get_replslot_entry(msg->m_slotname, false);
-
-		/*
-		 * Nothing to do if the given slot entry is not found.  This could
-		 * happen when the slot with the given name is removed and the
-		 * corresponding statistics entry is also removed before receiving the
-		 * reset message.
-		 */
-		if (!slotent)
-			return;
-
-		/* Reset the stats for the requested replication slot */
-		pgstat_reset_replslot(slotent, ts);
-	}
-}
-
-/* ----------
- * pgstat_recv_resetsubcounter() -
- *
- *	Reset some subscription statistics of the cluster.
- * ----------
- */
-static void
-pgstat_recv_resetsubcounter(PgStat_MsgResetsubcounter *msg, int len)
-{
-	PgStat_StatSubEntry *subentry;
-	TimestampTz ts;
-
-	/* Return if we don't have replication subscription statistics */
-	if (subscriptionStatHash == NULL)
-		return;
-
-	ts = GetCurrentTimestamp();
-	if (!OidIsValid(msg->m_subid))
-	{
-		HASH_SEQ_STATUS sstat;
-
-		/* Clear all subscription counters */
-		hash_seq_init(&sstat, subscriptionStatHash);
-		while ((subentry = (PgStat_StatSubEntry *) hash_seq_search(&sstat)) != NULL)
-			pgstat_reset_subscription(subentry, ts);
-	}
-	else
-	{
-		/* Get the subscription statistics to reset */
-		subentry = pgstat_get_subscription_entry(msg->m_subid, false);
-
-		/*
-		 * Nothing to do if the given subscription entry is not found.  This
-		 * could happen when the subscription with the subid is removed and
-		 * the corresponding statistics entry is also removed before receiving
-		 * the reset message.
-		 */
-		if (!subentry)
-			return;
-
-		/* Reset the stats for the requested subscription */
-		pgstat_reset_subscription(subentry, ts);
-	}
-}
-
-
-/* ----------
- * pgstat_recv_autovac() -
- *
- *	Process an autovacuum signaling message.
- * ----------
- */
-static void
-pgstat_recv_autovac(PgStat_MsgAutovacStart *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	/*
-	 * Store the last autovacuum time in the database's hashtable entry.
-	 */
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	dbentry->last_autovac_time = msg->m_start_time;
-}
-
-/* ----------
- * pgstat_recv_vacuum() -
- *
- *	Process a VACUUM message.
- * ----------
- */
-static void
-pgstat_recv_vacuum(PgStat_MsgVacuum *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-	PgStat_StatTabEntry *tabentry;
-
-	/*
-	 * Store the data in the table's hashtable entry.
-	 */
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	tabentry = pgstat_get_tab_entry(dbentry, msg->m_tableoid, true);
-
-	tabentry->n_live_tuples = msg->m_live_tuples;
-	tabentry->n_dead_tuples = msg->m_dead_tuples;
-
-	/*
-	 * It is quite possible that a non-aggressive VACUUM ended up skipping
-	 * various pages, however, we'll zero the insert counter here regardless.
-	 * It's currently used only to track when we need to perform an "insert"
-	 * autovacuum, which are mainly intended to freeze newly inserted tuples.
-	 * Zeroing this may just mean we'll not try to vacuum the table again
-	 * until enough tuples have been inserted to trigger another insert
-	 * autovacuum.  An anti-wraparound autovacuum will catch any persistent
-	 * stragglers.
-	 */
-	tabentry->inserts_since_vacuum = 0;
-
-	if (msg->m_autovacuum)
-	{
-		tabentry->autovac_vacuum_timestamp = msg->m_vacuumtime;
-		tabentry->autovac_vacuum_count++;
-	}
-	else
-	{
-		tabentry->vacuum_timestamp = msg->m_vacuumtime;
-		tabentry->vacuum_count++;
-	}
-}
-
-/* ----------
- * pgstat_recv_analyze() -
- *
- *	Process an ANALYZE message.
- * ----------
- */
-static void
-pgstat_recv_analyze(PgStat_MsgAnalyze *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-	PgStat_StatTabEntry *tabentry;
-
-	/*
-	 * Store the data in the table's hashtable entry.
-	 */
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	tabentry = pgstat_get_tab_entry(dbentry, msg->m_tableoid, true);
-
-	tabentry->n_live_tuples = msg->m_live_tuples;
-	tabentry->n_dead_tuples = msg->m_dead_tuples;
-
-	/*
-	 * If commanded, reset changes_since_analyze to zero.  This forgets any
-	 * changes that were committed while the ANALYZE was in progress, but we
-	 * have no good way to estimate how many of those there were.
-	 */
-	if (msg->m_resetcounter)
-		tabentry->changes_since_analyze = 0;
-
-	if (msg->m_autovacuum)
-	{
-		tabentry->autovac_analyze_timestamp = msg->m_analyzetime;
-		tabentry->autovac_analyze_count++;
-	}
-	else
-	{
-		tabentry->analyze_timestamp = msg->m_analyzetime;
-		tabentry->analyze_count++;
-	}
-}
-
-
-/* ----------
- * pgstat_recv_archiver() -
- *
- *	Process a ARCHIVER message.
- * ----------
- */
-static void
-pgstat_recv_archiver(PgStat_MsgArchiver *msg, int len)
-{
-	if (msg->m_failed)
-	{
-		/* Failed archival attempt */
-		++archiverStats.failed_count;
-		memcpy(archiverStats.last_failed_wal, msg->m_xlog,
-			   sizeof(archiverStats.last_failed_wal));
-		archiverStats.last_failed_timestamp = msg->m_timestamp;
-	}
-	else
-	{
-		/* Successful archival operation */
-		++archiverStats.archived_count;
-		memcpy(archiverStats.last_archived_wal, msg->m_xlog,
-			   sizeof(archiverStats.last_archived_wal));
-		archiverStats.last_archived_timestamp = msg->m_timestamp;
-	}
-}
-
-/* ----------
- * pgstat_recv_bgwriter() -
- *
- *	Process a BGWRITER message.
- * ----------
- */
-static void
-pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len)
-{
-	globalStats.bgwriter.buf_written_clean += msg->m_buf_written_clean;
-	globalStats.bgwriter.maxwritten_clean += msg->m_maxwritten_clean;
-	globalStats.bgwriter.buf_alloc += msg->m_buf_alloc;
-}
-
-/* ----------
- * pgstat_recv_checkpointer() -
- *
- *	Process a CHECKPOINTER message.
- * ----------
- */
-static void
-pgstat_recv_checkpointer(PgStat_MsgCheckpointer *msg, int len)
-{
-	globalStats.checkpointer.timed_checkpoints += msg->m_timed_checkpoints;
-	globalStats.checkpointer.requested_checkpoints += msg->m_requested_checkpoints;
-	globalStats.checkpointer.checkpoint_write_time += msg->m_checkpoint_write_time;
-	globalStats.checkpointer.checkpoint_sync_time += msg->m_checkpoint_sync_time;
-	globalStats.checkpointer.buf_written_checkpoints += msg->m_buf_written_checkpoints;
-	globalStats.checkpointer.buf_written_backend += msg->m_buf_written_backend;
-	globalStats.checkpointer.buf_fsync_backend += msg->m_buf_fsync_backend;
-}
-
-/* ----------
- * pgstat_recv_wal() -
- *
- *	Process a WAL message.
- * ----------
- */
-static void
-pgstat_recv_wal(PgStat_MsgWal *msg, int len)
-{
-	walStats.wal_records += msg->m_wal_records;
-	walStats.wal_fpi += msg->m_wal_fpi;
-	walStats.wal_bytes += msg->m_wal_bytes;
-	walStats.wal_buffers_full += msg->m_wal_buffers_full;
-	walStats.wal_write += msg->m_wal_write;
-	walStats.wal_sync += msg->m_wal_sync;
-	walStats.wal_write_time += msg->m_wal_write_time;
-	walStats.wal_sync_time += msg->m_wal_sync_time;
-}
-
-/* ----------
- * pgstat_recv_slru() -
- *
- *	Process a SLRU message.
- * ----------
- */
-static void
-pgstat_recv_slru(PgStat_MsgSLRU *msg, int len)
-{
-	slruStats[msg->m_index].blocks_zeroed += msg->m_blocks_zeroed;
-	slruStats[msg->m_index].blocks_hit += msg->m_blocks_hit;
-	slruStats[msg->m_index].blocks_read += msg->m_blocks_read;
-	slruStats[msg->m_index].blocks_written += msg->m_blocks_written;
-	slruStats[msg->m_index].blocks_exists += msg->m_blocks_exists;
-	slruStats[msg->m_index].flush += msg->m_flush;
-	slruStats[msg->m_index].truncate += msg->m_truncate;
-}
-
-/* ----------
- * pgstat_recv_recoveryconflict() -
- *
- *	Process a RECOVERYCONFLICT message.
- * ----------
- */
-static void
-pgstat_recv_recoveryconflict(PgStat_MsgRecoveryConflict *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	switch (msg->m_reason)
-	{
-		case PROCSIG_RECOVERY_CONFLICT_DATABASE:
-
-			/*
-			 * Since we drop the information about the database as soon as it
-			 * replicates, there is no point in counting these conflicts.
-			 */
-			break;
-		case PROCSIG_RECOVERY_CONFLICT_TABLESPACE:
-			dbentry->n_conflict_tablespace++;
-			break;
-		case PROCSIG_RECOVERY_CONFLICT_LOCK:
-			dbentry->n_conflict_lock++;
-			break;
-		case PROCSIG_RECOVERY_CONFLICT_SNAPSHOT:
-			dbentry->n_conflict_snapshot++;
-			break;
-		case PROCSIG_RECOVERY_CONFLICT_BUFFERPIN:
-			dbentry->n_conflict_bufferpin++;
-			break;
-		case PROCSIG_RECOVERY_CONFLICT_STARTUP_DEADLOCK:
-			dbentry->n_conflict_startup_deadlock++;
-			break;
-	}
-}
-
-/* ----------
- * pgstat_recv_deadlock() -
- *
- *	Process a DEADLOCK message.
- * ----------
- */
-static void
-pgstat_recv_deadlock(PgStat_MsgDeadlock *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	dbentry->n_deadlocks++;
-}
-
-/* ----------
- * pgstat_recv_checksum_failure() -
- *
- *	Process a CHECKSUMFAILURE message.
- * ----------
- */
-static void
-pgstat_recv_checksum_failure(PgStat_MsgChecksumFailure *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	dbentry->n_checksum_failures += msg->m_failurecount;
-	dbentry->last_checksum_failure = msg->m_failure_time;
-}
-
-/* ----------
- * pgstat_recv_replslot() -
- *
- *	Process a REPLSLOT message.
- * ----------
- */
-static void
-pgstat_recv_replslot(PgStat_MsgReplSlot *msg, int len)
-{
-	if (msg->m_drop)
-	{
-		Assert(!msg->m_create);
-
-		/* Remove the replication slot statistics with the given name */
-		if (replSlotStatHash != NULL)
-			(void) hash_search(replSlotStatHash,
-							   (void *) &(msg->m_slotname),
-							   HASH_REMOVE,
-							   NULL);
-	}
-	else
-	{
-		PgStat_StatReplSlotEntry *slotent;
-
-		slotent = pgstat_get_replslot_entry(msg->m_slotname, true);
-		Assert(slotent);
-
-		if (msg->m_create)
-		{
-			/*
-			 * If the message for dropping the slot with the same name gets
-			 * lost, slotent has stats for the old slot. So we initialize all
-			 * counters at slot creation.
-			 */
-			pgstat_reset_replslot(slotent, 0);
-		}
-		else
-		{
-			/* Update the replication slot statistics */
-			slotent->spill_txns += msg->m_spill_txns;
-			slotent->spill_count += msg->m_spill_count;
-			slotent->spill_bytes += msg->m_spill_bytes;
-			slotent->stream_txns += msg->m_stream_txns;
-			slotent->stream_count += msg->m_stream_count;
-			slotent->stream_bytes += msg->m_stream_bytes;
-			slotent->total_txns += msg->m_total_txns;
-			slotent->total_bytes += msg->m_total_bytes;
-		}
-	}
-}
-
-/* ----------
- * pgstat_recv_connect() -
- *
- *	Process a CONNECT message.
- * ----------
- */
-static void
-pgstat_recv_connect(PgStat_MsgConnect *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-	dbentry->n_sessions++;
-}
-
-/* ----------
- * pgstat_recv_disconnect() -
- *
- *	Process a DISCONNECT message.
- * ----------
- */
-static void
-pgstat_recv_disconnect(PgStat_MsgDisconnect *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	switch (msg->m_cause)
-	{
-		case DISCONNECT_NOT_YET:
-		case DISCONNECT_NORMAL:
-			/* we don't collect these */
-			break;
-		case DISCONNECT_CLIENT_EOF:
-			dbentry->n_sessions_abandoned++;
-			break;
-		case DISCONNECT_FATAL:
-			dbentry->n_sessions_fatal++;
-			break;
-		case DISCONNECT_KILLED:
-			dbentry->n_sessions_killed++;
-			break;
-	}
-}
-
-/* ----------
- * pgstat_recv_tempfile() -
- *
- *	Process a TEMPFILE message.
- * ----------
- */
-static void
-pgstat_recv_tempfile(PgStat_MsgTempFile *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	dbentry->n_temp_bytes += msg->m_filesize;
-	dbentry->n_temp_files += 1;
-}
-
-/* ----------
- * pgstat_recv_funcstat() -
- *
- *	Count what the backend has done.
- * ----------
- */
-static void
-pgstat_recv_funcstat(PgStat_MsgFuncstat *msg, int len)
-{
-	PgStat_FunctionEntry *funcmsg = &(msg->m_entry[0]);
-	PgStat_StatDBEntry *dbentry;
-	PgStat_StatFuncEntry *funcentry;
-	int			i;
-	bool		found;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, true);
-
-	/*
-	 * Process all function entries in the message.
-	 */
-	for (i = 0; i < msg->m_nentries; i++, funcmsg++)
-	{
-		funcentry = (PgStat_StatFuncEntry *) hash_search(dbentry->functions,
-														 (void *) &(funcmsg->f_id),
-														 HASH_ENTER, &found);
-
-		if (!found)
-		{
-			/*
-			 * If it's a new function entry, initialize counters to the values
-			 * we just got.
-			 */
-			funcentry->f_numcalls = funcmsg->f_numcalls;
-			funcentry->f_total_time = funcmsg->f_total_time;
-			funcentry->f_self_time = funcmsg->f_self_time;
-		}
-		else
-		{
-			/*
-			 * Otherwise add the values to the existing entry.
-			 */
-			funcentry->f_numcalls += funcmsg->f_numcalls;
-			funcentry->f_total_time += funcmsg->f_total_time;
-			funcentry->f_self_time += funcmsg->f_self_time;
-		}
-	}
-}
-
-/* ----------
- * pgstat_recv_funcpurge() -
- *
- *	Arrange for dead function removal.
- * ----------
- */
-static void
-pgstat_recv_funcpurge(PgStat_MsgFuncpurge *msg, int len)
-{
-	PgStat_StatDBEntry *dbentry;
-	int			i;
-
-	dbentry = pgstat_get_db_entry(msg->m_databaseid, false);
-
-	/*
-	 * No need to purge if we don't even know the database.
-	 */
-	if (!dbentry || !dbentry->functions)
-		return;
-
-	/*
-	 * Process all function entries in the message.
-	 */
-	for (i = 0; i < msg->m_nentries; i++)
-	{
-		/* Remove from hashtable if present; we don't care if it's not. */
-		(void) hash_search(dbentry->functions,
-						   (void *) &(msg->m_functionid[i]),
-						   HASH_REMOVE, NULL);
-	}
-}
-
-/* ----------
- * pgstat_recv_subscription_drop() -
- *
- *	Process a SUBSCRIPTIONDROP message.
- * ----------
- */
-static void
-pgstat_recv_subscription_drop(PgStat_MsgSubscriptionDrop *msg, int len)
-{
-	/* Return if we don't have replication subscription statistics */
-	if (subscriptionStatHash == NULL)
-		return;
-
-	/* Remove from hashtable if present; we don't care if it's not */
-	(void) hash_search(subscriptionStatHash, (void *) &(msg->m_subid),
-					   HASH_REMOVE, NULL);
-}
-
-/* ----------
- * pgstat_recv_subscription_error() -
- *
- *	Process a SUBSCRIPTIONERROR message.
- * ----------
- */
-static void
-pgstat_recv_subscription_error(PgStat_MsgSubscriptionError *msg, int len)
-{
-	PgStat_StatSubEntry *subentry;
-
-	/* Get the subscription stats */
-	subentry = pgstat_get_subscription_entry(msg->m_subid, true);
-	Assert(subentry);
-
-	if (msg->m_is_apply_error)
-		subentry->apply_error_count++;
-	else
-		subentry->sync_error_count++;
-}
-
-/* ----------
- * pgstat_write_statsfile_needed() -
- *
- *	Do we need to write out any stats files?
- * ----------
- */
-static bool
-pgstat_write_statsfile_needed(void)
-{
-	if (pending_write_requests != NIL)
-		return true;
-
-	/* Everything was written recently */
-	return false;
-}
-
-/* ----------
- * pgstat_db_requested() -
- *
- *	Checks whether stats for a particular DB need to be written to a file.
- * ----------
- */
-static bool
-pgstat_db_requested(Oid databaseid)
-{
-	/*
-	 * If any requests are outstanding at all, we should write the stats for
-	 * shared catalogs (the "database" with OID 0).  This ensures that
-	 * backends will see up-to-date stats for shared catalogs, even though
-	 * they send inquiry messages mentioning only their own DB.
-	 */
-	if (databaseid == InvalidOid && pending_write_requests != NIL)
-		return true;
-
-	/* Search to see if there's an open request to write this database. */
-	if (list_member_oid(pending_write_requests, databaseid))
-		return true;
-
-	return false;
-}
-
-/* ----------
- * pgstat_replslot_entry
- *
- * Return the entry of replication slot stats with the given name. Return
- * NULL if not found and the caller didn't request to create it.
- *
- * create tells whether to create the new slot entry if it is not found.
- * ----------
- */
-static PgStat_StatReplSlotEntry *
-pgstat_get_replslot_entry(NameData name, bool create)
-{
-	PgStat_StatReplSlotEntry *slotent;
-	bool		found;
-
-	if (replSlotStatHash == NULL)
-	{
-		HASHCTL		hash_ctl;
-
-		/*
-		 * Quick return NULL if the hash table is empty and the caller didn't
-		 * request to create the entry.
-		 */
-		if (!create)
-			return NULL;
-
-		hash_ctl.keysize = sizeof(NameData);
-		hash_ctl.entrysize = sizeof(PgStat_StatReplSlotEntry);
-		replSlotStatHash = hash_create("Replication slots hash",
-									   PGSTAT_REPLSLOT_HASH_SIZE,
-									   &hash_ctl,
-									   HASH_ELEM | HASH_BLOBS);
-	}
-
-	slotent = (PgStat_StatReplSlotEntry *) hash_search(replSlotStatHash,
-													   (void *) &name,
-													   create ? HASH_ENTER : HASH_FIND,
-													   &found);
-
-	if (!slotent)
-	{
-		/* not found */
-		Assert(!create && !found);
 		return NULL;
 	}
 
-	/* initialize the entry */
-	if (create && !found)
+	if (shfound)
 	{
-		namestrcpy(&(slotent->slotname), NameStr(name));
-		pgstat_reset_replslot(slotent, 0);
+		shheader = dsa_get_address(pgStatDSA, shhashent->body);
+
+		Assert(shheader->magic == 0xdeadbeef);
+		Assert(shhashent->dropped || pg_atomic_read_u32(&shhashent->refcount) > 0);
+
+		pg_atomic_fetch_add_u32(&shhashent->refcount, 1);
+
+		dshash_release_lock(pgStatSharedHash, shhashent);
+
+		shared_ref->shared_stats = shheader;
+		shared_ref->shared_entry = shhashent;
 	}
 
-	return slotent;
+	return shared_ref;
 }
 
-/* ----------
- * pgstat_reset_replslot
- *
- * Reset the given replication slot stats.
- * ----------
- */
 static void
-pgstat_reset_replslot(PgStat_StatReplSlotEntry *slotent, TimestampTz ts)
+pgstat_shared_ref_release(PgStatHashKey key, PgStatSharedRef *shared_ref)
 {
-	/* reset only counters. Don't clear slot name */
-	slotent->spill_txns = 0;
-	slotent->spill_count = 0;
-	slotent->spill_bytes = 0;
-	slotent->stream_txns = 0;
-	slotent->stream_count = 0;
-	slotent->stream_bytes = 0;
-	slotent->total_txns = 0;
-	slotent->total_bytes = 0;
-	slotent->stat_reset_timestamp = ts;
-}
+	Assert(shared_ref == NULL || shared_ref->pending == NULL);
 
-/* ----------
- * pgstat_get_subscription_entry
- *
- * Return the subscription statistics entry with the given subscription OID.
- * If no subscription entry exists, initialize it, if the create parameter is
- * true.  Else, return NULL.
- * ----------
- */
-static PgStat_StatSubEntry *
-pgstat_get_subscription_entry(Oid subid, bool create)
-{
-	PgStat_StatSubEntry *subentry;
-	bool		found;
-	HASHACTION	action = (create ? HASH_ENTER : HASH_FIND);
-
-	if (subscriptionStatHash == NULL)
+	if (shared_ref && shared_ref->shared_stats)
 	{
-		HASHCTL		hash_ctl;
+		Assert(shared_ref->shared_stats->magic == 0xdeadbeef);
+		Assert(shared_ref->pending == NULL);
 
 		/*
-		 * Quick return NULL if the hash table is empty and the caller didn't
-		 * request to create the entry.
+		 * AFIXME: this probably is racy. Another backend could look up the
+		 * stat, bump the refcount, as we free it.
 		 */
-		if (!create)
-			return NULL;
+		if (pg_atomic_fetch_sub_u32(&shared_ref->shared_entry->refcount, 1) == 1)
+		{
+			PgStatShmHashEntry *shent;
+			dsa_pointer dsap;
 
-		hash_ctl.keysize = sizeof(Oid);
-		hash_ctl.entrysize = sizeof(PgStat_StatSubEntry);
-		subscriptionStatHash = hash_create("Subscription hash",
-										   PGSTAT_SUBSCRIPTION_HASH_SIZE,
-										   &hash_ctl,
-										   HASH_ELEM | HASH_BLOBS);
+			/*
+			 * We're the last referrer to this entry, try to drop the shared
+			 * entry.
+			 */
+
+			/* only dropped entries can reach a 0 refcount */
+			Assert(shared_ref->shared_entry->dropped);
+
+			shent = dshash_find(pgStatSharedHash,
+								&shared_ref->shared_entry->key,
+								true);
+			if (!shent)
+				elog(PANIC, "could not find just referenced shared stats entry");
+
+			if (pg_atomic_read_u32(&shared_ref->shared_entry->refcount) != 0)
+				elog(PANIC, "concurrent access to stats entry during deletion");
+
+			Assert(shared_ref->shared_entry == shent);
+
+			/*
+			 * Fetch dsa pointer before deleting entry - that way we can free the
+			 * memory after releasing the lock.
+			 */
+			dsap = shent->body;
+
+			dshash_delete_entry(pgStatSharedHash, shent);
+
+			dsa_free(pgStatDSA, dsap);
+			shared_ref->shared_stats = NULL;
+		}
 	}
 
-	subentry = (PgStat_StatSubEntry *) hash_search(subscriptionStatHash,
-												   (void *) &subid,
-												   action, &found);
+	if (!pgstat_shared_ref_hash_delete(pgStatSharedRefHash, key))
+		elog(PANIC, "something has gone wrong");
 
-	if (!create && !found)
+	if (shared_ref)
+		pfree(shared_ref);
+}
+
+bool
+pgstat_shared_stat_lock(PgStatSharedRef *shared_ref, bool nowait)
+{
+	LWLock *lock = &shared_ref->shared_entry->lock;
+
+	if (nowait)
+		return LWLockConditionalAcquire(lock, LW_EXCLUSIVE);
+
+	LWLockAcquire(lock, LW_EXCLUSIVE);
+	return true;
+}
+
+void
+pgstat_shared_stat_unlock(PgStatSharedRef *shared_ref)
+{
+	LWLockRelease(&shared_ref->shared_entry->lock);
+}
+
+/*
+ * Helper function to fetch and lock shared stats.
+ */
+PgStatSharedRef *
+pgstat_shared_stat_locked(PgStatKind type, Oid dboid, Oid objoid, bool nowait)
+{
+	PgStatSharedRef *shared_ref;
+
+	/* find shared table stats entry corresponding to the local entry */
+	shared_ref = pgstat_shared_ref_get(type, dboid, objoid, true);
+
+	/* lock the shared entry to protect the content, skip if failed */
+	if (!pgstat_shared_stat_lock(shared_ref, nowait))
 		return NULL;
 
-	/* If not found, initialize the new one */
-	if (!found)
-		pgstat_reset_subscription(subentry, 0);
-
-	return subentry;
+	return shared_ref;
 }
 
-/* ----------
- * pgstat_reset_subscription
+/*
+ * The length of the data portion of a shared memory stats entry (i.e. without
+ * transient data such as refcoutns, lwlocks, ...).
+ */
+static inline size_t
+shared_stat_entry_len(PgStatKind kind)
+{
+	size_t		sz = pgstat_kind_infos[kind].shared_data_len;
+
+	AssertArg(kind <= PGSTAT_KIND_LAST);
+	Assert(sz != 0 && sz < PG_UINT32_MAX);
+
+	return sz;
+}
+
+/*
+ * Returns a pointer to the data portion of a shared memory stats entry.
+ */
+static inline void*
+shared_stat_entry_data(PgStatKind kind, PgStatShm_StatEntryHeader *entry)
+{
+	size_t		off = pgstat_kind_infos[kind].shared_data_off;
+
+	AssertArg(kind <= PGSTAT_KIND_LAST);
+	Assert(off != 0 && off < PG_UINT32_MAX);
+
+	return ((char *)(entry)) + off;
+}
+
+static bool
+pgstat_shared_refs_need_gc(void)
+{
+	uint64		currage;
+
+	if (!pgStatSharedRefHash)
+		return false;
+
+	currage = pg_atomic_read_u64(&pgStatShmem->gc_count);
+
+	return pgStatSharedRefAge != currage;
+}
+
+static void
+pgstat_shared_refs_gc(void)
+{
+	pgstat_shared_ref_hash_iterator i;
+	PgStatSharedRefHashEntry *ent;
+	uint64		currage;
+
+	currage = pg_atomic_read_u64(&pgStatShmem->gc_count);
+
+	/*
+	 * Some entries have been dropped. Invalidate cache pointer to
+	 * them.
+	 */
+	pgstat_shared_ref_hash_start_iterate(pgStatSharedRefHash, &i);
+	while ((ent = pgstat_shared_ref_hash_iterate(pgStatSharedRefHash, &i)) != NULL)
+	{
+		PgStatSharedRef *shared_ref = ent->shared_ref;
+
+		Assert(!shared_ref->shared_stats || shared_ref->shared_stats->magic == 0xdeadbeef);
+
+		/* cannot gc shared ref that has pending data */
+		if (shared_ref->pending != NULL)
+			continue;
+
+		if (shared_ref->shared_stats && shared_ref->shared_entry->dropped)
+			pgstat_shared_ref_release(ent->key, shared_ref);
+	}
+
+	pgStatSharedRefAge = currage;
+}
+
+/*
+ * Release all local references to shared stats entries.
  *
- * Reset the given subscription stats.
- * ----------
+ * When a process exits it cannot do so while still holding references onto
+ * stats entries, otherwise the shared stats entries could never be freed.
  */
 static void
-pgstat_reset_subscription(PgStat_StatSubEntry *subentry, TimestampTz ts)
+pgstat_shared_refs_release_all(void)
 {
-	subentry->apply_error_count = 0;
-	subentry->sync_error_count = 0;
-	subentry->stat_reset_timestamp = ts;
+	pgstat_shared_ref_hash_iterator i;
+	PgStatSharedRefHashEntry *ent;
+
+	if (pgStatSharedRefHash == NULL)
+		return;
+
+	pgstat_shared_ref_hash_start_iterate(pgStatSharedRefHash, &i);
+
+	while ((ent = pgstat_shared_ref_hash_iterate(pgStatSharedRefHash, &i))
+		   != NULL)
+	{
+		Assert(ent->shared_ref != NULL);
+
+		pgstat_shared_ref_release(ent->key, ent->shared_ref);
+	}
+
+	Assert(pgStatSharedRefHash->members == 0);
+	pgstat_shared_ref_hash_destroy(pgStatSharedRefHash);
+	pgStatSharedRefHash = NULL;
+}
+
+/*
+ * Returns the appropriate PgStatSharedRef, preparing it to receive pending
+ * stats if not already done.
+ */
+PgStatSharedRef*
+pgstat_pending_prepare(PgStatKind type, Oid dboid, Oid objoid)
+{
+	PgStatSharedRef *shared_ref;
+
+	shared_ref = pgstat_shared_ref_get(type, dboid, objoid, true);
+
+	if (shared_ref->pending == NULL)
+	{
+		size_t entrysize = pgstat_kind_infos[type].pending_size;
+
+		Assert(entrysize != (size_t)-1);
+
+		shared_ref->pending = MemoryContextAllocZero(TopMemoryContext, entrysize);
+		dlist_push_tail(&pgStatPending, &shared_ref->pending_node);
+	}
+
+	return shared_ref;
+}
+
+/*
+ * Return an existing stats entry, or NULL.
+ *
+ * This should only be used for helper function for pgstatfuncs.c - outside of
+ * that it shouldn't be needed.
+ */
+PgStatSharedRef*
+pgstat_pending_fetch(PgStatKind type, Oid dboid, Oid objoid)
+{
+	PgStatSharedRef *shared_ref;
+
+	shared_ref = pgstat_shared_ref_get(type, dboid, objoid, false);
+
+	if (shared_ref == NULL || shared_ref->pending == NULL)
+		return NULL;
+
+	return shared_ref;
+}
+
+static void
+pgstat_pending_delete(PgStatSharedRef *shared_ref)
+{
+	void *pending_data = shared_ref->pending;
+	PgStatKind kind;
+
+	Assert(pending_data != NULL);
+
+	/* FIXME: Move into a pgstat_kind_info callback */
+	kind = shared_ref->shared_entry->key.kind;
+	switch (kind)
+	{
+		case PGSTAT_KIND_TABLE:
+			pgstat_relation_delink(((PgStat_TableStatus *) pending_data)->relation);
+			break;
+		case PGSTAT_KIND_DB:
+		case PGSTAT_KIND_FUNCTION:
+		case PGSTAT_KIND_SUBSCRIPTION:
+			break;
+		default:
+			/* non-global stats should be explicitly handled */
+			Assert(pgstat_kind_infos[kind].is_global);
+			elog(ERROR, "unexpected");
+			break;
+	}
+
+	pfree(pending_data);
+	shared_ref->pending = NULL;
+
+	dlist_delete(&shared_ref->pending_node);
+}
+
+/*
+ * Flush out pending stats for database objects (databases, relations,
+ * functions).
+ */
+static bool
+pgstat_pending_flush_stats(bool nowait)
+{
+	bool		have_pending = false;
+	dlist_node *cur = NULL;
+
+	/*
+	 * Need to be a bit careful iterating over the list of pending
+	 * entries. Processing a pending entry may queue further pending entries
+	 * to the end of the list that we want to process, so a simple iteration
+	 * won't do. Further complicating matter is that we want to delete the
+	 * current entry in each iteration from the list if we flushed
+	 * successfully.
+	 *
+	 * So we just keep track of the next pointer in each loop iteration.
+	 */
+	if (!dlist_is_empty(&pgStatPending))
+		cur = dlist_head_node(&pgStatPending);
+
+	while (cur)
+	{
+		PgStatSharedRef *shared_ref
+			= dlist_container(PgStatSharedRef, pending_node, cur);
+		PgStatHashKey key;
+		bool		remove;
+		dlist_node *next;
+
+		key = shared_ref->shared_entry->key;
+
+		Assert(!pgstat_kind_infos[key.kind].is_global);
+		Assert(pgstat_kind_infos[key.kind].flush_pending_cb != NULL);
+
+		/* flush the stats, if possible */
+		remove = pgstat_kind_infos[key.kind].flush_pending_cb(shared_ref, nowait);
+
+		Assert(remove || nowait);
+
+		/* determine next entry, before deleting the pending entry */
+		if (dlist_has_next(&pgStatPending, cur))
+			next = dlist_next_node(&pgStatPending, cur);
+		else
+			next = NULL;
+
+		/* if successfully flushed, remove entry */
+		if (remove)
+			pgstat_pending_delete(shared_ref);
+		else
+			have_pending = true;
+
+		cur = next;
+	}
+
+	Assert(dlist_is_empty(&pgStatPending) == !have_pending);
+
+	return have_pending;
+}
+
+void
+pgstat_schedule_create(PgStatKind kind, Oid dboid, Oid objoid)
+{
+	int			nest_level = GetCurrentTransactionNestLevel();
+	PgStat_SubXactStatus *xact_state;
+	PgStat_PendingDroppedStatsItem *drop = (PgStat_PendingDroppedStatsItem *)
+		MemoryContextAlloc(TopTransactionContext, sizeof(PgStat_PendingDroppedStatsItem));
+
+	xact_state = pgstat_xact_stack_level_get(nest_level);
+
+	drop->is_create = true;
+	drop->item.kind = kind;
+	drop->item.dboid = dboid;
+	drop->item.objoid = objoid;
+
+	dlist_push_tail(&xact_state->pending_drops, &drop->node);
+	xact_state->pending_drops_count++;
+
+	/*
+	 * AFIXME: It would be a good idea to check if an object with that key
+	 * already exists. WARN if so, and reset the stats to 0.
+	 */
+}
+
+/*
+ * AFIXME: deduplicate with pgstat_schedule_create
+ * AFIXME: comment
+ * AFIXME: see notes about race conditions for functions in
+ *         pgstat_drop_function().
+ */
+void
+pgstat_schedule_drop(PgStatKind kind, Oid dboid, Oid objoid)
+{
+	int			nest_level = GetCurrentTransactionNestLevel();
+	PgStat_SubXactStatus *xact_state;
+	PgStat_PendingDroppedStatsItem *drop = (PgStat_PendingDroppedStatsItem *)
+		MemoryContextAlloc(TopTransactionContext, sizeof(PgStat_PendingDroppedStatsItem));
+
+	xact_state = pgstat_xact_stack_level_get(nest_level);
+
+	drop->is_create = false;
+	drop->item.kind = kind;
+	drop->item.dboid = dboid;
+	drop->item.objoid = objoid;
+
+	dlist_push_tail(&xact_state->pending_drops, &drop->node);
+	xact_state->pending_drops_count++;
+}
+
+/*
+ * Drop stats for the database and all the objects inside that database.
+ */
+void
+pgstat_drop_database_and_contents(Oid dboid)
+{
+	dshash_seq_status hstat;
+	PgStatShmHashEntry *p;
+	uint64		not_freed_count = 0;
+
+	Assert(OidIsValid(dboid));
+
+	Assert(pgStatSharedHash != NULL);
+
+	/* some of the dshash entries are to be removed, take exclusive lock. */
+	dshash_seq_init(&hstat, pgStatSharedHash, true);
+	while ((p = dshash_seq_next(&hstat)) != NULL)
+	{
+		if (p->dropped)
+			continue;
+
+		if (p->key.dboid == dboid)
+		{
+			/*
+			 * Even statistics for a dropped database might currently be
+			 * accessed (consider e.g. database stats for pg_stat_database).
+			 */
+			if (!pgstat_drop_stats_entry(&hstat))
+				not_freed_count++;
+		}
+	}
+	dshash_seq_term(&hstat);
+
+	/*
+	 * If some of the stats data could not be freed, signal the reference
+	 * holders to run garbage collection of their cached pgStatShmLookupCache.
+	 */
+	if (not_freed_count > 0)
+		pg_atomic_fetch_add_u64(&pgStatShmem->gc_count, 1);
+}
+
+/*
+ * Drop a shared stats entry. The entry must be exclusively locked.
+ *
+ * This marks the shared entry as dropped. However, the shared hash table
+ * entry and the stats entry are only deleted if there are no remaining
+ * references.
+ *
+ * Returns whether the stats data could be freed or not.
+ *
+ * AFIXME: This needs to be deduplicated with pgstat_shared_ref_release(). But
+ * it's not entirely trivial, because we can't use plain dshash_delete_entry()
+ * (but have to use dshash_delete_current()).
+ */
+static bool
+pgstat_drop_stats_entry(dshash_seq_status *hstat)
+{
+	PgStatShmHashEntry *ent;
+	PgStatHashKey key;
+	dsa_pointer pdsa;
+	bool		did_free;
+
+	ent = dshash_get_current(hstat);
+	key = ent->key;
+	pdsa = ent->body;
+
+	/*
+	 * Signal that the entry is dropped - this will eventually cause other
+	 * backends to release their references.
+	 */
+	Assert(!ent->dropped);
+	ent->dropped = true;
+
+	/*
+	 * This backend might very well be the only backend holding a
+	 * reference. Ensure that we're not preventing it from being cleaned up
+	 * till later.
+	 *
+	 * XXX: don't do this while holding the dshash lock.
+	 */
+	if (pgStatSharedRefHash)
+	{
+		PgStatSharedRefHashEntry *shared_ref_entry;
+
+		shared_ref_entry =
+			pgstat_shared_ref_hash_lookup(pgStatSharedRefHash, key);
+
+		if (shared_ref_entry && shared_ref_entry->shared_ref)
+		{
+			Assert(shared_ref_entry->shared_ref->shared_entry == ent);
+			pgstat_shared_ref_release(shared_ref_entry->key,
+									  shared_ref_entry->shared_ref);
+		}
+	}
+
+	/*
+	 * Now that the entry isn't needed anymore, remove the refcount
+	 * representing a valid entry. If that causes the refcount to reach 0 no
+	 * other backend can have a reference, so we can free.
+	 */
+	if (pg_atomic_fetch_sub_u32(&ent->refcount, 1) == 1)
+	{
+		dshash_delete_current(hstat);
+		dsa_free(pgStatDSA, pdsa);
+		did_free = true;
+	}
+	else
+	{
+		did_free = false;
+	}
+
+	return did_free;
+}
+
+static void
+pgstat_perform_drop(PgStat_DroppedStatsItem *drop)
+{
+	PgStatShmHashEntry *shent;
+	PgStatHashKey key;
+
+	key.kind = drop->kind;
+	key.dboid = drop->dboid;
+	key.objoid = drop->objoid;
+
+	if (pgStatSharedRefHash)
+	{
+		PgStatSharedRefHashEntry *lohashent;
+
+		lohashent = pgstat_shared_ref_hash_lookup(pgStatSharedRefHash, key);
+
+		if (lohashent)
+		{
+			if (lohashent->shared_ref && lohashent->shared_ref->pending)
+				pgstat_pending_delete(lohashent->shared_ref);
+
+			pgstat_shared_ref_release(lohashent->key, lohashent->shared_ref);
+		}
+	}
+
+	shent = dshash_find(pgStatSharedHash, &key, true);
+	if (shent)
+	{
+		dsa_pointer pdsa;
+
+		Assert(shent->body != InvalidDsaPointer);
+		pdsa = shent->body;
+
+		/*
+		 * Signal that the entry is dropped - this will eventually cause other
+		 * backends to release their references.
+		 */
+
+		if (shent->dropped)
+			elog(ERROR, "can only drop stats once");
+		shent->dropped = true;
+
+		if (pg_atomic_fetch_sub_u32(&shent->refcount, 1) == 1)
+		{
+			dshash_delete_entry(pgStatSharedHash, shent);
+			dsa_free(pgStatDSA, pdsa);
+		}
+		else
+		{
+			dshash_release_lock(pgStatSharedHash, shent);
+		}
+	}
+}
+
+
+/* ------------------------------------------------------------
+ * Fetching of stats
+ *------------------------------------------------------------
+ */
+
+static void
+pgstat_fetch_prepare(void)
+{
+	if (stats_snapshot.stats == NULL)
+	{
+		pgstat_setup_memcxt();
+
+		stats_snapshot.stats = pgstat_snapshot_create(pgStatSnapshotContext,
+													  PGSTAT_TABLE_HASH_SIZE,
+													  NULL);
+	}
+}
+
+static void
+pgstat_fetch_snapshot_build(void)
+{
+	dshash_seq_status hstat;
+	PgStatShmHashEntry *p;
+
+	pgstat_fetch_prepare();
+
+	Assert(stats_snapshot.stats->members == 0);
+
+	stats_snapshot.snapshot_timestamp = GetCurrentTimestamp();
+
+	/*
+	 * Build snapshot all variable stats.
+	 */
+	dshash_seq_init(&hstat, pgStatSharedHash, false);
+	while ((p = dshash_seq_next(&hstat)) != NULL)
+	{
+		bool found;
+		PgStatSnapshotEntry *entry = NULL;
+		size_t entry_len;
+		PgStatShm_StatEntryHeader *stats_data;
+
+		/*
+		 * Check if the stats object should be included in the
+		 * snapshot. Unless the stats kind can be accessed from all databases
+		 * (e.g. database stats themselves), we only include stats for the
+		 * current database or objects not associated with a database
+		 * (e.g. shared relations).
+		 */
+		if (p->key.dboid != MyDatabaseId &&
+			p->key.dboid != InvalidOid &&
+			!pgstat_kind_infos[p->key.kind].accessed_across_databases)
+			continue;
+
+		if (p->dropped)
+			continue;
+
+		Assert(pg_atomic_read_u32(&p->refcount) > 0);
+
+		stats_data = dsa_get_address(pgStatDSA, p->body);
+		Assert(stats_data);
+
+		entry = pgstat_snapshot_insert(stats_snapshot.stats, p->key, &found);
+		Assert(!found);
+
+		entry_len = pgstat_kind_infos[p->key.kind].shared_size;
+		entry->data = MemoryContextAlloc(pgStatSnapshotContext, entry_len);
+		memcpy(entry->data,
+			   shared_stat_entry_data(p->key.kind, stats_data),
+			   entry_len);
+	}
+	dshash_seq_term(&hstat);
+
+	/*
+	 * Build snapshot of all global stats.
+	 */
+	for (int kind = 0; kind < PGSTAT_KIND_LAST; kind++)
+	{
+		if (!pgstat_kind_infos[kind].is_global)
+		{
+			Assert(pgstat_kind_infos[kind].snapshot_cb == NULL);
+			continue;
+		}
+
+		Assert(pgstat_kind_infos[kind].snapshot_cb != NULL);
+
+		stats_snapshot.global_valid[kind] = false;
+
+		pgstat_kind_infos[kind].snapshot_cb();
+
+		Assert(!stats_snapshot.global_valid[kind]);
+		stats_snapshot.global_valid[kind] = true;
+	}
+
+	stats_snapshot.mode = STATS_FETCH_CONSISTENCY_SNAPSHOT;
+}
+
+void*
+pgstat_fetch_entry(PgStatKind type, Oid dboid, Oid objoid)
+{
+	PgStatHashKey key;
+	PgStatSharedRef *shared_ref;
+	void *stats_data;
+	size_t data_size;
+	size_t data_offset;
+
+	/* should be called from backends */
+	Assert(IsUnderPostmaster || !IsPostmasterEnvironment);
+
+	pgstat_fetch_prepare();
+
+	AssertArg(type <= PGSTAT_KIND_LAST);
+	AssertArg(!pgstat_kind_infos[type].is_global);
+
+	key.kind = type;
+	key.dboid = dboid;
+	key.objoid = objoid;
+
+	/* if we need to build a full snapshot, do so */
+	if (stats_snapshot.mode != STATS_FETCH_CONSISTENCY_SNAPSHOT &&
+		pgstat_fetch_consistency == STATS_FETCH_CONSISTENCY_SNAPSHOT)
+		pgstat_fetch_snapshot_build();
+
+	/* if caching is desired, look up in cache */
+	if (pgstat_fetch_consistency > STATS_FETCH_CONSISTENCY_NONE)
+	{
+		PgStatSnapshotEntry *entry = NULL;
+
+		entry = pgstat_snapshot_lookup(stats_snapshot.stats, key);
+
+		if (entry)
+			return entry->data;
+	}
+
+	/*
+	 * if we built a full snapshot and it's not in stats_snapshot.stats, it
+	 * doesn't exist.
+	 */
+	if (pgstat_fetch_consistency == STATS_FETCH_CONSISTENCY_SNAPSHOT)
+		return NULL;
+
+	stats_snapshot.mode = pgstat_fetch_consistency;
+
+	shared_ref = pgstat_shared_ref_get(type, dboid, objoid, false);
+
+	if (shared_ref == NULL || shared_ref->shared_entry->dropped)
+	{
+		/* FIXME: need to remember that STATS_FETCH_CONSISTENCY_CACHE */
+		return NULL;
+	}
+
+	/*
+	 * FIXME: For STATS_FETCH_CONSISTENCY_NONE, should we instead allocate
+	 * stats in calling context?
+	 */
+
+	data_size = pgstat_kind_infos[type].shared_data_len;
+	data_offset = pgstat_kind_infos[type].shared_data_off;
+	stats_data = MemoryContextAlloc(pgStatSnapshotContext, data_size);
+	memcpy(stats_data, ((char*) shared_ref->shared_stats) + data_offset, data_size);
+
+	if (pgstat_fetch_consistency > STATS_FETCH_CONSISTENCY_NONE)
+	{
+		PgStatSnapshotEntry *entry = NULL;
+		bool found;
+
+		entry = pgstat_snapshot_insert(stats_snapshot.stats, key, &found);
+		entry->data = stats_data;
+	}
+
+	return stats_data;
+}
+
+/*
+ * AFIXME: consistent naming
+ * AFIXME: deduplicate some of this code with pgstat_fetch_snapshot_build().
+ *
+ * AFIXME: it'd be nicer if we passed .snapshot_cb() the target memory
+ * location, instead of putting PgStatSnapshot into pgstat_internal.h
+ */
+void
+pgstat_snapshot_global(PgStatKind kind)
+{
+	AssertArg(kind <= PGSTAT_KIND_LAST);
+	AssertArg(pgstat_kind_infos[kind].is_global);
+
+	if (pgstat_fetch_consistency == STATS_FETCH_CONSISTENCY_SNAPSHOT)
+	{
+		if (stats_snapshot.mode != STATS_FETCH_CONSISTENCY_SNAPSHOT)
+			pgstat_fetch_snapshot_build();
+
+		Assert(stats_snapshot.global_valid[kind] == true);
+	}
+	else if (pgstat_fetch_consistency == STATS_FETCH_CONSISTENCY_NONE ||
+			 !stats_snapshot.global_valid[kind])
+	{
+		if (pgstat_fetch_consistency == STATS_FETCH_CONSISTENCY_NONE)
+			stats_snapshot.global_valid[kind] = false;
+
+		pgstat_kind_infos[kind].snapshot_cb();
+
+		Assert(!stats_snapshot.global_valid[kind]);
+		stats_snapshot.global_valid[kind] = true;
+	}
 }
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 80bb2695998..33e4bec9f68 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -255,7 +255,6 @@ static pid_t StartupPID = 0,
 			WalReceiverPID = 0,
 			AutoVacPID = 0,
 			PgArchPID = 0,
-			PgStatPID = 0,
 			SysLoggerPID = 0;
 
 /* Startup process's status */
@@ -517,7 +516,6 @@ typedef struct
 	PGPROC	   *AuxiliaryProcs;
 	PGPROC	   *PreparedXactProcs;
 	PMSignalData *PMSignalState;
-	InheritableSocket pgStatSock;
 	pid_t		PostmasterPid;
 	TimestampTz PgStartTime;
 	TimestampTz PgReloadTime;
@@ -652,9 +650,8 @@ PostmasterMain(int argc, char *argv[])
 	 * CAUTION: when changing this list, check for side-effects on the signal
 	 * handling setup of child processes.  See tcop/postgres.c,
 	 * bootstrap/bootstrap.c, postmaster/bgwriter.c, postmaster/walwriter.c,
-	 * postmaster/autovacuum.c, postmaster/pgarch.c, postmaster/pgstat.c,
-	 * postmaster/syslogger.c, postmaster/bgworker.c and
-	 * postmaster/checkpointer.c.
+	 * postmaster/autovacuum.c, postmaster/pgarch.c, postmaster/syslogger.c,
+	 * postmaster/bgworker.c and postmaster/checkpointer.c.
 	 */
 	pqinitmask();
 	PG_SETMASK(&BlockSig);
@@ -1391,12 +1388,6 @@ PostmasterMain(int argc, char *argv[])
 	 */
 	RemovePgTempFiles();
 
-	/*
-	 * Initialize stats collection subsystem (this does NOT start the
-	 * collector process!)
-	 */
-	pgstat_init();
-
 	/*
 	 * Initialize the autovacuum subsystem (again, no process start yet)
 	 */
@@ -1852,11 +1843,6 @@ ServerLoop(void)
 				start_autovac_launcher = false; /* signal processed */
 		}
 
-		/* If we have lost the stats collector, try to start a new one */
-		if (PgStatPID == 0 &&
-			(pmState == PM_RUN || pmState == PM_HOT_STANDBY))
-			PgStatPID = pgstat_start();
-
 		/* If we have lost the archiver, try to start a new one. */
 		if (PgArchPID == 0 && PgArchStartupAllowed())
 			PgArchPID = StartArchiver();
@@ -2791,8 +2777,6 @@ SIGHUP_handler(SIGNAL_ARGS)
 			signal_child(PgArchPID, SIGHUP);
 		if (SysLoggerPID != 0)
 			signal_child(SysLoggerPID, SIGHUP);
-		if (PgStatPID != 0)
-			signal_child(PgStatPID, SIGHUP);
 
 		/* Reload authentication config files too */
 		if (!load_hba())
@@ -3121,8 +3105,6 @@ reaper(SIGNAL_ARGS)
 				AutoVacPID = StartAutoVacLauncher();
 			if (PgArchStartupAllowed() && PgArchPID == 0)
 				PgArchPID = StartArchiver();
-			if (PgStatPID == 0)
-				PgStatPID = pgstat_start();
 
 			/* workers may be scheduled to start now */
 			maybe_start_bgworkers();
@@ -3189,13 +3171,6 @@ reaper(SIGNAL_ARGS)
 				SignalChildren(SIGUSR2);
 
 				pmState = PM_SHUTDOWN_2;
-
-				/*
-				 * We can also shut down the stats collector now; there's
-				 * nothing left for it to do.
-				 */
-				if (PgStatPID != 0)
-					signal_child(PgStatPID, SIGQUIT);
 			}
 			else
 			{
@@ -3274,22 +3249,6 @@ reaper(SIGNAL_ARGS)
 			continue;
 		}
 
-		/*
-		 * Was it the statistics collector?  If so, just try to start a new
-		 * one; no need to force reset of the rest of the system.  (If fail,
-		 * we'll try again in future cycles of the main loop.)
-		 */
-		if (pid == PgStatPID)
-		{
-			PgStatPID = 0;
-			if (!EXIT_STATUS_0(exitstatus))
-				LogChildExit(LOG, _("statistics collector process"),
-							 pid, exitstatus);
-			if (pmState == PM_RUN || pmState == PM_HOT_STANDBY)
-				PgStatPID = pgstat_start();
-			continue;
-		}
-
 		/* Was it the system logger?  If so, try to start a new one */
 		if (pid == SysLoggerPID)
 		{
@@ -3731,22 +3690,6 @@ HandleChildCrash(int pid, int exitstatus, const char *procname)
 		signal_child(PgArchPID, (SendStop ? SIGSTOP : SIGQUIT));
 	}
 
-	/*
-	 * Force a power-cycle of the pgstat process too.  (This isn't absolutely
-	 * necessary, but it seems like a good idea for robustness, and it
-	 * simplifies the state-machine logic in the case where a shutdown request
-	 * arrives during crash processing.)
-	 */
-	if (PgStatPID != 0 && take_action)
-	{
-		ereport(DEBUG2,
-				(errmsg_internal("sending %s to process %d",
-								 "SIGQUIT",
-								 (int) PgStatPID)));
-		signal_child(PgStatPID, SIGQUIT);
-		allow_immediate_pgstat_restart();
-	}
-
 	/* We do NOT restart the syslogger */
 
 	if (Shutdown != ImmediateShutdown)
@@ -3972,8 +3915,6 @@ PostmasterStateMachine(void)
 					SignalChildren(SIGQUIT);
 					if (PgArchPID != 0)
 						signal_child(PgArchPID, SIGQUIT);
-					if (PgStatPID != 0)
-						signal_child(PgStatPID, SIGQUIT);
 				}
 			}
 		}
@@ -3997,8 +3938,7 @@ PostmasterStateMachine(void)
 	{
 		/*
 		 * PM_WAIT_DEAD_END state ends when the BackendList is entirely empty
-		 * (ie, no dead_end children remain), and the archiver and stats
-		 * collector are gone too.
+		 * (ie, no dead_end children remain), and the archiveris gone too.
 		 *
 		 * The reason we wait for those two is to protect them against a new
 		 * postmaster starting conflicting subprocesses; this isn't an
@@ -4008,8 +3948,7 @@ PostmasterStateMachine(void)
 		 * normal state transition leading up to PM_WAIT_DEAD_END, or during
 		 * FatalError processing.
 		 */
-		if (dlist_is_empty(&BackendList) &&
-			PgArchPID == 0 && PgStatPID == 0)
+		if (dlist_is_empty(&BackendList) && PgArchPID == 0)
 		{
 			/* These other guys should be dead already */
 			Assert(StartupPID == 0);
@@ -4229,8 +4168,6 @@ TerminateChildren(int signal)
 		signal_child(AutoVacPID, signal);
 	if (PgArchPID != 0)
 		signal_child(PgArchPID, signal);
-	if (PgStatPID != 0)
-		signal_child(PgStatPID, signal);
 }
 
 /*
@@ -5162,12 +5099,6 @@ SubPostmasterMain(int argc, char *argv[])
 
 		StartBackgroundWorker();
 	}
-	if (strcmp(argv[1], "--forkcol") == 0)
-	{
-		/* Do not want to attach to shared memory */
-
-		PgstatCollectorMain(argc, argv);	/* does not return */
-	}
 	if (strcmp(argv[1], "--forklog") == 0)
 	{
 		/* Do not want to attach to shared memory */
@@ -5271,12 +5202,6 @@ sigusr1_handler(SIGNAL_ARGS)
 	if (CheckPostmasterSignal(PMSIGNAL_BEGIN_HOT_STANDBY) &&
 		pmState == PM_RECOVERY && Shutdown == NoShutdown)
 	{
-		/*
-		 * Likewise, start other special children as needed.
-		 */
-		Assert(PgStatPID == 0);
-		PgStatPID = pgstat_start();
-
 		ereport(LOG,
 				(errmsg("database system is ready to accept read-only connections")));
 
@@ -6192,7 +6117,6 @@ extern slock_t *ShmemLock;
 extern slock_t *ProcStructLock;
 extern PGPROC *AuxiliaryProcs;
 extern PMSignalData *PMSignalState;
-extern pgsocket pgStatSock;
 extern pg_time_t first_syslogger_file_time;
 
 #ifndef WIN32
@@ -6248,8 +6172,6 @@ save_backend_variables(BackendParameters *param, Port *port,
 	param->AuxiliaryProcs = AuxiliaryProcs;
 	param->PreparedXactProcs = PreparedXactProcs;
 	param->PMSignalState = PMSignalState;
-	if (!write_inheritable_socket(&param->pgStatSock, pgStatSock, childPid))
-		return false;
 
 	param->PostmasterPid = PostmasterPid;
 	param->PgStartTime = PgStartTime;
@@ -6483,7 +6405,6 @@ restore_backend_variables(BackendParameters *param, Port *port)
 	AuxiliaryProcs = param->AuxiliaryProcs;
 	PreparedXactProcs = param->PreparedXactProcs;
 	PMSignalState = param->PMSignalState;
-	read_inheritable_socket(&pgStatSock, &param->pgStatSock);
 
 	PostmasterPid = param->PostmasterPid;
 	PgStartTime = param->PgStartTime;
@@ -6522,8 +6443,6 @@ restore_backend_variables(BackendParameters *param, Port *port)
 	if (postmaster_alive_fds[1] >= 0)
 		ReserveExternalFD();
 #endif
-	if (pgStatSock != PGINVALID_SOCKET)
-		ReserveExternalFD();
 }
 
 
diff --git a/src/backend/postmaster/walwriter.c b/src/backend/postmaster/walwriter.c
index 102fa2a089f..202ad711e86 100644
--- a/src/backend/postmaster/walwriter.c
+++ b/src/backend/postmaster/walwriter.c
@@ -258,7 +258,7 @@ WalWriterMain(void)
 			left_till_hibernate--;
 
 		/* Send WAL statistics to the stats collector */
-		pgstat_send_wal(false);
+		pgstat_report_wal(false);
 
 		/*
 		 * Sleep until we are signaled or WalWriterDelay has elapsed.  If we
@@ -298,11 +298,11 @@ HandleWalWriterInterrupts(void)
 		 * Force to send remaining WAL statistics to the stats collector at
 		 * process exit.
 		 *
-		 * Since pgstat_send_wal is invoked with 'force' is false in main loop
+		 * Since pgstat_report_wal is invoked with 'force' is false in main loop
 		 * to avoid overloading to the stats collector, there may exist unsent
 		 * stats counters for the WAL writer.
 		 */
-		pgstat_send_wal(true);
+		pgstat_report_wal(true);
 
 		proc_exit(0);
 	}
diff --git a/src/backend/replication/basebackup.c b/src/backend/replication/basebackup.c
index 0bf28b55d7f..7fd8463ca29 100644
--- a/src/backend/replication/basebackup.c
+++ b/src/backend/replication/basebackup.c
@@ -1528,8 +1528,8 @@ is_checksummed_file(const char *fullpath, const char *filename)
  *
  * If 'missing_ok' is true, will not throw an error if the file is not found.
  *
- * If dboid is anything other than InvalidOid then any checksum failures detected
- * will get reported to the stats collector.
+ * If dboid is anything other than InvalidOid then any checksum failures
+ * detected will get reported to the activity stats facility.
  *
  * Returns true if the file was successfully sent, false if 'missing_ok',
  * and the file did not exist.
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 934aa13f2d3..59e5f9819ce 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -1919,7 +1919,8 @@ UpdateDecodingStats(LogicalDecodingContext *ctx)
 	repSlotStat.total_txns = rb->totalTxns;
 	repSlotStat.total_bytes = rb->totalBytes;
 
-	pgstat_report_replslot(&repSlotStat);
+	pgstat_report_replslot(ctx->slot - ReplicationSlotCtl->replication_slots,
+						   &repSlotStat);
 
 	rb->spillTxns = 0;
 	rb->spillCount = 0;
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index caa6b297560..bae1299f0c0 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -354,9 +354,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
 	 * stats for physical slots, so no need to create an entry for the same.
 	 * See ReplicationSlotDropPtr for why we need to do this before releasing
 	 * ReplicationSlotAllocationLock.
+	 *
+	 * XXX: lock nesting issues, comment update.
 	 */
 	if (SlotIsLogical(slot))
-		pgstat_report_replslot_create(NameStr(slot->data.name));
+		pgstat_report_replslot_create(NameStr(slot->data.name),
+									  slot - ReplicationSlotCtl->replication_slots);
 
 	/*
 	 * Now that the slot has been marked as in_use and active, it's safe to
@@ -721,23 +724,15 @@ ReplicationSlotDropPtr(ReplicationSlot *slot)
 				(errmsg("could not remove directory \"%s\"", tmppath)));
 
 	/*
-	 * Send a message to drop the replication slot to the stats collector.
-	 * Since there is no guarantee of the order of message transfer on a UDP
-	 * connection, it's possible that a message for creating a new slot
-	 * reaches before a message for removing the old slot. We send the drop
-	 * and create messages while holding ReplicationSlotAllocationLock to
-	 * reduce that possibility. If the messages reached in reverse, we would
-	 * lose one statistics update message. But the next update message will
-	 * create the statistics for the replication slot.
+	 * Drop the statistics entry for the replication slot.  Do this while
+	 * holding ReplicationSlotAllocationLock so that we don't drop a statistics
+	 * entry for another slot with the same name just created on another
+	 * session.
 	 *
-	 * XXX In case, the messages for creation and drop slot of the same name
-	 * get lost and create happens before (auto)vacuum cleans up the dead
-	 * slot, the stats will be accumulated into the old slot. One can imagine
-	 * having OIDs for each slot to avoid the accumulation of stats but that
-	 * doesn't seem worth doing as in practice this won't happen frequently.
+	 * XXX: lock nesting issues, comment update.
 	 */
 	if (SlotIsLogical(slot))
-		pgstat_report_replslot_drop(NameStr(slot->data.name));
+		pgstat_report_replslot_drop(slot - ReplicationSlotCtl->replication_slots);
 
 	/*
 	 * We release this at the very end, so that nobody starts trying to create
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index f5459c68f89..2b1187140c1 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -2147,7 +2147,7 @@ BufferSync(int flags)
 			if (SyncOneBuffer(buf_id, false, &wb_context) & BUF_WRITTEN)
 			{
 				TRACE_POSTGRESQL_BUFFER_SYNC_WRITTEN(buf_id);
-				PendingCheckpointerStats.m_buf_written_checkpoints++;
+				PendingCheckpointerStats.buf_written_checkpoints++;
 				num_written++;
 			}
 		}
@@ -2257,7 +2257,7 @@ BgBufferSync(WritebackContext *wb_context)
 	strategy_buf_id = StrategySyncStart(&strategy_passes, &recent_alloc);
 
 	/* Report buffer alloc counts to pgstat */
-	PendingBgWriterStats.m_buf_alloc += recent_alloc;
+	PendingBgWriterStats.buf_alloc += recent_alloc;
 
 	/*
 	 * If we're not running the LRU scan, just stop after doing the stats
@@ -2447,7 +2447,7 @@ BgBufferSync(WritebackContext *wb_context)
 			reusable_buffers++;
 			if (++num_written >= bgwriter_lru_maxpages)
 			{
-				PendingBgWriterStats.m_maxwritten_clean++;
+				PendingBgWriterStats.maxwritten_clean++;
 				break;
 			}
 		}
@@ -2455,7 +2455,7 @@ BgBufferSync(WritebackContext *wb_context)
 			reusable_buffers++;
 	}
 
-	PendingBgWriterStats.m_buf_written_clean += num_written;
+	PendingBgWriterStats.buf_written_clean += num_written;
 
 #ifdef BGW_DEBUG
 	elog(DEBUG1, "bgwriter: recent_alloc=%u smoothed=%.2f delta=%ld ahead=%d density=%.2f reusable_est=%d upcoming_est=%d scanned=%d wrote=%d reusable=%d",
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5e..88ff59c568f 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -145,6 +145,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, StatsShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -296,6 +297,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	StatsShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 8f7f1b2f7c3..26933d00441 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -176,7 +176,9 @@ static const char *const BuiltinTrancheNames[] = {
 	/* LWTRANCHE_PARALLEL_APPEND: */
 	"ParallelAppend",
 	/* LWTRANCHE_PER_XACT_PREDICATE_LIST: */
-	"PerXactPredicateList"
+	"PerXactPredicateList",
+	/* LWTRANCHE_STATS: */
+	"ActivityStatistics"
 };
 
 StaticAssertDecl(lengthof(BuiltinTrancheNames) ==
diff --git a/src/backend/storage/lmgr/lwlocknames.txt b/src/backend/storage/lmgr/lwlocknames.txt
index 6c7cf6c2956..8edb41c1cf5 100644
--- a/src/backend/storage/lmgr/lwlocknames.txt
+++ b/src/backend/storage/lmgr/lwlocknames.txt
@@ -53,3 +53,4 @@ XactTruncationLock					44
 # 45 was XactTruncationLock until removal of BackendRandomLock
 WrapLimitsVacuumLock				46
 NotifyQueueTailLock					47
+StatsLock							48
diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index d71a557a352..2c7a2b28572 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -415,11 +415,6 @@ smgrdounlinkall(SMgrRelation *rels, int nrels, bool isRedo)
 			smgrsw[which].smgr_close(rels[i], forknum);
 	}
 
-	/*
-	 * It'd be nice to tell the stats collector to forget them immediately,
-	 * too. But we can't because we don't know the OIDs.
-	 */
-
 	/*
 	 * Send a shared-inval message to force other backends to close any
 	 * dangling smgr references they may have for these rels.  We should do
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 34c13a11138..a33a072a711 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -3364,6 +3364,12 @@ ProcessInterrupts(void)
 
 	if (LogMemoryContextPending)
 		ProcessLogMemoryContextInterrupt();
+
+	if (IdleStatsUpdateTimeoutPending)
+	{
+		IdleStatsUpdateTimeoutPending = false;
+		pgstat_report_stat(true);
+	}
 }
 
 
@@ -4028,6 +4034,7 @@ PostgresMain(const char *dbname, const char *username)
 	volatile bool send_ready_for_query = true;
 	bool		idle_in_transaction_timeout_enabled = false;
 	bool		idle_session_timeout_enabled = false;
+	bool		idle_stats_update_timeout_enabled = false;
 
 	AssertArg(dbname != NULL);
 	AssertArg(username != NULL);
@@ -4353,11 +4360,12 @@ PostgresMain(const char *dbname, const char *username)
 		 * Note: this includes fflush()'ing the last of the prior output.
 		 *
 		 * This is also a good time to send collected statistics to the
-		 * collector, and to update the PS stats display.  We avoid doing
-		 * those every time through the message loop because it'd slow down
-		 * processing of batched messages, and because we don't want to report
-		 * uncommitted updates (that confuses autovacuum).  The notification
-		 * processor wants a call too, if we are not in a transaction block.
+		 * activity statistics, and to update the PS stats display.  We avoid
+		 * doing those every time through the message loop because it'd slow
+		 * down processing of batched messages, and because we don't want to
+		 * report uncommitted updates (that confuses autovacuum).  The
+		 * notification processor wants a call too, if we are not in a
+		 * transaction block.
 		 *
 		 * Also, if an idle timeout is enabled, start the timer for that.
 		 */
@@ -4391,6 +4399,8 @@ PostgresMain(const char *dbname, const char *username)
 			}
 			else
 			{
+				long stats_timeout;
+
 				/*
 				 * Process incoming notifies (including self-notifies), if
 				 * any, and send relevant messages to the client.  Doing it
@@ -4401,8 +4411,14 @@ PostgresMain(const char *dbname, const char *username)
 				if (notifyInterruptPending)
 					ProcessNotifyInterrupt(false);
 
-				pgstat_report_stat(false);
-
+				/* Start the idle-stats-update timer */
+				stats_timeout = pgstat_report_stat(false);
+				if (stats_timeout > 0)
+				{
+					idle_stats_update_timeout_enabled = true;
+					enable_timeout_after(IDLE_STATS_UPDATE_TIMEOUT,
+										 stats_timeout);
+				}
 				set_ps_display("idle");
 				pgstat_report_activity(STATE_IDLE, NULL);
 
@@ -4436,9 +4452,9 @@ PostgresMain(const char *dbname, const char *username)
 		firstchar = ReadCommand(&input_message);
 
 		/*
-		 * (4) turn off the idle-in-transaction and idle-session timeouts, if
-		 * active.  We do this before step (5) so that any last-moment timeout
-		 * is certain to be detected in step (5).
+		 * (4) turn off the idle-in-transaction, idle-session and
+		 * idle-state-update timeouts if active.  We do this before step (5) so
+		 * that any last-moment timeout is certain to be detected in step (5).
 		 *
 		 * At most one of these timeouts will be active, so there's no need to
 		 * worry about combining the timeout.c calls into one.
@@ -4453,6 +4469,11 @@ PostgresMain(const char *dbname, const char *username)
 			disable_timeout(IDLE_SESSION_TIMEOUT, false);
 			idle_session_timeout_enabled = false;
 		}
+		if (idle_stats_update_timeout_enabled)
+		{
+			disable_timeout(IDLE_STATS_UPDATE_TIMEOUT, false);
+			idle_stats_update_timeout_enabled = false;
+		}
 
 		/*
 		 * (5) disable async signal conditions again.
diff --git a/src/backend/utils/activity/pgstat_database.c b/src/backend/utils/activity/pgstat_database.c
index 7850829e36f..566f2f71ffe 100644
--- a/src/backend/utils/activity/pgstat_database.c
+++ b/src/backend/utils/activity/pgstat_database.c
@@ -19,6 +19,7 @@
 
 #include "utils/pgstat_internal.h"
 #include "utils/timestamp.h"
+#include "storage/procsignal.h"
 
 
 static bool pgstat_should_report_connstat(void);
@@ -39,89 +40,118 @@ 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().)
- * ----------
+ * Remove entry for the database being dropped.
+ *
+ * Some entries might be left alone due to lock failure or some stats are
+ * flushed after this but we will still clean the dead DB eventually via
+ * future invocations of pgstat_vacuum_stat().
+ *	----------
  */
 void
-pgstat_drop_database(Oid databaseid)
+pgstat_drop_database(Oid dboid)
 {
-	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));
+	/*
+	 * FIXME: need to do this using the transactional mechanism. Not so much
+	 * because of rollbacks, but so the stats are removed on a standby
+	 * too. Maybe a dedicated drop type?
+	 */
+	pgstat_drop_database_and_contents(dboid);
 }
 
 /* --------
  * pgstat_report_recovery_conflict() -
  *
- *	Tell the collector about a Hot Standby recovery conflict.
+ * Report a Hot Standby recovery conflict.
  * --------
  */
 void
 pgstat_report_recovery_conflict(int reason)
 {
-	PgStat_MsgRecoveryConflict msg;
+	PgStat_StatDBEntry *dbentry;
 
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+	Assert(IsUnderPostmaster);
+	if (!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));
+	dbentry = pgstat_pending_db_prepare(MyDatabaseId);
+
+	switch (reason)
+	{
+		case PROCSIG_RECOVERY_CONFLICT_DATABASE:
+
+			/*
+			 * Since we drop the information about the database as soon as it
+			 * replicates, there is no point in counting these conflicts.
+			 */
+			break;
+		case PROCSIG_RECOVERY_CONFLICT_TABLESPACE:
+			dbentry->n_conflict_tablespace++;
+			break;
+		case PROCSIG_RECOVERY_CONFLICT_LOCK:
+			dbentry->n_conflict_lock++;
+			break;
+		case PROCSIG_RECOVERY_CONFLICT_SNAPSHOT:
+			dbentry->n_conflict_snapshot++;
+			break;
+		case PROCSIG_RECOVERY_CONFLICT_BUFFERPIN:
+			dbentry->n_conflict_bufferpin++;
+			break;
+		case PROCSIG_RECOVERY_CONFLICT_STARTUP_DEADLOCK:
+			dbentry->n_conflict_startup_deadlock++;
+			break;
+	}
 }
 
 /* --------
  * pgstat_report_deadlock() -
  *
- *	Tell the collector about a deadlock detected.
+ * Report a detected deadlock.
  * --------
  */
 void
 pgstat_report_deadlock(void)
 {
-	PgStat_MsgDeadlock msg;
+	PgStat_StatDBEntry *dbent;
 
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+	if (!pgstat_track_counts)
 		return;
 
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_DEADLOCK);
-	msg.m_databaseid = MyDatabaseId;
-	pgstat_send(&msg, sizeof(msg));
+	dbent = pgstat_pending_db_prepare(MyDatabaseId);
+	dbent->n_deadlocks++;
 }
 
 /* --------
  * pgstat_report_checksum_failures_in_db() -
  *
- *	Tell the collector about one or more checksum failures.
+ * Report one or more checksum failures.
  * --------
  */
 void
 pgstat_report_checksum_failures_in_db(Oid dboid, int failurecount)
 {
-	PgStat_MsgChecksumFailure msg;
+	PgStatSharedRef *shared_ref;
+	PgStatShm_StatDBEntry *sharedent;
 
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+	if (!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();
+	/*
+	 * Update the shared stats directly - checksum failures should never be
+	 * common enough for that to be a problem.
+	 */
+	shared_ref =
+		pgstat_shared_stat_locked(PGSTAT_KIND_DB, dboid, InvalidOid, false);
 
-	pgstat_send(&msg, sizeof(msg));
+	sharedent = (PgStatShm_StatDBEntry *) shared_ref->shared_stats;
+	sharedent->stats.n_checksum_failures += failurecount;
+
+	pgstat_shared_stat_unlock(shared_ref);
 }
 
 /* --------
  * pgstat_report_checksum_failure() -
  *
- *	Tell the collector about a checksum failure.
+ * Reports one checksum failure in the current database.
  * --------
  */
 void
@@ -133,75 +163,23 @@ pgstat_report_checksum_failure(void)
 /* --------
  * pgstat_report_tempfile() -
  *
- *	Tell the collector about a temporary file.
+ * Report a temporary file.
  * --------
  */
 void
 pgstat_report_tempfile(size_t filesize)
 {
-	PgStat_MsgTempFile msg;
+	PgStat_StatDBEntry *dbent;
 
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+	if (!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));
-}
+	if (filesize == 0)			/* Is there a case where filesize is really 0? */
+		return;
 
-/*
- * 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;
-	}
+	dbent = pgstat_pending_db_prepare(MyDatabaseId);
+	dbent->n_temp_bytes += filesize; /* needs check overflow */
+	dbent->n_temp_files++;
 }
 
 /* --------
@@ -213,16 +191,13 @@ pgstat_update_dbstats(PgStat_MsgTabstat *tsmsg, TimestampTz now)
 void
 pgstat_report_connect(Oid dboid)
 {
-	PgStat_MsgConnect msg;
+	PgStat_StatDBEntry *dbentry;
 
 	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));
+	dbentry = pgstat_pending_db_prepare(MyDatabaseId);
+	dbentry->n_sessions++;
 }
 
 /* --------
@@ -234,15 +209,76 @@ pgstat_report_connect(Oid dboid)
 void
 pgstat_report_disconnect(Oid dboid)
 {
-	PgStat_MsgDisconnect msg;
+	PgStat_StatDBEntry *dbentry;
 
 	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));
+	dbentry = pgstat_pending_db_prepare(MyDatabaseId);
+
+    switch (pgStatSessionEndCause)
+    {
+        case DISCONNECT_NOT_YET:
+        case DISCONNECT_NORMAL:
+            /* we don't collect these */
+            break;
+        case DISCONNECT_CLIENT_EOF:
+            dbentry->n_sessions_abandoned++;
+            break;
+        case DISCONNECT_FATAL:
+            dbentry->n_sessions_fatal++;
+            break;
+        case DISCONNECT_KILLED:
+            dbentry->n_sessions_killed++;
+            break;
+    }
+}
+
+/* ----------
+ * pgstat_update_connstat() -
+ *
+ * Update pending connection stats.
+ * ----------
+ */
+void
+pgstat_update_dbstats(TimestampTz now)
+{
+	PgStat_StatDBEntry *dbentry;
+
+	dbentry = pgstat_pending_db_prepare(MyDatabaseId);
+
+	/*
+	 * Accumulate xact commit/rollback and I/O timings to stats entry of the
+	 * current database.
+	 */
+	dbentry->n_xact_commit += pgStatXactCommit;
+	dbentry->n_xact_rollback += pgStatXactRollback;
+	dbentry->n_block_read_time += pgStatBlockReadTime;
+	dbentry->n_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;
+
+		dbentry->total_session_time += (PgStat_Counter) secs * 1000000 + usecs;
+		dbentry->total_active_time += pgStatActiveTime;
+		dbentry->total_idle_in_xact_time += pgStatTransactionIdleTime;
+	}
+
+	pgStatXactCommit = 0;
+	pgStatXactRollback = 0;
+	pgStatBlockReadTime = 0;
+	pgStatBlockWriteTime = 0;
+	pgStatActiveTime = 0;
+	pgStatTransactionIdleTime = 0;
 }
 
 /* --------
@@ -261,6 +297,96 @@ pgstat_should_report_connstat(void)
 	return MyBackendType == B_BACKEND;
 }
 
+/*
+ * pgstat_flush_db - flush out a local database stats entry
+ *
+ * If nowait is true, this function returns false on lock failure. Otherwise
+ * this function always returns true.
+ *
+ * Returns true if the entry is successfully flushed out.
+ */
+bool
+pgstat_flush_database(PgStatSharedRef *shared_ref, bool nowait)
+{
+	PgStatShm_StatDBEntry *sharedent;
+	PgStat_StatDBEntry *pendingent = (PgStat_StatDBEntry *) shared_ref->pending;
+
+	if (shared_ref == NULL)
+		return false;
+
+	if (!pgstat_shared_stat_lock(shared_ref, nowait))
+		return false;			/* failed to acquire lock, skip */
+
+	sharedent = (PgStatShm_StatDBEntry *) shared_ref->shared_stats;
+
+#define PGSTAT_ACCUM_DBCOUNT(item)		\
+	(sharedent)->stats.item += (pendingent)->item
+
+	PGSTAT_ACCUM_DBCOUNT(n_tuples_returned);
+	PGSTAT_ACCUM_DBCOUNT(n_tuples_fetched);
+	PGSTAT_ACCUM_DBCOUNT(n_tuples_inserted);
+	PGSTAT_ACCUM_DBCOUNT(n_tuples_updated);
+	PGSTAT_ACCUM_DBCOUNT(n_tuples_deleted);
+	PGSTAT_ACCUM_DBCOUNT(n_blocks_fetched);
+	PGSTAT_ACCUM_DBCOUNT(n_blocks_hit);
+
+	PGSTAT_ACCUM_DBCOUNT(n_deadlocks);
+	PGSTAT_ACCUM_DBCOUNT(n_temp_bytes);
+	PGSTAT_ACCUM_DBCOUNT(n_temp_files);
+	PGSTAT_ACCUM_DBCOUNT(n_checksum_failures);
+
+	PGSTAT_ACCUM_DBCOUNT(n_xact_commit);
+	PGSTAT_ACCUM_DBCOUNT(n_xact_rollback);
+
+	PGSTAT_ACCUM_DBCOUNT(n_block_read_time);
+	PGSTAT_ACCUM_DBCOUNT(n_block_write_time);
+
+	PGSTAT_ACCUM_DBCOUNT(n_sessions);
+	PGSTAT_ACCUM_DBCOUNT(total_session_time);
+	PGSTAT_ACCUM_DBCOUNT(total_active_time);
+	PGSTAT_ACCUM_DBCOUNT(total_idle_in_xact_time);
+	PGSTAT_ACCUM_DBCOUNT(n_sessions_abandoned);
+	PGSTAT_ACCUM_DBCOUNT(n_sessions_fatal);
+	PGSTAT_ACCUM_DBCOUNT(n_sessions_killed);
+#undef PGSTAT_ACCUM_DBCOUNT
+
+	pgstat_shared_stat_unlock(shared_ref);
+
+	memset(pendingent, 0, sizeof(*pendingent));
+
+	return true;
+}
+
+/*
+ * Find or create a local PgStat_StatDBEntry entry for dboid.
+ */
+PgStat_StatDBEntry *
+pgstat_pending_db_prepare(Oid dboid)
+{
+	PgStatSharedRef *shared_ref;
+
+	shared_ref = pgstat_pending_prepare(PGSTAT_KIND_DB, dboid, InvalidOid);
+
+	return shared_ref->pending;
+
+}
+
+/* ----------
+ * pgstat_fetch_stat_dbentry() -
+ *
+ * Support function for the SQL-callable pgstat* functions. Returns the
+ * collected statistics for one database or NULL. NULL doesn't necessarily
+ * mean that the database doesn't exist, just that there are no statistics,
+ * so the caller is better off to report ZERO instead.
+ * ----------
+ */
+PgStat_StatDBEntry *
+pgstat_fetch_stat_dbentry(Oid dboid)
+{
+	return (PgStat_StatDBEntry *)
+		pgstat_fetch_entry(PGSTAT_KIND_DB, dboid, InvalidOid);
+}
+
 void
 AtEOXact_PgStat_Database(bool isCommit, bool parallel)
 {
diff --git a/src/backend/utils/activity/pgstat_function.c b/src/backend/utils/activity/pgstat_function.c
index 728bc20b133..66a7eb0747d 100644
--- a/src/backend/utils/activity/pgstat_function.c
+++ b/src/backend/utils/activity/pgstat_function.c
@@ -28,18 +28,6 @@
 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.
@@ -48,6 +36,43 @@ static HTAB *pgStatFunctions = NULL;
 static instr_time total_func_time;
 
 
+/*
+ * Ensure that stats are dropped if transaction aborts.
+ */
+void
+pgstat_create_function(Oid proid)
+{
+	pgstat_schedule_create(PGSTAT_KIND_FUNCTION,
+						   MyDatabaseId,
+						   proid);
+}
+
+/*
+ * Ensure that stats are dropped if transaction commits.
+ */
+void
+pgstat_drop_function(Oid proid)
+{
+	/*
+	 * AFIXME: This is not bulletproof, because functions are not locked, and
+	 * therefore no sinval processing occurs when calling a function. Which in
+	 * turn means that new stats entries might be created for stats whose
+	 * shared stats entry already has been dropped.
+	 *
+	 * A possible solution is to not allow dropping stats from
+	 * pgStatSharedHash until no transaction is alive that could potentially
+	 * still see the row. E.g. by adding an xid to PgStatShmHashEntry which
+	 * needs to be older than the horizon for the entry to be removed.
+	 *
+	 * Or we could decide not to care, and ensure that we reset stats in
+	 * pgstat_create_function() / pgstat_schedule_create().
+	 */
+
+	pgstat_schedule_drop(PGSTAT_KIND_FUNCTION,
+						 MyDatabaseId,
+						 proid);
+}
+
 /*
  * Initialize function call usage data.
  * Called by the executor before invoking a function.
@@ -56,8 +81,8 @@ void
 pgstat_init_function_usage(FunctionCallInfo fcinfo,
 						   PgStat_FunctionCallUsage *fcu)
 {
-	PgStat_BackendFunctionEntry *htabent;
-	bool		found;
+	PgStatSharedRef *shared_ref;
+	PgStat_BackendFunctionEntry *pending;
 
 	if (pgstat_track_functions <= fcinfo->flinfo->fn_stats)
 	{
@@ -66,29 +91,15 @@ pgstat_init_function_usage(FunctionCallInfo fcinfo,
 		return;
 	}
 
-	if (!pgStatFunctions)
-	{
-		/* First time through - initialize function stat table */
-		HASHCTL		hash_ctl;
+	shared_ref = pgstat_pending_prepare(PGSTAT_KIND_FUNCTION,
+											 MyDatabaseId,
+											 fcinfo->flinfo->fn_oid);
+	pending = shared_ref->pending;
 
-		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;
+	fcu->fs = &pending->f_counts;
 
 	/* save stats for this function, later used to compensate for recursion */
-	fcu->save_f_total_time = htabent->f_counts.f_total_time;
+	fcu->save_f_total_time = pending->f_counts.f_total_time;
 
 	/* save current backend-wide total time */
 	fcu->save_total = total_func_time;
@@ -145,81 +156,75 @@ pgstat_end_function_usage(PgStat_FunctionCallUsage *fcu, bool 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
+ * pgstat_flush_function - flush out a local function stats entry
+ *
+ * If nowait is true, this function returns false on lock failure. Otherwise
+ * this function always returns true.
+ *
+ * Returns true if the entry is successfully flushed out.
  */
-void
-pgstat_send_funcstats(void)
+bool
+pgstat_flush_function(PgStatSharedRef *shared_ref, bool nowait)
 {
-	/* we assume this inits to all zeroes: */
-	static const PgStat_FunctionCounts all_zeroes;
+	PgStat_BackendFunctionEntry *localent;	/* local stats entry */
+	PgStatShm_StatFuncEntry *shfuncent = NULL; /* shared stats entry */
 
-	PgStat_MsgFuncstat msg;
-	PgStat_BackendFunctionEntry *entry;
-	HASH_SEQ_STATUS fstat;
+	Assert(shared_ref->shared_entry->key.kind == PGSTAT_KIND_FUNCTION);
+	localent = (PgStat_BackendFunctionEntry *) shared_ref->pending;
 
-	if (pgStatFunctions == NULL)
-		return;
+	/* localent always has non-zero content */
 
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_FUNCSTAT);
-	msg.m_databaseid = MyDatabaseId;
-	msg.m_nentries = 0;
+	if (!pgstat_shared_stat_lock(shared_ref, nowait))
+		return false;			/* failed to acquire lock, skip */
 
-	hash_seq_init(&fstat, pgStatFunctions);
-	while ((entry = (PgStat_BackendFunctionEntry *) hash_seq_search(&fstat)) != NULL)
-	{
-		PgStat_FunctionEntry *m_ent;
+	shfuncent = (PgStatShm_StatFuncEntry *) shared_ref->shared_stats;
 
-		/* Skip it if no counts accumulated since last time */
-		if (memcmp(&entry->f_counts, &all_zeroes,
-				   sizeof(PgStat_FunctionCounts)) == 0)
-			continue;
+	shfuncent->stats.f_numcalls += localent->f_counts.f_numcalls;
+	shfuncent->stats.f_total_time +=
+		INSTR_TIME_GET_MICROSEC(localent->f_counts.f_total_time);
+	shfuncent->stats.f_self_time +=
+		INSTR_TIME_GET_MICROSEC(localent->f_counts.f_self_time);
 
-		/* 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);
+	pgstat_shared_stat_unlock(shared_ref);
 
-		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;
+	return true;
 }
 
-/*
+/* ----------
  * 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();
+	PgStatSharedRef *shared_ref;
 
-	if (pgStatFunctions == NULL)
-		return NULL;
+	shared_ref = pgstat_pending_fetch(PGSTAT_KIND_FUNCTION, MyDatabaseId, func_id);
 
-	return (PgStat_BackendFunctionEntry *) hash_search(pgStatFunctions,
-													   (void *) &func_id,
-													   HASH_FIND, NULL);
+	if (shared_ref)
+		return shared_ref->pending;
+	return NULL;
+}
+
+/* ----------
+ * pgstat_fetch_stat_funcentry() -
+ *
+ *	Support function for the SQL-callable pgstat* functions. Returns
+ *	the collected statistics for one function or NULL.
+ *
+ *  The returned entry is stored in static memory so the content is valid until
+ *	the next call of the same function for the different function id.
+ * ----------
+ */
+PgStat_StatFuncEntry *
+pgstat_fetch_stat_funcentry(Oid func_id)
+{
+	return (PgStat_StatFuncEntry *)
+		pgstat_fetch_entry(PGSTAT_KIND_FUNCTION, MyDatabaseId, func_id);
 }
diff --git a/src/backend/utils/activity/pgstat_global.c b/src/backend/utils/activity/pgstat_global.c
index 295763d8f3e..8ebef48c51a 100644
--- a/src/backend/utils/activity/pgstat_global.c
+++ b/src/backend/utils/activity/pgstat_global.c
@@ -20,6 +20,7 @@
 #include "executor/instrument.h"
 #include "replication/slot.h"
 #include "utils/builtins.h"
+#include "utils/memutils.h"
 #include "utils/pgstat_internal.h"
 #include "utils/timestamp.h"
 
@@ -29,27 +30,25 @@
  * ----------
  */
 
-/*
- * 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;
+PgStat_BgWriterStats PendingBgWriterStats = {0};
+PgStat_CheckpointerStats PendingCheckpointerStats = {0};
+PgStat_WalStats	WalStats = {0};
+
 
 /*
- * 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[].
+ * SLRU statistics counts waiting to be written to the shared activity
+ * statistics.  We assume this variable inits to zeroes.  Entries are
+ * one-to-one with slru_names[].
+ * Changes of SLRU counters are reported within critical sections so we use
+ * static memory in order to avoid memory allocation.
  */
-static PgStat_MsgSLRU SLRUStats[SLRU_NUM_ELEMENTS];
+static PgStat_SLRUStats pending_SLRUStats[SLRU_NUM_ELEMENTS];
+bool have_slrustats = false;
 
 /*
  * 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
+ * pgstat_report_wal(). This is used to calculate how much WAL usage
+ * happens between pgstat_report_wal() calls, by subtracting
  * the previous counters from the current ones.
  *
  * FIXME: It'd be better if this weren't global.
@@ -60,85 +59,181 @@ WalUsage prevWalUsage;
 /* ----------
  * pgstat_reset_shared_counters() -
  *
- *	Tell the statistics collector to reset cluster-wide shared counters.
+ *	Reset cluster-wide shared counters.
  *
  *	Permission checking for this function is managed through the normal
  *	GRANT system.
+ *
+ *  We don't scribble on shared stats while resetting to avoid locking on
+ *  shared stats struct. Instead, just record the current counters in another
+ *  shared struct, which is protected by StatsLock. See
+ *  pgstat_fetch_stat_(archiver|bgwriter|checkpointer) for the reader side.
  * ----------
  */
 void
 pgstat_reset_shared_counters(const char *target)
 {
-	PgStat_MsgResetsharedcounter msg;
-
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
+	TimestampTz now = GetCurrentTimestamp();
+	PgStat_Shared_Reset_Target t;
 
 	if (strcmp(target, "archiver") == 0)
-		msg.m_resettarget = RESET_ARCHIVER;
+		t = RESET_ARCHIVER;
 	else if (strcmp(target, "bgwriter") == 0)
-		msg.m_resettarget = RESET_BGWRITER;
+		t = RESET_BGWRITER;
 	else if (strcmp(target, "wal") == 0)
-		msg.m_resettarget = RESET_WAL;
+		t = 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));
+	/* Reset statistics for the cluster. */
+
+	switch (t)
+	{
+		case RESET_ARCHIVER:
+			LWLockAcquire(StatsLock, LW_EXCLUSIVE);
+			pgstat_copy_global_stats(&pgStatShmem->archiver.reset_offset,
+									 &pgStatShmem->archiver.stats,
+									 sizeof(PgStat_ArchiverStats),
+									 &pgStatShmem->archiver.changecount);
+			pgStatShmem->archiver.reset_offset.stat_reset_timestamp = now;
+			LWLockRelease(StatsLock);
+			break;
+
+		case RESET_BGWRITER:
+			LWLockAcquire(StatsLock, LW_EXCLUSIVE);
+			pgstat_copy_global_stats(&pgStatShmem->bgwriter.reset_offset,
+									 &pgStatShmem->bgwriter.stats,
+									 sizeof(PgStat_BgWriterStats),
+									 &pgStatShmem->bgwriter.changecount);
+			pgstat_copy_global_stats(&pgStatShmem->checkpointer.reset_offset,
+									 &pgStatShmem->checkpointer.stats,
+									 sizeof(PgStat_CheckpointerStats),
+									 &pgStatShmem->checkpointer.changecount);
+			pgStatShmem->bgwriter.reset_offset.stat_reset_timestamp = now;
+			LWLockRelease(StatsLock);
+			break;
+
+		case RESET_WAL:
+
+			/*
+			 * Differently from the two cases above, WAL statistics has many
+			 * writer processes with the shared stats protected by
+			 * pgStatShmem->wal.lock.
+			 */
+			LWLockAcquire(&pgStatShmem->wal.lock, LW_EXCLUSIVE);
+			MemSet(&pgStatShmem->wal.stats, 0, sizeof(PgStat_WalStats));
+			pgStatShmem->wal.stats.stat_reset_timestamp = now;
+			LWLockRelease(&pgStatShmem->wal.lock);
+			break;
+	}
 }
 
 /* ----------
- * pgstat_send_archiver() -
+ * pgstat_report_archiver() -
  *
- *	Tell the collector about the WAL file that we successfully
- *	archived or failed to archive.
+ * Report archiver statistics
  * ----------
  */
 void
-pgstat_send_archiver(const char *xlog, bool failed)
+pgstat_report_archiver(const char *xlog, bool failed)
 {
-	PgStat_MsgArchiver msg;
+	TimestampTz now = GetCurrentTimestamp();
 
-	/*
-	 * 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));
+	changecount_before_write(&pgStatShmem->archiver.changecount);
+
+	if (failed)
+	{
+		++pgStatShmem->archiver.stats.failed_count;
+		memcpy(&pgStatShmem->archiver.stats.last_failed_wal, xlog,
+			   sizeof(pgStatShmem->archiver.stats.last_failed_wal));
+		pgStatShmem->archiver.stats.last_failed_timestamp = now;
+	}
+	else
+	{
+		++pgStatShmem->archiver.stats.archived_count;
+		memcpy(&pgStatShmem->archiver.stats.last_archived_wal, xlog,
+			   sizeof(pgStatShmem->archiver.stats.last_archived_wal));
+		pgStatShmem->archiver.stats.last_archived_timestamp = now;
+	}
+
+	changecount_after_write(&pgStatShmem->archiver.changecount);
+}
+
+PgStat_ArchiverStats *
+pgstat_fetch_stat_archiver(void)
+{
+	pgstat_snapshot_global(PGSTAT_KIND_ARCHIVER);
+
+	return &stats_snapshot.archiver;
+}
+
+void
+pgstat_snapshot_archiver(void)
+{
+	PgStat_ArchiverStats reset;
+	PgStat_ArchiverStats *reset_offset = &pgStatShmem->archiver.reset_offset;
+
+	pgstat_copy_global_stats(&stats_snapshot.archiver,
+							 &pgStatShmem->archiver.stats,
+							 sizeof(PgStat_ArchiverStats),
+							 &pgStatShmem->archiver.changecount);
+
+	LWLockAcquire(StatsLock, LW_SHARED);
+	memcpy(&reset, reset_offset, sizeof(PgStat_ArchiverStats));
+	LWLockRelease(StatsLock);
+
+	/* compensate by reset offsets */
+	if (stats_snapshot.archiver.archived_count == reset.archived_count)
+	{
+		stats_snapshot.archiver.last_archived_wal[0] = 0;
+		stats_snapshot.archiver.last_archived_timestamp = 0;
+	}
+	stats_snapshot.archiver.archived_count -= reset.archived_count;
+
+	if (stats_snapshot.archiver.failed_count == reset.failed_count)
+	{
+		stats_snapshot.archiver.last_failed_wal[0] = 0;
+		stats_snapshot.archiver.last_failed_timestamp = 0;
+	}
+	stats_snapshot.archiver.failed_count -= reset.failed_count;
+
+	stats_snapshot.archiver.stat_reset_timestamp = reset.stat_reset_timestamp;
 }
 
 /* ----------
- * pgstat_send_bgwriter() -
+ * pgstat_report_bgwriter() -
  *
- *		Send bgwriter statistics to the collector
+ * Report bgwriter statistics
  * ----------
  */
 void
-pgstat_send_bgwriter(void)
+pgstat_report_bgwriter(void)
 {
-	/* We assume this initializes to zeroes */
-	static const PgStat_MsgBgWriter all_zeroes;
+	static const PgStat_BgWriterStats all_zeroes;
+	PgStat_BgWriterStats *s = &pgStatShmem->bgwriter.stats;
+	PgStat_BgWriterStats *l = &PendingBgWriterStats;
+
+	Assert(!pgStatShmem->is_shutdown);
 
 	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.
+	 * this case, avoid taking lock for a completely empty stats.
 	 */
-	if (memcmp(&PendingBgWriterStats, &all_zeroes, sizeof(PgStat_MsgBgWriter)) == 0)
+	if (memcmp(&PendingBgWriterStats, &all_zeroes, sizeof(PgStat_BgWriterStats)) == 0)
 		return;
 
-	/*
-	 * Prepare and send the message
-	 */
-	pgstat_setheader(&PendingBgWriterStats.m_hdr, PGSTAT_MTYPE_BGWRITER);
-	pgstat_send(&PendingBgWriterStats, sizeof(PendingBgWriterStats));
+	changecount_before_write(&pgStatShmem->bgwriter.changecount);
+
+	s->buf_written_clean += l->buf_written_clean;
+	s->maxwritten_clean += l->maxwritten_clean;
+	s->buf_alloc += l->buf_alloc;
+
+	changecount_after_write(&pgStatShmem->bgwriter.changecount);
 
 	/*
 	 * Clear out the statistics buffer, so it can be re-used.
@@ -146,31 +241,69 @@ pgstat_send_bgwriter(void)
 	MemSet(&PendingBgWriterStats, 0, sizeof(PendingBgWriterStats));
 }
 
+PgStat_BgWriterStats *
+pgstat_fetch_stat_bgwriter(void)
+{
+	pgstat_snapshot_global(PGSTAT_KIND_BGWRITER);
+
+	return &stats_snapshot.bgwriter;
+}
+
+void
+pgstat_snapshot_bgwriter(void)
+{
+	PgStat_BgWriterStats reset;
+	PgStat_BgWriterStats *reset_offset = &pgStatShmem->bgwriter.reset_offset;
+
+	pgstat_copy_global_stats(&stats_snapshot.bgwriter,
+							 &pgStatShmem->bgwriter.stats,
+							 sizeof(PgStat_BgWriterStats),
+							 &pgStatShmem->bgwriter.changecount);
+
+	LWLockAcquire(StatsLock, LW_SHARED);
+	memcpy(&reset, reset_offset, sizeof(PgStat_BgWriterStats));
+	LWLockRelease(StatsLock);
+
+	/* compensate by reset offsets */
+	stats_snapshot.bgwriter.buf_written_clean -= reset.buf_written_clean;
+	stats_snapshot.bgwriter.maxwritten_clean -= reset.maxwritten_clean;
+	stats_snapshot.bgwriter.buf_alloc -= reset.buf_alloc;
+	stats_snapshot.bgwriter.stat_reset_timestamp = reset.stat_reset_timestamp;
+}
+
 /* ----------
- * pgstat_send_checkpointer() -
+ * pgstat_report_checkpointer() -
  *
- *		Send checkpointer statistics to the collector
+ * Report checkpointer statistics
  * ----------
  */
 void
-pgstat_send_checkpointer(void)
+pgstat_report_checkpointer(void)
 {
 	/* We assume this initializes to zeroes */
-	static const PgStat_MsgCheckpointer all_zeroes;
+	static const PgStat_CheckpointerStats all_zeroes;
+	PgStat_CheckpointerStats *s = &pgStatShmem->checkpointer.stats;
+	PgStat_CheckpointerStats *l = &PendingCheckpointerStats;
 
 	/*
 	 * 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.
+	 * this case, avoid taking lock for a completely empty stats.
 	 */
-	if (memcmp(&PendingCheckpointerStats, &all_zeroes, sizeof(PgStat_MsgCheckpointer)) == 0)
+	if (memcmp(&PendingCheckpointerStats, &all_zeroes,
+			   sizeof(PgStat_CheckpointerStats)) == 0)
 		return;
 
-	/*
-	 * Prepare and send the message
-	 */
-	pgstat_setheader(&PendingCheckpointerStats.m_hdr, PGSTAT_MTYPE_CHECKPOINTER);
-	pgstat_send(&PendingCheckpointerStats, sizeof(PendingCheckpointerStats));
+	changecount_before_write(&pgStatShmem->checkpointer.changecount);
+
+	s->timed_checkpoints += l->timed_checkpoints;
+	s->requested_checkpoints += l->requested_checkpoints;
+	s->checkpoint_write_time += l->checkpoint_write_time;
+	s->checkpoint_sync_time += l->checkpoint_sync_time;
+	s->buf_written_checkpoints += l->buf_written_checkpoints;
+	s->buf_written_backend += l->buf_written_backend;
+	s->buf_fsync_backend += l->buf_fsync_backend;
+
+	changecount_after_write(&pgStatShmem->checkpointer.changecount);
 
 	/*
 	 * Clear out the statistics buffer, so it can be re-used.
@@ -178,6 +311,39 @@ pgstat_send_checkpointer(void)
 	MemSet(&PendingCheckpointerStats, 0, sizeof(PendingCheckpointerStats));
 }
 
+PgStat_CheckpointerStats *
+pgstat_fetch_stat_checkpointer(void)
+{
+	pgstat_snapshot_global(PGSTAT_KIND_CHECKPOINTER);
+
+	return &stats_snapshot.checkpointer;
+}
+
+void
+pgstat_snapshot_checkpointer(void)
+{
+	PgStat_CheckpointerStats reset;
+	PgStat_CheckpointerStats *reset_offset = &pgStatShmem->checkpointer.reset_offset;
+
+	pgstat_copy_global_stats(&stats_snapshot.checkpointer,
+							 &pgStatShmem->checkpointer.stats,
+							 sizeof(PgStat_CheckpointerStats),
+							 &pgStatShmem->checkpointer.changecount);
+
+	LWLockAcquire(StatsLock, LW_SHARED);
+	memcpy(&reset, reset_offset, sizeof(PgStat_CheckpointerStats));
+	LWLockRelease(StatsLock);
+
+	/* compensate by reset offsets */
+	stats_snapshot.checkpointer.timed_checkpoints -= reset.timed_checkpoints;
+	stats_snapshot.checkpointer.requested_checkpoints -= reset.requested_checkpoints;
+	stats_snapshot.checkpointer.buf_written_checkpoints -= reset.buf_written_checkpoints;
+	stats_snapshot.checkpointer.buf_written_backend -= reset.buf_written_backend;
+	stats_snapshot.checkpointer.buf_fsync_backend -= reset.buf_fsync_backend;
+	stats_snapshot.checkpointer.checkpoint_write_time -= reset.checkpoint_write_time;
+	stats_snapshot.checkpointer.checkpoint_sync_time -= reset.checkpoint_sync_time;
+}
+
 /* ----------
  * pgstat_reset_replslot_counter() -
  *
@@ -191,125 +357,269 @@ pgstat_send_checkpointer(void)
 void
 pgstat_reset_replslot_counter(const char *name)
 {
-	PgStat_MsgResetreplslotcounter msg;
+	int			startidx;
+	int			endidx;
+	int			i;
+	TimestampTz ts;
 
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
+	/*
+	 * AFIXME: pgstats has business no looking into slot.c structures at
+	 * this level of detail.
+	 */
 
 	if (name)
 	{
-		namestrcpy(&msg.m_slotname, name);
-		msg.clearall = false;
+		ReplicationSlot *slot;
+
+		/* Check if the slot exits with the given name. */
+		slot = SearchNamedReplicationSlot(name, LW_SHARED);
+
+		if (!slot)
+			ereport(ERROR,
+					(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+					 errmsg("replication slot \"%s\" does not exist",
+							name)));
+
+		/*
+		 * Nothing to do for physical slots as we collect stats only for
+		 * logical slots.
+		 */
+		if (SlotIsPhysical(slot))
+			return;
+
+		/* reset this one entry */
+		startidx = endidx = slot - ReplicationSlotCtl->replication_slots;
 	}
 	else
-		msg.clearall = true;
+	{
+		/* reset all existent entries */
+		startidx = 0;
+		endidx = max_replication_slots - 1;
+	}
 
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETREPLSLOTCOUNTER);
+	ts = GetCurrentTimestamp();
+	LWLockAcquire(&pgStatShmem->replslot.lock, LW_EXCLUSIVE);
+	for (i = startidx; i <= endidx; i++)
+	{
+		PgStat_StatReplSlotEntry *statent = &pgStatShmem->replslot.stats[i];
+		size_t off;
 
-	pgstat_send(&msg, sizeof(msg));
+		off = offsetof(PgStat_StatReplSlotEntry, slotname) + sizeof(NameData);
+
+		memset(((char *) statent) + off, 0, sizeof(pgStatShmem->replslot.stats[i]) - off);
+		statent->stat_reset_timestamp = ts;
+	}
+	LWLockRelease(&pgStatShmem->replslot.lock);
 }
 
 /* ----------
  * pgstat_report_replslot() -
  *
- *	Tell the collector about replication slot statistics.
+ * Report replication slot activity.
  * ----------
  */
 void
-pgstat_report_replslot(const PgStat_StatReplSlotEntry *repSlotStat)
+pgstat_report_replslot(uint32 index, const PgStat_StatReplSlotEntry *repSlotStat)
 {
-	PgStat_MsgReplSlot msg;
+	PgStat_StatReplSlotEntry *statent;
 
-	/*
-	 * 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));
+	Assert(index < max_replication_slots);
+
+	if (!pgstat_track_counts)
+		return;
+
+	statent = &pgStatShmem->replslot.stats[index];
+
+	LWLockAcquire(&pgStatShmem->replslot.lock, LW_EXCLUSIVE);
+
+	/* clear the counters if not used */
+	if (statent->index == -1)
+	{
+		memset(statent, 0, sizeof(*statent));
+		statent->index = index;
+		namestrcpy(&statent->slotname, NameStr(repSlotStat->slotname));
+	}
+	else if (namestrcmp(&statent->slotname, NameStr(statent->slotname)) != 0)
+	{
+		/* AFIXME: Is there a valid way this can happen? */
+		elog(ERROR, "stats out of sync");
+	}
+	else
+	{
+		Assert(statent->index == index);
+	}
+
+	/* Update the replication slot statistics */
+	statent->spill_txns += repSlotStat->spill_txns;
+	statent->spill_count += repSlotStat->spill_count;
+	statent->spill_bytes += repSlotStat->spill_bytes;
+	statent->stream_txns += repSlotStat->stream_txns;
+	statent->stream_count += repSlotStat->stream_count;
+	statent->stream_bytes += repSlotStat->stream_bytes;
+	statent->total_txns += repSlotStat->total_txns;
+	statent->total_bytes += repSlotStat->total_bytes;
+
+	LWLockRelease(&pgStatShmem->replslot.lock);
 }
 
+
 /* ----------
- * pgstat_report_replslot_create() -
+ * pgstat_report_replslot_drop() -
  *
- *	Tell the collector about creating the replication slot.
+ * Report replication slot drop.
  * ----------
  */
 void
-pgstat_report_replslot_create(const char *slotname)
+pgstat_report_replslot_create(const char *name, uint32 index)
 {
-	PgStat_MsgReplSlot msg;
+	PgStat_StatReplSlotEntry *statent;
 
-	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));
+	if (!pgstat_track_counts)
+		return;
+
+	statent = &pgStatShmem->replslot.stats[index];
+
+	LWLockAcquire(&pgStatShmem->replslot.lock, LW_EXCLUSIVE);
+	/*
+	 * NB: need to accept that there might be stats from an older slot,
+	 * e.g. if we previously crashed after dropping a slot.
+	 */
+	memset(statent, 0, sizeof(*statent));
+	statent->index = index;
+	namestrcpy(&statent->slotname, name);
+
+	LWLockRelease(&pgStatShmem->replslot.lock);
 }
 
 /* ----------
  * pgstat_report_replslot_drop() -
  *
- *	Tell the collector about dropping the replication slot.
+ * Report replication slot drop.
  * ----------
  */
 void
-pgstat_report_replslot_drop(const char *slotname)
+pgstat_report_replslot_drop(uint32 index)
 {
-	PgStat_MsgReplSlot msg;
+	PgStat_StatReplSlotEntry *statent;
 
-	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));
+	if (!pgstat_track_counts)
+		return;
+
+	statent = &pgStatShmem->replslot.stats[index];
+
+	LWLockAcquire(&pgStatShmem->replslot.lock, LW_EXCLUSIVE);
+	/*
+	 * NB: need to accept that there might not be any stats, e.g. if we threw
+	 * away stats after a crash restart.
+	 */
+	statent->index = -1;
+	LWLockRelease(&pgStatShmem->replslot.lock);
 }
 
-/* ----------
- * pgstat_send_slru() -
- *
- *		Send SLRU statistics to the collector
- * ----------
- */
-void
-pgstat_send_slru(void)
+static void
+pgstat_reset_slru_counter_internal(int index, TimestampTz ts)
 {
-	/* We assume this initializes to zeroes */
-	static const PgStat_MsgSLRU all_zeroes;
+	LWLockAcquire(&pgStatShmem->slru.lock, LW_EXCLUSIVE);
 
-	for (int i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	memset(&pgStatShmem->slru.stats[index], 0, sizeof(PgStat_SLRUStats));
+	pgStatShmem->slru.stats[index].stat_reset_timestamp = ts;
+
+	LWLockRelease(&pgStatShmem->slru.lock);
+}
+
+PgStat_StatReplSlotEntry *
+pgstat_fetch_replslot(NameData slotname)
+{
+	pgstat_snapshot_global(PGSTAT_KIND_REPLSLOT);
+
+	for (int i = 0; i < stats_snapshot.replslot_count; 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;
+		PgStat_StatReplSlotEntry *statentry = &stats_snapshot.replslot[i];
 
-		/* 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));
+		if (namestrcmp(&statentry->slotname, NameStr(slotname)) == 0)
+		{
+			return statentry;
+		}
 	}
+
+	return NULL;
+}
+
+void
+pgstat_snapshot_replslot(void)
+{
+	if (stats_snapshot.replslot == NULL)
+	{
+		stats_snapshot.replslot = (PgStat_StatReplSlotEntry *)
+			MemoryContextAlloc(TopMemoryContext,
+							   sizeof(PgStat_StatReplSlotEntry) * max_replication_slots);
+	}
+
+	stats_snapshot.replslot_count = 0;
+
+	LWLockAcquire(&pgStatShmem->replslot.lock, LW_EXCLUSIVE);
+
+	for (int i = 0; i < max_replication_slots; i++)
+	{
+		PgStat_StatReplSlotEntry *statent = &pgStatShmem->replslot.stats[i];
+
+		if (statent->index != -1)
+		{
+			stats_snapshot.replslot[stats_snapshot.replslot_count++] = *statent;
+		}
+	}
+
+	LWLockRelease(&pgStatShmem->replslot.lock);
+}
+
+/*
+ * pgstat_flush_slru - flush out locally pending SLRU stats entries
+ *
+ * If nowait is true, this function returns false on lock failure. Otherwise
+ * this function always returns true. Writer processes are mutually excluded
+ * using LWLock, but readers are expected to use change-count protocol to avoid
+ * interference with writers.
+ *
+ * Returns true if not all pending stats have been flushed out.
+ */
+bool
+pgstat_flush_slru(bool nowait)
+{
+	int			i;
+
+	if (!have_slrustats)
+		return false;
+
+	/* lock the shared entry to protect the content, skip if failed */
+	if (!nowait)
+		LWLockAcquire(&pgStatShmem->slru.lock, LW_EXCLUSIVE);
+	else if (!LWLockConditionalAcquire(&pgStatShmem->slru.lock,
+									   LW_EXCLUSIVE))
+		return true;			/* failed to acquire lock, skip */
+
+
+	for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+	{
+		PgStat_SLRUStats *sharedent = &pgStatShmem->slru.stats[i];
+		PgStat_SLRUStats *pendingent = &pending_SLRUStats[i];
+
+		sharedent->blocks_zeroed += pendingent->blocks_zeroed;
+		sharedent->blocks_hit += pendingent->blocks_hit;
+		sharedent->blocks_read += pendingent->blocks_read;
+		sharedent->blocks_written += pendingent->blocks_written;
+		sharedent->blocks_exists += pendingent->blocks_exists;
+		sharedent->flush += pendingent->flush;
+		sharedent->truncate += pendingent->truncate;
+	}
+
+	/* done, clear the pending entry */
+	MemSet(pending_SLRUStats, 0, SizeOfSlruStats);
+
+	LWLockRelease(&pgStatShmem->slru.lock);
+
+	have_slrustats = false;
+
+	return false;
 }
 
 /* ----------
@@ -325,15 +635,19 @@ pgstat_send_slru(void)
 void
 pgstat_reset_slru_counter(const char *name)
 {
-	PgStat_MsgResetslrucounter msg;
+	int			i;
+	TimestampTz ts = GetCurrentTimestamp();
 
-	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));
+	if (name)
+	{
+		i = pgstat_slru_index(name);
+		pgstat_reset_slru_counter_internal(i, ts);
+	}
+	else
+	{
+		for (i = 0; i < SLRU_NUM_ELEMENTS; i++)
+			pgstat_reset_slru_counter_internal(i, ts);
+	}
 }
 
 /*
@@ -342,7 +656,7 @@ pgstat_reset_slru_counter(const char *name)
  * Returns pointer to entry with counters for given SLRU (based on the name
  * stored in SlruCtl as lwlock tranche name).
  */
-static inline PgStat_MsgSLRU *
+static inline PgStat_SLRUStats *
 slru_entry(int slru_idx)
 {
 	pgstat_assert_is_up();
@@ -355,7 +669,9 @@ slru_entry(int slru_idx)
 
 	Assert((slru_idx >= 0) && (slru_idx < SLRU_NUM_ELEMENTS));
 
-	return &SLRUStats[slru_idx];
+	have_slrustats = true;
+
+	return &pending_SLRUStats[slru_idx];
 }
 
 /*
@@ -365,43 +681,43 @@ slru_entry(int slru_idx)
 void
 pgstat_count_slru_page_zeroed(int slru_idx)
 {
-	slru_entry(slru_idx)->m_blocks_zeroed += 1;
+	slru_entry(slru_idx)->blocks_zeroed += 1;
 }
 
 void
 pgstat_count_slru_page_hit(int slru_idx)
 {
-	slru_entry(slru_idx)->m_blocks_hit += 1;
+	slru_entry(slru_idx)->blocks_hit += 1;
 }
 
 void
 pgstat_count_slru_page_exists(int slru_idx)
 {
-	slru_entry(slru_idx)->m_blocks_exists += 1;
+	slru_entry(slru_idx)->blocks_exists += 1;
 }
 
 void
 pgstat_count_slru_page_read(int slru_idx)
 {
-	slru_entry(slru_idx)->m_blocks_read += 1;
+	slru_entry(slru_idx)->blocks_read += 1;
 }
 
 void
 pgstat_count_slru_page_written(int slru_idx)
 {
-	slru_entry(slru_idx)->m_blocks_written += 1;
+	slru_entry(slru_idx)->blocks_written += 1;
 }
 
 void
 pgstat_count_slru_flush(int slru_idx)
 {
-	slru_entry(slru_idx)->m_flush += 1;
+	slru_entry(slru_idx)->flush += 1;
 }
 
 void
 pgstat_count_slru_truncate(int slru_idx)
 {
-	slru_entry(slru_idx)->m_truncate += 1;
+	slru_entry(slru_idx)->truncate += 1;
 }
 
 /*
@@ -442,25 +758,115 @@ pgstat_slru_index(const char *name)
 	return (SLRU_NUM_ELEMENTS - 1);
 }
 
+void
+pgstat_snapshot_slru(void)
+{
+	LWLockAcquire(&pgStatShmem->slru.lock, LW_SHARED);
+
+	memcpy(stats_snapshot.slru, &pgStatShmem->slru.stats, SizeOfSlruStats);
+
+	LWLockRelease(&pgStatShmem->slru.lock);
+}
+
+PgStat_SLRUStats *
+pgstat_fetch_slru(void)
+{
+	pgstat_snapshot_global(PGSTAT_KIND_SLRU);
+
+	return stats_snapshot.slru;
+}
+
+/*
+ * Calculate how much WAL usage counters have increased by substracting the
+ * previous counters from the current ones.
+ *
+ * If nowait is true, this function returns true if the lock could not be
+ * acquired. Otherwise return false.
+ */
+bool
+pgstat_flush_wal(bool nowait)
+{
+	PgStat_WalStats *s = &pgStatShmem->wal.stats;
+	PgStat_WalStats *l = &WalStats;
+	WalUsage	all_zeroes PG_USED_FOR_ASSERTS_ONLY = {0};
+
+	Assert(IsUnderPostmaster || !IsPostmasterEnvironment);
+	Assert(pgStatShmem != NULL);
+
+	/*
+	 * We don't update the WAL usage portion of the local WalStats elsewhere.
+	 * Instead, fill in that portion with the difference of pgWalUsage since
+	 * the previous call.
+	 */
+	Assert(memcmp(&l->wal_usage, &all_zeroes, sizeof(WalUsage)) == 0);
+	WalUsageAccumDiff(&l->wal_usage, &pgWalUsage, &prevWalUsage);
+
+	/*
+	 * This function can be called even if nothing at all has happened. Avoid
+	 * taking lock for nothing in that case.
+	 */
+	if (!walstats_pending())
+		return false;
+
+	/* lock the shared entry to protect the content, skip if failed */
+	if (!nowait)
+		LWLockAcquire(&pgStatShmem->wal.lock, LW_EXCLUSIVE);
+	else if (!LWLockConditionalAcquire(&pgStatShmem->wal.lock,
+									   LW_EXCLUSIVE))
+	{
+		MemSet(l, 0, sizeof(WalStats));
+		return true;			/* failed to acquire lock, skip */
+	}
+
+	s->wal_usage.wal_records += l->wal_usage.wal_records;
+	s->wal_usage.wal_fpi += l->wal_usage.wal_fpi;
+	s->wal_usage.wal_bytes += l->wal_usage.wal_bytes;
+	s->wal_buffers_full += l->wal_buffers_full;
+	s->wal_write += l->wal_write;
+	s->wal_write_time += l->wal_write_time;
+	s->wal_sync += l->wal_sync;
+	s->wal_sync_time += l->wal_sync_time;
+	LWLockRelease(&pgStatShmem->wal.lock);
+
+	/*
+	 * Save the current counters for the subsequent calculation of WAL usage.
+	 */
+	prevWalUsage = pgWalUsage;
+
+	/*
+	 * Clear out the statistics buffer, so it can be re-used.
+	 */
+	MemSet(&WalStats, 0, sizeof(WalStats));
+
+	return false;
+}
+
 /* ----------
- * pgstat_send_wal() -
+ * pgstat_report_wal() -
  *
- *	Send WAL statistics to the collector.
+ * Calculate how much WAL usage counters have increased and update
+ * shared statistics.
  *
- * 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.
+ * Must be called by processes that generate WAL.
  * ----------
  */
 void
-pgstat_send_wal(bool force)
+pgstat_report_wal(bool force)
 {
-	static TimestampTz sendTime = 0;
+	Assert(!pgStatShmem->is_shutdown);
 
+	pgstat_flush_wal(force);
+}
+
+/*
+ * XXXX: always try to flush WAL stats. We don't want to manipulate another
+ * counter during XLogInsert so we don't have an effecient short cut to know
+ * whether any counter gets incremented.
+ */
+bool
+walstats_pending(void)
+{
 	/*
-	 * 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
@@ -477,72 +883,38 @@ pgstat_send_wal(bool force)
 	 * and syncs are also checked.
 	 */
 	if (pgWalUsage.wal_records == prevWalUsage.wal_records &&
-		WalStats.m_wal_write == 0 && WalStats.m_wal_sync == 0)
+		WalStats.wal_write == 0 && WalStats.wal_sync == 0)
 	{
-		Assert(WalStats.m_wal_buffers_full == 0);
-		return;
+		Assert(WalStats.wal_buffers_full == 0);
+		return false;
 	}
 
-	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));
+	return true;
 }
 
 void
 pgstat_wal_initialize(void)
 {
 	/*
-	 * Initialize prevWalUsage with pgWalUsage so that pgstat_send_wal() can
+	 * Initialize prevWalUsage with pgWalUsage so that pgstat_flush_wal() can
 	 * calculate how much pgWalUsage counters are increased by subtracting
 	 * prevWalUsage from pgWalUsage.
 	 */
 	prevWalUsage = pgWalUsage;
 }
+
+PgStat_WalStats *
+pgstat_fetch_stat_wal(void)
+{
+	pgstat_snapshot_global(PGSTAT_KIND_WAL);
+
+	return &stats_snapshot.wal;
+}
+
+void
+pgstat_snapshot_wal(void)
+{
+	LWLockAcquire(StatsLock, LW_SHARED);
+	memcpy(&stats_snapshot.wal, &pgStatShmem->wal.stats, sizeof(PgStat_WalStats));
+	LWLockRelease(StatsLock);
+}
diff --git a/src/backend/utils/activity/pgstat_relation.c b/src/backend/utils/activity/pgstat_relation.c
index 074c4b8958d..80c5193e644 100644
--- a/src/backend/utils/activity/pgstat_relation.c
+++ b/src/backend/utils/activity/pgstat_relation.c
@@ -27,38 +27,6 @@
 #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
 {
@@ -75,27 +43,13 @@ typedef struct TwoPhasePgStatRecord
 } TwoPhasePgStatRecord;
 
 
-static PgStat_TableStatus *get_tabstat_entry(Oid rel_id, bool isshared);
-static void pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg, TimestampTz now);
+static PgStat_TableStatus *pgstat_pending_tab_prepare(Oid rel_id, bool isshared);
 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;
-
-
 /*
  * Copy stats between relations. This is used for things like REINDEX
  * CONCURRENTLY.
@@ -104,48 +58,41 @@ void
 pgstat_copy_relation_stats(Relation dst, Relation src)
 {
 	PgStat_StatTabEntry *srcstats;
+	PgStatShm_StatTabEntry *dstshstats;
+	PgStatSharedRef *dst_ref;
 
-	srcstats = pgstat_fetch_stat_tabentry(RelationGetRelid(src));
-
+	srcstats = pgstat_fetch_stat_tabentry_extended(src->rd_rel->relisshared,
+												   RelationGetRelid(src));
 	if (!srcstats)
 		return;
 
-	if (pgstat_relation_should_count(dst))
-	{
-		/*
-		 * XXX: temporarily this does not actually quite do what the name says,
-		 * and just copy index related fields. A subsequent commit will do more.
-		 */
+	/*
+	 * XXX: Is it actually correct to copy all stats here? Probably fine with
+	 * current uses of stats for indexes, but what we tracked bloat for
+	 * indexes via stats?
+	 */
+	dst_ref = pgstat_shared_ref_get(PGSTAT_KIND_TABLE,
+									dst->rd_rel->relisshared ? InvalidOid : MyDatabaseId,
+									RelationGetRelid(dst),
+									true);
+	pgstat_shared_stat_lock(dst_ref, false);
 
-		dst->pgstat_info->t_counts.t_numscans = srcstats->numscans;
-		dst->pgstat_info->t_counts.t_tuples_returned = srcstats->tuples_returned;
-		dst->pgstat_info->t_counts.t_tuples_fetched = srcstats->tuples_fetched;
-		dst->pgstat_info->t_counts.t_blocks_fetched = srcstats->blocks_fetched;
-		dst->pgstat_info->t_counts.t_blocks_hit = srcstats->blocks_hit;
+	dstshstats = (PgStatShm_StatTabEntry *) dst_ref->shared_stats;
+	dstshstats->stats = *srcstats;
 
-		/*
-		 * The data will be sent by the next pgstat_report_stat()
-		 * call.
-		 */
-	}
+	pgstat_shared_stat_unlock(dst_ref);
 }
 
-/* ----------
- * pgstat_relation_init() -
+/*
+ * Initialize a relcache entry to count access statistics.  Called whenever a
+ * relation is opened.
  *
- *	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.
- * ----------
+ * 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.
  */
 void
 pgstat_relation_init(Relation rel)
 {
-	Oid			rel_id = rel->rd_id;
 	char		relkind = rel->rd_rel->relkind;
 
 	/*
@@ -158,7 +105,7 @@ pgstat_relation_init(Relation rel)
 		return;
 	}
 
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+	if (!pgstat_track_counts)
 	{
 		/* We're not counting at all */
 		rel->pgstat_enabled = false;
@@ -166,60 +113,89 @@ pgstat_relation_init(Relation rel)
 		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;
 }
 
+/*
+ * Prepare for statistics for this relation to be collected. This ensures we
+ * have a reference to the shared stats entry. That is important because a
+ * relation drop in another connection can otherwise lead to the shared stats
+ * entry being dropped, which we then later would re-create when flushing
+ * stats.
+ *
+ * This is separate from pgstat_relation_init() as it is not uncommon for
+ * relcache entries to be opened without ever getting stats reported.
+ */
 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);
+	/* Else find or make the PgStat_TableStatus entry, and update link */
+	rel->pgstat_info = pgstat_pending_tab_prepare(RelationGetRelid(rel),
+												  rel->rd_rel->relisshared);
+	/* mark this relation as the owner */
+
+	/* don't allow link a stats to multiple relcache entries */
+	Assert(rel->pgstat_info->relation == NULL);
+	rel->pgstat_info->relation = rel;
+}
+
+/*
+ * Break the mutual link between a relcache entry and a local stats entry.
+ * This must be called always when one end of the link is removed.
+ */
+void
+pgstat_relation_delink(Relation rel)
+{
+	/* remove the link to stats info if any */
+	if (rel && rel->pgstat_info)
+	{
+		/* link sanity check */
+		Assert(rel->pgstat_info->relation == rel);
+		rel->pgstat_info->relation = NULL;
+		rel->pgstat_info = NULL;
+	}
+}
+
+/*
+ * Ensure that stats are dropped if transaction aborts.
+ */
+void
+pgstat_create_relation(Relation rel)
+{
+	pgstat_schedule_create(PGSTAT_KIND_TABLE,
+						   rel->rd_rel->relisshared ? InvalidOid : MyDatabaseId,
+						   RelationGetRelid(rel));
 }
 
 /* ----------
  * 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.
+ * Ensure that stats are dropped if transaction commits.
  * ----------
  */
-#ifdef NOT_USED
 void
-pgstat_drop_relation(Oid relid)
+pgstat_drop_relation(Relation rel)
 {
-	PgStat_MsgTabpurge msg;
-	int			len;
+	int			nest_level = GetCurrentTransactionNestLevel();
+	PgStat_TableStatus *pgstat_info = rel->pgstat_info;
 
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
+	pgstat_schedule_drop(PGSTAT_KIND_TABLE,
+						 rel->rd_rel->relisshared ? InvalidOid : MyDatabaseId,
+						 RelationGetRelid(rel));
 
-	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);
+	if (pgstat_info &&
+		pgstat_info->trans != NULL &&
+		pgstat_info->trans->nest_level == nest_level)
+	{
+		pgstat_truncdrop_save_counters(pgstat_info->trans, true);
+		pgstat_info->trans->tuples_inserted = 0;
+		pgstat_info->trans->tuples_updated = 0;
+		pgstat_info->trans->tuples_deleted = 0;
+	}
 }
-#endif							/* NOT_USED */
 
 /* ----------
  * pgstat_report_autovac() -
@@ -232,47 +208,97 @@ pgstat_drop_relation(Oid relid)
 void
 pgstat_report_autovac(Oid dboid)
 {
-	PgStat_MsgAutovacStart msg;
+	PgStatSharedRef *shared_ref;
+	PgStatShm_StatDBEntry *dbentry;
 
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
+	/* can't get here in single user mode */
+	Assert(IsUnderPostmaster);
 
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_AUTOVAC_START);
-	msg.m_databaseid = dboid;
-	msg.m_start_time = GetCurrentTimestamp();
+	/*
+	 * End-of-vacuum is reported instantly. Report the start the same way for
+	 * consistency. Vacuum doesn't run frequently and is a long-lasting
+	 * operation so it doesn't matter if we get blocked here a little.
+	 */
+	shared_ref =
+		pgstat_shared_ref_get(PGSTAT_KIND_DB, dboid, InvalidOid, true);
 
-	pgstat_send(&msg, sizeof(msg));
+	pgstat_shared_stat_lock(shared_ref, false);
+
+	dbentry = (PgStatShm_StatDBEntry *) shared_ref->shared_stats;
+	dbentry->stats.last_autovac_time = GetCurrentTimestamp();
+
+	pgstat_shared_stat_unlock(shared_ref);
 }
 
 /* ---------
  * pgstat_report_vacuum() -
  *
- *	Tell the collector about the table we just vacuumed.
+ * Report that the table was just vacuumed.
  * ---------
  */
 void
 pgstat_report_vacuum(Oid tableoid, bool shared,
 					 PgStat_Counter livetuples, PgStat_Counter deadtuples)
 {
-	PgStat_MsgVacuum msg;
+	PgStatSharedRef *shared_ref;
+	PgStatShm_StatTabEntry *shtabentry;
+	PgStat_StatTabEntry *tabentry;
+	Oid			dboid = (shared ? InvalidOid : MyDatabaseId);
+	TimestampTz ts;
 
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+	if (!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));
+	/* Store the data in the table's hash table entry. */
+	ts = GetCurrentTimestamp();
+
+	/*
+	 * Differently from ordinary operations, maintenance commands take longer
+	 * time and getting blocked at the end of work doesn't matter.
+	 * Furthermore, this can prevent the stats updates made by the
+	 * transactions that ends after this vacuum from being canceled by a
+	 * delayed vacuum report. Update shared stats entry directly for the above
+	 * reasons.
+	 */
+	shared_ref =
+		pgstat_shared_stat_locked(PGSTAT_KIND_TABLE, dboid, tableoid, false);
+
+	shtabentry = (PgStatShm_StatTabEntry *) shared_ref->shared_stats;
+	tabentry = &shtabentry->stats;
+
+	tabentry->n_live_tuples = livetuples;
+	tabentry->n_dead_tuples = deadtuples;
+
+	/*
+	 * It is quite possible that a non-aggressive VACUUM ended up skipping
+	 * various pages, however, we'll zero the insert counter here regardless.
+	 * It's currently used only to track when we need to perform an "insert"
+	 * autovacuum, which are mainly intended to freeze newly inserted tuples.
+	 * Zeroing this may just mean we'll not try to vacuum the table again
+	 * until enough tuples have been inserted to trigger another insert
+	 * autovacuum.  An anti-wraparound autovacuum will catch any persistent
+	 * stragglers.
+	 */
+	tabentry->inserts_since_vacuum = 0;
+
+	if (IsAutoVacuumWorkerProcess())
+	{
+		tabentry->autovac_vacuum_timestamp = ts;
+		tabentry->autovac_vacuum_count++;
+	}
+	else
+	{
+		tabentry->vacuum_timestamp = ts;
+		tabentry->vacuum_count++;
+	}
+
+	pgstat_shared_stat_unlock(shared_ref);
 }
 
 /* --------
  * pgstat_report_analyze() -
  *
- *	Tell the collector about the table we just analyzed.
+ * Report that the table was 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.
@@ -283,9 +309,12 @@ pgstat_report_analyze(Relation rel,
 					  PgStat_Counter livetuples, PgStat_Counter deadtuples,
 					  bool resetcounter)
 {
-	PgStat_MsgAnalyze msg;
+	PgStatSharedRef *shared_ref;
+	PgStatShm_StatTabEntry *shtabentry;
+	PgStat_StatTabEntry *tabentry;
+	Oid			dboid = (rel->rd_rel->relisshared ? InvalidOid : MyDatabaseId);
 
-	if (pgStatSock == PGINVALID_SOCKET || !pgstat_track_counts)
+	if (!pgstat_track_counts)
 		return;
 
 	/*
@@ -293,10 +322,10 @@ pgstat_report_analyze(Relation rel,
 	 * 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.)
+	 * off these counts from what is already written to shared stats now, else
+	 * they'll be double-counted after commit.  (This approach also ensures
+	 * that the shared stats ends up with the right numbers if we abort
+	 * instead of committing.)
 	 *
 	 * Waste no time on partitioned tables, though.
 	 */
@@ -317,130 +346,133 @@ pgstat_report_analyze(Relation rel,
 		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));
-}
+	/*
+	 * Differently from ordinary operations, maintenance commands take longer
+	 * time and getting blocked at the end of work doesn't matter.
+	 * Furthermore, this can prevent the stats updates made by the
+	 * transactions that ends after this analyze from being canceled by a
+	 * delayed analyze report. Update shared stats entry directly for the
+	 * above reasons.
+	 */
+	shared_ref = pgstat_shared_stat_locked(PGSTAT_KIND_TABLE, dboid,
+										   RelationGetRelid(rel),
+										   false);
+	/* can't get dropped while accessed */
+	Assert(shared_ref != NULL && shared_ref->shared_stats != NULL);
 
-/*
- * 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;
+	shtabentry = (PgStatShm_StatTabEntry *) shared_ref->shared_stats;
+	tabentry = &shtabentry->stats;
+
+	tabentry->n_live_tuples = livetuples;
+	tabentry->n_dead_tuples = deadtuples;
 
 	/*
-	 * 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 commanded, reset changes_since_analyze to zero.  This forgets any
+	 * changes that were committed while the ANALYZE was in progress, but we
+	 * have no good way to estimate how many of those there were.
 	 */
-	if (pgStatTabHash)
-		hash_destroy(pgStatTabHash);
-	pgStatTabHash = NULL;
+	if (resetcounter)
+		tabentry->changes_since_analyze = 0;
 
-	/*
-	 * 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)
+	if (IsAutoVacuumWorkerProcess())
 	{
-		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;
+		tabentry->autovac_analyze_timestamp = GetCurrentTimestamp();
+		tabentry->autovac_analyze_count++;
+	}
+	else
+	{
+		tabentry->analyze_timestamp = GetCurrentTimestamp();
+		tabentry->analyze_count++;
 	}
 
-	/*
-	 * 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;
+	pgstat_shared_stat_unlock(shared_ref);
 }
 
 /*
- * Subroutine for pgstat_send_tabstats: finish and send one tabstat message
+ * pgstat_flush_relation - flush out a pending rlation stats entry
+ *
+ * Some of the stats numbers are copied to pending database stats entry after
+ * successful flush-out.
+ *
+ * If nowait is true, this function returns false on lock failure. Otherwise
+ * this function always returns true.
+ *
+ * Returns true if the entry is successfully flushed out.
  */
-static void
-pgstat_send_tabstat(PgStat_MsgTabstat *tsmsg, TimestampTz now)
+bool
+pgstat_flush_relation(PgStatSharedRef *shared_ref, bool nowait)
 {
-	int			n;
-	int			len;
+	static const PgStat_TableCounts all_zeroes;
+	Oid			dboid;			/* database OID of the table */
+	PgStat_TableStatus *lstats; /* pending stats entry  */
+	PgStatShm_StatTabEntry *shtabstats;
+	PgStat_StatTabEntry *tabentry;	/* table entry of shared stats */
+	PgStat_StatDBEntry *dbentry;	/* pending database entry */
 
-	/* It's unlikely we'd get here with no socket, but maybe not impossible */
-	if (pgStatSock == PGINVALID_SOCKET)
-		return;
+	Assert(shared_ref->shared_entry->key.kind == PGSTAT_KIND_TABLE);
+	lstats = (PgStat_TableStatus *) shared_ref->pending;
+	dboid = shared_ref->shared_entry->key.dboid;
 
 	/*
-	 * Report and reset accumulated xact commit/rollback and I/O timings
-	 * whenever we send a normal tabstat message
+	 * Ignore entries that didn't accumulate any actual counts, such as
+	 * indexes that were opened by the planner but not used.
 	 */
-	pgstat_update_dbstats(tsmsg, now);
+	if (memcmp(&lstats->t_counts, &all_zeroes,
+			   sizeof(PgStat_TableCounts)) == 0)
+	{
+		return true;
+	}
 
-	n = tsmsg->m_nentries;
-	len = offsetof(PgStat_MsgTabstat, m_entry[0]) +
-		n * sizeof(PgStat_TableEntry);
+	if (!pgstat_shared_stat_lock(shared_ref, nowait))
+		return false;			/* failed to acquire lock, skip */
 
-	pgstat_setheader(&tsmsg->m_hdr, PGSTAT_MTYPE_TABSTAT);
-	pgstat_send(tsmsg, len);
+	/* add the values to the shared entry. */
+	shtabstats = (PgStatShm_StatTabEntry *) shared_ref->shared_stats;
+	tabentry = &shtabstats->stats;
+	tabentry->numscans += lstats->t_counts.t_numscans;
+	tabentry->tuples_returned += lstats->t_counts.t_tuples_returned;
+	tabentry->tuples_fetched += lstats->t_counts.t_tuples_fetched;
+	tabentry->tuples_inserted += lstats->t_counts.t_tuples_inserted;
+	tabentry->tuples_updated += lstats->t_counts.t_tuples_updated;
+	tabentry->tuples_deleted += lstats->t_counts.t_tuples_deleted;
+	tabentry->tuples_hot_updated += lstats->t_counts.t_tuples_hot_updated;
+
+	/*
+	 * If table was truncated or vacuum/analyze has ran, first reset the
+	 * live/dead counters.
+	 */
+	if (lstats->t_counts.t_truncdropped)
+	{
+		tabentry->n_live_tuples = 0;
+		tabentry->n_dead_tuples = 0;
+		/* AFIXME: Why is inserts_since_vacuum not reset anymore? */
+	}
+
+	tabentry->n_live_tuples += lstats->t_counts.t_delta_live_tuples;
+	tabentry->n_dead_tuples += lstats->t_counts.t_delta_dead_tuples;
+	tabentry->changes_since_analyze += lstats->t_counts.t_changed_tuples;
+	tabentry->inserts_since_vacuum += lstats->t_counts.t_tuples_inserted;
+	tabentry->blocks_fetched += lstats->t_counts.t_blocks_fetched;
+	tabentry->blocks_hit += lstats->t_counts.t_blocks_hit;
+
+	/* Clamp n_live_tuples in case of negative delta_live_tuples */
+	tabentry->n_live_tuples = Max(tabentry->n_live_tuples, 0);
+	/* Likewise for n_dead_tuples */
+	tabentry->n_dead_tuples = Max(tabentry->n_dead_tuples, 0);
+
+	pgstat_shared_stat_unlock(shared_ref);
+
+	/* The entry is successfully flushed so the same to add to database stats */
+	dbentry = pgstat_pending_db_prepare(dboid);
+	dbentry->n_tuples_returned += lstats->t_counts.t_tuples_returned;
+	dbentry->n_tuples_fetched += lstats->t_counts.t_tuples_fetched;
+	dbentry->n_tuples_inserted += lstats->t_counts.t_tuples_inserted;
+	dbentry->n_tuples_updated += lstats->t_counts.t_tuples_updated;
+	dbentry->n_tuples_deleted += lstats->t_counts.t_tuples_deleted;
+	dbentry->n_blocks_fetched += lstats->t_counts.t_blocks_fetched;
+	dbentry->n_blocks_hit += lstats->t_counts.t_blocks_hit;
+
+	return true;
 }
 
 /*
@@ -687,7 +719,7 @@ AtPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state)
 
 	for (trans = xact_state->first; trans != NULL; trans = trans->next)
 	{
-		PgStat_TableStatus *tabstat;
+		PgStat_TableStatus *tabstat PG_USED_FOR_ASSERTS_ONLY;
 		TwoPhasePgStatRecord record;
 
 		Assert(trans->nest_level == 1);
@@ -701,8 +733,6 @@ AtPrepare_PgStat_Relations(PgStat_SubXactStatus *xact_state)
 		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,
@@ -745,7 +775,7 @@ pgstat_twophase_postcommit(TransactionId xid, uint16 info,
 	PgStat_TableStatus *pgstat_info;
 
 	/* Find or create a tabstat entry for the rel */
-	pgstat_info = get_tabstat_entry(rec->t_id, rec->t_shared);
+	pgstat_info = pgstat_pending_tab_prepare(rec->t_id, rec->t_shared);
 
 	/* Same math as in AtEOXact_PgStat, commit case */
 	pgstat_info->t_counts.t_tuples_inserted += rec->tuples_inserted;
@@ -781,7 +811,7 @@ pgstat_twophase_postabort(TransactionId xid, uint16 info,
 	PgStat_TableStatus *pgstat_info;
 
 	/* Find or create a tabstat entry for the rel */
-	pgstat_info = get_tabstat_entry(rec->t_id, rec->t_shared);
+	pgstat_info = pgstat_pending_tab_prepare(rec->t_id, rec->t_shared);
 
 	/* Same math as in AtEOXact_PgStat, abort case */
 	if (rec->t_truncdropped)
@@ -797,116 +827,81 @@ pgstat_twophase_postabort(TransactionId xid, uint16 info,
 		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
+ * Find any existing PgStat_TableStatus entry for rel_id in the current
+ * database. If not found, try finding from shared tables.
  *
- * 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".
+ * If no entry found, return NULL, don't create a new one
+ * ----------
  */
 PgStat_TableStatus *
 find_tabstat_entry(Oid rel_id)
 {
-	TabStatHashEntry *hash_entry;
+	PgStatSharedRef *shared_ref;
 
-	/* If hashtable doesn't exist, there are no entries at all */
-	if (!pgStatTabHash)
-		return NULL;
+	shared_ref = pgstat_pending_fetch(PGSTAT_KIND_TABLE, MyDatabaseId, rel_id);
+	if (!shared_ref)
+		shared_ref = pgstat_pending_fetch(PGSTAT_KIND_TABLE, InvalidOid, rel_id);
 
-	hash_entry = hash_search(pgStatTabHash, &rel_id, HASH_FIND, NULL);
-	if (!hash_entry)
-		return NULL;
+	if (shared_ref)
+		return shared_ref->pending;
+	return NULL;
+}
 
-	/* Note that this step could also return NULL, but that's correct */
-	return hash_entry->tsa_entry;
+/* ----------
+ * pgstat_fetch_stat_tabentry() -
+ *
+ *	Support function for the SQL-callable pgstat* functions. Returns the
+ *	collected statistics for one table or NULL. NULL doesn't necessarily mean
+ *	that the relation doesn't exist, just that there are no statistics, so the
+ *	caller is better off to report ZERO instead.
+ * ----------
+ */
+PgStat_StatTabEntry *
+pgstat_fetch_stat_tabentry(Oid relid)
+{
+	PgStat_StatTabEntry *tabentry;
+
+	tabentry = pgstat_fetch_stat_tabentry_extended(false, relid);
+	if (tabentry != NULL)
+		return tabentry;
+
+	/*
+	 * If we didn't find it, maybe it's a shared table.
+	 */
+	tabentry = pgstat_fetch_stat_tabentry_extended(true, relid);
+	return tabentry;
+}
+
+/*
+ * More efficient version of pgstat_fetch_stat_tabentry(), allowing to specify
+ * whether the to-be-accessed table is a shared relation or not.
+ */
+PgStat_StatTabEntry *
+pgstat_fetch_stat_tabentry_extended(bool shared, Oid reloid)
+{
+	Oid			dboid = (shared ? InvalidOid : MyDatabaseId);
+
+	return (PgStat_StatTabEntry *)
+		pgstat_fetch_entry(PGSTAT_KIND_TABLE, dboid, reloid);
+}
+
+/*
+ * Find or create a PgStat_TableStatus entry for rel. New entry is created and
+ * initialized if not exists.
+ */
+static PgStat_TableStatus *
+pgstat_pending_tab_prepare(Oid rel_id, bool isshared)
+{
+	PgStatSharedRef *shared_ref;
+
+	shared_ref = pgstat_pending_prepare(PGSTAT_KIND_TABLE,
+										isshared ? InvalidOid : MyDatabaseId,
+										rel_id);
+
+	return shared_ref->pending;
 }
 
 /*
diff --git a/src/backend/utils/activity/pgstat_subscription.c b/src/backend/utils/activity/pgstat_subscription.c
index 0bfbe814d76..af33814cec4 100644
--- a/src/backend/utils/activity/pgstat_subscription.c
+++ b/src/backend/utils/activity/pgstat_subscription.c
@@ -18,61 +18,88 @@
 #include "postgres.h"
 
 #include "utils/pgstat_internal.h"
+#include "utils/timestamp.h"
 
 
+/*
+ * pgstat_flush_subscription - flush out a local subscription stats entry
+ *
+ * If nowait is true, this function returns false on lock failure. Otherwise
+ * this function always returns true.
+ *
+ * Returns true if the entry is successfully flushed out.
+ */
+bool
+pgstat_flush_subscription(PgStatSharedRef *shared_ref, bool nowait)
+{
+	PgStat_BackendSubEntry *localent;	/* local stats entry */
+	PgStatShm_StatSubEntry *shsubent = NULL; /* shared stats entry */
+
+	Assert(shared_ref->shared_entry->key.kind == PGSTAT_KIND_SUBSCRIPTION);
+	localent = (PgStat_BackendSubEntry *) shared_ref->pending;
+
+	/* localent always has non-zero content */
+
+	if (!pgstat_shared_stat_lock(shared_ref, nowait))
+		return false;			/* failed to acquire lock, skip */
+
+	shsubent = (PgStatShm_StatSubEntry *) shared_ref->shared_stats;
+
+	shsubent->stats.apply_error_count += localent->apply_error_count;
+	shsubent->stats.sync_error_count += localent->sync_error_count;
+
+	pgstat_shared_stat_unlock(shared_ref);
+	return true;
+}
+
+/*
+ * Ensure that stats are dropped if transaction commits.
+ */
+void
+pgstat_report_subscription_drop(Oid subid)
+{
+
+	pgstat_schedule_drop(PGSTAT_KIND_SUBSCRIPTION,
+						 subid, InvalidOid);
+}
+
 /* ----------
  * pgstat_report_subscription_error() -
  *
- *	Tell the collector about the subscription error.
+ *	Tell the collector about the subscription worker error.
  * ----------
  */
 void
 pgstat_report_subscription_error(Oid subid, bool is_apply_error)
 {
-	PgStat_MsgSubscriptionError msg;
+	PgStatSharedRef *shared_ref;
+	PgStat_BackendSubEntry *pending;
 
-	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)
+	if (!pgstat_track_counts)
 		return;
 
-	msg.m_subid = subid;
-	pgstat_setheader(&msg.m_hdr, PGSTAT_MTYPE_RESETSUBCOUNTER);
+	shared_ref = pgstat_pending_prepare(PGSTAT_KIND_SUBSCRIPTION,
+										subid,
+										InvalidOid);
 
-	pgstat_send(&msg, sizeof(msg));
+	pending = shared_ref->pending;
+	if (is_apply_error)
+		pending->apply_error_count++;
+	else
+		pending->sync_error_count++;
+}
+
+/*
+ * ---------
+ * pgstat_fetch_stat_subscription() -
+ *
+ *	Support function for the SQL-callable pgstat* functions. Returns
+ *	the collected statistics for one subscription or NULL.
+ * ---------
+ */
+PgStat_StatSubEntry *
+pgstat_fetch_stat_subscription(Oid subid)
+{
+	return (PgStat_StatSubEntry *)
+		pgstat_fetch_entry(PGSTAT_KIND_SUBSCRIPTION, subid, InvalidOid);
 }
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index 60972c3a750..5ed26067c6b 100644
--- a/src/backend/utils/activity/wait_event.c
+++ b/src/backend/utils/activity/wait_event.c
@@ -230,8 +230,8 @@ pgstat_get_wait_activity(WaitEventActivity w)
 		case WAIT_EVENT_LOGICAL_LAUNCHER_MAIN:
 			event_name = "LogicalLauncherMain";
 			break;
-		case WAIT_EVENT_PGSTAT_MAIN:
-			event_name = "PgStatMain";
+		case WAIT_EVENT_READING_STATS_FILE:
+			event_name = "ReadingStatsFile";
 			break;
 		case WAIT_EVENT_RECOVERY_WAL_STREAM:
 			event_name = "RecoveryWalStream";
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index fd993d0d5fb..dd6bc755607 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1,7 +1,7 @@
 /*-------------------------------------------------------------------------
  *
  * pgstatfuncs.c
- *	  Functions for accessing the statistics collector data
+ *	  Functions for accessing the activity statistics data
  *
  * Portions Copyright (c) 1996-2022, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
@@ -1834,11 +1834,11 @@ pg_stat_get_wal(PG_FUNCTION_ARGS)
 	wal_stats = pgstat_fetch_stat_wal();
 
 	/* Fill values and NULLs */
-	values[0] = Int64GetDatum(wal_stats->wal_records);
-	values[1] = Int64GetDatum(wal_stats->wal_fpi);
+	values[0] = Int64GetDatum(wal_stats->wal_usage.wal_records);
+	values[1] = Int64GetDatum(wal_stats->wal_usage.wal_fpi);
 
 	/* Convert to numeric. */
-	snprintf(buf, sizeof buf, UINT64_FORMAT, wal_stats->wal_bytes);
+	snprintf(buf, sizeof buf, UINT64_FORMAT, wal_stats->wal_usage.wal_bytes);
 	values[2] = DirectFunctionCall3(numeric_in,
 									CStringGetDatum(buf),
 									ObjectIdGetDatum(0),
@@ -2124,7 +2124,15 @@ pg_stat_get_xact_function_self_time(PG_FUNCTION_ARGS)
 Datum
 pg_stat_get_snapshot_timestamp(PG_FUNCTION_ARGS)
 {
-	PG_RETURN_TIMESTAMPTZ(pgstat_fetch_global()->stats_timestamp);
+	bool		have_snapshot;
+	TimestampTz ts;
+
+	ts = pgstat_get_stat_snapshot_timestamp(&have_snapshot);
+
+	if (!have_snapshot)
+		PG_RETURN_NULL();
+
+	PG_RETURN_TIMESTAMPTZ(ts);
 }
 
 /* Discard the active statistics snapshot */
@@ -2136,6 +2144,14 @@ pg_stat_clear_snapshot(PG_FUNCTION_ARGS)
 	PG_RETURN_VOID();
 }
 
+/* Force statistics to be reported at the next occasion */
+Datum
+pg_stat_force_next_flush(PG_FUNCTION_ARGS)
+{
+	pgstat_force_next_flush();
+
+	PG_RETURN_VOID();
+}
 
 /* Reset all counters for the current database */
 Datum
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index fccffce5729..4d2b970a40e 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -72,6 +72,7 @@
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/optimizer.h"
+#include "pgstat.h"
 #include "rewrite/rewriteDefine.h"
 #include "rewrite/rowsecurity.h"
 #include "storage/lmgr.h"
@@ -2389,6 +2390,9 @@ RelationDestroyRelation(Relation relation, bool remember_tupdesc)
 	 */
 	RelationCloseSmgr(relation);
 
+	/* break mutual link with stats entry */
+	pgstat_relation_delink(relation);
+
 	/*
 	 * Free all the subsidiary data structures of the relcache entry, then the
 	 * entry itself.
diff --git a/src/backend/utils/init/globals.c b/src/backend/utils/init/globals.c
index 3419c099b28..1a5d29ac9ba 100644
--- a/src/backend/utils/init/globals.c
+++ b/src/backend/utils/init/globals.c
@@ -36,6 +36,7 @@ volatile sig_atomic_t IdleInTransactionSessionTimeoutPending = false;
 volatile sig_atomic_t IdleSessionTimeoutPending = false;
 volatile sig_atomic_t ProcSignalBarrierPending = false;
 volatile sig_atomic_t LogMemoryContextPending = false;
+volatile sig_atomic_t IdleStatsUpdateTimeoutPending = false;
 volatile uint32 InterruptHoldoffCount = 0;
 volatile uint32 QueryCancelHoldoffCount = 0;
 volatile uint32 CritSectionCount = 0;
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index bdc77af7194..0d3cfe8240b 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -288,9 +288,6 @@ GetBackendTypeDesc(BackendType backendType)
 		case B_ARCHIVER:
 			backendDesc = "archiver";
 			break;
-		case B_STATS_COLLECTOR:
-			backendDesc = "stats collector";
-			break;
 		case B_LOGGER:
 			backendDesc = "logger";
 			break;
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index 86d193c89fc..151a40c5c11 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -80,6 +80,7 @@ static void StatementTimeoutHandler(void);
 static void LockTimeoutHandler(void);
 static void IdleInTransactionSessionTimeoutHandler(void);
 static void IdleSessionTimeoutHandler(void);
+static void IdleStatsUpdateTimeoutHandler(void);
 static void ClientCheckTimeoutHandler(void);
 static bool ThereIsAtLeastOneRole(void);
 static void process_startup_options(Port *port, bool am_superuser);
@@ -706,6 +707,8 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
 						IdleInTransactionSessionTimeoutHandler);
 		RegisterTimeout(IDLE_SESSION_TIMEOUT, IdleSessionTimeoutHandler);
 		RegisterTimeout(CLIENT_CONNECTION_CHECK_TIMEOUT, ClientCheckTimeoutHandler);
+		RegisterTimeout(IDLE_STATS_UPDATE_TIMEOUT,
+						IdleStatsUpdateTimeoutHandler);
 	}
 
 	/*
@@ -733,6 +736,7 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
 		 * Use before_shmem_exit() so that ShutdownXLOG() can rely on DSM
 		 * segments etc to work (which in turn is required for pgstats).
 		 */
+		before_shmem_exit(pgstat_before_server_shutdown, 0);
 		before_shmem_exit(ShutdownXLOG, 0);
 	}
 
@@ -1333,6 +1337,14 @@ IdleSessionTimeoutHandler(void)
 	SetLatch(MyLatch);
 }
 
+static void
+IdleStatsUpdateTimeoutHandler(void)
+{
+	IdleStatsUpdateTimeoutPending = true;
+	InterruptPending = true;
+	SetLatch(MyLatch);
+}
+
 static void
 ClientCheckTimeoutHandler(void)
 {
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 1e3650184b1..61901a1f479 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -373,6 +373,16 @@ static const struct config_enum_entry track_function_options[] = {
 StaticAssertDecl(lengthof(track_function_options) == (TRACK_FUNC_ALL + 2),
 				 "array length mismatch");
 
+static const struct config_enum_entry stats_fetch_consistency[] = {
+	{"none", STATS_FETCH_CONSISTENCY_NONE, false},
+	{"cache", STATS_FETCH_CONSISTENCY_CACHE, false},
+	{"snapshot", STATS_FETCH_CONSISTENCY_SNAPSHOT, false},
+	{NULL, 0, false}
+};
+
+StaticAssertDecl(lengthof(stats_fetch_consistency) == (STATS_FETCH_CONSISTENCY_SNAPSHOT + 2),
+				 "array length mismatch");
+
 static const struct config_enum_entry xmlbinary_options[] = {
 	{"base64", XMLBINARY_BASE64, false},
 	{"hex", XMLBINARY_HEX, false},
@@ -4901,6 +4911,16 @@ static struct config_enum ConfigureNamesEnum[] =
 		NULL, NULL, NULL
 	},
 
+	{
+		{"stats_fetch_consistency", PGC_SUSET, STATS_COLLECTOR,
+			gettext_noop("Stats consistency model XXXX"),
+			NULL
+		},
+		&pgstat_fetch_consistency,
+		STATS_FETCH_CONSISTENCY_NONE, stats_fetch_consistency,
+		NULL, NULL, NULL
+	},
+
 	{
 		{"wal_level", PGC_POSTMASTER, WAL_SETTINGS,
 			gettext_noop("Sets the level of information written to the WAL."),
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 4a094bb38be..8c2e4fb048d 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -603,7 +603,7 @@
 # STATISTICS
 #------------------------------------------------------------------------------
 
-# - Query and Index Statistics Collector -
+# - Query and Index Activity Statistics -
 
 #track_activities = on
 #track_activity_query_size = 1024	# (change requires restart)
@@ -612,6 +612,7 @@
 #track_wal_io_timing = off
 #track_functions = none			# none, pl, all
 #stats_temp_directory = 'pg_stat_tmp'
+#stats_fetch_consistency = none
 
 
 # - Monitoring -
diff --git a/src/bin/pg_basebackup/t/010_pg_basebackup.pl b/src/bin/pg_basebackup/t/010_pg_basebackup.pl
index 8cb8cfe045e..961504d2167 100644
--- a/src/bin/pg_basebackup/t/010_pg_basebackup.pl
+++ b/src/bin/pg_basebackup/t/010_pg_basebackup.pl
@@ -150,7 +150,7 @@ is_deeply(
 
 # Contents of these directories should not be copied.
 foreach my $dirname (
-	qw(pg_dynshmem pg_notify pg_replslot pg_serial pg_snapshots pg_stat_tmp pg_subtrans)
+	qw(pg_dynshmem pg_notify pg_replslot pg_serial pg_snapshots pg_subtrans)
   )
 {
 	is_deeply(
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 9fb62fec8ec..fbf8b983612 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -5129,7 +5129,8 @@ SELECT pid, wait_event_type, wait_event FROM pg_stat_activity WHERE wait_event i
         <returnvalue>timestamp with time zone</returnvalue>
        </para>
        <para>
-        Returns the timestamp of the current statistics snapshot.
+        Returns the timestamp of the current statistics snapshot, or NULL if
+        no statistics snapshot has been taken.
        </para></entry>
       </row>
 
diff --git a/src/test/regress/expected/stats.out b/src/test/regress/expected/stats.out
index 3e9ab0915fe..b9821cd8c76 100644
--- a/src/test/regress/expected/stats.out
+++ b/src/test/regress/expected/stats.out
@@ -17,6 +17,8 @@ SET enable_indexscan TO on;
 -- for the moment, we don't want index-only scans here
 SET enable_indexonlyscan TO off;
 -- save counters
+BEGIN;
+SET LOCAL stats_fetch_consistency = snapshot;
 CREATE TABLE prevstats AS
 SELECT t.seq_scan, t.seq_tup_read, t.idx_scan, t.idx_tup_fetch,
        (b.heap_blks_read + b.heap_blks_hit) AS heap_blks,
@@ -25,6 +27,7 @@ SELECT t.seq_scan, t.seq_tup_read, t.idx_scan, t.idx_tup_fetch,
   FROM pg_catalog.pg_stat_user_tables AS t,
        pg_catalog.pg_statio_user_tables AS b
  WHERE t.relname='tenk2' AND b.relname='tenk2';
+COMMIT;
 -- function to wait for counters to advance
 create function wait_for_stats() returns void as $$
 declare
@@ -34,6 +37,8 @@ declare
   updated3 bool;
   updated4 bool;
 begin
+  SET LOCAL stats_fetch_consistency = snapshot;
+
   -- we don't want to wait forever; loop will exit after 30 seconds
   for i in 1 .. 300 loop
 
@@ -160,6 +165,8 @@ SELECT wait_for_stats();
 (1 row)
 
 -- check effects
+BEGIN;
+SET LOCAL stats_fetch_consistency = snapshot;
 SELECT relname, n_tup_ins, n_tup_upd, n_tup_del, n_live_tup, n_dead_tup
   FROM pg_stat_user_tables
  WHERE relname like 'trunc_stats_test%' order by relname;
@@ -199,6 +206,7 @@ FROM prevstats AS pr;
  t
 (1 row)
 
+COMMIT;
 DROP TABLE trunc_stats_test, trunc_stats_test1, trunc_stats_test2, trunc_stats_test3, trunc_stats_test4;
 DROP TABLE prevstats;
 -- test BRIN index doesn't block HOT update - we include this test here, as it
diff --git a/src/test/regress/sql/stats.sql b/src/test/regress/sql/stats.sql
index 82e6f24c391..9dfd44748e5 100644
--- a/src/test/regress/sql/stats.sql
+++ b/src/test/regress/sql/stats.sql
@@ -15,6 +15,8 @@ SET enable_indexscan TO on;
 SET enable_indexonlyscan TO off;
 
 -- save counters
+BEGIN;
+SET LOCAL stats_fetch_consistency = snapshot;
 CREATE TABLE prevstats AS
 SELECT t.seq_scan, t.seq_tup_read, t.idx_scan, t.idx_tup_fetch,
        (b.heap_blks_read + b.heap_blks_hit) AS heap_blks,
@@ -23,6 +25,7 @@ SELECT t.seq_scan, t.seq_tup_read, t.idx_scan, t.idx_tup_fetch,
   FROM pg_catalog.pg_stat_user_tables AS t,
        pg_catalog.pg_statio_user_tables AS b
  WHERE t.relname='tenk2' AND b.relname='tenk2';
+COMMIT;
 
 -- function to wait for counters to advance
 create function wait_for_stats() returns void as $$
@@ -33,6 +36,8 @@ declare
   updated3 bool;
   updated4 bool;
 begin
+  SET LOCAL stats_fetch_consistency = snapshot;
+
   -- we don't want to wait forever; loop will exit after 30 seconds
   for i in 1 .. 300 loop
 
@@ -155,6 +160,9 @@ RESET enable_bitmapscan;
 SELECT wait_for_stats();
 
 -- check effects
+BEGIN;
+SET LOCAL stats_fetch_consistency = snapshot;
+
 SELECT relname, n_tup_ins, n_tup_upd, n_tup_del, n_live_tup, n_dead_tup
   FROM pg_stat_user_tables
  WHERE relname like 'trunc_stats_test%' order by relname;
@@ -174,6 +182,8 @@ SELECT st.heap_blks_read + st.heap_blks_hit >= pr.heap_blks + cl.relpages,
 SELECT pr.snap_ts < pg_stat_get_snapshot_timestamp() as snapshot_newer
 FROM prevstats AS pr;
 
+COMMIT;
+
 DROP TABLE trunc_stats_test, trunc_stats_test1, trunc_stats_test2, trunc_stats_test3, trunc_stats_test4;
 DROP TABLE prevstats;
 
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index eaf3e7a8d44..9749a8fbb05 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1913,45 +1913,24 @@ PgFdwPathExtraData
 PgFdwRelationInfo
 PgFdwScanState
 PgIfAddrCallback
+PgStatHashEntry
+PgStatHashKey
+PgStatLocalHashEntry
+PgStatSharedSLRUStats
+PgStatShm_StatDBEntry
+PgStatShm_StatEntryHeader
+PgStatShm_StatFuncEntry
+PgStatShm_StatTabEntry
+PgStatShmemGlobal
+PgStatTypes
 PgStat_ArchiverStats
 PgStat_BackendFunctionEntry
+PgStat_BgWriterStats
+PgStat_CheckPointerStats
 PgStat_Counter
 PgStat_FunctionCallUsage
 PgStat_FunctionCounts
-PgStat_FunctionEntry
-PgStat_GlobalStats
-PgStat_Msg
-PgStat_MsgAnalyze
-PgStat_MsgAnlAncestors
-PgStat_MsgArchiver
-PgStat_MsgAutovacStart
-PgStat_MsgBgWriter
-PgStat_MsgChecksumFailure
-PgStat_MsgConnect
-PgStat_MsgDeadlock
-PgStat_MsgDisconnect
-PgStat_MsgDropdb
-PgStat_MsgDummy
-PgStat_MsgFuncpurge
-PgStat_MsgFuncstat
-PgStat_MsgHdr
-PgStat_MsgInquiry
-PgStat_MsgRecoveryConflict
-PgStat_MsgReplSlot
-PgStat_MsgResetcounter
-PgStat_MsgResetreplslotcounter
-PgStat_MsgResetsharedcounter
-PgStat_MsgResetsinglecounter
-PgStat_MsgResetslrucounter
-PgStat_MsgResetsubcounter
-PgStat_MsgSLRU
-PgStat_MsgSubscriptionDrop
-PgStat_MsgSubscriptionError
-PgStat_MsgTabpurge
-PgStat_MsgTabstat
-PgStat_MsgTempFile
-PgStat_MsgVacuum
-PgStat_MsgWal
+PgStat_ReplSlotStats
 PgStat_SLRUStats
 PgStat_Shared_Reset_Target
 PgStat_Single_Reset_Type
@@ -1962,7 +1941,6 @@ PgStat_StatSubEntry
 PgStat_StatTabEntry
 PgStat_SubXactStatus
 PgStat_TableCounts
-PgStat_TableEntry
 PgStat_TableStatus
 PgStat_TableXactStatus
 PgStat_WalStats
@@ -2510,7 +2488,6 @@ StartReplicationCmd
 StartupStatusEnum
 StatEntry
 StatExtEntry
-StatMsgType
 StateFileChunk
 StatisticExtInfo
 Stats
@@ -2624,8 +2601,6 @@ TXNEntryFile
 TYPCATEGORY
 T_Action
 T_WorkerStatus
-TabStatHashEntry
-TabStatusArray
 TableAmRoutine
 TableAttachInfo
 TableDataInfo
-- 
2.35.1.354.g715d08a9e5

