diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 793bd33cb4..85e9b26040 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -165,6 +165,7 @@ typedef struct LVRelState
 	/* Error reporting state */
 	char	   *dbname;
 	char	   *relnamespace;
+	Oid			reloid;
 	char	   *relname;
 	char	   *indname;		/* Current index name */
 	BlockNumber blkno;			/* used only for heap operations */
@@ -192,6 +193,8 @@ typedef struct LVRelState
 	BlockNumber lpdead_item_pages;	/* # pages with LP_DEAD items */
 	BlockNumber missed_dead_pages;	/* # pages with missed dead tuples */
 	BlockNumber nonempty_pages; /* actually, last nonempty page + 1 */
+	BlockNumber set_frozen_pages; /* pages are marked as frozen in vm during vacuum */
+	BlockNumber set_all_visible_pages;	/* pages are marked as all-visible in vm during vacuum */
 
 	/* Statistics output by us, for table */
 	double		new_rel_tuples; /* new estimated total # of tuples */
@@ -224,6 +227,18 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/*
+ * Counters and usage data for extended stats tracking.
+ */
+typedef struct LVExtStatCounters
+{
+	TimestampTz starttime;
+	WalUsage	walusage;
+	BufferUsage bufusage;
+	double		VacuumDelayTime;
+	PgStat_Counter blocks_fetched;
+	PgStat_Counter blocks_hit;
+} LVExtStatCounters;
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel);
@@ -277,6 +292,97 @@ static void update_vacuum_error_info(LVRelState *vacrel,
 static void restore_vacuum_error_info(LVRelState *vacrel,
 									  const LVSavedErrInfo *saved_vacrel);
 
+/*
+ * extvac_stats_start() -
+ *
+ * Save extended stats counters before start of relation processing.
+ */
+static void
+extvac_stats_start(Relation rel, LVExtStatCounters *counters)
+{
+	TimestampTz	starttime;
+
+	memset(counters, 0, sizeof(LVExtStatCounters));
+
+	starttime = GetCurrentTimestamp();
+
+	counters->starttime = starttime;
+	counters->walusage = pgWalUsage;
+	counters->bufusage = pgBufferUsage;
+	counters->VacuumDelayTime = VacuumDelayTime;
+	counters->blocks_fetched = 0;
+	counters->blocks_hit = 0;
+
+	if (!rel->pgstat_info || !pgstat_track_counts)
+		/*
+		 * if something goes wrong or user doesn't want to track a database
+		 * activity - just suppress it.
+		 */
+		return;
+
+	counters->blocks_fetched = rel->pgstat_info->counts.blocks_fetched;
+	counters->blocks_hit = rel->pgstat_info->counts.blocks_hit;
+}
+
+/*
+ * extvac_stats_end() -
+ *
+ * Called to finish an extended vacuum statistic gathering and form a report.
+ */
+static void
+extvac_stats_end(Relation rel, LVExtStatCounters *counters,
+				  ExtVacReport *report)
+{
+	WalUsage	walusage;
+	BufferUsage	bufusage;
+	TimestampTz endtime;
+	long		secs;
+	int			usecs;
+
+	/* Calculate diffs of global stat parameters on WAL and buffer usage. */
+	memset(&walusage, 0, sizeof(WalUsage));
+	WalUsageAccumDiff(&walusage, &pgWalUsage, &counters->walusage);
+
+	memset(&bufusage, 0, sizeof(BufferUsage));
+	BufferUsageAccumDiff(&bufusage, &pgBufferUsage, &counters->bufusage);
+
+	endtime = GetCurrentTimestamp();
+	TimestampDifference(counters->starttime, endtime, &secs, &usecs);
+
+	memset(report, 0, sizeof(ExtVacReport));
+
+	/*
+	 * Fill additional statistics on a vacuum processing operation.
+	 */
+	report->total_blks_read = bufusage.local_blks_read + bufusage.shared_blks_read;
+	report->total_blks_hit = bufusage.local_blks_hit + bufusage.shared_blks_hit;
+	report->total_blks_dirtied = bufusage.local_blks_dirtied + bufusage.shared_blks_dirtied;
+	report->total_blks_written = bufusage.shared_blks_written;
+
+	report->wal_records = walusage.wal_records;
+	report->wal_fpi = walusage.wal_fpi;
+	report->wal_bytes = walusage.wal_bytes;
+
+	report->blk_read_time = INSTR_TIME_GET_MILLISEC(bufusage.local_blk_read_time);
+	report->blk_read_time += INSTR_TIME_GET_MILLISEC(bufusage.shared_blk_read_time);
+	report->blk_write_time = INSTR_TIME_GET_MILLISEC(bufusage.local_blk_write_time);
+	report->blk_write_time = INSTR_TIME_GET_MILLISEC(bufusage.shared_blk_write_time);
+	report->delay_time = VacuumDelayTime - counters->VacuumDelayTime;
+
+	report->total_time = secs * 1000. + usecs / 1000.;
+
+	if (!rel->pgstat_info || !pgstat_track_counts)
+		/*
+		 * if something goes wrong or an user doesn't want to track a database
+		 * activity - just suppress it.
+		 */
+		return;
+
+	report->blks_fetched =
+		rel->pgstat_info->counts.blocks_fetched - counters->blocks_fetched;
+	report->blks_hit =
+		rel->pgstat_info->counts.blocks_hit - counters->blocks_hit;
+}
 
 /*
  *	heap_vacuum_rel() -- perform VACUUM for one heap relation
@@ -309,6 +415,8 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	WalUsage	startwalusage = pgWalUsage;
 	BufferUsage startbufferusage = pgBufferUsage;
 	ErrorContextCallback errcallback;
+	LVExtStatCounters extVacCounters;
+	ExtVacReport extVacReport;
 	char	  **indnames = NULL;
 
 	verbose = (params->options & VACOPT_VERBOSE) != 0;
@@ -327,7 +435,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
-
+	extvac_stats_start(rel, &extVacCounters);
 	/*
 	 * Setup error traceback support for ereport() first.  The idea is to set
 	 * up an error context callback to display additional information on any
@@ -344,6 +452,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->dbname = get_database_name(MyDatabaseId);
 	vacrel->relnamespace = get_namespace_name(RelationGetNamespace(rel));
 	vacrel->relname = pstrdup(RelationGetRelationName(rel));
+	vacrel->reloid = RelationGetRelid(rel);
 	vacrel->indname = NULL;
 	vacrel->phase = VACUUM_ERRCB_PHASE_UNKNOWN;
 	vacrel->verbose = verbose;
@@ -411,6 +520,8 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->lpdead_item_pages = 0;
 	vacrel->missed_dead_pages = 0;
 	vacrel->nonempty_pages = 0;
+	vacrel->set_frozen_pages = 0;
+	vacrel->set_all_visible_pages = 0;
 	/* dead_items_alloc allocates vacrel->dead_items later on */
 
 	/* Allocate/initialize output statistics state */
@@ -572,6 +683,19 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 						vacrel->NewRelfrozenXid, vacrel->NewRelminMxid,
 						&frozenxid_updated, &minmulti_updated, false);
 
+	/* Make generic extended vacuum stats report */
+	extvac_stats_end(rel, &extVacCounters, &extVacReport);
+
+	/* Fill heap-specific extended stats fields */
+	extVacReport.pages_scanned = vacrel->scanned_pages;
+	extVacReport.pages_removed = vacrel->removed_pages;
+	extVacReport.pages_frozen = vacrel->set_frozen_pages;
+	extVacReport.pages_all_visible = vacrel->set_all_visible_pages;
+	extVacReport.tuples_deleted = vacrel->tuples_deleted;
+	extVacReport.tuples_frozen = vacrel->tuples_frozen;
+	extVacReport.recently_dead_tuples = vacrel->recently_dead_tuples;
+	extVacReport.index_vacuum_count = vacrel->num_index_scans;
+
 	/*
 	 * Report results to the cumulative stats system, too.
 	 *
@@ -586,7 +710,8 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 						 rel->rd_rel->relisshared,
 						 Max(vacrel->new_live_tuples, 0),
 						 vacrel->recently_dead_tuples +
-						 vacrel->missed_dead_tuples);
+						 vacrel->missed_dead_tuples,
+						 &extVacReport);
 	pgstat_progress_end_command();
 
 	if (instrument)
@@ -1378,6 +1503,8 @@ lazy_scan_new_or_empty(LVRelState *vacrel, Buffer buf, BlockNumber blkno,
 							  vmbuffer, InvalidTransactionId,
 							  VISIBILITYMAP_ALL_VISIBLE | VISIBILITYMAP_ALL_FROZEN);
 			END_CRIT_SECTION();
+			vacrel->set_all_visible_pages++;
+			vacrel->set_frozen_pages++;
 		}
 
 		freespace = PageGetHeapFreeSpace(page);
@@ -2275,11 +2402,13 @@ lazy_vacuum_heap_page(LVRelState *vacrel, BlockNumber blkno, Buffer buffer,
 								 &all_frozen))
 	{
 		uint8		flags = VISIBILITYMAP_ALL_VISIBLE;
+		vacrel->set_all_visible_pages++;
 
 		if (all_frozen)
 		{
 			Assert(!TransactionIdIsValid(visibility_cutoff_xid));
 			flags |= VISIBILITYMAP_ALL_FROZEN;
+			vacrel->set_frozen_pages++;
 		}
 
 		PageSetAllVisible(page);
@@ -3120,6 +3249,8 @@ vacuum_error_callback(void *arg)
 	switch (errinfo->phase)
 	{
 		case VACUUM_ERRCB_PHASE_SCAN_HEAP:
+			if(geterrelevel() == ERROR)
+				pgstat_report_vacuum_error(errinfo->reloid);
 			if (BlockNumberIsValid(errinfo->blkno))
 			{
 				if (OffsetNumberIsValid(errinfo->offnum))
@@ -3135,6 +3266,8 @@ vacuum_error_callback(void *arg)
 			break;
 
 		case VACUUM_ERRCB_PHASE_VACUUM_HEAP:
+			if(geterrelevel() == ERROR)
+				pgstat_report_vacuum_error(errinfo->reloid);
 			if (BlockNumberIsValid(errinfo->blkno))
 			{
 				if (OffsetNumberIsValid(errinfo->offnum))
diff --git a/src/backend/access/heap/visibilitymap.c b/src/backend/access/heap/visibilitymap.c
index 8b24e7bc33..d25c9a3679 100644
--- a/src/backend/access/heap/visibilitymap.c
+++ b/src/backend/access/heap/visibilitymap.c
@@ -91,6 +91,7 @@
 #include "access/xloginsert.h"
 #include "access/xlogutils.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "port/pg_bitutils.h"
 #include "storage/bufmgr.h"
 #include "storage/smgr.h"
@@ -160,6 +161,15 @@ visibilitymap_clear(Relation rel, BlockNumber heapBlk, Buffer vmbuf, uint8 flags
 
 	if (map[mapByte] & mask)
 	{
+		/*
+		 * As part of vacuum stats, track how often all-visible or all-frozen
+		 * bits are cleared.
+		 */
+		if (map[mapByte] >> mapOffset & flags & VISIBILITYMAP_ALL_VISIBLE)
+			pgstat_count_vm_rev_all_visible(rel);
+		if (map[mapByte] >> mapOffset & flags & VISIBILITYMAP_ALL_FROZEN)
+			pgstat_count_vm_rev_all_frozen(rel);
+
 		map[mapByte] &= ~mask;
 
 		MarkBufferDirty(vmbuf);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..33ff7c81aa 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -691,7 +691,9 @@ CREATE VIEW pg_stat_all_tables AS
             pg_stat_get_vacuum_count(C.oid) AS vacuum_count,
             pg_stat_get_autovacuum_count(C.oid) AS autovacuum_count,
             pg_stat_get_analyze_count(C.oid) AS analyze_count,
-            pg_stat_get_autoanalyze_count(C.oid) AS autoanalyze_count
+            pg_stat_get_autoanalyze_count(C.oid) AS autoanalyze_count,
+            pg_stat_get_rev_all_frozen_pages(C.oid) AS rev_all_frozen_pages,
+            pg_stat_get_rev_all_visible_pages(C.oid) AS rev_all_visible_pages
     FROM pg_class C LEFT JOIN
          pg_index I ON C.oid = I.indrelid
          LEFT JOIN pg_namespace N ON (N.oid = C.relnamespace)
@@ -1381,3 +1383,47 @@ CREATE VIEW pg_stat_subscription_stats AS
 
 CREATE VIEW pg_wait_events AS
     SELECT * FROM pg_get_wait_events();
+--
+-- Show extended cumulative statistics on a vacuum operation over all tables and
+-- databases of the instance.
+-- Use Invalid Oid "0" as an input relation id to get stat on each table in a
+-- database.
+--
+
+CREATE VIEW pg_stat_vacuum_tables AS
+SELECT
+  ns.nspname AS "schema",
+  rel.relname AS relname,
+  stats.relid as relid,
+
+  stats.total_blks_read AS total_blks_read,
+  stats.total_blks_hit AS total_blks_hit,
+  stats.total_blks_dirtied AS total_blks_dirtied,
+  stats.total_blks_written AS total_blks_written,
+
+  stats.rel_blks_read AS rel_blks_read,
+  stats.rel_blks_hit AS rel_blks_hit,
+
+  stats.pages_scanned AS pages_scanned,
+  stats.pages_removed AS pages_removed,
+  stats.pages_frozen AS pages_frozen,
+  stats.pages_all_visible AS pages_all_visible,
+  stats.tuples_deleted AS tuples_deleted,
+  stats.tuples_frozen AS tuples_frozen,
+  stats.dead_tuples AS dead_tuples,
+
+  stats.index_vacuum_count AS index_vacuum_count,
+  stats.wal_records AS wal_records,
+  stats.wal_fpi AS wal_fpi,
+  stats.wal_bytes AS wal_bytes,
+
+  stats.blk_read_time AS blk_read_time,
+  stats.blk_write_time AS blk_write_time,
+
+  stats.delay_time AS delay_time,
+  stats.total_time AS total_time
+
+FROM pg_class rel
+  JOIN pg_namespace ns ON ns.oid = rel.relnamespace,
+  LATERAL pg_stat_get_vacuum_tables(rel.oid) stats
+WHERE rel.relkind = 'r';
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index 86f36b3695..1cff446fb9 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -102,6 +102,9 @@ pg_atomic_uint32 *VacuumSharedCostBalance = NULL;
 pg_atomic_uint32 *VacuumActiveNWorkers = NULL;
 int			VacuumCostBalanceLocal = 0;
 
+/* Cumulative storage to report total vacuum delay time. */
+double VacuumDelayTime = 0; /* msec. */
+
 /* non-export function prototypes */
 static List *expand_vacuum_rel(VacuumRelation *vrel,
 							   MemoryContext vac_context, int options);
@@ -2418,6 +2421,7 @@ vacuum_delay_point(void)
 			exit(1);
 
 		VacuumCostBalance = 0;
+		VacuumDelayTime += msec;
 
 		/*
 		 * Balance and update limit values for autovacuum workers. We must do
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 4fd6574e12..7f7c7c16e2 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -1048,6 +1048,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	/* Set cost-based vacuum delay */
 	VacuumUpdateCosts();
 	VacuumCostBalance = 0;
+	VacuumDelayTime = 0;
 	VacuumCostBalanceLocal = 0;
 	VacuumSharedCostBalance = &(shared->cost_balance);
 	VacuumActiveNWorkers = &(shared->active_nworkers);
diff --git a/src/backend/utils/activity/pgstat.c b/src/backend/utils/activity/pgstat.c
index ea8c5691e8..3f2e9f5468 100644
--- a/src/backend/utils/activity/pgstat.c
+++ b/src/backend/utils/activity/pgstat.c
@@ -189,7 +189,7 @@ static void pgstat_reset_after_failure(void);
 static bool pgstat_flush_pending_entries(bool nowait);
 
 static void pgstat_prep_snapshot(void);
-static void pgstat_build_snapshot(void);
+static void pgstat_build_snapshot(PgStat_Kind statKind);
 static void pgstat_build_snapshot_fixed(PgStat_Kind kind);
 
 static inline bool pgstat_is_kind_valid(PgStat_Kind kind);
@@ -259,7 +259,6 @@ static bool pgstat_is_initialized = false;
 static bool pgstat_is_shutdown = false;
 #endif
 
-
 /*
  * The different kinds of built-in statistics.
  *
@@ -878,7 +877,6 @@ pgstat_reset_of_kind(PgStat_Kind kind)
 		pgstat_reset_entries_of_kind(kind, ts);
 }
 
-
 /* ------------------------------------------------------------
  * Fetching of stats
  * ------------------------------------------------------------
@@ -947,7 +945,7 @@ pgstat_fetch_entry(PgStat_Kind kind, Oid dboid, uint64 objid)
 
 	/* if we need to build a full snapshot, do so */
 	if (pgstat_fetch_consistency == PGSTAT_FETCH_CONSISTENCY_SNAPSHOT)
-		pgstat_build_snapshot();
+		pgstat_build_snapshot(PGSTAT_KIND_INVALID);
 
 	/* if caching is desired, look up in cache */
 	if (pgstat_fetch_consistency > PGSTAT_FETCH_CONSISTENCY_NONE)
@@ -1063,7 +1061,7 @@ pgstat_snapshot_fixed(PgStat_Kind kind)
 		pgstat_clear_snapshot();
 
 	if (pgstat_fetch_consistency == PGSTAT_FETCH_CONSISTENCY_SNAPSHOT)
-		pgstat_build_snapshot();
+		pgstat_build_snapshot(PGSTAT_KIND_INVALID);
 	else
 		pgstat_build_snapshot_fixed(kind);
 
@@ -1113,8 +1111,30 @@ pgstat_prep_snapshot(void)
 							   NULL);
 }
 
+
+/*
+ * Trivial external interface to build a snapshot for table statistics only.
+ */
+void
+pgstat_update_snapshot(PgStat_Kind kind)
+{
+	int save_consistency_guc = pgstat_fetch_consistency;
+	pgstat_clear_snapshot();
+
+	PG_TRY();
+	{
+		pgstat_fetch_consistency = PGSTAT_FETCH_CONSISTENCY_SNAPSHOT;
+		pgstat_build_snapshot(PGSTAT_KIND_RELATION);
+	}
+	PG_FINALLY();
+	{
+		pgstat_fetch_consistency = save_consistency_guc;
+	}
+	PG_END_TRY();
+}
+
 static void
-pgstat_build_snapshot(void)
+pgstat_build_snapshot(PgStat_Kind statKind)
 {
 	dshash_seq_status hstat;
 	PgStatShared_HashEntry *p;
diff --git a/src/backend/utils/activity/pgstat_relation.c b/src/backend/utils/activity/pgstat_relation.c
index faba8b64d2..5dd8275671 100644
--- a/src/backend/utils/activity/pgstat_relation.c
+++ b/src/backend/utils/activity/pgstat_relation.c
@@ -47,6 +47,8 @@ static void add_tabstat_xact_level(PgStat_TableStatus *pgstat_info, int nest_lev
 static void ensure_tabstat_xact_level(PgStat_TableStatus *pgstat_info);
 static void save_truncdrop_counters(PgStat_TableXactStatus *trans, bool is_drop);
 static void restore_truncdrop_counters(PgStat_TableXactStatus *trans);
+static void pgstat_accumulate_extvac_stats(ExtVacReport *dst, ExtVacReport *src,
+							   bool accumulate_reltype_specific_info);
 
 
 /*
@@ -203,12 +205,37 @@ pgstat_drop_relation(Relation rel)
 	}
 }
 
+/*
+ * Report an error while vacuuming.
+ */
+void
+pgstat_report_vacuum_error(Oid tableoid)
+{
+	PgStat_EntryRef *entry_ref;
+	PgStatShared_Relation *shtabentry;
+	PgStat_StatTabEntry *tabentry;
+	Oid			dboid =  MyDatabaseId;
+
+	if (!pgstat_track_counts)
+		return;
+
+	entry_ref = pgstat_get_entry_ref_locked(PGSTAT_KIND_RELATION,
+											dboid, tableoid, false);
+
+	shtabentry = (PgStatShared_Relation *) entry_ref->shared_stats;
+	tabentry = &shtabentry->stats;
+
+	tabentry->vacuum_ext.errors++;
+	pgstat_unlock_entry(entry_ref);
+}
+
 /*
  * Report that the table was just vacuumed and flush IO statistics.
  */
 void
 pgstat_report_vacuum(Oid tableoid, bool shared,
-					 PgStat_Counter livetuples, PgStat_Counter deadtuples)
+					 PgStat_Counter livetuples, PgStat_Counter deadtuples,
+					 ExtVacReport *params)
 {
 	PgStat_EntryRef *entry_ref;
 	PgStatShared_Relation *shtabentry;
@@ -232,6 +259,8 @@ pgstat_report_vacuum(Oid tableoid, bool shared,
 	tabentry->live_tuples = livetuples;
 	tabentry->dead_tuples = deadtuples;
 
+	pgstat_accumulate_extvac_stats(&tabentry->vacuum_ext, params, true);
+
 	/*
 	 * It is quite possible that a non-aggressive VACUUM ended up skipping
 	 * various pages, however, we'll zero the insert counter here regardless.
@@ -857,6 +886,9 @@ pgstat_relation_flush_cb(PgStat_EntryRef *entry_ref, bool nowait)
 	tabentry->blocks_fetched += lstats->counts.blocks_fetched;
 	tabentry->blocks_hit += lstats->counts.blocks_hit;
 
+	tabentry->rev_all_frozen_pages += lstats->counts.rev_all_frozen_pages;
+	tabentry->rev_all_visible_pages += lstats->counts.rev_all_visible_pages;
+
 	/* Clamp live_tuples in case of negative delta_live_tuples */
 	tabentry->live_tuples = Max(tabentry->live_tuples, 0);
 	/* Likewise for dead_tuples */
@@ -980,3 +1012,36 @@ restore_truncdrop_counters(PgStat_TableXactStatus *trans)
 		trans->tuples_deleted = trans->deleted_pre_truncdrop;
 	}
 }
+
+static void
+pgstat_accumulate_extvac_stats(ExtVacReport *dst, ExtVacReport *src,
+							   bool accumulate_reltype_specific_info)
+{
+	dst->total_blks_read += src->total_blks_read;
+	dst->total_blks_hit += src->total_blks_hit;
+	dst->total_blks_dirtied += src->total_blks_dirtied;
+	dst->total_blks_written += src->total_blks_written;
+	dst->wal_bytes += src->wal_bytes;
+	dst->wal_fpi += src->wal_fpi;
+	dst->wal_records += src->wal_records;
+	dst->blk_read_time += src->blk_read_time;
+	dst->blk_write_time += src->blk_write_time;
+	dst->delay_time += src->delay_time;
+	dst->total_time += src->total_time;
+	dst->errors += src->errors;
+
+	if (!accumulate_reltype_specific_info)
+		return;
+
+	dst->blks_fetched += src->blks_fetched;
+	dst->blks_hit += src->blks_hit;
+
+	dst->pages_scanned += src->pages_scanned;
+	dst->pages_removed += src->pages_removed;
+	dst->pages_frozen += src->pages_frozen;
+	dst->pages_all_visible += src->pages_all_visible;
+	dst->tuples_deleted += src->tuples_deleted;
+	dst->tuples_frozen += src->tuples_frozen;
+	dst->recently_dead_tuples += src->recently_dead_tuples;
+	dst->index_vacuum_count += src->index_vacuum_count;
+}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..ab3fba3314 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -31,6 +31,42 @@
 #include "utils/acl.h"
 #include "utils/builtins.h"
 #include "utils/timestamp.h"
+#include "utils/pgstat_internal.h"
+
+/* hash table for statistics snapshots entry */
+typedef struct PgStat_SnapshotEntry
+{
+	PgStat_HashKey key;
+	char		status;			/* for simplehash use */
+	void	   *data;			/* the stats data itself */
+} PgStat_SnapshotEntry;
+
+/* ----------
+ * Backend-local Hash Table Definitions
+ * ----------
+ */
+
+/* for stats snapshot entries */
+#define SH_PREFIX pgstat_snapshot
+#define SH_ELEMENT_TYPE PgStat_SnapshotEntry
+#define SH_KEY_TYPE PgStat_HashKey
+#define SH_KEY key
+#define SH_HASH_KEY(tb, key) \
+	pgstat_hash_hash_key(&key, sizeof(PgStat_HashKey), NULL)
+#define SH_EQUAL(tb, a, b) \
+	pgstat_cmp_hash_key(&a, &b, sizeof(PgStat_HashKey), NULL) == 0
+#define SH_SCOPE static inline
+#define SH_DEFINE
+#define SH_DECLARE
+#include "lib/simplehash.h"
+
+typedef pgstat_snapshot_iterator SnapshotIterator;
+
+#define InitSnapshotIterator(htable, iter) \
+	pgstat_snapshot_start_iterate(htable, iter);
+#define ScanStatSnapshot(htable, iter) \
+	pgstat_snapshot_iterate(htable, iter)
+
 
 #define UINT32_ACCESS_ONCE(var)		 ((uint32)(*((volatile uint32 *)&(var))))
 
@@ -106,6 +142,12 @@ PG_STAT_GET_RELENTRY_INT64(tuples_updated)
 /* pg_stat_get_vacuum_count */
 PG_STAT_GET_RELENTRY_INT64(vacuum_count)
 
+/* pg_stat_get_rev_frozen_pages */
+PG_STAT_GET_RELENTRY_INT64(rev_all_frozen_pages)
+
+/* pg_stat_get_rev_all_visible_pages */
+PG_STAT_GET_RELENTRY_INT64(rev_all_visible_pages)
+
 #define PG_STAT_GET_RELENTRY_TIMESTAMPTZ(stat)					\
 Datum															\
 CppConcat(pg_stat_get_,stat)(PG_FUNCTION_ARGS)					\
@@ -2069,3 +2111,132 @@ pg_stat_have_stats(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(pgstat_have_entry(kind, dboid, objid));
 }
+
+
+/*
+ * Get the vacuum statistics for the heap tables.
+ */
+Datum
+pg_stat_get_vacuum_tables(PG_FUNCTION_ARGS)
+{
+	#define PG_STAT_GET_VACUUM_TABLES_STATS_COLS 22
+
+	Oid						relid = PG_GETARG_OID(0);
+	PgStat_StatTabEntry     *tabentry;
+	ExtVacReport 			*extvacuum;
+	TupleDesc				 tupdesc;
+	Datum					 values[PG_STAT_GET_VACUUM_TABLES_STATS_COLS] = {0};
+	bool					 nulls[PG_STAT_GET_VACUUM_TABLES_STATS_COLS] = {0};
+	char					 buf[256];
+	int						 i = 0;
+	ExtVacReport allzero;
+
+	/* Initialise attributes information in the tuple descriptor */
+	tupdesc = CreateTemplateTupleDesc(PG_STAT_GET_VACUUM_TABLES_STATS_COLS);
+
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "relid",
+					   INT4OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "total_blks_read",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "total_blks_hit",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "total_blks_dirtied",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "total_blks_written",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "rel_blks_read",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "rel_blks_hit",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "pages_scanned",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "pages_removed",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "pages_frozen",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "pages_all_visible",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "tuples_deleted",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "tuples_frozen",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "dead_tuples",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "index_vacuum_count",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "wal_records",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "wal_fpi",
+					   INT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "wal_bytes",
+					   NUMERICOID, -1, 0);
+
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "blk_read_time",
+					   FLOAT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "blk_write_time",
+					   FLOAT8OID, -1, 0);
+
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "delay_time",
+					   FLOAT8OID, -1, 0);
+	TupleDescInitEntry(tupdesc, (AttrNumber) ++i, "total_time",
+					   FLOAT8OID, -1, 0);
+
+	Assert(i == PG_STAT_GET_VACUUM_TABLES_STATS_COLS);
+
+	BlessTupleDesc(tupdesc);
+
+	tabentry = pgstat_fetch_stat_tabentry(relid);
+
+	if (tabentry == NULL)
+	{
+		/* If the subscription is not found, initialise its stats */
+		memset(&allzero, 0, sizeof(ExtVacReport));
+		extvacuum = &allzero;
+	}
+	else
+	{
+		extvacuum = &(tabentry->vacuum_ext);
+	}
+
+	i = 0;
+
+	values[i++] = ObjectIdGetDatum(relid);
+
+	values[i++] = Int64GetDatum(extvacuum->total_blks_read);
+	values[i++] = Int64GetDatum(extvacuum->total_blks_hit);
+	values[i++] = Int64GetDatum(extvacuum->total_blks_dirtied);
+	values[i++] = Int64GetDatum(extvacuum->total_blks_written);
+
+	values[i++] = Int64GetDatum(extvacuum->blks_fetched -
+									extvacuum->blks_hit);
+	values[i++] = Int64GetDatum(extvacuum->blks_hit);
+
+	values[i++] = Int64GetDatum(extvacuum->pages_scanned);
+	values[i++] = Int64GetDatum(extvacuum->pages_removed);
+	values[i++] = Int64GetDatum(extvacuum->pages_frozen);
+	values[i++] = Int64GetDatum(extvacuum->pages_all_visible);
+	values[i++] = Int64GetDatum(extvacuum->tuples_deleted);
+	values[i++] = Int64GetDatum(extvacuum->tuples_frozen);
+	values[i++] = Int64GetDatum(extvacuum->recently_dead_tuples);
+	values[i++] = Int64GetDatum(extvacuum->index_vacuum_count);
+
+	values[i++] = Int64GetDatum(extvacuum->wal_records);
+	values[i++] = Int64GetDatum(extvacuum->wal_fpi);
+
+	/* Convert to numeric, like pg_stat_statements */
+	snprintf(buf, sizeof buf, UINT64_FORMAT, extvacuum->wal_bytes);
+	values[i++] = DirectFunctionCall3(numeric_in,
+									  CStringGetDatum(buf),
+									  ObjectIdGetDatum(0),
+									  Int32GetDatum(-1));
+
+	values[i++] = Float8GetDatum(extvacuum->blk_read_time);
+	values[i++] = Float8GetDatum(extvacuum->blk_write_time);
+	values[i++] = Float8GetDatum(extvacuum->delay_time);
+	values[i++] = Float8GetDatum(extvacuum->total_time);
+
+	Assert(i == PG_STAT_GET_VACUUM_TABLES_STATS_COLS);
+
+	/* Returns the record as Datum */
+	PG_RETURN_DATUM(HeapTupleGetDatum(heap_form_tuple(tupdesc, values, nulls)));
+}
diff --git a/src/backend/utils/error/elog.c b/src/backend/utils/error/elog.c
index 8acca3e0a0..fe554547f5 100644
--- a/src/backend/utils/error/elog.c
+++ b/src/backend/utils/error/elog.c
@@ -1619,6 +1619,19 @@ getinternalerrposition(void)
 	return edata->internalpos;
 }
 
+/*
+ * Return elevel of errors
+ */
+int
+geterrelevel(void)
+{
+	ErrorData  *edata = &errordata[errordata_stack_depth];
+
+	/* we don't bother incrementing recursion_depth */
+	CHECK_STACK_DEPTH();
+
+	return edata->elevel;
+}
 
 /*
  * Functions to allow construction of error message strings separately from
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index cbbe8acd38..aacbbfabae 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -12402,4 +12402,22 @@
   proargtypes => 'int2',
   prosrc => 'gist_stratnum_identity' },
 
+{ oid => '8001',
+  descr => 'pg_stat_get_vacuum_tables return stats values',
+  proname => 'pg_stat_get_vacuum_tables', prorows => 1000, provolatile => 's', prorettype => 'record',proisstrict => 'f',
+  proretset => 't',
+  proargtypes => 'oid',
+  proallargtypes => '{oid,oid,int8,int8,int8,int8,int8,int8,int8,int8,int8,int8,int8,int8,int8,int8,int8,int8,numeric,float8,float8,float8,float8}',
+  proargmodes => '{i,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o,o}',
+  proargnames => '{reloid,relid,total_blks_read,total_blks_hit,total_blks_dirtied,total_blks_written,rel_blks_read,rel_blks_hit,pages_scanned,pages_removed,pages_frozen,pages_all_visible,tuples_deleted,tuples_frozen,dead_tuples,index_vacuum_count,wal_records,wal_fpi,wal_bytes,blk_read_time,blk_write_time,delay_time,total_time}',
+  prosrc => 'pg_stat_get_vacuum_tables' },
+
+  { oid => '8002', descr => 'statistics: number of times the all-visible pages in the visibility map was removed for pages of table',
+  proname => 'pg_stat_get_rev_all_visible_pages', provolatile => 's',
+  proparallel => 'r', prorettype => 'int8', proargtypes => 'oid',
+  prosrc => 'pg_stat_get_rev_all_visible_pages' },
+  { oid => '8003', descr => 'statistics: number of times the all-frozen pages in the visibility map was removed for pages of table',
+  proname => 'pg_stat_get_rev_all_frozen_pages', provolatile => 's',
+  proparallel => 'r', prorettype => 'int8', proargtypes => 'oid',
+  prosrc => 'pg_stat_get_rev_all_frozen_pages' },
 ]
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 759f9a87d3..07b28b15d9 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -308,6 +308,7 @@ extern PGDLLIMPORT int vacuum_multixact_failsafe_age;
 extern PGDLLIMPORT pg_atomic_uint32 *VacuumSharedCostBalance;
 extern PGDLLIMPORT pg_atomic_uint32 *VacuumActiveNWorkers;
 extern PGDLLIMPORT int VacuumCostBalanceLocal;
+extern PGDLLIMPORT double VacuumDelayTime;
 
 extern PGDLLIMPORT bool VacuumFailsafeActive;
 extern PGDLLIMPORT double vacuum_cost_delay;
diff --git a/src/include/pgstat.h b/src/include/pgstat.h
index 59c28b4aca..a6f363ac0d 100644
--- a/src/include/pgstat.h
+++ b/src/include/pgstat.h
@@ -169,6 +169,47 @@ typedef struct PgStat_BackendSubEntry
 	PgStat_Counter conflict_count[CONFLICT_NUM_TYPES];
 } PgStat_BackendSubEntry;
 
+/* 
+ * ExtVacReport
+ *
+ * Additional statistics of vacuum processing over a single heap relation.
+ */
+typedef struct ExtVacReport
+{
+	/* number of blocks missed, hit, dirtied and written */
+	int64		total_blks_read;
+	int64		total_blks_hit;
+	int64		total_blks_dirtied;
+	int64		total_blks_written;
+
+	/* blocks missed and hit for just the heap */
+	int64		blks_fetched;
+	int64		blks_hit;
+
+	/* Vacuum WAL usage stats */
+	int64		wal_records;	/* wal usage: number of WAL records */
+	int64		wal_fpi;		/* wal usage: number of WAL full page images produced */
+	uint64		wal_bytes;		/* wal usage: size of WAL records produced */
+
+	/* Time stats. */
+	double		blk_read_time;	/* time spent reading pages, in msec */
+	double		blk_write_time; /* time spent writing pages, in msec */
+	double		delay_time;		/* how long vacuum slept in vacuum delay point, in msec */
+	double		total_time;		/* total time of a vacuum operation, in msec */
+
+	/* Interruptions on any errors. */
+	int32		errors;
+
+	int64		pages_scanned;		/* heap pages examined (not skipped by VM) */
+	int64		pages_removed;		/* heap pages removed by vacuum "truncation" */
+	int64		pages_frozen;		/* pages marked in VM as frozen */
+	int64		pages_all_visible;	/* pages marked in VM as all-visible */
+	int64		tuples_deleted;		/* tuples deleted by vacuum */
+	int64		tuples_frozen;		/* tuples frozen up by vacuum */
+	int64		recently_dead_tuples;	/* deleted tuples that are still visible to some transaction */
+	int64		index_vacuum_count;	/* number of index vacuumings */
+} ExtVacReport;
+
 /* ----------
  * PgStat_TableCounts			The actual per-table counts kept by a backend
  *
@@ -209,6 +250,16 @@ typedef struct PgStat_TableCounts
 
 	PgStat_Counter blocks_fetched;
 	PgStat_Counter blocks_hit;
+
+	PgStat_Counter rev_all_visible_pages;
+	PgStat_Counter rev_all_frozen_pages;
+
+	/*
+	 * Additional cumulative stat on vacuum operations.
+	 * Use an expensive structure as an abstraction for different types of
+	 * relations.
+	 */
+	ExtVacReport	vacuum_ext;
 } PgStat_TableCounts;
 
 /* ----------
@@ -267,7 +318,7 @@ typedef struct PgStat_TableXactStatus
  * ------------------------------------------------------------
  */
 
-#define PGSTAT_FILE_FORMAT_ID	0x01A5BCAF
+#define PGSTAT_FILE_FORMAT_ID	0x01A5BCB1
 
 typedef struct PgStat_ArchiverStats
 {
@@ -388,6 +439,7 @@ typedef struct PgStat_StatDBEntry
 	PgStat_Counter sessions_killed;
 	PgStat_Counter parallel_workers_to_launch;
 	PgStat_Counter parallel_workers_launched;
+	ExtVacReport vacuum_ext;		/* extended vacuum statistics */
 
 	TimestampTz stat_reset_timestamp;
 } PgStat_StatDBEntry;
@@ -463,6 +515,11 @@ typedef struct PgStat_StatTabEntry
 	PgStat_Counter analyze_count;
 	TimestampTz last_autoanalyze_time;	/* autovacuum initiated */
 	PgStat_Counter autoanalyze_count;
+
+	PgStat_Counter rev_all_visible_pages;
+	PgStat_Counter rev_all_frozen_pages;
+
+	ExtVacReport vacuum_ext;
 } PgStat_StatTabEntry;
 
 typedef struct PgStat_WalStats
@@ -630,10 +687,12 @@ extern void pgstat_assoc_relation(Relation rel);
 extern void pgstat_unlink_relation(Relation rel);
 
 extern void pgstat_report_vacuum(Oid tableoid, bool shared,
-								 PgStat_Counter livetuples, PgStat_Counter deadtuples);
+								 PgStat_Counter livetuples, PgStat_Counter deadtuples,
+								 ExtVacReport *params);
 extern void pgstat_report_analyze(Relation rel,
 								  PgStat_Counter livetuples, PgStat_Counter deadtuples,
 								  bool resetcounter);
+extern void pgstat_report_vacuum_error(Oid tableoid);
 
 /*
  * If stats are enabled, but pending data hasn't been prepared yet, call
@@ -681,6 +740,17 @@ extern void pgstat_report_analyze(Relation rel,
 		if (pgstat_should_count_relation(rel))						\
 			(rel)->pgstat_info->counts.blocks_hit++;				\
 	} while (0)
+/* accumulate unfrozen all-visible and all-frozen pages */
+#define pgstat_count_vm_rev_all_visible(rel)						\
+	do {															\
+		if (pgstat_should_count_relation(rel))						\
+			(rel)->pgstat_info->counts.rev_all_visible_pages++;	\
+	} while (0)
+#define pgstat_count_vm_rev_all_frozen(rel)						\
+	do {															\
+		if (pgstat_should_count_relation(rel))						\
+			(rel)->pgstat_info->counts.rev_all_frozen_pages++;	\
+	} while (0)
 
 extern void pgstat_count_heap_insert(Relation rel, PgStat_Counter n);
 extern void pgstat_count_heap_update(Relation rel, bool hot, bool newpage);
@@ -698,7 +768,6 @@ extern PgStat_StatTabEntry *pgstat_fetch_stat_tabentry_ext(bool shared,
 														   Oid reloid);
 extern PgStat_TableStatus *find_tabstat_entry(Oid rel_id);
 
-
 /*
  * Functions in pgstat_replslot.c
  */
diff --git a/src/include/utils/elog.h b/src/include/utils/elog.h
index e54eca5b48..e752c0ce01 100644
--- a/src/include/utils/elog.h
+++ b/src/include/utils/elog.h
@@ -230,6 +230,7 @@ extern int	geterrlevel(void);
 extern int	geterrposition(void);
 extern int	getinternalerrposition(void);
 
+extern int	geterrelevel(void);
 
 /*----------
  * Old-style error reporting API: to be used in this way:
diff --git a/src/include/utils/pgstat_internal.h b/src/include/utils/pgstat_internal.h
index 61b2e1f96b..2c0e55d63f 100644
--- a/src/include/utils/pgstat_internal.h
+++ b/src/include/utils/pgstat_internal.h
@@ -573,7 +573,7 @@ extern PgStat_EntryRef *pgstat_fetch_pending_entry(PgStat_Kind kind,
 extern void *pgstat_fetch_entry(PgStat_Kind kind, Oid dboid, uint64 objid);
 extern void pgstat_snapshot_fixed(PgStat_Kind kind);
 
-
+extern void pgstat_update_snapshot(PgStat_Kind kind);
 /*
  * Functions in pgstat_archiver.c
  */
diff --git a/src/test/isolation/isolation_schedule b/src/test/isolation/isolation_schedule
index 143109aa4d..e93dd4f626 100644
--- a/src/test/isolation/isolation_schedule
+++ b/src/test/isolation/isolation_schedule
@@ -95,6 +95,7 @@ test: timeouts
 test: vacuum-concurrent-drop
 test: vacuum-conflict
 test: vacuum-skip-locked
+test: vacuum-extending-in-repetable-read
 test: stats
 test: horizons
 test: predicate-hash
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..ca7c64f55f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1804,7 +1804,9 @@ pg_stat_all_tables| SELECT c.oid AS relid,
     pg_stat_get_vacuum_count(c.oid) AS vacuum_count,
     pg_stat_get_autovacuum_count(c.oid) AS autovacuum_count,
     pg_stat_get_analyze_count(c.oid) AS analyze_count,
-    pg_stat_get_autoanalyze_count(c.oid) AS autoanalyze_count
+    pg_stat_get_autoanalyze_count(c.oid) AS autoanalyze_count,
+    pg_stat_get_rev_all_frozen_pages(c.oid) AS rev_all_frozen_pages,
+    pg_stat_get_rev_all_visible_pages(c.oid) AS rev_all_visible_pages
    FROM ((pg_class c
      LEFT JOIN pg_index i ON ((c.oid = i.indrelid)))
      LEFT JOIN pg_namespace n ON ((n.oid = c.relnamespace)))
@@ -1881,6 +1883,34 @@ pg_stat_database_conflicts| SELECT oid AS datid,
     pg_stat_get_db_conflict_startup_deadlock(oid) AS confl_deadlock,
     pg_stat_get_db_conflict_logicalslot(oid) AS confl_active_logicalslot
    FROM pg_database d;
+pg_stat_get_vacuum_tables| SELECT ns.nspname AS schema,
+    rel.relname,
+    stats.relid,
+    stats.total_blks_read,
+    stats.total_blks_hit,
+    stats.total_blks_dirtied,
+    stats.total_blks_written,
+    stats.rel_blks_read,
+    stats.rel_blks_hit,
+    stats.pages_scanned,
+    stats.pages_removed,
+    stats.pages_frozen,
+    stats.pages_all_visible,
+    stats.tuples_deleted,
+    stats.tuples_frozen,
+    stats.dead_tuples,
+    stats.index_vacuum_count,
+    stats.wal_records,
+    stats.wal_fpi,
+    stats.wal_bytes,
+    stats.blk_read_time,
+    stats.blk_write_time,
+    stats.delay_time,
+    stats.total_time
+   FROM (pg_class rel
+     JOIN pg_namespace ns ON ((ns.oid = rel.relnamespace))),
+    LATERAL pg_stat_get_vacuum_tables(rel.oid) stats(relid, total_blks_read, total_blks_hit, total_blks_dirtied, total_blks_written, rel_blks_read, rel_blks_hit, pages_scanned, pages_removed, pages_frozen, pages_all_visible, tuples_deleted, tuples_frozen, dead_tuples, index_vacuum_count, wal_records, wal_fpi, wal_bytes, blk_read_time, blk_write_time, delay_time, total_time)
+  WHERE (rel.relkind = 'r'::"char");
 pg_stat_gssapi| SELECT pid,
     gss_auth AS gss_authenticated,
     gss_princ AS principal,
@@ -2188,7 +2218,9 @@ pg_stat_sys_tables| SELECT relid,
     vacuum_count,
     autovacuum_count,
     analyze_count,
-    autoanalyze_count
+    autoanalyze_count,
+    rev_all_frozen_pages,
+    rev_all_visible_pages
    FROM pg_stat_all_tables
   WHERE ((schemaname = ANY (ARRAY['pg_catalog'::name, 'information_schema'::name])) OR (schemaname ~ '^pg_toast'::text));
 pg_stat_user_functions| SELECT p.oid AS funcid,
@@ -2236,7 +2268,9 @@ pg_stat_user_tables| SELECT relid,
     vacuum_count,
     autovacuum_count,
     analyze_count,
-    autoanalyze_count
+    autoanalyze_count,
+    rev_all_frozen_pages,
+    rev_all_visible_pages
    FROM pg_stat_all_tables
   WHERE ((schemaname <> ALL (ARRAY['pg_catalog'::name, 'information_schema'::name])) AND (schemaname !~ '^pg_toast'::text));
 pg_stat_wal| SELECT wal_records,
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 81e4222d26..977a047202 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -136,3 +136,8 @@ test: fast_default
 # run tablespace test at the end because it drops the tablespace created during
 # setup that other tests may use.
 test: tablespace
+
+# ----------
+# Check vacuum statistics
+# ----------
+test: vacuum_tables_statistics
\ No newline at end of file
