Add index scan progress to pg_stat_progress_vacuum

Started by Imseih (AWS), Samiabout 4 years ago138 messages
#1Imseih (AWS), Sami
simseih@amazon.com
1 attachment(s)

The current implementation of pg_stat_progress_vacuum does not provide progress on which index is being vacuumed making it difficult for a user to determine if the "vacuuming indexes" phase is making progress. By exposing which index is being scanned as well as the total progress the scan has made for the current cycle, a user can make better estimations on when the vacuum will complete.

The proposed patch adds 4 new columns to pg_stat_progress_vacuum:

1. indrelid - the relid of the index being vacuumed
2. index_blks_total - total number of blocks to be scanned in the current cycle
3. index_blks_scanned - number of blocks scanned in the current cycle
4. leader_pid - if the pid for the pg_stat_progress_vacuum entry is a leader or a vacuum worker. This patch places an entry for every worker pid ( if parallel ) as well as the leader pid

Attached is the patch.

Here is a sample output of a parallel vacuum for table with relid = 16638

postgres=# select * from pg_stat_progress_vacuum ;
-[ RECORD 1 ]------+------------------
pid | 18180
datid | 13732
datname | postgres
relid | 16638
phase | vacuuming indexes
heap_blks_total | 5149825
heap_blks_scanned | 5149825
heap_blks_vacuumed | 3686381
index_vacuum_count | 2
max_dead_tuples | 178956969
num_dead_tuples | 142086544
indrelid | 0 <<-----
index_blks_total | 0 <<-----
index_blks_scanned | 0 <<-----
leader_pid | <<-----
-[ RECORD 2 ]------+------------------
pid | 1543
datid | 13732
datname | postgres
relid | 16638
phase | vacuuming indexes
heap_blks_total | 0
heap_blks_scanned | 0
heap_blks_vacuumed | 0
index_vacuum_count | 0
max_dead_tuples | 0
num_dead_tuples | 0
indrelid | 16646
index_blks_total | 3030305
index_blks_scanned | 2356564
leader_pid | 18180
-[ RECORD 3 ]------+------------------
pid | 1544
datid | 13732
datname | postgres
relid | 16638
phase | vacuuming indexes
heap_blks_total | 0
heap_blks_scanned | 0
heap_blks_vacuumed | 0
index_vacuum_count | 0
max_dead_tuples | 0
num_dead_tuples | 0
indrelid | 16651
index_blks_total | 2685921
index_blks_scanned | 2119179
leader_pid | 18180

Regards,

Sami Imseih
Database Engineer @ Amazon Web Services

Attachments:

patch.txttext/plain; name=patch.txtDownload
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index ccc9fa0959..a4025e376c 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -39,6 +39,8 @@
 #include "utils/index_selfuncs.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 
 /*
@@ -77,7 +79,7 @@ static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRang
 static void form_and_insert_tuple(BrinBuildState *state);
 static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
 						 BrinTuple *b);
-static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
+static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, bool report_progress);
 static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
 								BrinMemTuple *dtup, Datum *values, bool *nulls);
 static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
@@ -952,7 +954,7 @@ brinvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 	heapRel = table_open(IndexGetRelation(RelationGetRelid(info->index), false),
 						 AccessShareLock);
 
-	brin_vacuum_scan(info->index, info->strategy);
+	brin_vacuum_scan(info->index, info->strategy, info->report_progress);
 
 	brinsummarize(info->index, heapRel, BRIN_ALL_BLOCKRANGES, false,
 				  &stats->num_index_tuples, &stats->num_index_tuples);
@@ -1634,7 +1636,7 @@ union_tuples(BrinDesc *bdesc, BrinMemTuple *a, BrinTuple *b)
  * and such.
  */
 static void
-brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
+brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, bool report_progress)
 {
 	BlockNumber nblocks;
 	BlockNumber blkno;
@@ -1644,6 +1646,9 @@ brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
 	 * each page.
 	 */
 	nblocks = RelationGetNumberOfBlocks(idxrel);
+	if (report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 nblocks);
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
 		Buffer		buf;
@@ -1655,9 +1660,21 @@ brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
 
 		brin_page_cleanup(idxrel, buf);
 
+		if (report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blkno + 1);
+
 		ReleaseBuffer(buf);
 	}
 
+	if (report_progress)
+	{
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									 0);
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 0);
+	}
+
 	/*
 	 * Update all upper pages in the index's FSM, as well.  This ensures not
 	 * only that we propagate leaf-page FSM updates made by brin_page_cleanup,
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index a276eb020b..0171454999 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -24,6 +24,8 @@
 #include "storage/lmgr.h"
 #include "storage/predicate.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 struct GinVacuumState
 {
@@ -571,6 +573,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	Buffer		buffer;
 	BlockNumber rootOfPostingTree[BLCKSZ / (sizeof(IndexTupleData) + sizeof(ItemId))];
 	uint32		nRoot;
+	BlockNumber	num_pages;
+	bool		needLock;
+	int		blocks_scanned = 0;
 
 	gvs.tmpCxt = AllocSetContextCreate(CurrentMemoryContext,
 									   "Gin vacuum temporary context",
@@ -635,6 +640,19 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 									RBM_NORMAL, info->strategy);
 	}
 
+	needLock = !RELATION_IS_LOCAL(index);
+
+	/* Get the current relation length */
+	if (needLock)
+		LockRelationForExtension(index, ExclusiveLock);
+	num_pages = RelationGetNumberOfBlocks(index);
+	if (needLock)
+		UnlockRelationForExtension(index, ExclusiveLock);
+
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 num_pages);
+
 	/* right now we found leftmost page in entry's BTree */
 
 	for (;;)
@@ -676,9 +694,21 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
+		blocks_scanned++;
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
 		LockBuffer(buffer, GIN_EXCLUSIVE);
 	}
 
+	if (info->report_progress)
+	{
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									 0);
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 0);
+	}
+
 	MemoryContextDelete(gvs.tmpCxt);
 
 	return gvs.result;
@@ -694,6 +724,7 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 	BlockNumber totFreePages;
 	GinState	ginstate;
 	GinStatsData idxStat;
+	int         blocks_scanned = 0;
 
 	/*
 	 * In an autovacuum analyze, we want to clean up pending insertions.
@@ -744,6 +775,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 	totFreePages = 0;
 
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 npages);
 	for (blkno = GIN_ROOT_BLKNO; blkno < npages; blkno++)
 	{
 		Buffer		buffer;
@@ -774,9 +808,22 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 				idxStat.nEntries += PageGetMaxOffsetNumber(page);
 		}
 
+		blocks_scanned++;
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
+
 		UnlockReleaseBuffer(buffer);
 	}
 
+	if (info->report_progress)
+	{
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									 0);
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 0);
+	}
+
 	/* Update the metapage with accurate page and entry counts */
 	idxStat.nTotalPages = npages;
 	ginUpdateStats(info->index, &idxStat, false);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0663193531..3be8bb4282 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -23,6 +23,8 @@
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 /* Working state needed by gistbulkdelete */
 typedef struct
@@ -215,9 +217,27 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 		/* Quit if we've scanned the whole relation */
 		if (blkno >= num_pages)
 			break;
+
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+										 num_pages);
+
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_progress)
+				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+											 blkno + 1);
+		}
+	}
+
+	if (info->report_progress)
+	{
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									 0);
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 0);
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index eb3810494f..851f33fd97 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -31,6 +31,7 @@
 #include "utils/builtins.h"
 #include "utils/index_selfuncs.h"
 #include "utils/rel.h"
+#include "storage/lmgr.h"
 
 /* Working state for hashbuild and its callback */
 typedef struct
@@ -468,9 +469,15 @@ hashbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	Buffer		metabuf = InvalidBuffer;
 	HashMetaPage metap;
 	HashMetaPage cachedmetap;
+	int         blocks_scanned;
+	int         bucket_blocks_scanned;
+	BlockNumber num_pages;
+	bool		needLock;
 
 	tuples_removed = 0;
 	num_index_tuples = 0;
+	blocks_scanned = 0;
+	bucket_blocks_scanned = 0;
 
 	/*
 	 * We need a copy of the metapage so that we can use its hashm_spares[]
@@ -488,6 +495,19 @@ hashbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	cur_bucket = 0;
 	cur_maxbucket = orig_maxbucket;
 
+	needLock = !RELATION_IS_LOCAL(rel);
+
+	/* Get the current relation length */
+	if (needLock)
+		LockRelationForExtension(rel, ExclusiveLock);
+	num_pages = RelationGetNumberOfBlocks(rel);
+	if (needLock)
+		UnlockRelationForExtension(rel, ExclusiveLock);
+
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									num_pages);
+
 loop_top:
 	while (cur_bucket <= cur_maxbucket)
 	{
@@ -503,6 +523,7 @@ loop_top:
 		bucket_blkno = BUCKET_TO_BLKNO(cachedmetap, cur_bucket);
 
 		blkno = bucket_blkno;
+		blocks_scanned++;
 
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
@@ -549,10 +570,14 @@ loop_top:
 						  cachedmetap->hashm_highmask,
 						  cachedmetap->hashm_lowmask, &tuples_removed,
 						  &num_index_tuples, split_cleanup,
-						  callback, callback_state);
+						  callback, callback_state, info->report_progress,
+						  &bucket_blocks_scanned);
 
 		_hash_dropbuf(rel, bucket_buf);
 
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									blocks_scanned + bucket_blocks_scanned);
+
 		/* Advance to next bucket */
 		cur_bucket++;
 	}
@@ -632,6 +657,14 @@ loop_top:
 	stats->tuples_removed += tuples_removed;
 	/* hashvacuumcleanup will fill in num_pages */
 
+	if (info->report_progress)
+	{
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									 0);
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 0);
+	}
+
 	return stats;
 }
 
@@ -685,7 +718,8 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 				  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 				  double *tuples_removed, double *num_index_tuples,
 				  bool split_cleanup,
-				  IndexBulkDeleteCallback callback, void *callback_state)
+				  IndexBulkDeleteCallback callback, void *callback_state,
+				  bool report_progress, int *bucket_blocks_scanned)
 {
 	BlockNumber blkno;
 	Buffer		buf;
@@ -717,6 +751,8 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 		page = BufferGetPage(buf);
 		opaque = (HashPageOpaque) PageGetSpecialPointer(page);
 
+		bucket_blocks_scanned++;
+
 		/* Scan each tuple in page */
 		maxoffno = PageGetMaxOffsetNumber(page);
 		for (offno = FirstOffsetNumber;
@@ -915,4 +951,5 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 							bstrategy);
 	else
 		LockBuffer(bucket_buf, BUFFER_LOCK_UNLOCK);
+
 }
diff --git a/src/backend/access/hash/hashpage.c b/src/backend/access/hash/hashpage.c
index 159646c7c3..538d153df7 100644
--- a/src/backend/access/hash/hashpage.c
+++ b/src/backend/access/hash/hashpage.c
@@ -758,7 +758,7 @@ restart_expand:
 
 		hashbucketcleanup(rel, old_bucket, buf_oblkno, start_oblkno, NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL, NULL);
 
 		_hash_dropbuf(rel, buf_oblkno);
 
@@ -1326,7 +1326,7 @@ _hash_splitbucket(Relation rel,
 		hashbucketcleanup(rel, obucket, bucket_obuf,
 						  BufferGetBlockNumber(bucket_obuf), NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL, NULL);
 	}
 	else
 	{
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index a00947ea1c..28a3797088 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -528,6 +528,9 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID,
+								 MyProcPid);
+
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
 						  params->freeze_table_age,
@@ -3025,12 +3028,17 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	IndexVacuumInfo ivinfo;
 	PGRUsage	ru0;
 	LVSavedErrInfo saved_err_info;
+	const int    initprog_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_CURRENT_INDRELID
+	};
+	int64        initprog_val[2];
 
 	pg_rusage_init(&ru0);
 
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
-	ivinfo.report_progress = false;
+	ivinfo.report_progress = true;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = elevel;
 	ivinfo.num_heap_tuples = reltuples;
@@ -3048,10 +3056,18 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 							 VACUUM_ERRCB_PHASE_VACUUM_INDEX,
 							 InvalidBlockNumber, InvalidOffsetNumber);
 
+	/* Report that we're vacuuming the index, advertising the indrelid */
+	initprog_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	initprog_val[1] = RelationGetRelid(indrel);
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	/* Do bulk deletion */
 	istat = index_bulk_delete(&ivinfo, istat, lazy_tid_reaped,
 							  (void *) vacrel->dead_tuples);
 
+	/* Report that we're done vacuuming the index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID,
+								 0);
 	ereport(elevel,
 			(errmsg("scanned index \"%s\" to remove %d row versions",
 					vacrel->indname, vacrel->dead_tuples->num_tuples),
@@ -3081,12 +3097,17 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	IndexVacuumInfo ivinfo;
 	PGRUsage	ru0;
 	LVSavedErrInfo saved_err_info;
+	const int    initprog_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_CURRENT_INDRELID
+	};
+	int64        initprog_val[2];
 
 	pg_rusage_init(&ru0);
 
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
-	ivinfo.report_progress = false;
+	ivinfo.report_progress = true;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = elevel;
 
@@ -3105,8 +3126,18 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 							 VACUUM_ERRCB_PHASE_INDEX_CLEANUP,
 							 InvalidBlockNumber, InvalidOffsetNumber);
 
+	/* Report that we're cleaning the index, advertising the indrelid */
+	initprog_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	initprog_val[1] = RelationGetRelid(indrel);
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	istat = index_vacuum_cleanup(&ivinfo, istat);
 
+	/* Report that we're done cleaning the index */
+	initprog_val[0] = 0;
+	initprog_val[1] = 0;
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	if (istat)
 	{
 		ereport(elevel,
@@ -4158,6 +4189,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	char	   *sharedquery;
 	LVRelState	vacrel;
 	ErrorContextCallback errcallback;
+	PGPROC     *leader = MyProc->lockGroupLeader;
 
 	/*
 	 * A parallel vacuum worker must have only PROC_IN_VACUUM flag since we
@@ -4186,6 +4218,14 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 */
 	rel = table_open(lvshared->relid, ShareUpdateExclusiveLock);
 
+	/*
+	 * Track progress of current index being vacuumed
+	 */
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
+								  RelationGetRelid(rel));
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, leader->pid);
+
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
 	 * matched to the leader's one.
@@ -4252,6 +4292,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(vacrel.bstrategy);
+	pgstat_progress_end_command();
 	pfree(vacrel.indstats);
 }
 
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 40ad0956e0..e90fd77b86 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -996,10 +996,18 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			btvacuumpage(&vstate, scanblkno);
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
-											 scanblkno);
+											 scanblkno + 1);
 		}
 	}
 
+	if (info->report_progress)
+	{
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									 0);
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 0);
+	}
+
 	/* Set statistics num_pages field to final size of index */
 	stats->num_pages = num_pages;
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 76fb0374c4..19768f11ce 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -27,6 +27,8 @@
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/snapmgr.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 
 /* Entry in pending-list of TIDs we need to revisit */
@@ -797,6 +799,7 @@ spgvacuumscan(spgBulkDeleteState *bds)
 	bool		needLock;
 	BlockNumber num_pages,
 				blkno;
+	int         blocks_scanned = 0;
 
 	/* Finish setting up spgBulkDeleteState */
 	initSpGistState(&bds->spgstate, index);
@@ -836,6 +839,11 @@ spgvacuumscan(spgBulkDeleteState *bds)
 		/* Quit if we've scanned the whole relation */
 		if (blkno >= num_pages)
 			break;
+
+		if (bds->info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+										num_pages);
+
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
 		{
@@ -843,9 +851,22 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+
+			blocks_scanned++;
+			if (bds->info->report_progress)
+				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
 		}
 	}
 
+	if (bds->info->report_progress)
+	{
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									 0);
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 0);
+	}
+
 	/* Propagate local lastUsedPages cache to metablock */
 	SpGistUpdateMetaPage(index);
 
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index eb560955cd..953d238925 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1124,7 +1124,10 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indrelid, S.param16 index_blks_total,
+        S.param17 AS index_blks_scanned,
+        CASE WHEN S.param9 = S.pid THEN NULL ELSE S.param9 END AS leader_pid
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/include/access/hash.h b/src/include/access/hash.h
index 1cce865be2..c00bb76e3e 100644
--- a/src/include/access/hash.h
+++ b/src/include/access/hash.h
@@ -478,6 +478,7 @@ extern void hashbucketcleanup(Relation rel, Bucket cur_bucket,
 							  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 							  double *tuples_removed, double *num_index_tuples,
 							  bool split_cleanup,
-							  IndexBulkDeleteCallback callback, void *callback_state);
+							  IndexBulkDeleteCallback callback, void *callback_state,
+							  bool report_progress, int *bucket_blocks_scanned);
 
 #endif							/* HASH_H */
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index d7bf16368b..4387a7c1f1 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_CURRENT_INDRELID        7
+#define PROGRESS_VACUUM_LEADER_PID              8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
#2Bossart, Nathan
bossartn@amazon.com
In reply to: Imseih (AWS), Sami (#1)
Re: Add index scan progress to pg_stat_progress_vacuum

On 12/1/21, 3:02 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

The current implementation of pg_stat_progress_vacuum does not
provide progress on which index is being vacuumed making it
difficult for a user to determine if the "vacuuming indexes" phase
is making progress. By exposing which index is being scanned as well
as the total progress the scan has made for the current cycle, a
user can make better estimations on when the vacuum will complete.

+1

The proposed patch adds 4 new columns to pg_stat_progress_vacuum:

1. indrelid - the relid of the index being vacuumed
2. index_blks_total - total number of blocks to be scanned in the
current cycle
3. index_blks_scanned - number of blocks scanned in the current
cycle
4. leader_pid - if the pid for the pg_stat_progress_vacuum entry is
a leader or a vacuum worker. This patch places an entry for every
worker pid ( if parallel ) as well as the leader pid

nitpick: Shouldn't index_blks_scanned be index_blks_vacuumed? IMO it
is more analogous to heap_blks_vacuumed.

This will tell us which indexes are currently being vacuumed and the
current progress of those operations, but it doesn't tell us which
indexes have already been vacuumed or which ones are pending vacuum.
I think such information is necessary to truly understand the current
progress of vacuuming indexes, and I can think of a couple of ways we
might provide it:

1. Make the new columns you've proposed return arrays. This isn't
very clean, but it would keep all the information for a given
vacuum operation in a single row. The indrelids column would be
populated with all the indexes that have been vacuumed, need to
be vacuumed, or are presently being vacuumed. The other index-
related columns would then have the associated stats and the
worker PID (which might be the same as the pid column depending
on whether parallel index vacuum was being done). Alternatively,
the index column could have an array of records, each containing
all the information for a given index.
2. Create a new view for just index vacuum progress information.
This would have similar information as 1. There would be an
entry for each index that has been vacuumed, needs to be
vacuumed, or is currently being vacuumed. And there would be an
easy way to join with pg_stat_progress_vacuum (e.g., leader_pid,
which again might be the same as our index vacuum PID depending
on whether we were doing parallel index vacuum). Note that it
would be possible for the PID of these entries to be null before
and after we process the index.
3. Instead of adding columns to pg_stat_progress_vacuum, adjust the
current ones to be more general, and then add new entries for
each of the indexes that have been, need to be, or currently are
being vacuumed. This is the most similar option to your current
proposal, but instead of introducing a column like
index_blks_total, we'd rename heap_blks_total to blks_total and
use that for both the heap and indexes. I think we'd still want
to add a leader_pid column. Again, we have to be prepared for
the PID to be null in this case. Or we could just make the pid
column always refer to the leader, and we could introduce a
worker_pid column. That might create confusion, though.

I wish option #1 was cleaner, because I think it would be really nice
to have all this information in a single row. However, I don't expect
much support for a 3-dimensional view, so I suspect option #2
(creating a separate view for index vacuum progress) is the way to go.
The other benefit of option #2 versus option #3 or your original
proposal is that it cleanly separates the top-level vacuum operations
and the index vacuum operations, which are related at the moment, but
which might not always be tied so closely together.

Nathan

#3Imseih (AWS), Sami
simseih@amazon.com
In reply to: Bossart, Nathan (#2)
Re: Add index scan progress to pg_stat_progress_vacuum

On 12/15/21, 4:10 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 12/1/21, 3:02 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

The current implementation of pg_stat_progress_vacuum does not
provide progress on which index is being vacuumed making it
difficult for a user to determine if the "vacuuming indexes" phase
is making progress. By exposing which index is being scanned as well
as the total progress the scan has made for the current cycle, a
user can make better estimations on when the vacuum will complete.

+1

The proposed patch adds 4 new columns to pg_stat_progress_vacuum:

1. indrelid - the relid of the index being vacuumed
2. index_blks_total - total number of blocks to be scanned in the
current cycle
3. index_blks_scanned - number of blocks scanned in the current
cycle
4. leader_pid - if the pid for the pg_stat_progress_vacuum entry is
a leader or a vacuum worker. This patch places an entry for every
worker pid ( if parallel ) as well as the leader pid

nitpick: Shouldn't index_blks_scanned be index_blks_vacuumed? IMO it
is more analogous to heap_blks_vacuumed.

No, What is being tracked is the number of index blocks scanned from the total index blocks. The block will be scanned regardless if it will be vacuumed or not.

This will tell us which indexes are currently being vacuumed and the
current progress of those operations, but it doesn't tell us which
indexes have already been vacuumed or which ones are pending vacuum.
I think such information is necessary to truly understand the current
progress of vacuuming indexes, and I can think of a couple of ways we
might provide it:

1. Make the new columns you've proposed return arrays. This isn't
very clean, but it would keep all the information for a given
vacuum operation in a single row. The indrelids column would be
populated with all the indexes that have been vacuumed, need to
be vacuumed, or are presently being vacuumed. The other index-
related columns would then have the associated stats and the
worker PID (which might be the same as the pid column depending
on whether parallel index vacuum was being done). Alternatively,
the index column could have an array of records, each containing
all the information for a given index.
2. Create a new view for just index vacuum progress information.
This would have similar information as 1. There would be an
entry for each index that has been vacuumed, needs to be
vacuumed, or is currently being vacuumed. And there would be an
easy way to join with pg_stat_progress_vacuum (e.g., leader_pid,
which again might be the same as our index vacuum PID depending
on whether we were doing parallel index vacuum). Note that it
would be possible for the PID of these entries to be null before
and after we process the index.
3. Instead of adding columns to pg_stat_progress_vacuum, adjust the
current ones to be more general, and then add new entries for
each of the indexes that have been, need to be, or currently are
being vacuumed. This is the most similar option to your current
proposal, but instead of introducing a column like
index_blks_total, we'd rename heap_blks_total to blks_total and
use that for both the heap and indexes. I think we'd still want
to add a leader_pid column. Again, we have to be prepared for
the PID to be null in this case. Or we could just make the pid
column always refer to the leader, and we could introduce a
worker_pid column. That might create confusion, though.

I wish option #1 was cleaner, because I think it would be really nice
to have all this information in a single row. However, I don't expect
much support for a 3-dimensional view, so I suspect option #2
(creating a separate view for index vacuum progress) is the way to go.
The other benefit of option #2 versus option #3 or your original
proposal is that it cleanly separates the top-level vacuum operations
and the index vacuum operations, which are related at the moment, but
which might not always be tied so closely together.

Option #1 is not clean as you will need to unnest the array to make sense out of it. It will be too complex to use.
Option #3 I am reluctant to spent time looking at this option. It's more valuable to see progress per index instead of total.
Option #2 was one that I originally designed but backed away as it was introducing a new view. Thinking about it a bit more, this is a cleaner approach.
1. Having a view called pg_stat_progress_vacuum_worker to join with pg_stat_progress_vacuum is clean
2. No changes required to pg_stat_progress_vacuum
3. I’ll lean towards calling the view " pg_stat_progress_vacuum_worker" instead of " pg_stat_progress_vacuum_index", to perhaps allow us to track other items a vacuum worker may do in future releases. As of now, only indexes are vacuumed by workers.
I will rework the patch for option #2

Nathan

#4Greg Stark
stark@mit.edu
In reply to: Imseih (AWS), Sami (#3)
Re: Add index scan progress to pg_stat_progress_vacuum

I had a similar question. And I'm still not clear from the response
what exactly index_blks_total is and whether it addresses it.

I think I agree that a user is likely to want to see the progress in a
way they can understand which means for a single index at a time.

I think what you're describing is that index_blks_total and
index_blks_scanned are the totals across all the indexes? That isn't
clear from the definitions but if that's what you intend then I think
that would work.

(For what it's worth what I was imagining was having a pair of
counters for blocks scanned and max blocks in this index and a second
counter for number of indexes processed and max number of indexes. But
I don't think that's necessarily any better than what you have)

#5Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#3)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Here is a V2 attempt of the patch to include a new view called pg_stat_progress_vacuum_worker. Also, scans for index cleanups will also have an entry in the new view.

- here is the new view which reports an entry for every worker ( or leader ) that is doing index vacuum/index cleanup work.
postgres=# select * from pg_stat_progress_vacuum_worker ;
-[ RECORD 1 ]------+------
pid | 29355
leader_pid | 26501
indrelid | 16391
index_blks_total | 68894
index_blks_scanned | 35618

- the view can be joined with pg_stat_progress_vacuum. Sample output below

postgres=# select a.*, b.phase, b.heap_blks_total, b.heap_blks_scanned from pg_stat_progress_vacuum_worker a full outer join pg_stat_progress_vacuum b on a.pid = b.pid ;
pid | leader_pid | indrelid | index_blks_total | index_blks_scanned | phase | heap_blks_total | heap_blks_scanned
-------+------------+----------+------------------+--------------------+---------------------+-----------------+-------------------
26667 | 26667 | 16391 | 9165 | 401 | cleaning up indexes | 20082 | 20082
(1 row)

postgres=# select a.*, b.phase, b.heap_blks_total, b.heap_blks_scanned from pg_stat_progress_vacuum_worker a full outer join pg_stat_progress_vacuum b on a.pid = b.pid ;
-[ RECORD 1 ]------+------------------
pid | 26501
leader_pid | 26501
indrelid | 16393
index_blks_total | 145107
index_blks_scanned | 11060
phase | vacuuming indexes
heap_blks_total | 165375
heap_blks_scanned | 165375
-[ RECORD 2 ]------+------------------
pid | 28982
leader_pid | 26501
indrelid | 16392
index_blks_total | 47616
index_blks_scanned | 11861
phase | vacuuming indexes
heap_blks_total | 0
heap_blks_scanned | 0
-[ RECORD 3 ]------+------------------
pid | 28983
leader_pid | 26501
indrelid | 16391
index_blks_total | 56936
index_blks_scanned | 9138
phase | vacuuming indexes
heap_blks_total | 0
heap_blks_scanned | 0

On 12/15/21, 4:10 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 12/1/21, 3:02 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

The current implementation of pg_stat_progress_vacuum does not
provide progress on which index is being vacuumed making it
difficult for a user to determine if the "vacuuming indexes" phase
is making progress. By exposing which index is being scanned as well
as the total progress the scan has made for the current cycle, a
user can make better estimations on when the vacuum will complete.

+1

The proposed patch adds 4 new columns to pg_stat_progress_vacuum:

1. indrelid - the relid of the index being vacuumed
2. index_blks_total - total number of blocks to be scanned in the
current cycle
3. index_blks_scanned - number of blocks scanned in the current
cycle
4. leader_pid - if the pid for the pg_stat_progress_vacuum entry is
a leader or a vacuum worker. This patch places an entry for every
worker pid ( if parallel ) as well as the leader pid

nitpick: Shouldn't index_blks_scanned be index_blks_vacuumed? IMO it
is more analogous to heap_blks_vacuumed.

No, What is being tracked is the number of index blocks scanned from the total index blocks. The block will be scanned regardless if it will be vacuumed or not.

This will tell us which indexes are currently being vacuumed and the
current progress of those operations, but it doesn't tell us which
indexes have already been vacuumed or which ones are pending vacuum.
I think such information is necessary to truly understand the current
progress of vacuuming indexes, and I can think of a couple of ways we
might provide it:

1. Make the new columns you've proposed return arrays. This isn't
very clean, but it would keep all the information for a given
vacuum operation in a single row. The indrelids column would be
populated with all the indexes that have been vacuumed, need to
be vacuumed, or are presently being vacuumed. The other index-
related columns would then have the associated stats and the
worker PID (which might be the same as the pid column depending
on whether parallel index vacuum was being done). Alternatively,
the index column could have an array of records, each containing
all the information for a given index.
2. Create a new view for just index vacuum progress information.
This would have similar information as 1. There would be an
entry for each index that has been vacuumed, needs to be
vacuumed, or is currently being vacuumed. And there would be an
easy way to join with pg_stat_progress_vacuum (e.g., leader_pid,
which again might be the same as our index vacuum PID depending
on whether we were doing parallel index vacuum). Note that it
would be possible for the PID of these entries to be null before
and after we process the index.
3. Instead of adding columns to pg_stat_progress_vacuum, adjust the
current ones to be more general, and then add new entries for
each of the indexes that have been, need to be, or currently are
being vacuumed. This is the most similar option to your current
proposal, but instead of introducing a column like
index_blks_total, we'd rename heap_blks_total to blks_total and
use that for both the heap and indexes. I think we'd still want
to add a leader_pid column. Again, we have to be prepared for
the PID to be null in this case. Or we could just make the pid
column always refer to the leader, and we could introduce a
worker_pid column. That might create confusion, though.

I wish option #1 was cleaner, because I think it would be really nice
to have all this information in a single row. However, I don't expect
much support for a 3-dimensional view, so I suspect option #2
(creating a separate view for index vacuum progress) is the way to go.
The other benefit of option #2 versus option #3 or your original
proposal is that it cleanly separates the top-level vacuum operations
and the index vacuum operations, which are related at the moment, but
which might not always be tied so closely together.

Option #1 is not clean as you will need to unnest the array to make sense out of it. It will be too complex to use.
Option #3 I am reluctant to spent time looking at this option. It's more valuable to see progress per index instead of total.
Option #2 was one that I originally designed but backed away as it was introducing a new view. Thinking about it a bit more, this is a cleaner approach.
1. Having a view called pg_stat_progress_vacuum_worker to join with pg_stat_progress_vacuum is clean
2. No changes required to pg_stat_progress_vacuum
3. I’ll lean towards calling the view " pg_stat_progress_vacuum_worker" instead of " pg_stat_progress_vacuum_index", to perhaps allow us to track other items a vacuum worker may do in future releases. As of now, only indexes are vacuumed by workers.
I will rework the patch for option #2

Nathan

Attachments:

patch.v2.txttext/plain; name=patch.v2.txtDownload
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index f521bb9635..97b2f8bc13 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -39,6 +39,8 @@
 #include "utils/index_selfuncs.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 
 /*
@@ -77,7 +79,7 @@ static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRang
 static void form_and_insert_tuple(BrinBuildState *state);
 static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
 						 BrinTuple *b);
-static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
+static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, bool report_progress);
 static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
 								BrinMemTuple *dtup, Datum *values, bool *nulls);
 static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
@@ -953,7 +955,7 @@ brinvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 	heapRel = table_open(IndexGetRelation(RelationGetRelid(info->index), false),
 						 AccessShareLock);
 
-	brin_vacuum_scan(info->index, info->strategy);
+	brin_vacuum_scan(info->index, info->strategy, info->report_progress);
 
 	brinsummarize(info->index, heapRel, BRIN_ALL_BLOCKRANGES, false,
 				  &stats->num_index_tuples, &stats->num_index_tuples);
@@ -1635,16 +1637,24 @@ union_tuples(BrinDesc *bdesc, BrinMemTuple *a, BrinTuple *b)
  * and such.
  */
 static void
-brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
+brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, bool report_progress)
 {
 	BlockNumber nblocks;
 	BlockNumber blkno;
+	const int    initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/*
 	 * Scan the index in physical order, and clean up any possible mess in
 	 * each page.
 	 */
 	nblocks = RelationGetNumberOfBlocks(idxrel);
+	if (report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 nblocks);
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
 		Buffer		buf;
@@ -1656,9 +1666,20 @@ brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
 
 		brin_page_cleanup(idxrel, buf);
 
+		if (report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blkno + 1);
+
 		ReleaseBuffer(buf);
 	}
 
+	if (report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	/*
 	 * Update all upper pages in the index's FSM, as well.  This ensures not
 	 * only that we propagate leaf-page FSM updates made by brin_page_cleanup,
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index a276eb020b..714586040a 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -24,6 +24,8 @@
 #include "storage/lmgr.h"
 #include "storage/predicate.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 struct GinVacuumState
 {
@@ -571,6 +573,14 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	Buffer		buffer;
 	BlockNumber rootOfPostingTree[BLCKSZ / (sizeof(IndexTupleData) + sizeof(ItemId))];
 	uint32		nRoot;
+	BlockNumber	num_pages;
+	bool		needLock;
+	int		blocks_scanned = 0;
+	const int	initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	gvs.tmpCxt = AllocSetContextCreate(CurrentMemoryContext,
 									   "Gin vacuum temporary context",
@@ -635,6 +645,19 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 									RBM_NORMAL, info->strategy);
 	}
 
+	needLock = !RELATION_IS_LOCAL(index);
+
+	/* Get the current relation length */
+	if (needLock)
+		LockRelationForExtension(index, ExclusiveLock);
+	num_pages = RelationGetNumberOfBlocks(index);
+	if (needLock)
+		UnlockRelationForExtension(index, ExclusiveLock);
+
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 num_pages);
+
 	/* right now we found leftmost page in entry's BTree */
 
 	for (;;)
@@ -676,9 +699,20 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
+		blocks_scanned++;
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
 		LockBuffer(buffer, GIN_EXCLUSIVE);
 	}
 
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	MemoryContextDelete(gvs.tmpCxt);
 
 	return gvs.result;
@@ -694,6 +728,12 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 	BlockNumber totFreePages;
 	GinState	ginstate;
 	GinStatsData idxStat;
+	int         blocks_scanned = 0;
+	const int	initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/*
 	 * In an autovacuum analyze, we want to clean up pending insertions.
@@ -744,6 +784,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 	totFreePages = 0;
 
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 npages);
 	for (blkno = GIN_ROOT_BLKNO; blkno < npages; blkno++)
 	{
 		Buffer		buffer;
@@ -774,9 +817,21 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 				idxStat.nEntries += PageGetMaxOffsetNumber(page);
 		}
 
+		blocks_scanned++;
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
+
 		UnlockReleaseBuffer(buffer);
 	}
 
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	/* Update the metapage with accurate page and entry counts */
 	idxStat.nTotalPages = npages;
 	ginUpdateStats(info->index, &idxStat, false);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0663193531..e7d13c9eb6 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -23,6 +23,8 @@
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 /* Working state needed by gistbulkdelete */
 typedef struct
@@ -131,6 +133,11 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	bool		needLock;
 	BlockNumber blkno;
 	MemoryContext oldctx;
+	const int	initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/*
 	 * Reset fields that track information about the entire index now.  This
@@ -215,9 +222,26 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 		/* Quit if we've scanned the whole relation */
 		if (blkno >= num_pages)
 			break;
+
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+										 num_pages);
+
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_progress)
+				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+											 blkno + 1);
+		}
+	}
+
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 81c7da7ec6..2b4eed6aae 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -31,6 +31,7 @@
 #include "utils/builtins.h"
 #include "utils/index_selfuncs.h"
 #include "utils/rel.h"
+#include "storage/lmgr.h"
 
 /* Working state for hashbuild and its callback */
 typedef struct
@@ -469,9 +470,21 @@ hashbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	Buffer		metabuf = InvalidBuffer;
 	HashMetaPage metap;
 	HashMetaPage cachedmetap;
+	int         blocks_scanned;
+	int         bucket_blocks_scanned;
+	BlockNumber num_pages;
+	bool		needLock;
+	const int initprog_index[] = {
+		PROGRESS_SCAN_BLOCKS_DONE,
+		PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
+
 
 	tuples_removed = 0;
 	num_index_tuples = 0;
+	blocks_scanned = 0;
+	bucket_blocks_scanned = 0;
 
 	/*
 	 * We need a copy of the metapage so that we can use its hashm_spares[]
@@ -489,6 +502,19 @@ hashbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	cur_bucket = 0;
 	cur_maxbucket = orig_maxbucket;
 
+	needLock = !RELATION_IS_LOCAL(rel);
+
+	/* Get the current relation length */
+	if (needLock)
+		LockRelationForExtension(rel, ExclusiveLock);
+	num_pages = RelationGetNumberOfBlocks(rel);
+	if (needLock)
+		UnlockRelationForExtension(rel, ExclusiveLock);
+
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									num_pages);
+
 loop_top:
 	while (cur_bucket <= cur_maxbucket)
 	{
@@ -504,6 +530,7 @@ loop_top:
 		bucket_blkno = BUCKET_TO_BLKNO(cachedmetap, cur_bucket);
 
 		blkno = bucket_blkno;
+		blocks_scanned++;
 
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
@@ -550,10 +577,14 @@ loop_top:
 						  cachedmetap->hashm_highmask,
 						  cachedmetap->hashm_lowmask, &tuples_removed,
 						  &num_index_tuples, split_cleanup,
-						  callback, callback_state);
+						  callback, callback_state, info->report_progress,
+						  &bucket_blocks_scanned);
 
 		_hash_dropbuf(rel, bucket_buf);
 
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									blocks_scanned + bucket_blocks_scanned);
+
 		/* Advance to next bucket */
 		cur_bucket++;
 	}
@@ -633,6 +664,13 @@ loop_top:
 	stats->tuples_removed += tuples_removed;
 	/* hashvacuumcleanup will fill in num_pages */
 
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	return stats;
 }
 
@@ -686,7 +724,8 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 				  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 				  double *tuples_removed, double *num_index_tuples,
 				  bool split_cleanup,
-				  IndexBulkDeleteCallback callback, void *callback_state)
+				  IndexBulkDeleteCallback callback, void *callback_state,
+				  bool report_progress, int *bucket_blocks_scanned)
 {
 	BlockNumber blkno;
 	Buffer		buf;
@@ -718,6 +757,8 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 		page = BufferGetPage(buf);
 		opaque = (HashPageOpaque) PageGetSpecialPointer(page);
 
+		bucket_blocks_scanned++;
+
 		/* Scan each tuple in page */
 		maxoffno = PageGetMaxOffsetNumber(page);
 		for (offno = FirstOffsetNumber;
@@ -916,4 +957,5 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 							bstrategy);
 	else
 		LockBuffer(bucket_buf, BUFFER_LOCK_UNLOCK);
+
 }
diff --git a/src/backend/access/hash/hashpage.c b/src/backend/access/hash/hashpage.c
index 159646c7c3..538d153df7 100644
--- a/src/backend/access/hash/hashpage.c
+++ b/src/backend/access/hash/hashpage.c
@@ -758,7 +758,7 @@ restart_expand:
 
 		hashbucketcleanup(rel, old_bucket, buf_oblkno, start_oblkno, NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL, NULL);
 
 		_hash_dropbuf(rel, buf_oblkno);
 
@@ -1326,7 +1326,7 @@ _hash_splitbucket(Relation rel,
 		hashbucketcleanup(rel, obucket, bucket_obuf,
 						  BufferGetBlockNumber(bucket_obuf), NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL, NULL);
 	}
 	else
 	{
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index db6becfed5..ca26d5344e 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -517,6 +517,9 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID,
+								 MyProcPid);
+
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
 						  params->freeze_table_age,
@@ -3021,12 +3024,17 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	IndexVacuumInfo ivinfo;
 	PGRUsage	ru0;
 	LVSavedErrInfo saved_err_info;
+	const int    initprog_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_CURRENT_INDRELID
+	};
+	int64        initprog_val[2];
 
 	pg_rusage_init(&ru0);
 
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
-	ivinfo.report_progress = false;
+	ivinfo.report_progress = true;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = elevel;
 	ivinfo.num_heap_tuples = reltuples;
@@ -3044,10 +3052,19 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 							 VACUUM_ERRCB_PHASE_VACUUM_INDEX,
 							 InvalidBlockNumber, InvalidOffsetNumber);
 
+	/* Report that we're vacuuming the index, advertising the indrelid */
+	initprog_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	initprog_val[1] = RelationGetRelid(indrel);
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	/* Do bulk deletion */
 	istat = index_bulk_delete(&ivinfo, istat, lazy_tid_reaped,
 							  (void *) vacrel->dead_items);
 
+	/* Report that we're done vacuuming the index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID,
+								 0);
+
 	ereport(elevel,
 			(errmsg("scanned index \"%s\" to remove %d row versions",
 					vacrel->indname, vacrel->dead_items->num_items),
@@ -3078,12 +3095,17 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	IndexVacuumInfo ivinfo;
 	PGRUsage	ru0;
 	LVSavedErrInfo saved_err_info;
+	const int    initprog_index[] = {
+			PROGRESS_VACUUM_PHASE,
+			PROGRESS_VACUUM_CURRENT_INDRELID
+	};
+	int64        initprog_val[2];
 
 	pg_rusage_init(&ru0);
 
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
-	ivinfo.report_progress = false;
+	ivinfo.report_progress = true;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = elevel;
 
@@ -3102,8 +3124,18 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 							 VACUUM_ERRCB_PHASE_INDEX_CLEANUP,
 							 InvalidBlockNumber, InvalidOffsetNumber);
 
+	/* Report that we're cleaning the index, advertising the indrelid */
+	initprog_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	initprog_val[1] = RelationGetRelid(indrel);
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	istat = index_vacuum_cleanup(&ivinfo, istat);
 
+	/* Report that we're done cleaning the index */
+	initprog_val[0] = 0;
+	initprog_val[1] = 0;
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	if (istat)
 	{
 		ereport(elevel,
@@ -4145,6 +4177,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	char	   *sharedquery;
 	LVRelState	vacrel;
 	ErrorContextCallback errcallback;
+	PGPROC     *leader = MyProc->lockGroupLeader;
 
 	/*
 	 * A parallel vacuum worker must have only PROC_IN_VACUUM flag since we
@@ -4170,6 +4203,14 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 */
 	rel = table_open(lvshared->relid, ShareUpdateExclusiveLock);
 
+	/*
+	 *Track progress of current index being vacuumed
+	 */
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
+								  RelationGetRelid(rel));
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, leader->pid);
+
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
 	 * matched to the leader's one.
@@ -4241,6 +4282,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(vacrel.bstrategy);
+	pgstat_progress_end_command();
 	pfree(vacrel.indstats);
 }
 
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index dfce06dc49..15c11fcab8 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -909,6 +909,11 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	BlockNumber num_pages;
 	BlockNumber scanblkno;
 	bool		needLock;
+	const int       initprog_index[] = {
+		PROGRESS_SCAN_BLOCKS_DONE,
+		PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/*
 	 * Reset fields that track information about the entire index now.  This
@@ -997,10 +1002,17 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			btvacuumpage(&vstate, scanblkno);
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
-											 scanblkno);
+											 scanblkno + 1);
 		}
 	}
 
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	/* Set statistics num_pages field to final size of index */
 	stats->num_pages = num_pages;
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 76fb0374c4..c71345fceb 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -27,6 +27,8 @@
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/snapmgr.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 
 /* Entry in pending-list of TIDs we need to revisit */
@@ -797,6 +799,12 @@ spgvacuumscan(spgBulkDeleteState *bds)
 	bool		needLock;
 	BlockNumber num_pages,
 				blkno;
+	int         blocks_scanned = 0;
+	const int	initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/* Finish setting up spgBulkDeleteState */
 	initSpGistState(&bds->spgstate, index);
@@ -836,6 +844,11 @@ spgvacuumscan(spgBulkDeleteState *bds)
 		/* Quit if we've scanned the whole relation */
 		if (blkno >= num_pages)
 			break;
+
+		if (bds->info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+										num_pages);
+
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
 		{
@@ -843,9 +856,21 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+
+			blocks_scanned++;
+			if (bds->info->report_progress)
+				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
 		}
 	}
 
+	if (bds->info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	/* Propagate local lastUsedPages cache to metablock */
 	SpGistUpdateMetaPage(index);
 
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 61b515cdb8..9e0dc39314 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1128,6 +1128,17 @@ CREATE VIEW pg_stat_progress_vacuum AS
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_worker AS
+	SELECT
+		S.pid,
+		S.param9 leader_pid,
+		S.param8 AS indrelid,
+		S.param16 index_blks_total,
+		S.param17 AS index_blks_scanned
+	FROM pg_stat_get_progress_info('VACUUM') AS S
+		LEFT JOIN pg_database D ON S.datid = D.oid
+	WHERE S.param8 > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/include/access/hash.h b/src/include/access/hash.h
index 1cce865be2..c00bb76e3e 100644
--- a/src/include/access/hash.h
+++ b/src/include/access/hash.h
@@ -478,6 +478,7 @@ extern void hashbucketcleanup(Relation rel, Bucket cur_bucket,
 							  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 							  double *tuples_removed, double *num_index_tuples,
 							  bool split_cleanup,
-							  IndexBulkDeleteCallback callback, void *callback_state);
+							  IndexBulkDeleteCallback callback, void *callback_state,
+							  bool report_progress, int *bucket_blocks_scanned);
 
 #endif							/* HASH_H */
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index d7bf16368b..4387a7c1f1 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_CURRENT_INDRELID        7
+#define PROGRESS_VACUUM_LEADER_PID              8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
In reply to: Bossart, Nathan (#2)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Dec 15, 2021 at 2:10 PM Bossart, Nathan <bossartn@amazon.com> wrote:

nitpick: Shouldn't index_blks_scanned be index_blks_vacuumed? IMO it
is more analogous to heap_blks_vacuumed.

+1.

This will tell us which indexes are currently being vacuumed and the
current progress of those operations, but it doesn't tell us which
indexes have already been vacuumed or which ones are pending vacuum.

VACUUM will process a table's indexes in pg_class OID order (outside
of parallel VACUUM, I suppose). See comments about sort order above
RelationGetIndexList().

Anyway, it might be useful to add ordinal numbers to each index, that
line up with this processing/OID order. It would also be reasonable to
display the same number in log_autovacuum* (and VACUUM VERBOSE)
per-index output, to reinforce the idea. Note that we don't
necessarily display a distinct line for each distinct index in this
log output, which is why including the ordinal number there makes
sense.

I wish option #1 was cleaner, because I think it would be really nice
to have all this information in a single row.

I do too. I agree with the specific points you raise in your remarks
about what you've called options #2 and #3, but those options still
seem unappealing to me.

--
Peter Geoghegan

In reply to: Imseih (AWS), Sami (#1)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Dec 1, 2021 at 2:59 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

The current implementation of pg_stat_progress_vacuum does not provide progress on which index is being vacuumed making it difficult for a user to determine if the "vacuuming indexes" phase is making progress.

I notice that your patch largely assumes that indexes can be treated
like heap relations, in the sense that they're scanned sequentially,
and process each block exactly once (or exactly once per "pass"). But
that isn't quite true. There are a few differences that seem like they
might matter:

* An ambulkdelete() scan of an index cannot take the size of the
relation once, at the start, and ignore any blocks that are added
after the scan begins. And so the code may need to re-establish the
total size of the index multiple times, to make sure no index tuples
are missed -- there may be index tuples that VACUUM needs to process
that appear in later pages due to concurrent page splits. You don't
have the issue with things like IndexBulkDeleteResult.num_pages,
because they report on the index after ambulkdelete/amvacuumcleanup
return (they're not granular progress indicators).

* Some index AMs don't work like nbtree and GiST in that they cannot
do their scan sequentially -- they have to do something like a
logical/keyspace order scan instead, which is *totally* different to
heapam (not just a bit different). There is no telling how many times
each page will be accessed in these other index AMs, and in what
order, even under optimal conditions. We should arguably not even try
to provide any granular progress information here, since it'll
probably be too messy.

I'm not sure what to recommend for your patch, in light of this. Maybe
you should change the names of the new columns to own the squishiness.
For example, instead of using the name index_blks_total, you might
instead use the name index_blks_initial. That might be enough to avoid
user confusion when we scan more blocks than the index initially
contained (within a single ambulkdelete scan).

Note also that we have to do something called backtracking in
btvacuumpage(), which you've ignored -- that's another reasonably
common way that we'll end up scanning a page twice. But that probably
should just be ignored -- it's too narrow a case to be worth caring
about.

--
Peter Geoghegan

#8Justin Pryzby
pryzby@telsasoft.com
In reply to: Imseih (AWS), Sami (#1)
Re: Add index scan progress to pg_stat_progress_vacuum

This view also doesn't show vacuum progress across a partitioned table.

For comparison:

pg_stat_progress_create_index (added in v12) has:
partitions_total
partitions_done

pg_stat_progress_analyze (added in v13) has:
child_tables_total
child_tables_done

pg_stat_progress_cluster should have something similar.

--
Justin Pryzby
System Administrator
Telsasoft
+1-952-707-8581

#9Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Peter Geoghegan (#6)
Re: Add index scan progress to pg_stat_progress_vacuum

On Tue, Dec 21, 2021 at 3:37 AM Peter Geoghegan <pg@bowt.ie> wrote:

On Wed, Dec 15, 2021 at 2:10 PM Bossart, Nathan <bossartn@amazon.com> wrote:

nitpick: Shouldn't index_blks_scanned be index_blks_vacuumed? IMO it
is more analogous to heap_blks_vacuumed.

+1.

This will tell us which indexes are currently being vacuumed and the
current progress of those operations, but it doesn't tell us which
indexes have already been vacuumed or which ones are pending vacuum.

VACUUM will process a table's indexes in pg_class OID order (outside
of parallel VACUUM, I suppose). See comments about sort order above
RelationGetIndexList().

Right.

Anyway, it might be useful to add ordinal numbers to each index, that
line up with this processing/OID order. It would also be reasonable to
display the same number in log_autovacuum* (and VACUUM VERBOSE)
per-index output, to reinforce the idea. Note that we don't
necessarily display a distinct line for each distinct index in this
log output, which is why including the ordinal number there makes
sense.

An alternative idea would be to show the number of indexes on the
table and the number of indexes that have been processed in the
leader's entry of pg_stat_progress_vacuum. Even in parallel vacuum
cases, since we have index vacuum status for each index it would not
be hard for the leader process to count how many indexes have been
processed.

Regarding the details of the progress of index vacuum, I'm not sure
this progress information can fit for pg_stat_progress_vacuum. As
Peter already mentioned, the behavior quite varies depending on index
AM.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#10Andrey Lepikhov
a.lepikhov@postgrespro.ru
In reply to: Peter Geoghegan (#7)
Re: Add index scan progress to pg_stat_progress_vacuum

On 21/12/2021 00:05, Peter Geoghegan wrote:

* Some index AMs don't work like nbtree and GiST in that they cannot
do their scan sequentially -- they have to do something like a
logical/keyspace order scan instead, which is *totally* different to
heapam (not just a bit different). There is no telling how many times
each page will be accessed in these other index AMs, and in what
order, even under optimal conditions. We should arguably not even try
to provide any granular progress information here, since it'll
probably be too messy.

Maybe we could add callbacks into AM interface for
send/receive/representation implementation of progress?
So AM would define a set of parameters to send into stat collector and
show to users.

--
regards,
Andrey Lepikhov
Postgres Professional

#11Justin Pryzby
pryzby@telsasoft.com
In reply to: Justin Pryzby (#8)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Please send your patches as *.diff or *.patch, so they're processed by the
patch tester. Preferably with commit messages; git format-patch is the usual
tool for this.
http://cfbot.cputube.org/sami-imseih.html

(Occasionally, it's also useful to send a *.txt to avoid the cfbot processing
the wrong thing, in case one sends an unrelated, secondary patch, or sends
fixes to a patch as a "relative patch" which doesn't include the main patch.)

I'm including a patch rebased on 8e1fae193.

Attachments:

0001-Add-index-scan-progress-to-pg_stat_progress_vacuum.patchtext/x-diff; charset=us-asciiDownload
From 1d034ff6317919e52c70ff8a4f3af9ac1c101368 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS), Sami" <simseih@amazon.com>
Date: Mon, 20 Dec 2021 17:55:03 +0000
Subject: [PATCH] Add index scan progress to pg_stat_progress_vacuum

Here is a V2 attempt of the patch to include a new view called pg_stat_progress_vacuum_worker. Also, scans for index cleanups will also have an entry in the new view.

Re: Add index scan progress to pg_stat_progress_vacuum
---
 src/backend/access/brin/brin.c        | 27 +++++++++++--
 src/backend/access/gin/ginvacuum.c    | 55 +++++++++++++++++++++++++++
 src/backend/access/gist/gistvacuum.c  | 24 ++++++++++++
 src/backend/access/hash/hash.c        | 46 +++++++++++++++++++++-
 src/backend/access/hash/hashpage.c    |  4 +-
 src/backend/access/heap/vacuumlazy.c  | 36 +++++++++++++++++-
 src/backend/access/nbtree/nbtree.c    | 14 ++++++-
 src/backend/access/spgist/spgvacuum.c | 25 ++++++++++++
 src/backend/catalog/system_views.sql  | 11 ++++++
 src/backend/commands/vacuumparallel.c | 11 ++++++
 src/include/access/hash.h             |  3 +-
 src/include/commands/progress.h       |  2 +
 12 files changed, 247 insertions(+), 11 deletions(-)

diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index f521bb96356..97b2f8bc13a 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -39,6 +39,8 @@
 #include "utils/index_selfuncs.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 
 /*
@@ -77,7 +79,7 @@ static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRang
 static void form_and_insert_tuple(BrinBuildState *state);
 static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
 						 BrinTuple *b);
-static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
+static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, bool report_progress);
 static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
 								BrinMemTuple *dtup, Datum *values, bool *nulls);
 static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
@@ -953,7 +955,7 @@ brinvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 	heapRel = table_open(IndexGetRelation(RelationGetRelid(info->index), false),
 						 AccessShareLock);
 
-	brin_vacuum_scan(info->index, info->strategy);
+	brin_vacuum_scan(info->index, info->strategy, info->report_progress);
 
 	brinsummarize(info->index, heapRel, BRIN_ALL_BLOCKRANGES, false,
 				  &stats->num_index_tuples, &stats->num_index_tuples);
@@ -1635,16 +1637,24 @@ union_tuples(BrinDesc *bdesc, BrinMemTuple *a, BrinTuple *b)
  * and such.
  */
 static void
-brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
+brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, bool report_progress)
 {
 	BlockNumber nblocks;
 	BlockNumber blkno;
+	const int    initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/*
 	 * Scan the index in physical order, and clean up any possible mess in
 	 * each page.
 	 */
 	nblocks = RelationGetNumberOfBlocks(idxrel);
+	if (report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 nblocks);
 	for (blkno = 0; blkno < nblocks; blkno++)
 	{
 		Buffer		buf;
@@ -1656,9 +1666,20 @@ brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
 
 		brin_page_cleanup(idxrel, buf);
 
+		if (report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blkno + 1);
+
 		ReleaseBuffer(buf);
 	}
 
+	if (report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	/*
 	 * Update all upper pages in the index's FSM, as well.  This ensures not
 	 * only that we propagate leaf-page FSM updates made by brin_page_cleanup,
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index a276eb020b5..714586040aa 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -24,6 +24,8 @@
 #include "storage/lmgr.h"
 #include "storage/predicate.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 struct GinVacuumState
 {
@@ -571,6 +573,14 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	Buffer		buffer;
 	BlockNumber rootOfPostingTree[BLCKSZ / (sizeof(IndexTupleData) + sizeof(ItemId))];
 	uint32		nRoot;
+	BlockNumber	num_pages;
+	bool		needLock;
+	int		blocks_scanned = 0;
+	const int	initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	gvs.tmpCxt = AllocSetContextCreate(CurrentMemoryContext,
 									   "Gin vacuum temporary context",
@@ -635,6 +645,19 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 									RBM_NORMAL, info->strategy);
 	}
 
+	needLock = !RELATION_IS_LOCAL(index);
+
+	/* Get the current relation length */
+	if (needLock)
+		LockRelationForExtension(index, ExclusiveLock);
+	num_pages = RelationGetNumberOfBlocks(index);
+	if (needLock)
+		UnlockRelationForExtension(index, ExclusiveLock);
+
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 num_pages);
+
 	/* right now we found leftmost page in entry's BTree */
 
 	for (;;)
@@ -676,9 +699,20 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
+		blocks_scanned++;
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
 		LockBuffer(buffer, GIN_EXCLUSIVE);
 	}
 
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	MemoryContextDelete(gvs.tmpCxt);
 
 	return gvs.result;
@@ -694,6 +728,12 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 	BlockNumber totFreePages;
 	GinState	ginstate;
 	GinStatsData idxStat;
+	int         blocks_scanned = 0;
+	const int	initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/*
 	 * In an autovacuum analyze, we want to clean up pending insertions.
@@ -744,6 +784,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 	totFreePages = 0;
 
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									 npages);
 	for (blkno = GIN_ROOT_BLKNO; blkno < npages; blkno++)
 	{
 		Buffer		buffer;
@@ -774,9 +817,21 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 				idxStat.nEntries += PageGetMaxOffsetNumber(page);
 		}
 
+		blocks_scanned++;
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
+
 		UnlockReleaseBuffer(buffer);
 	}
 
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	/* Update the metapage with accurate page and entry counts */
 	idxStat.nTotalPages = npages;
 	ginUpdateStats(info->index, &idxStat, false);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0663193531a..e7d13c9eb6e 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -23,6 +23,8 @@
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 /* Working state needed by gistbulkdelete */
 typedef struct
@@ -131,6 +133,11 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	bool		needLock;
 	BlockNumber blkno;
 	MemoryContext oldctx;
+	const int	initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/*
 	 * Reset fields that track information about the entire index now.  This
@@ -215,9 +222,26 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 		/* Quit if we've scanned the whole relation */
 		if (blkno >= num_pages)
 			break;
+
+		if (info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+										 num_pages);
+
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_progress)
+				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+											 blkno + 1);
+		}
+	}
+
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 81c7da7ec69..2b4eed6aae1 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -31,6 +31,7 @@
 #include "utils/builtins.h"
 #include "utils/index_selfuncs.h"
 #include "utils/rel.h"
+#include "storage/lmgr.h"
 
 /* Working state for hashbuild and its callback */
 typedef struct
@@ -469,9 +470,21 @@ hashbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	Buffer		metabuf = InvalidBuffer;
 	HashMetaPage metap;
 	HashMetaPage cachedmetap;
+	int         blocks_scanned;
+	int         bucket_blocks_scanned;
+	BlockNumber num_pages;
+	bool		needLock;
+	const int initprog_index[] = {
+		PROGRESS_SCAN_BLOCKS_DONE,
+		PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
+
 
 	tuples_removed = 0;
 	num_index_tuples = 0;
+	blocks_scanned = 0;
+	bucket_blocks_scanned = 0;
 
 	/*
 	 * We need a copy of the metapage so that we can use its hashm_spares[]
@@ -489,6 +502,19 @@ hashbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	cur_bucket = 0;
 	cur_maxbucket = orig_maxbucket;
 
+	needLock = !RELATION_IS_LOCAL(rel);
+
+	/* Get the current relation length */
+	if (needLock)
+		LockRelationForExtension(rel, ExclusiveLock);
+	num_pages = RelationGetNumberOfBlocks(rel);
+	if (needLock)
+		UnlockRelationForExtension(rel, ExclusiveLock);
+
+	if (info->report_progress)
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+									num_pages);
+
 loop_top:
 	while (cur_bucket <= cur_maxbucket)
 	{
@@ -504,6 +530,7 @@ loop_top:
 		bucket_blkno = BUCKET_TO_BLKNO(cachedmetap, cur_bucket);
 
 		blkno = bucket_blkno;
+		blocks_scanned++;
 
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
@@ -550,10 +577,14 @@ loop_top:
 						  cachedmetap->hashm_highmask,
 						  cachedmetap->hashm_lowmask, &tuples_removed,
 						  &num_index_tuples, split_cleanup,
-						  callback, callback_state);
+						  callback, callback_state, info->report_progress,
+						  &bucket_blocks_scanned);
 
 		_hash_dropbuf(rel, bucket_buf);
 
+		pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+									blocks_scanned + bucket_blocks_scanned);
+
 		/* Advance to next bucket */
 		cur_bucket++;
 	}
@@ -633,6 +664,13 @@ loop_top:
 	stats->tuples_removed += tuples_removed;
 	/* hashvacuumcleanup will fill in num_pages */
 
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	return stats;
 }
 
@@ -686,7 +724,8 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 				  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 				  double *tuples_removed, double *num_index_tuples,
 				  bool split_cleanup,
-				  IndexBulkDeleteCallback callback, void *callback_state)
+				  IndexBulkDeleteCallback callback, void *callback_state,
+				  bool report_progress, int *bucket_blocks_scanned)
 {
 	BlockNumber blkno;
 	Buffer		buf;
@@ -718,6 +757,8 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 		page = BufferGetPage(buf);
 		opaque = (HashPageOpaque) PageGetSpecialPointer(page);
 
+		bucket_blocks_scanned++;
+
 		/* Scan each tuple in page */
 		maxoffno = PageGetMaxOffsetNumber(page);
 		for (offno = FirstOffsetNumber;
@@ -916,4 +957,5 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 							bstrategy);
 	else
 		LockBuffer(bucket_buf, BUFFER_LOCK_UNLOCK);
+
 }
diff --git a/src/backend/access/hash/hashpage.c b/src/backend/access/hash/hashpage.c
index 159646c7c3e..538d153df7c 100644
--- a/src/backend/access/hash/hashpage.c
+++ b/src/backend/access/hash/hashpage.c
@@ -758,7 +758,7 @@ restart_expand:
 
 		hashbucketcleanup(rel, old_bucket, buf_oblkno, start_oblkno, NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL, NULL);
 
 		_hash_dropbuf(rel, buf_oblkno);
 
@@ -1326,7 +1326,7 @@ _hash_splitbucket(Relation rel,
 		hashbucketcleanup(rel, obucket, bucket_obuf,
 						  BufferGetBlockNumber(bucket_obuf), NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL, NULL);
 	}
 	else
 	{
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index cd603e6aa41..6900b9ff5db 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -344,6 +344,9 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID,
+								 MyProcPid);
+
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
 						  params->freeze_table_age,
@@ -2508,10 +2511,15 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 {
 	IndexVacuumInfo ivinfo;
 	LVSavedErrInfo saved_err_info;
+	const int    initprog_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_CURRENT_INDRELID
+	};
+	int64        initprog_val[2];
 
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
-	ivinfo.report_progress = false;
+	ivinfo.report_progress = true;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = elevel;
 	ivinfo.num_heap_tuples = reltuples;
@@ -2529,9 +2537,18 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 							 VACUUM_ERRCB_PHASE_VACUUM_INDEX,
 							 InvalidBlockNumber, InvalidOffsetNumber);
 
+	/* Report that we're vacuuming the index, advertising the indrelid */
+	initprog_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	initprog_val[1] = RelationGetRelid(indrel);
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	/* Do bulk deletion */
 	istat = vac_bulkdel_one_index(&ivinfo, istat, (void *) vacrel->dead_items);
 
+	/* Report that we're done vacuuming the index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID,
+								 0);
+
 	/* Revert to the previous phase information for error traceback */
 	restore_vacuum_error_info(vacrel, &saved_err_info);
 	pfree(vacrel->indname);
@@ -2556,10 +2573,15 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 {
 	IndexVacuumInfo ivinfo;
 	LVSavedErrInfo saved_err_info;
+	const int    initprog_index[] = {
+			PROGRESS_VACUUM_PHASE,
+			PROGRESS_VACUUM_CURRENT_INDRELID
+	};
+	int64        initprog_val[2];
 
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
-	ivinfo.report_progress = false;
+	ivinfo.report_progress = true;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = elevel;
 
@@ -2578,8 +2600,18 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 							 VACUUM_ERRCB_PHASE_INDEX_CLEANUP,
 							 InvalidBlockNumber, InvalidOffsetNumber);
 
+	/* Report that we're cleaning the index, advertising the indrelid */
+	initprog_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	initprog_val[1] = RelationGetRelid(indrel);
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	istat = vac_cleanup_one_index(&ivinfo, istat);
 
+	/* Report that we're done cleaning the index */
+	initprog_val[0] = 0;
+	initprog_val[1] = 0;
+	pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+
 	/* Revert to the previous phase information for error traceback */
 	restore_vacuum_error_info(vacrel, &saved_err_info);
 	pfree(vacrel->indname);
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index dfce06dc49f..15c11fcab81 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -909,6 +909,11 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 	BlockNumber num_pages;
 	BlockNumber scanblkno;
 	bool		needLock;
+	const int       initprog_index[] = {
+		PROGRESS_SCAN_BLOCKS_DONE,
+		PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/*
 	 * Reset fields that track information about the entire index now.  This
@@ -997,10 +1002,17 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			btvacuumpage(&vstate, scanblkno);
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
-											 scanblkno);
+											 scanblkno + 1);
 		}
 	}
 
+	if (info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	/* Set statistics num_pages field to final size of index */
 	stats->num_pages = num_pages;
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 76fb0374c42..c71345fcebe 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -27,6 +27,8 @@
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/snapmgr.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 
 /* Entry in pending-list of TIDs we need to revisit */
@@ -797,6 +799,12 @@ spgvacuumscan(spgBulkDeleteState *bds)
 	bool		needLock;
 	BlockNumber num_pages,
 				blkno;
+	int         blocks_scanned = 0;
+	const int	initprog_index[] = {
+			PROGRESS_SCAN_BLOCKS_DONE,
+			PROGRESS_SCAN_BLOCKS_TOTAL
+	};
+	int64        initprog_val[2];
 
 	/* Finish setting up spgBulkDeleteState */
 	initSpGistState(&bds->spgstate, index);
@@ -836,6 +844,11 @@ spgvacuumscan(spgBulkDeleteState *bds)
 		/* Quit if we've scanned the whole relation */
 		if (blkno >= num_pages)
 			break;
+
+		if (bds->info->report_progress)
+			pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_TOTAL,
+										num_pages);
+
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
 		{
@@ -843,9 +856,21 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+
+			blocks_scanned++;
+			if (bds->info->report_progress)
+				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
+										 blocks_scanned + 1);
 		}
 	}
 
+	if (bds->info->report_progress)
+	{
+		initprog_val[0] = 0;
+		initprog_val[1] = 0;
+		pgstat_progress_update_multi_param(2, initprog_index, initprog_val);
+	}
+
 	/* Propagate local lastUsedPages cache to metablock */
 	SpGistUpdateMetaPage(index);
 
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 61b515cdb85..9e0dc39314c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1128,6 +1128,17 @@ CREATE VIEW pg_stat_progress_vacuum AS
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_worker AS
+	SELECT
+		S.pid,
+		S.param9 leader_pid,
+		S.param8 AS indrelid,
+		S.param16 index_blks_total,
+		S.param17 AS index_blks_scanned
+	FROM pg_stat_get_progress_info('VACUUM') AS S
+		LEFT JOIN pg_database D ON S.datid = D.oid
+	WHERE S.param8 > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 0d61c8ec74a..aed186c0c0c 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -942,6 +943,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	int			nindexes;
 	char	   *sharedquery;
 	ErrorContextCallback errcallback;
+	PGPROC     *leader = MyProc->lockGroupLeader;
 
 	/*
 	 * A parallel vacuum worker must have only PROC_IN_VACUUM flag since we
@@ -965,6 +967,14 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
 
+	/*
+	 * Track progress of current index being vacuumed
+	 */
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
+			RelationGetRelid(rel));
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, leader->pid);
+
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
 	 * matched to the leader's one.
@@ -1035,6 +1045,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
+	pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/include/access/hash.h b/src/include/access/hash.h
index 1cce865be2b..c00bb76e3e2 100644
--- a/src/include/access/hash.h
+++ b/src/include/access/hash.h
@@ -478,6 +478,7 @@ extern void hashbucketcleanup(Relation rel, Bucket cur_bucket,
 							  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 							  double *tuples_removed, double *num_index_tuples,
 							  bool split_cleanup,
-							  IndexBulkDeleteCallback callback, void *callback_state);
+							  IndexBulkDeleteCallback callback, void *callback_state,
+							  bool report_progress, int *bucket_blocks_scanned);
 
 #endif							/* HASH_H */
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index d7bf16368bd..4387a7c1f16 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_CURRENT_INDRELID        7
+#define PROGRESS_VACUUM_LEADER_PID              8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
-- 
2.17.1

#12Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#9)
Re: Add index scan progress to pg_stat_progress_vacuum

I do agree that tracking progress by # of blocks scanned is not deterministic for all index types.

Based on this feedback, I went back to the drawing board on this.

Something like below may make more sense.

In pg_stat_progress_vacuum, introduce 2 new columns:

1. total_index_vacuum - total # of indexes to vacuum
2. max_cycle_time - the time in seconds of the longest index cycle.

Introduce another view called pg_stat_progress_vacuum_index_cycle:

postgres=# \d pg_stat_progress_vacuum_index_cycle
View "public.pg_stat_progress_vacuum_worker"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | | <<<-- the PID of the vacuum worker ( or leader if it's doing index vacuuming )
leader_pid | bigint | | | <<<-- the leader PID to allow this view to be joined back to pg_stat_progress_vacuum
indrelid | bigint | | | <<<- the index relid of the index being vacuumed
ordinal_position | bigint | | | <<<- the processing position, which will give an idea of the processing position of the index being vacuumed.
dead_tuples_removed | bigint | | <<<- the number of dead rows removed in the current cycle for the index.

Having this information, one can

1. Determine which index is being vacuumed. For monitoring tools, this can help identify the index that accounts for most of the index vacuuming time.
2. Having the processing order of the current index will allow the user to determine how many of the total indexes has been completed in the current cycle.
3. dead_tuples_removed will show progress on the index vacuum in the current cycle.
4. the max_cycle_time will give an idea on how long the longest index cycle took for the current vacuum operation.

On 12/23/21, 2:46 AM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can confirm the sender and know the content is safe.

On Tue, Dec 21, 2021 at 3:37 AM Peter Geoghegan <pg@bowt.ie> wrote:

On Wed, Dec 15, 2021 at 2:10 PM Bossart, Nathan <bossartn@amazon.com> wrote:

nitpick: Shouldn't index_blks_scanned be index_blks_vacuumed? IMO it
is more analogous to heap_blks_vacuumed.

+1.

This will tell us which indexes are currently being vacuumed and the
current progress of those operations, but it doesn't tell us which
indexes have already been vacuumed or which ones are pending vacuum.

VACUUM will process a table's indexes in pg_class OID order (outside
of parallel VACUUM, I suppose). See comments about sort order above
RelationGetIndexList().

Right.

Anyway, it might be useful to add ordinal numbers to each index, that
line up with this processing/OID order. It would also be reasonable to
display the same number in log_autovacuum* (and VACUUM VERBOSE)
per-index output, to reinforce the idea. Note that we don't
necessarily display a distinct line for each distinct index in this
log output, which is why including the ordinal number there makes
sense.

An alternative idea would be to show the number of indexes on the
table and the number of indexes that have been processed in the
leader's entry of pg_stat_progress_vacuum. Even in parallel vacuum
cases, since we have index vacuum status for each index it would not
be hard for the leader process to count how many indexes have been
processed.

Regarding the details of the progress of index vacuum, I'm not sure
this progress information can fit for pg_stat_progress_vacuum. As
Peter already mentioned, the behavior quite varies depending on index
AM.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#13Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#12)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Attached is the latest revision of the patch.

In "pg_stat_progress_vacuum", introduce 2 columns:

* total_index_vacuum : This is the # of indexes that will be vacuumed. Keep in mind that if failsafe mode kicks in mid-flight to the vacuum, Postgres may choose to forgo index scans. This value will be adjusted accordingly.
* max_index_vacuum_cycle_time : The total elapsed time for a index vacuum cycle is calculated and this value will be updated to reflect the longest vacuum cycle. Until the first cycle completes, this value will be 0. The purpose of this column is to give the user an idea of how long an index vacuum cycle takes to complete.

postgres=# \d pg_stat_progress_vacuum
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default
-----------------------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
relid | oid | | |
phase | text | | |
heap_blks_total | bigint | | |
heap_blks_scanned | bigint | | |
heap_blks_vacuumed | bigint | | |
index_vacuum_count | bigint | | |
max_dead_tuples | bigint | | |
num_dead_tuples | bigint | | |
total_index_vacuum | bigint | | |
max_index_vacuum_cycle_time | bigint | | |

Introduce a new view called "pg_stat_progress_vacuum_index". This view will track the progress of a worker ( or leader PID ) while it's vacuuming an index. It will expose some key columns:

* pid: The PID of the worker process

* leader_pid: The PID of the leader process. This is the column that can be joined with "pg_stat_progress_vacuum". leader_pid and pid can have the same value as a leader can also perform an index vacuum.

* indrelid: The relid of the index currently being vacuumed

* vacuum_cycle_ordinal_position: The processing position of the index being vacuumed. This can be useful to determine how many indexes out of the total indexes ( pg_stat_progress_vacuum.total_index_vacuum ) have been vacuumed

* index_tuples_vacuumed: This is the number of index tuples vacuumed for the index overall. This is useful to show that the vacuum is actually doing work, as the # of tuples keeps increasing.

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
-------------------------------+---------+-----------+----------+---------
pid | integer | | |
leader_pid | bigint | | |
indrelid | bigint | | |
vacuum_cycle_ordinal_position | bigint | | |
index_tuples_vacuumed | bigint | | |

On 12/27/21, 6:12 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

I do agree that tracking progress by # of blocks scanned is not deterministic for all index types.

Based on this feedback, I went back to the drawing board on this.

Something like below may make more sense.

In pg_stat_progress_vacuum, introduce 2 new columns:

1. total_index_vacuum - total # of indexes to vacuum
2. max_cycle_time - the time in seconds of the longest index cycle.

Introduce another view called pg_stat_progress_vacuum_index_cycle:

postgres=# \d pg_stat_progress_vacuum_index_cycle
View "public.pg_stat_progress_vacuum_worker"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | | <<<-- the PID of the vacuum worker ( or leader if it's doing index vacuuming )
leader_pid | bigint | | | <<<-- the leader PID to allow this view to be joined back to pg_stat_progress_vacuum
indrelid | bigint | | | <<<- the index relid of the index being vacuumed
ordinal_position | bigint | | | <<<- the processing position, which will give an idea of the processing position of the index being vacuumed.
dead_tuples_removed | bigint | | <<<- the number of dead rows removed in the current cycle for the index.

Having this information, one can

1. Determine which index is being vacuumed. For monitoring tools, this can help identify the index that accounts for most of the index vacuuming time.
2. Having the processing order of the current index will allow the user to determine how many of the total indexes has been completed in the current cycle.
3. dead_tuples_removed will show progress on the index vacuum in the current cycle.
4. the max_cycle_time will give an idea on how long the longest index cycle took for the current vacuum operation.

On 12/23/21, 2:46 AM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can confirm the sender and know the content is safe.

On Tue, Dec 21, 2021 at 3:37 AM Peter Geoghegan <pg@bowt.ie> wrote:

On Wed, Dec 15, 2021 at 2:10 PM Bossart, Nathan <bossartn@amazon.com> wrote:

nitpick: Shouldn't index_blks_scanned be index_blks_vacuumed? IMO it
is more analogous to heap_blks_vacuumed.

+1.

This will tell us which indexes are currently being vacuumed and the
current progress of those operations, but it doesn't tell us which
indexes have already been vacuumed or which ones are pending vacuum.

VACUUM will process a table's indexes in pg_class OID order (outside
of parallel VACUUM, I suppose). See comments about sort order above
RelationGetIndexList().

Right.

Anyway, it might be useful to add ordinal numbers to each index, that
line up with this processing/OID order. It would also be reasonable to
display the same number in log_autovacuum* (and VACUUM VERBOSE)
per-index output, to reinforce the idea. Note that we don't
necessarily display a distinct line for each distinct index in this
log output, which is why including the ordinal number there makes
sense.

An alternative idea would be to show the number of indexes on the
table and the number of indexes that have been processed in the
leader's entry of pg_stat_progress_vacuum. Even in parallel vacuum
cases, since we have index vacuum status for each index it would not
be hard for the leader process to count how many indexes have been
processed.

Regarding the details of the progress of index vacuum, I'm not sure
this progress information can fit for pg_stat_progress_vacuum. As
Peter already mentioned, the behavior quite varies depending on index
AM.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

Attachments:

0001-Add-index-scan-progress-to-pg_stat_progress_vacuum.patchapplication/octet-stream; name=0001-Add-index-scan-progress-to-pg_stat_progress_vacuum.patchDownload
From 149631ce5c8c5cb5b05c92a7df1de3d8e431a3c2 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS), Sami" <simseih@amazon.com>
Date: Tue, 28 Dec 2021 22:00:43 +0000
Subject: [PATCH 1/1] Add index scan progress to pg_stat_progress_vacuum.

Expose progress for the "vacuuming indexes" phase
of a VACUUM operation.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 src/backend/access/gin/ginvacuum.c    |  4 ++
 src/backend/access/gist/gistvacuum.c  |  4 ++
 src/backend/access/hash/hash.c        |  4 ++
 src/backend/access/heap/vacuumlazy.c  | 32 ++++++++++++++-
 src/backend/access/nbtree/nbtree.c    |  4 ++
 src/backend/access/spgist/spgvacuum.c |  4 ++
 src/backend/catalog/system_views.sql  | 15 ++++++-
 src/backend/commands/vacuumparallel.c | 22 +++++++++--
 src/backend/utils/misc/pg_rusage.c    | 56 ++++++++++++++++++++-------
 src/include/commands/progress.h       | 20 ++++++----
 src/include/utils/pg_rusage.h         |  1 +
 11 files changed, 137 insertions(+), 29 deletions(-)

diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index a276eb020b..07fd5271a1 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -24,6 +24,8 @@
 #include "storage/lmgr.h"
 #include "storage/predicate.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 struct GinVacuumState
 {
@@ -60,6 +62,8 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, gvs->result->tuples_removed);
+
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0663193531..ee77e5e405 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -23,6 +23,8 @@
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 /* Working state needed by gistbulkdelete */
 typedef struct
@@ -375,6 +377,8 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, vstate->stats->tuples_removed);
+
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 81c7da7ec6..0ac59fdb7e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -31,6 +31,8 @@
 #include "utils/builtins.h"
 #include "utils/index_selfuncs.h"
 #include "utils/rel.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 /* Working state for hashbuild and its callback */
 typedef struct
@@ -631,6 +633,8 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, stats->tuples_removed);
+
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index cd603e6aa4..f0cda3d162 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -167,6 +167,8 @@ typedef struct LVRelState
 	/* VACUUM operation's cutoff for freezing XIDs and MultiXactIds */
 	TransactionId FreezeLimit;
 	MultiXactId MultiXactCutoff;
+	/* VACCUM operation's longest index scan cycle */
+	int64 max_index_cycle_time;
 
 	/* Error reporting state */
 	char	   *relnamespace;
@@ -343,6 +345,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
@@ -2038,6 +2041,8 @@ static bool
 lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
+	PGRUsage	ru0;
+	int64		index_cycle_elapsed_ms = 0;
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2056,6 +2061,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, vacrel->nindexes);
+
+	pg_rusage_init(&ru0);
+
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
@@ -2063,14 +2072,22 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index that is being vacuumed and the vacuum order of the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, indrel->rd_id);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_ORDINAL, idx + 1);
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/* Advertise that we are done vacuuming the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
 				allindexes = false;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, 0);
 				break;
 			}
 		}
@@ -2085,8 +2102,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
 		 * that parallel VACUUM only gets the precheck and this postcheck.
 		 */
-		if (lazy_check_wraparound_failsafe(vacrel))
+		if (lazy_check_wraparound_failsafe(vacrel)) {
 			allindexes = false;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, 0);
+		}
 	}
 
 	/*
@@ -2110,6 +2129,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
 
+	/*
+	 * Set the Maximum index vacuum cycle time.
+	 *
+	 * If this cycle took longer than any previous cycle, reset the max_index_cycle_time time.
+	 */
+	index_cycle_elapsed_ms = pg_rusage_elapsed_ms(&ru0);
+	if (index_cycle_elapsed_ms > vacrel->max_index_cycle_time) {
+		vacrel->max_index_cycle_time = index_cycle_elapsed_ms;
+		pgstat_progress_update_param(PROGRESS_VACUUM_MAX_CYCLE_TIME, index_cycle_elapsed_ms);
+	}
+
 	return allindexes;
 }
 
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index dfce06dc49..ed6e6ac42c 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -36,6 +36,8 @@
 #include "utils/builtins.h"
 #include "utils/index_selfuncs.h"
 #include "utils/memutils.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 
 /*
@@ -1272,6 +1274,8 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, stats->tuples_removed);
+
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 76fb0374c4..6040761f33 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -27,6 +27,8 @@
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/snapmgr.h"
+#include "commands/progress.h"
+#include "pgstat.h"
 
 
 /* Entry in pending-list of TIDs we need to revisit */
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 61b515cdb8..70f35092f6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1124,9 +1124,20 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+		S.param11 AS total_index_vacuum, S.param12 AS max_index_vacuum_cycle_time
     FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        LEFT JOIN pg_database D ON S.datid = D.oid
+	WHERE S.pid = S.param9; -- show vacuum progress for the leader PID only
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+	SELECT
+		S.pid AS pid, S.param9 AS leader_pid,
+		S.param8 AS indrelid, S.param10 vacuum_cycle_ordinal_position,
+		S.param13 index_rows_vacuumed
+	FROM pg_stat_get_progress_info('VACUUM') AS S
+		LEFT JOIN pg_database D ON S.datid = D.oid
+	WHERE S.param8 > 0; -- show vacuum progress for a PID that has advertised an index relid
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 0d61c8ec74..983fc823bb 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -35,6 +35,7 @@
 #include "storage/bufmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
+#include "commands/progress.h"
 
 /*
  * DSM keys for parallel vacuum.  Unlike other parallel execution code, since
@@ -206,7 +207,7 @@ static void parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int nu
 static void parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs);
 static void parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs);
 static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-											  PVIndStats *indstats);
+											  PVIndStats *indstats, int ordinal_position);
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
@@ -754,7 +755,7 @@ parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats, idx + 1);
 	}
 
 	/*
@@ -795,7 +796,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats, i + 1);
 	}
 
 	/*
@@ -814,7 +815,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
  */
 static void
 parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-								  PVIndStats *indstats)
+								  PVIndStats *indstats, int ordinal_position)
 {
 	IndexBulkDeleteResult *istat = NULL;
 	IndexBulkDeleteResult *istat_res;
@@ -842,7 +843,14 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			/* Advertise the index that is being vacuumed and the vacuum order of the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, indrel->rd_id);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_ORDINAL, ordinal_position);
+
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+
+			/* Advertise that we are done vacuuming the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, 0);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
@@ -942,6 +950,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	int			nindexes;
 	char	   *sharedquery;
 	ErrorContextCallback errcallback;
+	PGPROC     *leader = MyProc->lockGroupLeader;
 
 	/*
 	 * A parallel vacuum worker must have only PROC_IN_VACUUM flag since we
@@ -965,6 +974,10 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
 
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
+								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, leader->pid);
+
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
 	 * matched to the leader's one.
@@ -1035,6 +1048,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
+	pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/backend/utils/misc/pg_rusage.c b/src/backend/utils/misc/pg_rusage.c
index bb5d9e7c85..14362d7d65 100644
--- a/src/backend/utils/misc/pg_rusage.c
+++ b/src/backend/utils/misc/pg_rusage.c
@@ -19,6 +19,27 @@
 
 #include "utils/pg_rusage.h"
 
+void rusage_adjust(const PGRUsage *ru0, PGRUsage *ru1);
+
+void
+rusage_adjust(const PGRUsage *ru0, PGRUsage *ru1)
+{
+	if (ru1->tv.tv_usec < ru0->tv.tv_usec)
+	{
+		ru1->tv.tv_sec--;
+		ru1->tv.tv_usec += 1000000;
+	}
+	if (ru1->ru.ru_stime.tv_usec < ru0->ru.ru_stime.tv_usec)
+	{
+		ru1->ru.ru_stime.tv_sec--;
+		ru1->ru.ru_stime.tv_usec += 1000000;
+	}
+	if (ru1->ru.ru_utime.tv_usec < ru0->ru.ru_utime.tv_usec)
+	{
+		ru1->ru.ru_utime.tv_sec--;
+		ru1->ru.ru_utime.tv_usec += 1000000;
+	}
+}
 
 /*
  * Initialize usage snapshot.
@@ -44,21 +65,7 @@ pg_rusage_show(const PGRUsage *ru0)
 
 	pg_rusage_init(&ru1);
 
-	if (ru1.tv.tv_usec < ru0->tv.tv_usec)
-	{
-		ru1.tv.tv_sec--;
-		ru1.tv.tv_usec += 1000000;
-	}
-	if (ru1.ru.ru_stime.tv_usec < ru0->ru.ru_stime.tv_usec)
-	{
-		ru1.ru.ru_stime.tv_sec--;
-		ru1.ru.ru_stime.tv_usec += 1000000;
-	}
-	if (ru1.ru.ru_utime.tv_usec < ru0->ru.ru_utime.tv_usec)
-	{
-		ru1.ru.ru_utime.tv_sec--;
-		ru1.ru.ru_utime.tv_usec += 1000000;
-	}
+	rusage_adjust(ru0, &ru1);
 
 	snprintf(result, sizeof(result),
 			 _("CPU: user: %d.%02d s, system: %d.%02d s, elapsed: %d.%02d s"),
@@ -71,3 +78,22 @@ pg_rusage_show(const PGRUsage *ru0)
 
 	return result;
 }
+
+/*
+ * Compute elapsed time since ru0 usage snapshot, and return the
+ * value in Milliseconds.
+ */
+const int64
+pg_rusage_elapsed_ms(const PGRUsage *ru0)
+{
+	static int64 result;
+	PGRUsage    ru1;
+
+	pg_rusage_init(&ru1);
+
+	rusage_adjust(ru0, &ru1);
+
+	result = (int64) (ru1.tv.tv_sec - ru0->tv.tv_sec) * 1000 + ((int64) (ru1.tv.tv_usec - ru0->tv.tv_usec) / 1000);
+
+	return result;
+}
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index d7bf16368b..1d25a6b345 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -18,13 +18,19 @@
 #define PROGRESS_H
 
 /* Progress parameters for (lazy) vacuum */
-#define PROGRESS_VACUUM_PHASE					0
-#define PROGRESS_VACUUM_TOTAL_HEAP_BLKS			1
-#define PROGRESS_VACUUM_HEAP_BLKS_SCANNED		2
-#define PROGRESS_VACUUM_HEAP_BLKS_VACUUMED		3
-#define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
-#define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
-#define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_PHASE					 0
+#define PROGRESS_VACUUM_TOTAL_HEAP_BLKS			 1
+#define PROGRESS_VACUUM_HEAP_BLKS_SCANNED		 2
+#define PROGRESS_VACUUM_HEAP_BLKS_VACUUMED		 3
+#define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		 4
+#define PROGRESS_VACUUM_MAX_DEAD_TUPLES			 5
+#define PROGRESS_VACUUM_NUM_DEAD_TUPLES			 6
+#define PROGRESS_VACUUM_CURRENT_INDRELID         7
+#define PROGRESS_VACUUM_LEADER_PID               8
+#define PROGRESS_VACUUM_INDEX_ORDINAL			 9
+#define PROGRESS_VACUUM_TOTAL_INDEX_VACUUM		 10
+#define PROGRESS_VACUUM_MAX_CYCLE_TIME			 11
+#define PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED	 12
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/utils/pg_rusage.h b/src/include/utils/pg_rusage.h
index c0def804ee..026f823155 100644
--- a/src/include/utils/pg_rusage.h
+++ b/src/include/utils/pg_rusage.h
@@ -33,5 +33,6 @@ typedef struct PGRUsage
 
 extern void pg_rusage_init(PGRUsage *ru0);
 extern const char *pg_rusage_show(const PGRUsage *ru0);
+extern const int64 pg_rusage_elapsed_ms(const PGRUsage *ru0);
 
 #endif							/* PG_RUSAGE_H */
-- 
2.32.0

#14Justin Pryzby
pryzby@telsasoft.com
In reply to: Imseih (AWS), Sami (#13)
Re: Add index scan progress to pg_stat_progress_vacuum

http://cfbot.cputube.org/sami-imseih.html
You should run "make check" and update rules.out.

You should also use make check-world - usually something like:
make check-world -j4 >check-world.out 2>&1 ; echo ret $?

indrelid: The relid of the index currently being vacuumed

I think it should be called indexrelid not indrelid, for consistency with
pg_index.

S.param10 vacuum_cycle_ordinal_position,
S.param13 index_rows_vacuumed

These should both say "AS" for consistency.

system_views.sql is using tabs, but should use spaces for consistency.

#include "commands/progress.h"

The postgres convention is to alphabetize the includes.

/* VACCUM operation's longest index scan cycle */

VACCUM => VACUUM

Ultimately you'll also need to update the docs.

#15Imseih (AWS), Sami
simseih@amazon.com
In reply to: Justin Pryzby (#14)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Attaching the latest revision of the patch with the fixes suggested. Also ran make check and make check-world successfully.

On 12/29/21, 11:51 AM, "Justin Pryzby" <pryzby@telsasoft.com> wrote:

CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can confirm the sender and know the content is safe.

http://cfbot.cputube.org/sami-imseih.html
You should run "make check" and update rules.out.

You should also use make check-world - usually something like:
make check-world -j4 >check-world.out 2>&1 ; echo ret $?

indrelid: The relid of the index currently being vacuumed

I think it should be called indexrelid not indrelid, for consistency with
pg_index.

S.param10 vacuum_cycle_ordinal_position,
S.param13 index_rows_vacuumed

These should both say "AS" for consistency.

system_views.sql is using tabs, but should use spaces for consistency.

#include "commands/progress.h"

The postgres convention is to alphabetize the includes.

/* VACCUM operation's longest index scan cycle */

VACCUM => VACUUM

Ultimately you'll also need to update the docs.

Attachments:

0001-Add-index-scan-progress-to-pg_stat_progress_vacuum.patchapplication/octet-stream; name=0001-Add-index-scan-progress-to-pg_stat_progress_vacuum.patchDownload
From 62af0b730ff30f9d4ff5c5b9663bb062440a9fb7 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS), Sami" <simseih@amazon.com>
Date: Fri, 31 Dec 2021 04:43:31 +0000
Subject: [PATCH 1/1] Add index scan progress to pg_stat_progress_vacuum.

Expose progress for the "vacuuming indexes" phase
of a VACUUM operation.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 src/backend/access/gin/ginvacuum.c    |  4 ++
 src/backend/access/gist/gistvacuum.c  |  4 ++
 src/backend/access/hash/hash.c        |  2 +
 src/backend/access/heap/vacuumlazy.c  | 32 ++++++++++++++-
 src/backend/access/nbtree/nbtree.c    |  2 +
 src/backend/access/spgist/spgvacuum.c |  4 ++
 src/backend/catalog/system_views.sql  | 15 ++++++-
 src/backend/commands/vacuumparallel.c | 22 +++++++++--
 src/backend/utils/misc/pg_rusage.c    | 56 ++++++++++++++++++++-------
 src/include/commands/progress.h       | 20 ++++++----
 src/include/utils/pg_rusage.h         |  1 +
 src/test/regress/expected/rules.out   | 15 ++++++-
 12 files changed, 146 insertions(+), 31 deletions(-)

diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index a276eb020b..90706be135 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,8 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, gvs->result->tuples_removed);
+
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0663193531..9241175329 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,8 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, vstate->stats->tuples_removed);
+
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 81c7da7ec6..45d2942c9e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -631,6 +631,8 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, stats->tuples_removed);
+
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index cd603e6aa4..5b45ee3e2e 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -167,6 +167,8 @@ typedef struct LVRelState
 	/* VACUUM operation's cutoff for freezing XIDs and MultiXactIds */
 	TransactionId FreezeLimit;
 	MultiXactId MultiXactCutoff;
+	/* VACUUM operation's longest index scan cycle */
+	int64 max_index_cycle_time;
 
 	/* Error reporting state */
 	char	   *relnamespace;
@@ -343,6 +345,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
@@ -2038,6 +2041,8 @@ static bool
 lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
+	PGRUsage	ru0;
+	int64		index_cycle_elapsed_ms = 0;
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2056,6 +2061,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, vacrel->nindexes);
+
+	pg_rusage_init(&ru0);
+
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
@@ -2063,14 +2072,22 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index that is being vacuumed and the vacuum order of the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, indrel->rd_id);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_ORDINAL, idx + 1);
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/* Advertise that we are done vacuuming the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
 				allindexes = false;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, 0);
 				break;
 			}
 		}
@@ -2085,8 +2102,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
 		 * that parallel VACUUM only gets the precheck and this postcheck.
 		 */
-		if (lazy_check_wraparound_failsafe(vacrel))
+		if (lazy_check_wraparound_failsafe(vacrel)) {
 			allindexes = false;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, 0);
+		}
 	}
 
 	/*
@@ -2110,6 +2129,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
 
+	/*
+	 * Set the Maximum index vacuum cycle time.
+	 *
+	 * If this cycle took longer than any previous cycle, reset the max_index_cycle_time time.
+	 */
+	index_cycle_elapsed_ms = pg_rusage_elapsed_ms(&ru0);
+	if (index_cycle_elapsed_ms > vacrel->max_index_cycle_time) {
+		vacrel->max_index_cycle_time = index_cycle_elapsed_ms;
+		pgstat_progress_update_param(PROGRESS_VACUUM_MAX_CYCLE_TIME, index_cycle_elapsed_ms);
+	}
+
 	return allindexes;
 }
 
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index dfce06dc49..1481fb41ea 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1272,6 +1272,8 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, stats->tuples_removed);
+
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 76fb0374c4..989a50ef86 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 61b515cdb8..788a1824cc 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1124,9 +1124,20 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param11 AS total_index_vacuum, S.param12 AS max_index_vacuum_cycle_time
     FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.pid = S.param9; -- show vacuum progress for the leader PID only
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid AS pid, S.param9 AS leader_pid,
+        S.param8 AS indexrelid, S.param10 AS vacuum_cycle_ordinal_position,
+        S.param13 AS index_rows_vacuumed
+    FROM pg_stat_get_progress_info('VACUUM') AS S
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.param8 > 0; -- show vacuum progress for a PID that has advertised an index relid
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 0d61c8ec74..1eda5785e5 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -206,7 +207,7 @@ static void parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int nu
 static void parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs);
 static void parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs);
 static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-											  PVIndStats *indstats);
+											  PVIndStats *indstats, int ordinal_position);
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
@@ -754,7 +755,7 @@ parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats, idx + 1);
 	}
 
 	/*
@@ -795,7 +796,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats, i + 1);
 	}
 
 	/*
@@ -814,7 +815,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
  */
 static void
 parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-								  PVIndStats *indstats)
+								  PVIndStats *indstats, int ordinal_position)
 {
 	IndexBulkDeleteResult *istat = NULL;
 	IndexBulkDeleteResult *istat_res;
@@ -842,7 +843,14 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			/* Advertise the index that is being vacuumed and the vacuum order of the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, indrel->rd_id);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_ORDINAL, ordinal_position);
+
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+
+			/* Advertise that we are done vacuuming the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, 0);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
@@ -942,6 +950,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	int			nindexes;
 	char	   *sharedquery;
 	ErrorContextCallback errcallback;
+	PGPROC     *leader = MyProc->lockGroupLeader;
 
 	/*
 	 * A parallel vacuum worker must have only PROC_IN_VACUUM flag since we
@@ -965,6 +974,10 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
 
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
+								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, leader->pid);
+
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
 	 * matched to the leader's one.
@@ -1035,6 +1048,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
+	pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/backend/utils/misc/pg_rusage.c b/src/backend/utils/misc/pg_rusage.c
index bb5d9e7c85..14362d7d65 100644
--- a/src/backend/utils/misc/pg_rusage.c
+++ b/src/backend/utils/misc/pg_rusage.c
@@ -19,6 +19,27 @@
 
 #include "utils/pg_rusage.h"
 
+void rusage_adjust(const PGRUsage *ru0, PGRUsage *ru1);
+
+void
+rusage_adjust(const PGRUsage *ru0, PGRUsage *ru1)
+{
+	if (ru1->tv.tv_usec < ru0->tv.tv_usec)
+	{
+		ru1->tv.tv_sec--;
+		ru1->tv.tv_usec += 1000000;
+	}
+	if (ru1->ru.ru_stime.tv_usec < ru0->ru.ru_stime.tv_usec)
+	{
+		ru1->ru.ru_stime.tv_sec--;
+		ru1->ru.ru_stime.tv_usec += 1000000;
+	}
+	if (ru1->ru.ru_utime.tv_usec < ru0->ru.ru_utime.tv_usec)
+	{
+		ru1->ru.ru_utime.tv_sec--;
+		ru1->ru.ru_utime.tv_usec += 1000000;
+	}
+}
 
 /*
  * Initialize usage snapshot.
@@ -44,21 +65,7 @@ pg_rusage_show(const PGRUsage *ru0)
 
 	pg_rusage_init(&ru1);
 
-	if (ru1.tv.tv_usec < ru0->tv.tv_usec)
-	{
-		ru1.tv.tv_sec--;
-		ru1.tv.tv_usec += 1000000;
-	}
-	if (ru1.ru.ru_stime.tv_usec < ru0->ru.ru_stime.tv_usec)
-	{
-		ru1.ru.ru_stime.tv_sec--;
-		ru1.ru.ru_stime.tv_usec += 1000000;
-	}
-	if (ru1.ru.ru_utime.tv_usec < ru0->ru.ru_utime.tv_usec)
-	{
-		ru1.ru.ru_utime.tv_sec--;
-		ru1.ru.ru_utime.tv_usec += 1000000;
-	}
+	rusage_adjust(ru0, &ru1);
 
 	snprintf(result, sizeof(result),
 			 _("CPU: user: %d.%02d s, system: %d.%02d s, elapsed: %d.%02d s"),
@@ -71,3 +78,22 @@ pg_rusage_show(const PGRUsage *ru0)
 
 	return result;
 }
+
+/*
+ * Compute elapsed time since ru0 usage snapshot, and return the
+ * value in Milliseconds.
+ */
+const int64
+pg_rusage_elapsed_ms(const PGRUsage *ru0)
+{
+	static int64 result;
+	PGRUsage    ru1;
+
+	pg_rusage_init(&ru1);
+
+	rusage_adjust(ru0, &ru1);
+
+	result = (int64) (ru1.tv.tv_sec - ru0->tv.tv_sec) * 1000 + ((int64) (ru1.tv.tv_usec - ru0->tv.tv_usec) / 1000);
+
+	return result;
+}
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index d7bf16368b..1d25a6b345 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -18,13 +18,19 @@
 #define PROGRESS_H
 
 /* Progress parameters for (lazy) vacuum */
-#define PROGRESS_VACUUM_PHASE					0
-#define PROGRESS_VACUUM_TOTAL_HEAP_BLKS			1
-#define PROGRESS_VACUUM_HEAP_BLKS_SCANNED		2
-#define PROGRESS_VACUUM_HEAP_BLKS_VACUUMED		3
-#define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
-#define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
-#define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_PHASE					 0
+#define PROGRESS_VACUUM_TOTAL_HEAP_BLKS			 1
+#define PROGRESS_VACUUM_HEAP_BLKS_SCANNED		 2
+#define PROGRESS_VACUUM_HEAP_BLKS_VACUUMED		 3
+#define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		 4
+#define PROGRESS_VACUUM_MAX_DEAD_TUPLES			 5
+#define PROGRESS_VACUUM_NUM_DEAD_TUPLES			 6
+#define PROGRESS_VACUUM_CURRENT_INDRELID         7
+#define PROGRESS_VACUUM_LEADER_PID               8
+#define PROGRESS_VACUUM_INDEX_ORDINAL			 9
+#define PROGRESS_VACUUM_TOTAL_INDEX_VACUUM		 10
+#define PROGRESS_VACUUM_MAX_CYCLE_TIME			 11
+#define PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED	 12
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/utils/pg_rusage.h b/src/include/utils/pg_rusage.h
index c0def804ee..026f823155 100644
--- a/src/include/utils/pg_rusage.h
+++ b/src/include/utils/pg_rusage.h
@@ -33,5 +33,6 @@ typedef struct PGRUsage
 
 extern void pg_rusage_init(PGRUsage *ru0);
 extern const char *pg_rusage_show(const PGRUsage *ru0);
+extern const int64 pg_rusage_elapsed_ms(const PGRUsage *ru0);
 
 #endif							/* PG_RUSAGE_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index b58b062b10..22c54b36f1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2024,9 +2024,20 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param11 AS total_index_vacuum,
+    s.param12 AS max_index_vacuum_cycle_time
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
-     LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE (s.pid = s.param9);
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.param9 AS leader_pid,
+    s.param8 AS indexrelid,
+    s.param10 AS vacuum_cycle_ordinal_position,
+    s.param13 AS index_rows_vacuumed
+   FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE (s.param8 > 0);
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

#16Bossart, Nathan
bossartn@amazon.com
In reply to: Imseih (AWS), Sami (#15)
Re: Add index scan progress to pg_stat_progress_vacuum

On 12/29/21, 8:44 AM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

In "pg_stat_progress_vacuum", introduce 2 columns:

* total_index_vacuum : This is the # of indexes that will be vacuumed. Keep in mind that if failsafe mode kicks in mid-flight to the vacuum, Postgres may choose to forgo index scans. This value will be adjusted accordingly.
* max_index_vacuum_cycle_time : The total elapsed time for a index vacuum cycle is calculated and this value will be updated to reflect the longest vacuum cycle. Until the first cycle completes, this value will be 0. The purpose of this column is to give the user an idea of how long an index vacuum cycle takes to complete.

I think that total_index_vacuum is a good thing to have. I would
expect this to usually just be the number of indexes on the table, but
as you pointed out, this can be different when we are skipping
indexes. My only concern with this new column is the potential for
confusion when compared with the index_vacuum_count value.
index_vacuum_count indicates the number of vacuum cycles completed,
but total_index_vacuum indicates the number of indexes that will be
vacuumed. However, the names sound like they could refer to the same
thing to me. Perhaps we should rename index_vacuum_count to
index_vacuum_cycles/index_vacuum_cycle_count, and the new column
should be something like num_indexes_to_vacuum or index_vacuum_total.

I don't think we need the max_index_vacuum_cycle_time column. While
the idea is to give users a rough estimate for how long an index cycle
will take, I don't think it will help generate any meaningful
estimates for how much longer the vacuum operation will take. IIUC we
won't have any idea how many total index vacuum cycles will be needed.
Even if we did, the current cycle could take much more or much less
time. Also, none of the other progress views seem to provide any
timing information, which I suspect is by design to avoid inaccurate
estimates.

Introduce a new view called "pg_stat_progress_vacuum_index". This view will track the progress of a worker ( or leader PID ) while it's vacuuming an index. It will expose some key columns:

* pid: The PID of the worker process

* leader_pid: The PID of the leader process. This is the column that can be joined with "pg_stat_progress_vacuum". leader_pid and pid can have the same value as a leader can also perform an index vacuum.

* indrelid: The relid of the index currently being vacuumed

* vacuum_cycle_ordinal_position: The processing position of the index being vacuumed. This can be useful to determine how many indexes out of the total indexes ( pg_stat_progress_vacuum.total_index_vacuum ) have been vacuumed

* index_tuples_vacuumed: This is the number of index tuples vacuumed for the index overall. This is useful to show that the vacuum is actually doing work, as the # of tuples keeps increasing.

Should we also provide some information for determining the progress
of the current cycle? Perhaps there should be an
index_tuples_vacuumed_current_cycle column that users can compare with
the num_dead_tuples value in pg_stat_progress_vacuum. However,
perhaps the number of tuples vacuumed in the current cycle can already
be discovered via index_tuples_vacuumed % max_dead_tuples.

+void
+rusage_adjust(const PGRUsage *ru0, PGRUsage *ru1)
+{
+	if (ru1->tv.tv_usec < ru0->tv.tv_usec)
+	{
+		ru1->tv.tv_sec--;
+		ru1->tv.tv_usec += 1000000;
+	}
+	if (ru1->ru.ru_stime.tv_usec < ru0->ru.ru_stime.tv_usec)
+	{
+		ru1->ru.ru_stime.tv_sec--;
+		ru1->ru.ru_stime.tv_usec += 1000000;
+	}
+	if (ru1->ru.ru_utime.tv_usec < ru0->ru.ru_utime.tv_usec)
+	{
+		ru1->ru.ru_utime.tv_sec--;
+		ru1->ru.ru_utime.tv_usec += 1000000;
+	}
+}

I think this function could benefit from a comment. Without going
through it line by line, it is not clear to me exactly what it is
doing.

I know we're still working on what exactly this stuff should look
like, but I would suggest adding the documentation changes in the near
future.

Nathan

#17Imseih (AWS), Sami
simseih@amazon.com
In reply to: Bossart, Nathan (#16)
Re: Add index scan progress to pg_stat_progress_vacuum

Thanks for the review.

I am hesitant to make column name changes for obvious reasons, as it breaks existing tooling. However, I think there is a really good case to change "index_vacuum_count" as the name is confusing. "index_vacuum_cycles_completed" is the name I suggest if we agree to rename.

For the new column, "num_indexes_to_vacuum" is good with me.

As far as max_index_vacuum_cycle_time goes, Besides the points you make, another reason is that until one cycle completes, this value will remain at 0. It will not be helpful data for most vacuum cases. Removing it also reduces the complexity of the patch.

On 1/6/22, 2:41 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 12/29/21, 8:44 AM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

In "pg_stat_progress_vacuum", introduce 2 columns:

* total_index_vacuum : This is the # of indexes that will be vacuumed. Keep in mind that if failsafe mode kicks in mid-flight to the vacuum, Postgres may choose to forgo index scans. This value will be adjusted accordingly.
* max_index_vacuum_cycle_time : The total elapsed time for a index vacuum cycle is calculated and this value will be updated to reflect the longest vacuum cycle. Until the first cycle completes, this value will be 0. The purpose of this column is to give the user an idea of how long an index vacuum cycle takes to complete.

I think that total_index_vacuum is a good thing to have. I would
expect this to usually just be the number of indexes on the table, but
as you pointed out, this can be different when we are skipping
indexes. My only concern with this new column is the potential for
confusion when compared with the index_vacuum_count value.
index_vacuum_count indicates the number of vacuum cycles completed,
but total_index_vacuum indicates the number of indexes that will be
vacuumed. However, the names sound like they could refer to the same
thing to me. Perhaps we should rename index_vacuum_count to
index_vacuum_cycles/index_vacuum_cycle_count, and the new column
should be something like num_indexes_to_vacuum or index_vacuum_total.

I don't think we need the max_index_vacuum_cycle_time column. While
the idea is to give users a rough estimate for how long an index cycle
will take, I don't think it will help generate any meaningful
estimates for how much longer the vacuum operation will take. IIUC we
won't have any idea how many total index vacuum cycles will be needed.
Even if we did, the current cycle could take much more or much less
time. Also, none of the other progress views seem to provide any
timing information, which I suspect is by design to avoid inaccurate
estimates.

Introduce a new view called "pg_stat_progress_vacuum_index". This view will track the progress of a worker ( or leader PID ) while it's vacuuming an index. It will expose some key columns:

* pid: The PID of the worker process

* leader_pid: The PID of the leader process. This is the column that can be joined with "pg_stat_progress_vacuum". leader_pid and pid can have the same value as a leader can also perform an index vacuum.

* indrelid: The relid of the index currently being vacuumed

* vacuum_cycle_ordinal_position: The processing position of the index being vacuumed. This can be useful to determine how many indexes out of the total indexes ( pg_stat_progress_vacuum.total_index_vacuum ) have been vacuumed

* index_tuples_vacuumed: This is the number of index tuples vacuumed for the index overall. This is useful to show that the vacuum is actually doing work, as the # of tuples keeps increasing.

Should we also provide some information for determining the progress
of the current cycle? Perhaps there should be an
index_tuples_vacuumed_current_cycle column that users can compare with
the num_dead_tuples value in pg_stat_progress_vacuum. However,
perhaps the number of tuples vacuumed in the current cycle can already
be discovered via index_tuples_vacuumed % max_dead_tuples.

    +void
    +rusage_adjust(const PGRUsage *ru0, PGRUsage *ru1)
    +{
    +	if (ru1->tv.tv_usec < ru0->tv.tv_usec)
    +	{
    +		ru1->tv.tv_sec--;
    +		ru1->tv.tv_usec += 1000000;
    +	}
    +	if (ru1->ru.ru_stime.tv_usec < ru0->ru.ru_stime.tv_usec)
    +	{
    +		ru1->ru.ru_stime.tv_sec--;
    +		ru1->ru.ru_stime.tv_usec += 1000000;
    +	}
    +	if (ru1->ru.ru_utime.tv_usec < ru0->ru.ru_utime.tv_usec)
    +	{
    +		ru1->ru.ru_utime.tv_sec--;
    +		ru1->ru.ru_utime.tv_usec += 1000000;
    +	}
    +}

I think this function could benefit from a comment. Without going
through it line by line, it is not clear to me exactly what it is
doing.

I know we're still working on what exactly this stuff should look
like, but I would suggest adding the documentation changes in the near
future.

Nathan

#18Bossart, Nathan
bossartn@amazon.com
In reply to: Imseih (AWS), Sami (#17)
Re: Add index scan progress to pg_stat_progress_vacuum

On 1/6/22, 6:14 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

I am hesitant to make column name changes for obvious reasons, as it breaks existing tooling. However, I think there is a really good case to change "index_vacuum_count" as the name is confusing. "index_vacuum_cycles_completed" is the name I suggest if we agree to rename.

For the new column, "num_indexes_to_vacuum" is good with me.

Yeah, I think we can skip renaming index_vacuum_count for now. In any
case, it would probably be good to discuss that in a separate thread.

Nathan

#19Imseih (AWS), Sami
simseih@amazon.com
In reply to: Bossart, Nathan (#18)
2 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

I agree, Renaming "index_vacuum_count" can be taken up in a separate discussion.

I have attached the 3rd revision of the patch which also includes the documentation changes. Also attached is a rendered html of the docs for review.

"max_index_vacuum_cycle_time" has been removed.
"index_rows_vacuumed" renamed to "index_tuples_removed". "tuples" is a more consistent with the terminology used.
"vacuum_cycle_ordinal_position" renamed to "index_ordinal_position".

On 1/10/22, 12:30 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/6/22, 6:14 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

I am hesitant to make column name changes for obvious reasons, as it breaks existing tooling. However, I think there is a really good case to change "index_vacuum_count" as the name is confusing. "index_vacuum_cycles_completed" is the name I suggest if we agree to rename.

For the new column, "num_indexes_to_vacuum" is good with me.

Yeah, I think we can skip renaming index_vacuum_count for now. In any
case, it would probably be good to discuss that in a separate thread.

Nathan

Attachments:

progress-reporting.htmltext/html; name=progress-reporting.htmlDownload
0001-Expose-progress-for-the-vacuuming-indexes-phase-of-a.patchapplication/octet-stream; name=0001-Expose-progress-for-the-vacuuming-indexes-phase-of-a.patchDownload
From 61c5dd12727c94491131a53e15566c475b5d47f2 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS), Sami" <simseih@amazon.com>
Date: Mon, 10 Jan 2022 04:13:03 +0000
Subject: [PATCH 1/1] Expose progress for the "vacuuming indexes" phase of a
 VACUUM operation.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 doc/src/sgml/monitoring.sgml          | 110 +++++++++++++++++++++++++-
 src/backend/access/gin/ginvacuum.c    |   4 +
 src/backend/access/gist/gistvacuum.c  |   4 +
 src/backend/access/hash/hash.c        |   2 +
 src/backend/access/heap/vacuumlazy.c  |  19 ++++-
 src/backend/access/nbtree/nbtree.c    |   2 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  17 +++-
 src/backend/commands/vacuumparallel.c |  22 +++++-
 src/include/commands/progress.h       |  19 +++--
 src/test/regress/expected/rules.out   |  20 ++++-
 11 files changed, 205 insertions(+), 18 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 62f2a3332b..e82e2753bf 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6135,8 +6135,10 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    Whenever <command>VACUUM</command> is running, the
    <structname>pg_stat_progress_vacuum</structname> view will contain
    one row for each backend (including autovacuum worker processes) that is
-   currently vacuuming.  The tables below describe the information
-   that will be reported and provide information about how to interpret it.
+   currently vacuuming. The <structname>pg_stat_progress_vacuum_index</structname> 
+   view will also contain one row for each backend that is currently vacuuming an index. 
+   The tables below describe the information that will be reported and provide 
+   information about how to interpret it.
    Progress for <command>VACUUM FULL</command> commands is reported via
    <structname>pg_stat_progress_cluster</structname>
    because both <command>VACUUM FULL</command> and <command>CLUSTER</command>
@@ -6269,6 +6271,110 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>num_indexes_to_vacuum</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes that will be vacuumed. Only indexes with 
+       <literal>pg_index.indisready</literal> set to "true" will be vacuumed.
+	   Whenever <xref linkend="guc-vacuum-failsafe-age"/> is triggered, index
+       vacuuming will be bypassed.
+      </para></entry>
+     </row>
+    </tbody>
+   </tgroup>
+  </table>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>relid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the table being vacuumed.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Process ID of the parallel group leader. This field is <literal>NULL</literal> 
+       if this process is a parallel group leader or the
+       <literal>vacuuming indexes</literal> phase is not performed in parallel.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>bigint</type>
+      </para>
+      <para>
+       OID of the index being vacuumed.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>index_ordinal_position</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The order in which the index is being vacuumed. Indexes are vacuumed by OID in ascending order.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>index_tuples_removed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of tuple indexes removed during the <literal>vacuuming indexes</literal> phase of <literal>VACUUM</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..b7beeab751 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,8 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, gvs->result->tuples_removed);
+
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..d288edc8cd 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,8 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, vstate->stats->tuples_removed);
+
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index d48c8a4549..c1df85bce8 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -631,6 +631,8 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, stats->tuples_removed);
+
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 5e4a5d0d82..c73d2013a0 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -343,6 +343,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, -1);
 
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
@@ -447,6 +448,9 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	errcallback.previous = error_context_stack;
 	error_context_stack = &errcallback;
 
+	/* Advertise the number of indexes to vacuum if we are not in failsafe mode */
+	if (!lazy_check_wraparound_failsafe(vacrel))
+		pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, vacrel->nindexes);
 	/*
 	 * Call lazy_scan_heap to perform all required heap pruning, index
 	 * vacuuming, and heap vacuuming (plus related processing)
@@ -2038,6 +2042,7 @@ static bool
 lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
+	PGRUsage	ru0;
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2056,6 +2061,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 
+	pg_rusage_init(&ru0);
+
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
@@ -2063,14 +2070,22 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index that is being vacuumed and the vacuum order of the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, indrel->rd_id);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_ORDINAL, idx + 1);
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/* Advertise that we are done vacuuming the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
 				allindexes = false;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, 0);
 				break;
 			}
 		}
@@ -2085,8 +2100,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
 		 * that parallel VACUUM only gets the precheck and this postcheck.
 		 */
-		if (lazy_check_wraparound_failsafe(vacrel))
+		if (lazy_check_wraparound_failsafe(vacrel)) {
 			allindexes = false;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, 0);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 13024af2fa..27cedf20b9 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1272,6 +1272,8 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, stats->tuples_removed);
+
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..9753eb8090 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 701ff38f76..5bba12f678 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1124,9 +1124,22 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param11 AS num_indexes_to_vacuum
     FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE (S.pid = S.param9 OR S.param9 = -1); -- show vacuum progress for the leader PID only
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid AS pid, S.datid AS datid, D.datname AS datname,
+        S.relid AS relid,
+        CASE S.param9 WHEN -1 THEN NULL ELSE S.param9 END AS leader_pid,
+        S.param8 AS indexrelid,
+        S.param10 AS index_ordinal_position, S.param12 AS index_tuples_removed
+    FROM pg_stat_get_progress_info('VACUUM') AS S
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.param8 > 0; -- show vacuum progress for a PID that has advertised an index relid
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 53a59da2d6..6bb4a37d96 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -206,7 +207,7 @@ static void parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int nu
 static void parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs);
 static void parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs);
 static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-											  PVIndStats *indstats);
+											  PVIndStats *indstats, int ordinal_position);
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
@@ -754,7 +755,7 @@ parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats, idx + 1);
 	}
 
 	/*
@@ -795,7 +796,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats, i + 1);
 	}
 
 	/*
@@ -814,7 +815,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
  */
 static void
 parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-								  PVIndStats *indstats)
+								  PVIndStats *indstats, int ordinal_position)
 {
 	IndexBulkDeleteResult *istat = NULL;
 	IndexBulkDeleteResult *istat_res;
@@ -842,7 +843,14 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			/* Advertise the index that is being vacuumed and the vacuum order of the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, indrel->rd_id);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_ORDINAL, ordinal_position);
+
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+
+			/* Advertise that we are done vacuuming the index */
+			pgstat_progress_update_param(PROGRESS_VACUUM_CURRENT_INDRELID, 0);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
@@ -942,6 +950,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	int			nindexes;
 	char	   *sharedquery;
 	ErrorContextCallback errcallback;
+	PGPROC     *leader = MyProc->lockGroupLeader;
 
 	/*
 	 * A parallel vacuum worker must have only PROC_IN_VACUUM flag since we
@@ -965,6 +974,10 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
 
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
+								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, leader->pid);
+
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
 	 * matched to the leader's one.
@@ -1035,6 +1048,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
+	pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..fe3971f075 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -18,13 +18,18 @@
 #define PROGRESS_H
 
 /* Progress parameters for (lazy) vacuum */
-#define PROGRESS_VACUUM_PHASE					0
-#define PROGRESS_VACUUM_TOTAL_HEAP_BLKS			1
-#define PROGRESS_VACUUM_HEAP_BLKS_SCANNED		2
-#define PROGRESS_VACUUM_HEAP_BLKS_VACUUMED		3
-#define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
-#define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
-#define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_PHASE					 0
+#define PROGRESS_VACUUM_TOTAL_HEAP_BLKS			 1
+#define PROGRESS_VACUUM_HEAP_BLKS_SCANNED		 2
+#define PROGRESS_VACUUM_HEAP_BLKS_VACUUMED		 3
+#define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		 4
+#define PROGRESS_VACUUM_MAX_DEAD_TUPLES			 5
+#define PROGRESS_VACUUM_NUM_DEAD_TUPLES			 6
+#define PROGRESS_VACUUM_CURRENT_INDRELID         7
+#define PROGRESS_VACUUM_LEADER_PID               8
+#define PROGRESS_VACUUM_INDEX_ORDINAL			 9
+#define PROGRESS_VACUUM_TOTAL_INDEX_VACUUM		 10
+#define PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED	 11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index b58b062b10..a00e0db28c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2024,9 +2024,25 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param11 AS num_indexes_to_vacuum
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
-     LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.pid = s.param9) OR (s.param9 = '-1'::integer));
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    d.datname,
+    s.relid,
+        CASE s.param9
+            WHEN '-1'::integer THEN NULL::bigint
+            ELSE s.param9
+        END AS leader_pid,
+    s.param8 AS indexrelid,
+    s.param10 AS index_ordinal_position,
+    s.param12 AS index_tuples_removed
+   FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE (s.param8 > 0);
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

#20Bossart, Nathan
bossartn@amazon.com
In reply to: Imseih (AWS), Sami (#19)
Re: Add index scan progress to pg_stat_progress_vacuum

On 1/10/22, 5:01 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

I have attached the 3rd revision of the patch which also includes the documentation changes. Also attached is a rendered html of the docs for review.

"max_index_vacuum_cycle_time" has been removed.
"index_rows_vacuumed" renamed to "index_tuples_removed". "tuples" is a more consistent with the terminology used.
"vacuum_cycle_ordinal_position" renamed to "index_ordinal_position".

Thanks for the new version of the patch!

nitpick: I get one whitespace error when applying the patch.

Applying: Expose progress for the "vacuuming indexes" phase of a VACUUM operation.
.git/rebase-apply/patch:44: tab in indent.
Whenever <xref linkend="guc-vacuum-failsafe-age"/> is triggered, index
warning: 1 line adds whitespace errors.

+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>num_indexes_to_vacuum</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes that will be vacuumed. Only indexes with
+       <literal>pg_index.indisready</literal> set to "true" will be vacuumed.
+       Whenever <xref linkend="guc-vacuum-failsafe-age"/> is triggered, index
+       vacuuming will be bypassed.
+      </para></entry>
+     </row>
+    </tbody>
+   </tgroup>
+  </table>

We may want to avoid exhaustively listing the cases when this value
will be zero. I would suggest saying, "When index cleanup is skipped,
this value will be zero" instead.

+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>relid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the table being vacuumed.
+      </para></entry>
+     </row>

Do we need to include this field? I would expect indexrelid to go
here.

+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Process ID of the parallel group leader. This field is <literal>NULL</literal>
+       if this process is a parallel group leader or the
+       <literal>vacuuming indexes</literal> phase is not performed in parallel.
+      </para></entry>
+     </row>

Are there cases where the parallel group leader will have an entry in
this view when parallelism is enabled?

+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>index_ordinal_position</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The order in which the index is being vacuumed. Indexes are vacuumed by OID in ascending order.
+      </para></entry>
+     </row>

Should we include the bit about the OID ordering? I suppose that is
unlikely to change in the near future, but I don't know if it is
relevant information. Also, do we need to include the "index_"
prefix? This view is specific for indexes. (I have the same question
for index_tuples_removed.)

Should this new table go after the "VACUUM phases" table? It might
make sense to keep the phases table closer to where it is referenced.

+    /* Advertise the number of indexes to vacuum if we are not in failsafe mode */
+    if (!lazy_check_wraparound_failsafe(vacrel))
+        pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, vacrel->nindexes);

Shouldn't this be 0 when INDEX_CLEANUP is off, too?

+#define PROGRESS_VACUUM_CURRENT_INDRELID         7
+#define PROGRESS_VACUUM_LEADER_PID               8
+#define PROGRESS_VACUUM_INDEX_ORDINAL            9
+#define PROGRESS_VACUUM_TOTAL_INDEX_VACUUM       10
+#define PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED     11

nitpick: I would suggest the following names to match the existing
style:

PROGRESS_VACUUM_NUM_INDEXES_TO_VACUUM
PROGRESS_VACUUM_INDEX_LEADER_PID
PROGRESS_VACUUM_INDEX_INDEXRELID
PROGRESS_VACUUM_INDEX_ORDINAL_POSITION
PROGRESS_VACUUM_INDEX_TUPLES_REMOVED

Nathan

#21Imseih (AWS), Sami
simseih@amazon.com
In reply to: Bossart, Nathan (#20)
Re: Add index scan progress to pg_stat_progress_vacuum

On 1/11/22, 1:01 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/10/22, 5:01 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

I have attached the 3rd revision of the patch which also includes the documentation changes. Also attached is a rendered html of the docs for review.

"max_index_vacuum_cycle_time" has been removed.
"index_rows_vacuumed" renamed to "index_tuples_removed". "tuples" is a more consistent with the terminology used.
"vacuum_cycle_ordinal_position" renamed to "index_ordinal_position".

Thanks for the new version of the patch!

nitpick: I get one whitespace error when applying the patch.

Applying: Expose progress for the "vacuuming indexes" phase of a VACUUM operation.
.git/rebase-apply/patch:44: tab in indent.
Whenever <xref linkend="guc-vacuum-failsafe-age"/> is triggered, index
warning: 1 line adds whitespace errors.

That was missed. Will fix it.

    +     <row>
    +      <entry role="catalog_table_entry"><para role="column_definition">
    +       <structfield>num_indexes_to_vacuum</structfield> <type>bigint</type>
    +      </para>
    +      <para>
    +       The number of indexes that will be vacuumed. Only indexes with
    +       <literal>pg_index.indisready</literal> set to "true" will be vacuumed.
    +       Whenever <xref linkend="guc-vacuum-failsafe-age"/> is triggered, index
    +       vacuuming will be bypassed.
    +      </para></entry>
    +     </row>
    +    </tbody>
    +   </tgroup>
    +  </table>

We may want to avoid exhaustively listing the cases when this value
will be zero. I would suggest saying, "When index cleanup is skipped,
this value will be zero" instead.

What about something like "The number of indexes that are eligible for vacuuming".
This covers the cases where either an individual index is skipped or the entire "index vacuuming" phase is skipped.

    +     <row>
    +      <entry role="catalog_table_entry"><para role="column_definition">
    +       <structfield>relid</structfield> <type>oid</type>
    +      </para>
    +      <para>
    +       OID of the table being vacuumed.
    +      </para></entry>
    +     </row>

Do we need to include this field? I would expect indexrelid to go
here.

Having indexrelid and relid makes the pg_stat_progress_vacuum_index view "self-contained". A user can lookup the index and table being vacuumed without joining back to pg_stat_progress_vacuum.

    +     <row>
    +      <entry role="catalog_table_entry"><para role="column_definition">
    +       <structfield>leader_pid</structfield> <type>bigint</type>
    +      </para>
    +      <para>
    +       Process ID of the parallel group leader. This field is <literal>NULL</literal>
    +       if this process is a parallel group leader or the
    +       <literal>vacuuming indexes</literal> phase is not performed in parallel.
    +      </para></entry>
    +     </row>

Are there cases where the parallel group leader will have an entry in
this view when parallelism is enabled?

Yes. A parallel group leader can perform an index vacuum just like a parallel worker. If you do something like "vacuum (parallel 3) ", you may have up to 4 processes vacuuming indexes. The leader + 3 workers.

    +     <row>
    +      <entry role="catalog_table_entry"><para role="column_definition">
    +       <structfield>index_ordinal_position</structfield> <type>bigint</type>
    +      </para>
    +      <para>
    +       The order in which the index is being vacuumed. Indexes are vacuumed by OID in ascending order.
    +      </para></entry>
    +     </row>

Should we include the bit about the OID ordering? I suppose that is
unlikely to change in the near future, but I don't know if it is
relevant information. Also, do we need to include the "index_"
prefix? This view is specific for indexes. (I have the same question
for index_tuples_removed.)

I was on the fence about both of these as well. Will make a change to this.

Should this new table go after the "VACUUM phases" table? It might
make sense to keep the phases table closer to where it is referenced.

I did not think that would read better. The introduction discusses both views and the "phase" table is linked from the pg_stat_progress_vacuum

    +    /* Advertise the number of indexes to vacuum if we are not in failsafe mode */
    +    if (!lazy_check_wraparound_failsafe(vacrel))
    +        pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEX_VACUUM, vacrel->nindexes);

Shouldn't this be 0 when INDEX_CLEANUP is off, too?

This view is only covering the "vacuum index" phase, but it should also cover index_cleanup phase as well. Will update the patch.

    +#define PROGRESS_VACUUM_CURRENT_INDRELID         7
    +#define PROGRESS_VACUUM_LEADER_PID               8
    +#define PROGRESS_VACUUM_INDEX_ORDINAL            9
    +#define PROGRESS_VACUUM_TOTAL_INDEX_VACUUM       10
    +#define PROGRESS_VACUUM_DEAD_TUPLES_VACUUMED     11

nitpick: I would suggest the following names to match the existing
style:

PROGRESS_VACUUM_NUM_INDEXES_TO_VACUUM
PROGRESS_VACUUM_INDEX_LEADER_PID
PROGRESS_VACUUM_INDEX_INDEXRELID
PROGRESS_VACUUM_INDEX_ORDINAL_POSITION
PROGRESS_VACUUM_INDEX_TUPLES_REMOVED

That looks better.

Nathan

#22Bossart, Nathan
bossartn@amazon.com
In reply to: Imseih (AWS), Sami (#21)
Re: Add index scan progress to pg_stat_progress_vacuum

On 1/11/22, 12:33 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

What about something like "The number of indexes that are eligible for vacuuming".
This covers the cases where either an individual index is skipped or the entire "index vacuuming" phase is skipped.

Hm. I don't know if "eligible" is the right word. An index can be
eligible for vacuuming but skipped because we set INDEX_CLEANUP to
false. Maybe we should just stick with "The number of indexes that
will be vacuumed." The only thing we may want to clarify is whether
this value will change in some cases (e.g., vacuum failsafe takes
effect).

Nathan

#23Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#19)
Re: Add index scan progress to pg_stat_progress_vacuum

(We had better avoid top-posting[1]https://en.wikipedia.org/wiki/Posting_style#Top-posting)

On Tue, Jan 11, 2022 at 10:01 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

I agree, Renaming "index_vacuum_count" can be taken up in a separate discussion.

I have attached the 3rd revision of the patch which also includes the documentation changes. Also attached is a rendered html of the docs for review.

Thank you for updating the patch!

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

Regards,

[1]: https://en.wikipedia.org/wiki/Posting_style#Top-posting

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#24Bossart, Nathan
bossartn@amazon.com
In reply to: Masahiko Sawada (#23)
Re: Add index scan progress to pg_stat_progress_vacuum

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

#25Imseih (AWS), Sami
simseih@amazon.com
In reply to: Bossart, Nathan (#24)
Re: Add index scan progress to pg_stat_progress_vacuum

On 1/12/22, 1:28 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

To add, since a vacuum can utilize parallel worker processes + the main vacuum process to perform index vacuuming, it made sense to separate the backends doing index vacuum/cleanup in a separate view.
Besides what Nathan suggested, the only other clean option I can think of is to perhaps create a json column in pg_stat_progress_vacuum which will include all the new fields. My concern with this approach is that it will make usability, to flatten the json, difficult for users.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

The main purpose is to really show that the "index vacuum" phase is actually making progress. Note that for certain types of indexes, i.e. GIN/GIST the number of tuples_removed will end up exceeding the number of num_dead_tuples.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

#26Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#25)
2 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Attached is the latest patch and associated documentation.

This version addresses the index_ordinal_position column confusion. Rather than displaying the index position, the pg_stat_progress_vacuum view now has 2 new column(s):
index_total - this column will show the total number of indexes to be vacuumed
index_complete_count - this column will show the total number of indexes processed so far. In order to deal with the parallel vacuums, the parallel_workers ( planned workers ) value had to be exposed and each backends performing an index vacuum/cleanup in parallel had to advertise the number of indexes it vacuumed/cleaned. The # of indexes vacuumed for the parallel cleanup can then be derived the pg_stat_progress_vacuum view.

postgres=# \d pg_stat_progress_vacuum
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default
----------------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
relid | oid | | |
phase | text | | |
heap_blks_total | bigint | | |
heap_blks_scanned | bigint | | |
heap_blks_vacuumed | bigint | | |
index_vacuum_count | bigint | | |
max_dead_tuples | bigint | | |
num_dead_tuples | bigint | | |
index_total | bigint | | |. <<<---------------------
index_complete_count | numeric | | |. <<<---------------------

The pg_stat_progress_vacuum_index view includes:

Indexrelid - the currently vacuumed index
Leader_pid - the pid of the leader process. NULL if the process is the leader or vacuum is not parallel
tuples_removed - the amount of indexes tuples removed. The user can use this column to see that the index vacuum has movement.

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
phase | text | | |
leader_pid | bigint | | |
tuples_removed | bigint | | |

On 1/12/22, 9:52 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

On 1/12/22, 1:28 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

To add, since a vacuum can utilize parallel worker processes + the main vacuum process to perform index vacuuming, it made sense to separate the backends doing index vacuum/cleanup in a separate view.
Besides what Nathan suggested, the only other clean option I can think of is to perhaps create a json column in pg_stat_progress_vacuum which will include all the new fields. My concern with this approach is that it will make usability, to flatten the json, difficult for users.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

The main purpose is to really show that the "index vacuum" phase is actually making progress. Note that for certain types of indexes, i.e. GIN/GIST the number of tuples_removed will end up exceeding the number of num_dead_tuples.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

Attachments:

monitoring.htmltext/html; name=monitoring.htmlDownload
0004-Expose-progress-for-the-vacuuming-indexes-cleanup-ph.patchapplication/octet-stream; name=0004-Expose-progress-for-the-vacuuming-indexes-cleanup-ph.patchDownload
From 33d4116a6b2ea4eef7d00ada279a329854514068 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)"
 <simseih@dev-dsk-simseih-1e-fba9b514.us-east-1.amazon.com>
Date: Thu, 27 Jan 2022 01:41:20 +0000
Subject: [PATCH 1/1] Expose progress for the "vacuuming indexes/cleanup" phase
 of a VACUUM operation.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 doc/src/sgml/monitoring.sgml          | 131 ++++++++++++++++++++++++++
 src/backend/access/gin/ginvacuum.c    |   4 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  11 +++
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  70 ++++++++++----
 src/backend/commands/vacuumparallel.c |  56 ++++++++---
 src/include/commands/progress.h       |   6 ++
 src/test/regress/expected/rules.out   |  63 ++++++++++---
 11 files changed, 309 insertions(+), 41 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 62f2a3332b..b2b33acb66 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend (including autovacuum worker processes) running
+       the <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase of a 
+       <command>VACUUM</command>, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6269,6 +6278,128 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>index_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The total number of indexes that will be processed by the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>index_complete_count</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The total number of indexes processed by the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> of the vacuum.
+       For vacuums with mutiple index vacuum cycles, this value will be set to <literal>0</literal>
+       at the start of each cycle.
+      </para></entry>
+     </row>
+    </tbody>
+   </tgroup>
+  </table>
+
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or 
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal> 
+       phase.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..73660719b5 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,8 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
+
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index d48c8a4549..84341aa7e7 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -631,6 +631,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 1749cc2a47..432229adea 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -340,6 +340,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
@@ -455,6 +456,9 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->FreezeLimit = FreezeLimit;
 	vacrel->MultiXactCutoff = MultiXactCutoff;
 
+	/* Advertise the total indexes to vacuum */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
+
 	/*
 	 * Call lazy_scan_heap to perform all required heap pruning, index
 	 * vacuuming, and heap vacuuming (plus related processing)
@@ -2010,9 +2014,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
+			pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_COMPLETED, idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
 
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
@@ -2419,9 +2427,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+			pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_COMPLETED, idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 13024af2fa..7455d73d13 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1272,6 +1272,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3cb69b1f87..159605cb3f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1113,22 +1113,60 @@ CREATE VIEW pg_stat_progress_analyze AS
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
 CREATE VIEW pg_stat_progress_vacuum AS
-    SELECT
-        S.pid AS pid, S.datid AS datid, D.datname AS datname,
-        S.relid AS relid,
-        CASE S.param1 WHEN 0 THEN 'initializing'
-                      WHEN 1 THEN 'scanning heap'
-                      WHEN 2 THEN 'vacuuming indexes'
-                      WHEN 3 THEN 'vacuuming heap'
-                      WHEN 4 THEN 'cleaning up indexes'
-                      WHEN 5 THEN 'truncating heap'
-                      WHEN 6 THEN 'performing final cleanup'
-                      END AS phase,
-        S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
-    FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        SELECT
+            S.pid,
+            S.datid,
+            S.datname,
+            S.relid,
+            S.phase,
+            S.heap_blks_total,
+            S.heap_blks_scanned,
+            S.heap_blks_vacuumed,
+            S.index_vacuum_count,
+            S.max_dead_tuples,
+            S.num_dead_tuples,
+            S.index_total,
+            S.index_complete_count
+        FROM (SELECT
+                  S.pid AS pid, S.datid AS datid, D.datname AS datname,
+                  S.relid AS relid, S.param12 as leader_pid,
+                  CASE S.param1 WHEN 0 THEN 'initializing'
+                                WHEN 1 THEN 'scanning heap'
+                                WHEN 2 THEN 'vacuuming indexes'
+                                WHEN 3 THEN 'vacuuming heap'
+                                WHEN 4 THEN 'cleaning up indexes'
+                                WHEN 5 THEN 'truncating heap'
+                                WHEN 6 THEN 'performing final cleanup'
+                                END AS phase,
+                  S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
+                  S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+                  S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+                  S.param8 AS index_total,
+                  CASE WHEN S.param1 IN (2,4) AND S.param9 > 0
+                                              AND S.param10 > 0
+                                              AND S.param10 > (COUNT(*) OVER (PARTITION BY S.param12) - 1)
+                                              THEN S.param8 - (COUNT(*) OVER (PARTITION BY S.param12) - 1)
+                       ELSE SUM(S.param9) OVER(PARTITION BY S.param12)
+                       END AS index_complete_count
+              FROM pg_stat_get_progress_info('VACUUM') AS S
+                  LEFT JOIN pg_database D ON S.datid = D.oid) AS S
+        WHERE S.pid = S.leader_pid;
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+        SELECT
+            S.pid AS pid, S.datid AS datid,
+            D.datname AS datname,
+            S.param11 AS indexrelid,
+            CASE S.param1 WHEN 2 THEN 'vacuuming indexes'
+                          WHEN 4 THEN 'cleaning up indexes'
+                          END AS phase,
+            CASE WHEN S.param12 = s.pid THEN NULL
+                 ELSE S.param12
+                 END AS leader_pid,
+            S.param13 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM') AS S
+            LEFT JOIN pg_database D ON S.datid =D.oid
+        WHERE S.param1 IN (2, 4) AND S.param11 NOT IN (0);
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..eb4becf900 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,12 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID */
+	int leader_pid;
+
+	/* Number of planned parallel workers */
+	int parallel_workers;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -204,10 +211,10 @@ static int	parallel_vacuum_compute_workers(Relation *indrels, int nindexes, int
 											bool *will_parallel_vacuum);
 static void parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scans,
 												bool vacuum);
-static void parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs);
-static void parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs);
-static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-											  PVIndStats *indstats);
+static void parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs, int *idx_vacuumed);
+static void parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs, int *idx_vacuumed);
+static void	parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
+											  PVIndStats *indstats, int *idx_vacuumed);
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
@@ -259,6 +266,8 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		return NULL;
 	}
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_WORKERS_PLANNED, parallel_workers);
+
 	pvs = (ParallelVacuumState *) palloc0(sizeof(ParallelVacuumState));
 	pvs->indrels = indrels;
 	pvs->nindexes = nindexes;
@@ -357,6 +366,8 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->parallel_workers = parallel_workers;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -565,6 +576,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 {
 	int			nworkers;
 	PVIndVacStatus new_status;
+	int idx_vacuumed = 0;
 
 	Assert(!IsParallelWorker());
 
@@ -671,13 +683,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	}
 
 	/* Vacuum the indexes that can be processed by only leader process */
-	parallel_vacuum_process_unsafe_indexes(pvs);
+	parallel_vacuum_process_unsafe_indexes(pvs, &idx_vacuumed);
 
 	/*
 	 * Join as a parallel worker.  The leader vacuums alone processes all
 	 * parallel-safe indexes in the case where no workers are launched.
 	 */
-	parallel_vacuum_process_safe_indexes(pvs);
+	parallel_vacuum_process_safe_indexes(pvs, &idx_vacuumed);
 
 	/*
 	 * Next, accumulate buffer and WAL usage.  (This must wait for the workers
@@ -723,7 +735,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
  * vacuum worker processes to vacuum the indexes in parallel.
  */
 static void
-parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
+parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs, int *idx_vacuumed)
 {
 	/*
 	 * Increment the active worker count if we are able to launch any worker.
@@ -755,7 +767,8 @@ parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats, idx_vacuumed);
+
 	}
 
 	/*
@@ -777,7 +790,7 @@ parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
  * enforced by parallel_vacuum_compute_workers().
  */
 static void
-parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
+parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs, int *idx_vacuumed)
 {
 	Assert(!IsParallelWorker());
 
@@ -796,7 +809,8 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats, idx_vacuumed);
+
 	}
 
 	/*
@@ -805,6 +819,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
 	 */
 	if (VacuumActiveNWorkers)
 		pg_atomic_sub_fetch_u32(VacuumActiveNWorkers, 1);
+
 }
 
 /*
@@ -815,7 +830,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
  */
 static void
 parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-								  PVIndStats *indstats)
+								  PVIndStats *indstats, int *idx_vacuumed)
 {
 	IndexBulkDeleteResult *istat = NULL;
 	IndexBulkDeleteResult *istat_res;
@@ -840,13 +855,19 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			(*idx_vacuumed)++;
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			(*idx_vacuumed)++;
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
@@ -885,6 +906,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_COMPLETED, *idx_vacuumed);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
 }
 
 /*
@@ -943,6 +968,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	int			nindexes;
 	char	   *sharedquery;
 	ErrorContextCallback errcallback;
+	int			idx_vacuumed = 0;
 
 	/*
 	 * A parallel vacuum worker must have only PROC_IN_VACUUM flag since we
@@ -965,6 +991,9 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
+	pgstat_progress_update_param(PROGRESS_VACUUM_WORKERS_PLANNED, shared->parallel_workers);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -973,6 +1002,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_open_indexes(rel, RowExclusiveLock, &nindexes, &indrels);
 	Assert(nindexes > 0);
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, nindexes);
+
 	if (shared->maintenance_work_mem_worker > 0)
 		maintenance_work_mem = shared->maintenance_work_mem_worker;
 
@@ -1022,7 +1053,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	InstrStartParallelQuery();
 
 	/* Process indexes to perform vacuum/cleanup */
-	parallel_vacuum_process_safe_indexes(&pvs);
+	parallel_vacuum_process_safe_indexes(&pvs, &idx_vacuumed);
 
 	/* Report buffer/WAL usage during parallel execution */
 	buffer_usage = shm_toc_lookup(toc, PARALLEL_VACUUM_KEY_BUFFER_USAGE, false);
@@ -1036,6 +1067,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
+	pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..83e74e44b3 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,12 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_NUM_INDEX_COMPLETED		8
+#define PROGRESS_VACUUM_WORKERS_PLANNED			9
+#define PROGRESS_VACUUM_INDEXRELID				10
+#define PROGRESS_VACUUM_LEADER_PID				11
+#define PROGRESS_VACUUM_TUPLES_REMOVED          12
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d652f7b5fb..ca2e31cd90 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2007,26 +2007,63 @@ pg_stat_progress_create_index| SELECT s.pid,
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_progress_vacuum| SELECT s.pid,
     s.datid,
-    d.datname,
+    s.datname,
     s.relid,
+    s.phase,
+    s.heap_blks_total,
+    s.heap_blks_scanned,
+    s.heap_blks_vacuumed,
+    s.index_vacuum_count,
+    s.max_dead_tuples,
+    s.num_dead_tuples,
+    s.index_total,
+    s.index_complete_count
+   FROM ( SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.relid,
+            s_1.param12 AS leader_pid,
+                CASE s_1.param1
+                    WHEN 0 THEN 'initializing'::text
+                    WHEN 1 THEN 'scanning heap'::text
+                    WHEN 2 THEN 'vacuuming indexes'::text
+                    WHEN 3 THEN 'vacuuming heap'::text
+                    WHEN 4 THEN 'cleaning up indexes'::text
+                    WHEN 5 THEN 'truncating heap'::text
+                    WHEN 6 THEN 'performing final cleanup'::text
+                    ELSE NULL::text
+                END AS phase,
+            s_1.param2 AS heap_blks_total,
+            s_1.param3 AS heap_blks_scanned,
+            s_1.param4 AS heap_blks_vacuumed,
+            s_1.param5 AS index_vacuum_count,
+            s_1.param6 AS max_dead_tuples,
+            s_1.param7 AS num_dead_tuples,
+            s_1.param8 AS index_total,
+                CASE
+                    WHEN ((s_1.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s_1.param9 > 0) AND (s_1.param10 > 0) AND (s_1.param10 > (count(*) OVER (PARTITION BY s_1.param12) - 1))) THEN ((s_1.param8 - (count(*) OVER (PARTITION BY s_1.param12) - 1)))::numeric
+                    ELSE sum(s_1.param9) OVER (PARTITION BY s_1.param12)
+                END AS index_complete_count
+           FROM (pg_stat_get_progress_info('VACUUM'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))) s
+  WHERE (s.pid = s.leader_pid);
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    d.datname,
+    s.param11 AS indexrelid,
         CASE s.param1
-            WHEN 0 THEN 'initializing'::text
-            WHEN 1 THEN 'scanning heap'::text
             WHEN 2 THEN 'vacuuming indexes'::text
-            WHEN 3 THEN 'vacuuming heap'::text
             WHEN 4 THEN 'cleaning up indexes'::text
-            WHEN 5 THEN 'truncating heap'::text
-            WHEN 6 THEN 'performing final cleanup'::text
             ELSE NULL::text
         END AS phase,
-    s.param2 AS heap_blks_total,
-    s.param3 AS heap_blks_scanned,
-    s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
-    s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+        CASE
+            WHEN (s.param12 = s.pid) THEN NULL::bigint
+            ELSE s.param12
+        END AS leader_pid,
+    s.param13 AS tuples_removed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
-     LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s.param11 <> 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

#27Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#26)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

My apologies. The last attachment of documentation was the wrong file. Attached is the correct documentation file.

Thanks

On 1/26/22, 8:07 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

Attached is the latest patch and associated documentation.

This version addresses the index_ordinal_position column confusion. Rather than displaying the index position, the pg_stat_progress_vacuum view now has 2 new column(s):
index_total - this column will show the total number of indexes to be vacuumed
index_complete_count - this column will show the total number of indexes processed so far. In order to deal with the parallel vacuums, the parallel_workers ( planned workers ) value had to be exposed and each backends performing an index vacuum/cleanup in parallel had to advertise the number of indexes it vacuumed/cleaned. The # of indexes vacuumed for the parallel cleanup can then be derived the pg_stat_progress_vacuum view.

postgres=# \d pg_stat_progress_vacuum
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default
----------------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
relid | oid | | |
phase | text | | |
heap_blks_total | bigint | | |
heap_blks_scanned | bigint | | |
heap_blks_vacuumed | bigint | | |
index_vacuum_count | bigint | | |
max_dead_tuples | bigint | | |
num_dead_tuples | bigint | | |
index_total | bigint | | |. <<<---------------------
index_complete_count | numeric | | |. <<<---------------------

The pg_stat_progress_vacuum_index view includes:

Indexrelid - the currently vacuumed index
Leader_pid - the pid of the leader process. NULL if the process is the leader or vacuum is not parallel
tuples_removed - the amount of indexes tuples removed. The user can use this column to see that the index vacuum has movement.

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
phase | text | | |
leader_pid | bigint | | |
tuples_removed | bigint | | |

On 1/12/22, 9:52 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

On 1/12/22, 1:28 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

To add, since a vacuum can utilize parallel worker processes + the main vacuum process to perform index vacuuming, it made sense to separate the backends doing index vacuum/cleanup in a separate view.
Besides what Nathan suggested, the only other clean option I can think of is to perhaps create a json column in pg_stat_progress_vacuum which will include all the new fields. My concern with this approach is that it will make usability, to flatten the json, difficult for users.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

The main purpose is to really show that the "index vacuum" phase is actually making progress. Note that for certain types of indexes, i.e. GIN/GIST the number of tuples_removed will end up exceeding the number of num_dead_tuples.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

Attachments:

progress-reporting.htmltext/html; name=progress-reporting.htmlDownload
#28Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#26)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Resending patch as I see the last attachment was not annotated to the commitfest entry.

On 1/26/22, 8:07 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

Attached is the latest patch and associated documentation.

This version addresses the index_ordinal_position column confusion. Rather than displaying the index position, the pg_stat_progress_vacuum view now has 2 new column(s):
index_total - this column will show the total number of indexes to be vacuumed
index_complete_count - this column will show the total number of indexes processed so far. In order to deal with the parallel vacuums, the parallel_workers ( planned workers ) value had to be exposed and each backends performing an index vacuum/cleanup in parallel had to advertise the number of indexes it vacuumed/cleaned. The # of indexes vacuumed for the parallel cleanup can then be derived the pg_stat_progress_vacuum view.

postgres=# \d pg_stat_progress_vacuum
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default
----------------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
relid | oid | | |
phase | text | | |
heap_blks_total | bigint | | |
heap_blks_scanned | bigint | | |
heap_blks_vacuumed | bigint | | |
index_vacuum_count | bigint | | |
max_dead_tuples | bigint | | |
num_dead_tuples | bigint | | |
index_total | bigint | | |. <<<---------------------
index_complete_count | numeric | | |. <<<---------------------

The pg_stat_progress_vacuum_index view includes:

Indexrelid - the currently vacuumed index
Leader_pid - the pid of the leader process. NULL if the process is the leader or vacuum is not parallel
tuples_removed - the amount of indexes tuples removed. The user can use this column to see that the index vacuum has movement.

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
phase | text | | |
leader_pid | bigint | | |
tuples_removed | bigint | | |

On 1/12/22, 9:52 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

On 1/12/22, 1:28 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

To add, since a vacuum can utilize parallel worker processes + the main vacuum process to perform index vacuuming, it made sense to separate the backends doing index vacuum/cleanup in a separate view.
Besides what Nathan suggested, the only other clean option I can think of is to perhaps create a json column in pg_stat_progress_vacuum which will include all the new fields. My concern with this approach is that it will make usability, to flatten the json, difficult for users.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

The main purpose is to really show that the "index vacuum" phase is actually making progress. Note that for certain types of indexes, i.e. GIN/GIST the number of tuples_removed will end up exceeding the number of num_dead_tuples.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

Attachments:

0004-Expose-progress-for-the-vacuuming-indexes-cleanup-ph.patchapplication/octet-stream; name=0004-Expose-progress-for-the-vacuuming-indexes-cleanup-ph.patchDownload
From 33d4116a6b2ea4eef7d00ada279a329854514068 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)"
 <simseih@dev-dsk-simseih-1e-fba9b514.us-east-1.amazon.com>
Date: Thu, 27 Jan 2022 01:41:20 +0000
Subject: [PATCH 1/1] Expose progress for the "vacuuming indexes/cleanup" phase
 of a VACUUM operation.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 doc/src/sgml/monitoring.sgml          | 131 ++++++++++++++++++++++++++
 src/backend/access/gin/ginvacuum.c    |   4 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  11 +++
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  70 ++++++++++----
 src/backend/commands/vacuumparallel.c |  56 ++++++++---
 src/include/commands/progress.h       |   6 ++
 src/test/regress/expected/rules.out   |  63 ++++++++++---
 11 files changed, 309 insertions(+), 41 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 62f2a3332b..b2b33acb66 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend (including autovacuum worker processes) running
+       the <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase of a 
+       <command>VACUUM</command>, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6269,6 +6278,128 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>index_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The total number of indexes that will be processed by the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>index_complete_count</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The total number of indexes processed by the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> of the vacuum.
+       For vacuums with mutiple index vacuum cycles, this value will be set to <literal>0</literal>
+       at the start of each cycle.
+      </para></entry>
+     </row>
+    </tbody>
+   </tgroup>
+  </table>
+
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or 
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal> 
+       phase.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..73660719b5 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,8 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
+
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index d48c8a4549..84341aa7e7 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -631,6 +631,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 1749cc2a47..432229adea 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -340,6 +340,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
@@ -455,6 +456,9 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->FreezeLimit = FreezeLimit;
 	vacrel->MultiXactCutoff = MultiXactCutoff;
 
+	/* Advertise the total indexes to vacuum */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
+
 	/*
 	 * Call lazy_scan_heap to perform all required heap pruning, index
 	 * vacuuming, and heap vacuuming (plus related processing)
@@ -2010,9 +2014,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
+			pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_COMPLETED, idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
 
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
@@ -2419,9 +2427,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+			pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_COMPLETED, idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 13024af2fa..7455d73d13 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1272,6 +1272,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3cb69b1f87..159605cb3f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1113,22 +1113,60 @@ CREATE VIEW pg_stat_progress_analyze AS
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
 CREATE VIEW pg_stat_progress_vacuum AS
-    SELECT
-        S.pid AS pid, S.datid AS datid, D.datname AS datname,
-        S.relid AS relid,
-        CASE S.param1 WHEN 0 THEN 'initializing'
-                      WHEN 1 THEN 'scanning heap'
-                      WHEN 2 THEN 'vacuuming indexes'
-                      WHEN 3 THEN 'vacuuming heap'
-                      WHEN 4 THEN 'cleaning up indexes'
-                      WHEN 5 THEN 'truncating heap'
-                      WHEN 6 THEN 'performing final cleanup'
-                      END AS phase,
-        S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
-    FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        SELECT
+            S.pid,
+            S.datid,
+            S.datname,
+            S.relid,
+            S.phase,
+            S.heap_blks_total,
+            S.heap_blks_scanned,
+            S.heap_blks_vacuumed,
+            S.index_vacuum_count,
+            S.max_dead_tuples,
+            S.num_dead_tuples,
+            S.index_total,
+            S.index_complete_count
+        FROM (SELECT
+                  S.pid AS pid, S.datid AS datid, D.datname AS datname,
+                  S.relid AS relid, S.param12 as leader_pid,
+                  CASE S.param1 WHEN 0 THEN 'initializing'
+                                WHEN 1 THEN 'scanning heap'
+                                WHEN 2 THEN 'vacuuming indexes'
+                                WHEN 3 THEN 'vacuuming heap'
+                                WHEN 4 THEN 'cleaning up indexes'
+                                WHEN 5 THEN 'truncating heap'
+                                WHEN 6 THEN 'performing final cleanup'
+                                END AS phase,
+                  S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
+                  S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+                  S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+                  S.param8 AS index_total,
+                  CASE WHEN S.param1 IN (2,4) AND S.param9 > 0
+                                              AND S.param10 > 0
+                                              AND S.param10 > (COUNT(*) OVER (PARTITION BY S.param12) - 1)
+                                              THEN S.param8 - (COUNT(*) OVER (PARTITION BY S.param12) - 1)
+                       ELSE SUM(S.param9) OVER(PARTITION BY S.param12)
+                       END AS index_complete_count
+              FROM pg_stat_get_progress_info('VACUUM') AS S
+                  LEFT JOIN pg_database D ON S.datid = D.oid) AS S
+        WHERE S.pid = S.leader_pid;
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+        SELECT
+            S.pid AS pid, S.datid AS datid,
+            D.datname AS datname,
+            S.param11 AS indexrelid,
+            CASE S.param1 WHEN 2 THEN 'vacuuming indexes'
+                          WHEN 4 THEN 'cleaning up indexes'
+                          END AS phase,
+            CASE WHEN S.param12 = s.pid THEN NULL
+                 ELSE S.param12
+                 END AS leader_pid,
+            S.param13 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM') AS S
+            LEFT JOIN pg_database D ON S.datid =D.oid
+        WHERE S.param1 IN (2, 4) AND S.param11 NOT IN (0);
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..eb4becf900 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,12 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID */
+	int leader_pid;
+
+	/* Number of planned parallel workers */
+	int parallel_workers;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -204,10 +211,10 @@ static int	parallel_vacuum_compute_workers(Relation *indrels, int nindexes, int
 											bool *will_parallel_vacuum);
 static void parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scans,
 												bool vacuum);
-static void parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs);
-static void parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs);
-static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-											  PVIndStats *indstats);
+static void parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs, int *idx_vacuumed);
+static void parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs, int *idx_vacuumed);
+static void	parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
+											  PVIndStats *indstats, int *idx_vacuumed);
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
@@ -259,6 +266,8 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		return NULL;
 	}
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_WORKERS_PLANNED, parallel_workers);
+
 	pvs = (ParallelVacuumState *) palloc0(sizeof(ParallelVacuumState));
 	pvs->indrels = indrels;
 	pvs->nindexes = nindexes;
@@ -357,6 +366,8 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->parallel_workers = parallel_workers;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -565,6 +576,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 {
 	int			nworkers;
 	PVIndVacStatus new_status;
+	int idx_vacuumed = 0;
 
 	Assert(!IsParallelWorker());
 
@@ -671,13 +683,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	}
 
 	/* Vacuum the indexes that can be processed by only leader process */
-	parallel_vacuum_process_unsafe_indexes(pvs);
+	parallel_vacuum_process_unsafe_indexes(pvs, &idx_vacuumed);
 
 	/*
 	 * Join as a parallel worker.  The leader vacuums alone processes all
 	 * parallel-safe indexes in the case where no workers are launched.
 	 */
-	parallel_vacuum_process_safe_indexes(pvs);
+	parallel_vacuum_process_safe_indexes(pvs, &idx_vacuumed);
 
 	/*
 	 * Next, accumulate buffer and WAL usage.  (This must wait for the workers
@@ -723,7 +735,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
  * vacuum worker processes to vacuum the indexes in parallel.
  */
 static void
-parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
+parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs, int *idx_vacuumed)
 {
 	/*
 	 * Increment the active worker count if we are able to launch any worker.
@@ -755,7 +767,8 @@ parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[idx], indstats, idx_vacuumed);
+
 	}
 
 	/*
@@ -777,7 +790,7 @@ parallel_vacuum_process_safe_indexes(ParallelVacuumState *pvs)
  * enforced by parallel_vacuum_compute_workers().
  */
 static void
-parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
+parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs, int *idx_vacuumed)
 {
 	Assert(!IsParallelWorker());
 
@@ -796,7 +809,8 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
 			continue;
 
 		/* Do vacuum or cleanup of the index */
-		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats);
+		parallel_vacuum_process_one_index(pvs, pvs->indrels[i], indstats, idx_vacuumed);
+
 	}
 
 	/*
@@ -805,6 +819,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
 	 */
 	if (VacuumActiveNWorkers)
 		pg_atomic_sub_fetch_u32(VacuumActiveNWorkers, 1);
+
 }
 
 /*
@@ -815,7 +830,7 @@ parallel_vacuum_process_unsafe_indexes(ParallelVacuumState *pvs)
  */
 static void
 parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
-								  PVIndStats *indstats)
+								  PVIndStats *indstats, int *idx_vacuumed)
 {
 	IndexBulkDeleteResult *istat = NULL;
 	IndexBulkDeleteResult *istat_res;
@@ -840,13 +855,19 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			(*idx_vacuumed)++;
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			(*idx_vacuumed)++;
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
@@ -885,6 +906,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_COMPLETED, *idx_vacuumed);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
 }
 
 /*
@@ -943,6 +968,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	int			nindexes;
 	char	   *sharedquery;
 	ErrorContextCallback errcallback;
+	int			idx_vacuumed = 0;
 
 	/*
 	 * A parallel vacuum worker must have only PROC_IN_VACUUM flag since we
@@ -965,6 +991,9 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
+	pgstat_progress_update_param(PROGRESS_VACUUM_WORKERS_PLANNED, shared->parallel_workers);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -973,6 +1002,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_open_indexes(rel, RowExclusiveLock, &nindexes, &indrels);
 	Assert(nindexes > 0);
 
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, nindexes);
+
 	if (shared->maintenance_work_mem_worker > 0)
 		maintenance_work_mem = shared->maintenance_work_mem_worker;
 
@@ -1022,7 +1053,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	InstrStartParallelQuery();
 
 	/* Process indexes to perform vacuum/cleanup */
-	parallel_vacuum_process_safe_indexes(&pvs);
+	parallel_vacuum_process_safe_indexes(&pvs, &idx_vacuumed);
 
 	/* Report buffer/WAL usage during parallel execution */
 	buffer_usage = shm_toc_lookup(toc, PARALLEL_VACUUM_KEY_BUFFER_USAGE, false);
@@ -1036,6 +1067,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
+	pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..83e74e44b3 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,12 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_NUM_INDEX_COMPLETED		8
+#define PROGRESS_VACUUM_WORKERS_PLANNED			9
+#define PROGRESS_VACUUM_INDEXRELID				10
+#define PROGRESS_VACUUM_LEADER_PID				11
+#define PROGRESS_VACUUM_TUPLES_REMOVED          12
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d652f7b5fb..ca2e31cd90 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2007,26 +2007,63 @@ pg_stat_progress_create_index| SELECT s.pid,
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_progress_vacuum| SELECT s.pid,
     s.datid,
-    d.datname,
+    s.datname,
     s.relid,
+    s.phase,
+    s.heap_blks_total,
+    s.heap_blks_scanned,
+    s.heap_blks_vacuumed,
+    s.index_vacuum_count,
+    s.max_dead_tuples,
+    s.num_dead_tuples,
+    s.index_total,
+    s.index_complete_count
+   FROM ( SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.relid,
+            s_1.param12 AS leader_pid,
+                CASE s_1.param1
+                    WHEN 0 THEN 'initializing'::text
+                    WHEN 1 THEN 'scanning heap'::text
+                    WHEN 2 THEN 'vacuuming indexes'::text
+                    WHEN 3 THEN 'vacuuming heap'::text
+                    WHEN 4 THEN 'cleaning up indexes'::text
+                    WHEN 5 THEN 'truncating heap'::text
+                    WHEN 6 THEN 'performing final cleanup'::text
+                    ELSE NULL::text
+                END AS phase,
+            s_1.param2 AS heap_blks_total,
+            s_1.param3 AS heap_blks_scanned,
+            s_1.param4 AS heap_blks_vacuumed,
+            s_1.param5 AS index_vacuum_count,
+            s_1.param6 AS max_dead_tuples,
+            s_1.param7 AS num_dead_tuples,
+            s_1.param8 AS index_total,
+                CASE
+                    WHEN ((s_1.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s_1.param9 > 0) AND (s_1.param10 > 0) AND (s_1.param10 > (count(*) OVER (PARTITION BY s_1.param12) - 1))) THEN ((s_1.param8 - (count(*) OVER (PARTITION BY s_1.param12) - 1)))::numeric
+                    ELSE sum(s_1.param9) OVER (PARTITION BY s_1.param12)
+                END AS index_complete_count
+           FROM (pg_stat_get_progress_info('VACUUM'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))) s
+  WHERE (s.pid = s.leader_pid);
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    d.datname,
+    s.param11 AS indexrelid,
         CASE s.param1
-            WHEN 0 THEN 'initializing'::text
-            WHEN 1 THEN 'scanning heap'::text
             WHEN 2 THEN 'vacuuming indexes'::text
-            WHEN 3 THEN 'vacuuming heap'::text
             WHEN 4 THEN 'cleaning up indexes'::text
-            WHEN 5 THEN 'truncating heap'::text
-            WHEN 6 THEN 'performing final cleanup'::text
             ELSE NULL::text
         END AS phase,
-    s.param2 AS heap_blks_total,
-    s.param3 AS heap_blks_scanned,
-    s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
-    s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+        CASE
+            WHEN (s.param12 = s.pid) THEN NULL::bigint
+            ELSE s.param12
+        END AS leader_pid,
+    s.param13 AS tuples_removed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
-     LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s.param11 <> 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

#29Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#26)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

After speaking with Nathan offline, A few changes have been made to the patch.

As mentioned earlier in the thread, tracking how many indexes are processed in PARALLEL vacuum mode is not very straightforward since only the workers or leader process have ability to inspect the Vacuum shared parallel state.

The latest version of the patch introduces a shared memory to track indexes vacuumed/cleaned by each worker ( or leader ) in a PARALLEL vacuum. In order to present this data in the pg_stat_progress_vacuum view, the value of the new column "indexes_processed" is retrieved from shared memory by pg_stat_get_progress_info. For non-parallel vacuums, the value of "indexes_processed" is retrieved from the backend progress array directly.

The patch also includes the changes to implement the new view pg_stat_progress_vacuum_index which exposes the index being vacuumed/cleaned up.

postgres=# \d+ pg_stat_progress_vacuum ;
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default | Storage | Description
--------------------+---------+-----------+----------+---------+----------+-------------
pid | integer | | | | plain |
datid | oid | | | | plain |
datname | name | | | | plain |
relid | oid | | | | plain |
phase | text | | | | extended |
heap_blks_total | bigint | | | | plain |
heap_blks_scanned | bigint | | | | plain |
heap_blks_vacuumed | bigint | | | | plain |
index_vacuum_count | bigint | | | | plain |
max_dead_tuples | bigint | | | | plain |
num_dead_tuples | bigint | | | | plain |
indexes_total | bigint | | | | plain | <<<-- new column
indexes_processed | bigint | | | | plain | <<<-- new column

<<<--- new view --->>>

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
leader_pid | bigint | | |
phase | text | | |
tuples_removed | bigint | | |

On 1/26/22, 8:07 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

Attached is the latest patch and associated documentation.

This version addresses the index_ordinal_position column confusion. Rather than displaying the index position, the pg_stat_progress_vacuum view now has 2 new column(s):
index_total - this column will show the total number of indexes to be vacuumed
index_complete_count - this column will show the total number of indexes processed so far. In order to deal with the parallel vacuums, the parallel_workers ( planned workers ) value had to be exposed and each backends performing an index vacuum/cleanup in parallel had to advertise the number of indexes it vacuumed/cleaned. The # of indexes vacuumed for the parallel cleanup can then be derived the pg_stat_progress_vacuum view.

postgres=# \d pg_stat_progress_vacuum
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default
----------------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
relid | oid | | |
phase | text | | |
heap_blks_total | bigint | | |
heap_blks_scanned | bigint | | |
heap_blks_vacuumed | bigint | | |
index_vacuum_count | bigint | | |
max_dead_tuples | bigint | | |
num_dead_tuples | bigint | | |
index_total | bigint | | |. <<<---------------------
index_complete_count | numeric | | |. <<<---------------------

The pg_stat_progress_vacuum_index view includes:

Indexrelid - the currently vacuumed index
Leader_pid - the pid of the leader process. NULL if the process is the leader or vacuum is not parallel
tuples_removed - the amount of indexes tuples removed. The user can use this column to see that the index vacuum has movement.

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
phase | text | | |
leader_pid | bigint | | |
tuples_removed | bigint | | |

On 1/12/22, 9:52 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

On 1/12/22, 1:28 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

To add, since a vacuum can utilize parallel worker processes + the main vacuum process to perform index vacuuming, it made sense to separate the backends doing index vacuum/cleanup in a separate view.
Besides what Nathan suggested, the only other clean option I can think of is to perhaps create a json column in pg_stat_progress_vacuum which will include all the new fields. My concern with this approach is that it will make usability, to flatten the json, difficult for users.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

The main purpose is to really show that the "index vacuum" phase is actually making progress. Note that for certain types of indexes, i.e. GIN/GIST the number of tuples_removed will end up exceeding the number of num_dead_tuples.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

Attachments:

0005-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchapplication/octet-stream; name=0005-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchDownload
From f9c0fc69d99a8c681ee3326f12acbf752c66856c Mon Sep 17 00:00:00 2001
From: "Imseih (AWS), Sami" <simseih@amazon.com>
Date: Tue, 1 Feb 2022 20:12:18 +0000
Subject: [PATCH 1/1] Expose progress for the "vacuuming indexes" and "cleaning
   up indexes" phase of a VACUUM operation.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 doc/src/sgml/monitoring.sgml          | 121 +++++++++++++
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  | 236 ++++++++++++++++++++++++--
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  42 ++++-
 src/backend/commands/vacuumparallel.c |  25 +++
 src/backend/storage/ipc/ipci.c        |   3 +
 src/backend/utils/adt/pgstatfuncs.c   |  20 +++
 src/include/commands/progress.h       |   7 +
 src/include/commands/vacuum.h         |   8 +
 src/include/utils/backend_progress.h  |   1 +
 src/test/regress/expected/rules.out   |  38 ++++-
 15 files changed, 499 insertions(+), 14 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 62f2a3332b..0e2532effc 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6269,6 +6269,127 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes processed so far in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of the vacuum.
+      </para></entry>
+     </row>
+    </tbody>
+   </tgroup>
+  </table>
+
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index f09ad8f271..2d092f194b 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -239,6 +240,25 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/* Structs for tracking shared Progress information
+ * amongst worker ( and leader ) processes of a vacuum.
+ */
+typedef struct VacOneWorkerProgressInfo
+{
+	int                             leader_pid;
+	int                             indexes_processed;
+} VacOneWorkerProgressInfo;
+
+typedef struct VacWorkerProgressInfo
+{
+	int                     num_vacuums;    /* number of active VACUUMS with parallel workers */
+	int                     max_vacuums;    /* max number of VACUUMS with parallel workers */
+	slock_t			mutex;
+	VacOneWorkerProgressInfo vacuums[FLEXIBLE_ARRAY_MEMBER];
+} VacWorkerProgressInfo;
+
+static VacWorkerProgressInfo *vacworkerprogress;
+
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel, VacuumParams *params,
@@ -341,6 +361,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
@@ -404,6 +425,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2000,21 +2025,34 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
-
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		/* Report that we are now vacuuming indexes */
+		pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+									PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/* For the non-parallel variant of a vacuum, the array position
+			 * of the index determines how many indexes are processed so far.
+			 * Add 1 to the posititon as this is 0-based array.
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
+
+			/* Advertise we are done vacuuming indexes */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2025,9 +2063,28 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
+		/* Report that we are now vacuuming indexes in parallel
+		 * and Outsource everything to parallel variant.
+		 */
+		pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+									PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL);
+
+		/*
+		 * For the parallel variant of a vacuum, we will be populating shared memory
+		 * for the index completion progress. This is done with a call to
+		 * vacuum_worker_update inside vacuumparallel.c.
+		 *
+		 * Make sure we are properly cleaning up this shared memory on failure
+		 * or we will end up with a leak in the slots.
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
 											vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2405,33 +2462,55 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
-
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		double		reltuples = vacrel->new_rel_tuples;
 		bool		estimated_count =
 		vacrel->tupcount_pages < vacrel->rel_pages;
 
+		/* Report that we are now cleaning up indexes */
+		pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+									PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* See the lazy_vacuum_all_indexes comments */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
+
+			/* Advertise we are done cleaning indexes */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
+		/* Report that we are now cleaning up indexes in parallel
+		 * and Outsource everything to parallel variant.
+		 */
+		pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+									PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL);
+
+		/* See the lazy_vacuum_all_indexes comments */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
 											vacrel->num_index_scans,
 											(vacrel->tupcount_pages < vacrel->rel_pages));
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 	}
+
 }
 
 /*
@@ -3205,3 +3284,136 @@ restore_vacuum_error_info(LVRelState *vacrel,
 	vacrel->offnum = saved_vacrel->offnum;
 	vacrel->phase = saved_vacrel->phase;
 }
+
+void
+vacuum_worker_end(int leader_pid)
+{
+	SpinLockAcquire(&vacworkerprogress->mutex);
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		VacOneWorkerProgressInfo *vac = &vacworkerprogress->vacuums[i];
+
+		if (vac->leader_pid == leader_pid)
+		{
+			*vac = vacworkerprogress->vacuums[vacworkerprogress->num_vacuums - 1];
+			vacworkerprogress->num_vacuums--;
+			SpinLockRelease(&vacworkerprogress->mutex);
+			break;
+		}
+	}
+	SpinLockRelease(&vacworkerprogress->mutex);
+}
+
+/*
+ * vacuum_worker_end wrapped as an on_shmem_exit callback function
+ */
+void
+vacuum_worker_end_callback(int code, Datum arg)
+{
+	vacuum_worker_end(DatumGetInt32(arg));
+}
+
+/*
+ * vacuum_worker_update sets the number of indexes processed so far
+ * in a parallel vacuum. This routine can be
+ * expanded to other progress tracking amongst parallel
+ * workers ( and leader ).
+ */
+void
+vacuum_worker_update(int leader_pid)
+{
+	VacOneWorkerProgressInfo *vac;
+
+	SpinLockAcquire(&vacworkerprogress->mutex);
+
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		int next_leader_pid;
+
+		vac = &vacworkerprogress->vacuums[i];
+
+		next_leader_pid = vac->leader_pid;
+
+		if (next_leader_pid == leader_pid)
+		{
+			vac->indexes_processed++;
+			SpinLockRelease(&vacworkerprogress->mutex);
+			return;
+		}
+	}
+
+	if (vacworkerprogress->num_vacuums >= vacworkerprogress->max_vacuums)
+	{
+		SpinLockRelease(&vacworkerprogress->mutex);
+		elog(ERROR, "out of vacuum worker progress slots");
+	}
+
+	vac = &vacworkerprogress->vacuums[vacworkerprogress->num_vacuums];
+	vac->leader_pid = leader_pid;
+	vac->indexes_processed = 1;
+	vacworkerprogress->num_vacuums++;
+	SpinLockRelease(&vacworkerprogress->mutex);
+}
+
+/*
+ * vacuum_progress_cb updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+vacuum_progress_cb(Datum *values, int offset)
+{
+	VacOneWorkerProgressInfo *vac;
+	int leader_pid = values[0];
+
+	/* If we are vacuuming in parallel, set the number of indexes vacuumed
+	 * from the shared memory counter.
+	 * */
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		int next_leader_pid;
+
+		vac = &vacworkerprogress->vacuums[i];
+
+		next_leader_pid = vac->leader_pid;
+
+		if (next_leader_pid == leader_pid)
+			values[PROGRESS_VACUUM_INDEXES_COMPLETED + offset] = vac->indexes_processed;
+	}
+}
+
+/*
+ * VacuumWorkerProgressShmemSize --- report amount of shared memory space needed
+ */
+Size
+VacuumWorkerProgressShmemSize(void)
+{
+	Size            size;
+
+	size = offsetof(VacWorkerProgressInfo, vacuums);
+	size = add_size(size, mul_size(MaxBackends, sizeof(VacOneWorkerProgressInfo)));
+	return size;
+}
+
+/*
+ * VacuumWorkerProgressShmemInit --- initialize this module's shared memory
+ */
+void
+VacuumWorkerProgressShmemInit(void)
+{
+	bool            found;
+
+	vacworkerprogress = (VacWorkerProgressInfo *) ShmemInitStruct("Vacuum Worker Progress Stats",
+										VacuumWorkerProgressShmemSize(),
+										&found);
+
+	if (!IsUnderPostmaster)
+	{
+		/* Initialize shared memory area */
+		Assert(!found);
+
+		vacworkerprogress->max_vacuums = MaxBackends;
+		SpinLockInit(&vacworkerprogress->mutex);
+	}
+	else
+		Assert(found);
+}
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3cb69b1f87..0f91f66c73 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1123,13 +1123,53 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 4 THEN 'cleaning up indexes'
                       WHEN 5 THEN 'truncating heap'
                       WHEN 6 THEN 'performing final cleanup'
+                      WHEN 7 THEN 'vacuuming indexes'
+                      WHEN 8 THEN 'cleaning up indexes'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid,
+        S.datid,
+        S.datname,
+        S.indexrelid,
+        S.leader_pid,
+        CASE S.phase WHEN 7 THEN 'vacuuming indexes'
+                     WHEN 8 THEN 'cleaning up indexes'
+                     END AS phase,
+        S.tuples_removed
+    FROM (
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        WHERE S.param1 IN (2, 4, 7, 8) AND S.param10 > 0
+        UNION ALL
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM_PARALLEL') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        ) AS S
+    WHERE S.phase IN (2, 4, 7, 8) AND S.indexrelid > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..1332203047 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int		 leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -716,6 +721,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 		VacuumSharedCostBalance = NULL;
 		VacuumActiveNWorkers = NULL;
 	}
+
 }
 
 /*
@@ -840,13 +846,24 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			/* Report that we are now vacuuming indexes in parallel */
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+										PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			/* Report that we are now cleaning indexes in parallel */
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+										PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
@@ -881,6 +898,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -965,6 +986,9 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL,
+								RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1036,6 +1060,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
+	pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index 9f26e41c46..8682578ac3 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -23,6 +23,7 @@
 #include "access/syncscan.h"
 #include "access/twophase.h"
 #include "commands/async.h"
+#include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -143,6 +144,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, VacuumWorkerProgressShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -293,6 +295,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	VacuumWorkerProgressShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 15cb17ace4..5dece71596 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,7 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/vacuum.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -452,6 +453,14 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 
 /*
  * Returns command progress information for the named command.
+ *
+ * A command type can optionally define a callback function
+ * which will derive Datum values rather than use values
+ * directly from the backends progress array.
+ *
+ * Derived values are useful to calculate values form multiple backends
+ * as is the case with parallel operations, in which progress values
+ * are calculated form multiple workers.
  */
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
@@ -466,6 +475,7 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	MemoryContext per_query_ctx;
 	MemoryContext oldcontext;
+	void (*callback)(Datum *, int) = NULL;
 
 	/* check to see if caller supports us returning a tuplestore */
 	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
@@ -483,7 +493,14 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 
 	/* Translate command name into command type code. */
 	if (pg_strcasecmp(cmd, "VACUUM") == 0)
+	{
 		cmdtype = PROGRESS_COMMAND_VACUUM;
+		callback = vacuum_progress_cb;
+	}
+	else if (pg_strcasecmp(cmd, "VACUUM_PARALLEL") == 0)
+	{
+		cmdtype = PROGRESS_COMMAND_VACUUM_PARALLEL;
+	}
 	else if (pg_strcasecmp(cmd, "ANALYZE") == 0)
 		cmdtype = PROGRESS_COMMAND_ANALYZE;
 	else if (pg_strcasecmp(cmd, "CLUSTER") == 0)
@@ -552,6 +569,9 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 				nulls[i + 3] = true;
 		}
 
+		if (callback)
+			callback(values, 3);
+
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..37659882ee 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,11 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED		8
+#define PROGRESS_VACUUM_INDEXRELID			9
+#define PROGRESS_VACUUM_TUPLES_REMOVED			10
+#define PROGRESS_VACUUM_LEADER_PID			11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
@@ -33,6 +38,8 @@
 #define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP		4
 #define PROGRESS_VACUUM_PHASE_TRUNCATE			5
 #define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP		6
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL	7
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL	8
 
 /* Progress parameters for analyze */
 #define PROGRESS_ANALYZE_PHASE						0
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 5d0bdfa427..119924694d 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -336,4 +336,12 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+/* in commands/vacuumparallel.c */
+extern Size VacuumWorkerProgressShmemSize(void);
+extern void VacuumWorkerProgressShmemInit(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_progress_cb(Datum *values, int offset);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..4651e45c40 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -23,6 +23,7 @@ typedef enum ProgressCommandType
 {
 	PROGRESS_COMMAND_INVALID,
 	PROGRESS_COMMAND_VACUUM,
+	PROGRESS_COMMAND_VACUUM_PARALLEL,
 	PROGRESS_COMMAND_ANALYZE,
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d652f7b5fb..0fafe60e53 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2017,6 +2017,8 @@ pg_stat_progress_vacuum| SELECT s.pid,
             WHEN 4 THEN 'cleaning up indexes'::text
             WHEN 5 THEN 'truncating heap'::text
             WHEN 6 THEN 'performing final cleanup'::text
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
             ELSE NULL::text
         END AS phase,
     s.param2 AS heap_blks_total,
@@ -2024,9 +2026,43 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    s.datname,
+    s.indexrelid,
+    s.leader_pid,
+        CASE s.phase
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.tuples_removed
+   FROM ( SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))
+          WHERE ((s_1.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s_1.param10 > 0))
+        UNION ALL
+         SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM_PARALLEL'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))) s
+  WHERE ((s.phase = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s.indexrelid > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

#30Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#29)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Attached is the latest version of the patch to deal with the changes in the recent commit aa64f23b02924724eafbd9eadbf26d85df30a12b

On 2/1/22, 2:32 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

After speaking with Nathan offline, A few changes have been made to the patch.

As mentioned earlier in the thread, tracking how many indexes are processed in PARALLEL vacuum mode is not very straightforward since only the workers or leader process have ability to inspect the Vacuum shared parallel state.

The latest version of the patch introduces a shared memory to track indexes vacuumed/cleaned by each worker ( or leader ) in a PARALLEL vacuum. In order to present this data in the pg_stat_progress_vacuum view, the value of the new column "indexes_processed" is retrieved from shared memory by pg_stat_get_progress_info. For non-parallel vacuums, the value of "indexes_processed" is retrieved from the backend progress array directly.

The patch also includes the changes to implement the new view pg_stat_progress_vacuum_index which exposes the index being vacuumed/cleaned up.

postgres=# \d+ pg_stat_progress_vacuum ;
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default | Storage | Description
--------------------+---------+-----------+----------+---------+----------+-------------
pid | integer | | | | plain |
datid | oid | | | | plain |
datname | name | | | | plain |
relid | oid | | | | plain |
phase | text | | | | extended |
heap_blks_total | bigint | | | | plain |
heap_blks_scanned | bigint | | | | plain |
heap_blks_vacuumed | bigint | | | | plain |
index_vacuum_count | bigint | | | | plain |
max_dead_tuples | bigint | | | | plain |
num_dead_tuples | bigint | | | | plain |
indexes_total | bigint | | | | plain | <<<-- new column
indexes_processed | bigint | | | | plain | <<<-- new column

<<<--- new view --->>>

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
leader_pid | bigint | | |
phase | text | | |
tuples_removed | bigint | | |

On 1/26/22, 8:07 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

Attached is the latest patch and associated documentation.

This version addresses the index_ordinal_position column confusion. Rather than displaying the index position, the pg_stat_progress_vacuum view now has 2 new column(s):
index_total - this column will show the total number of indexes to be vacuumed
index_complete_count - this column will show the total number of indexes processed so far. In order to deal with the parallel vacuums, the parallel_workers ( planned workers ) value had to be exposed and each backends performing an index vacuum/cleanup in parallel had to advertise the number of indexes it vacuumed/cleaned. The # of indexes vacuumed for the parallel cleanup can then be derived the pg_stat_progress_vacuum view.

postgres=# \d pg_stat_progress_vacuum
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default
----------------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
relid | oid | | |
phase | text | | |
heap_blks_total | bigint | | |
heap_blks_scanned | bigint | | |
heap_blks_vacuumed | bigint | | |
index_vacuum_count | bigint | | |
max_dead_tuples | bigint | | |
num_dead_tuples | bigint | | |
index_total | bigint | | |. <<<---------------------
index_complete_count | numeric | | |. <<<---------------------

The pg_stat_progress_vacuum_index view includes:

Indexrelid - the currently vacuumed index
Leader_pid - the pid of the leader process. NULL if the process is the leader or vacuum is not parallel
tuples_removed - the amount of indexes tuples removed. The user can use this column to see that the index vacuum has movement.

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
phase | text | | |
leader_pid | bigint | | |
tuples_removed | bigint | | |

On 1/12/22, 9:52 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

On 1/12/22, 1:28 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

To add, since a vacuum can utilize parallel worker processes + the main vacuum process to perform index vacuuming, it made sense to separate the backends doing index vacuum/cleanup in a separate view.
Besides what Nathan suggested, the only other clean option I can think of is to perhaps create a json column in pg_stat_progress_vacuum which will include all the new fields. My concern with this approach is that it will make usability, to flatten the json, difficult for users.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

The main purpose is to really show that the "index vacuum" phase is actually making progress. Note that for certain types of indexes, i.e. GIN/GIST the number of tuples_removed will end up exceeding the number of num_dead_tuples.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

Attachments:

0006-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchapplication/octet-stream; name=0006-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchDownload
From 4460de90eda020c5e49f0ba264282a2a4bd6565d Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 10 Feb 2022 19:28:27 +0000
Subject: [PATCH 1/1] Expose progress for the "vacuuming indexes" and "cleaning
 up indexes" phase of a VACUUM operation.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 doc/src/sgml/monitoring.sgml          | 121 +++++++++++++
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  | 236 ++++++++++++++++++++++++--
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  42 ++++-
 src/backend/commands/vacuumparallel.c |  25 +++
 src/backend/storage/ipc/ipci.c        |   3 +
 src/backend/utils/adt/pgstatfuncs.c   |  20 +++
 src/include/commands/progress.h       |   7 +
 src/include/commands/vacuum.h         |   8 +
 src/include/utils/backend_progress.h  |   1 +
 src/test/regress/expected/rules.out   |  38 ++++-
 15 files changed, 499 insertions(+), 14 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 62f2a3332b..0e2532effc 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6269,6 +6269,127 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes processed so far in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of the vacuum.
+      </para></entry>
+     </row>
+    </tbody>
+   </tgroup>
+  </table>
+
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index f09ad8f271..fb8c9b1030 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -239,6 +240,25 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/* Structs for tracking shared Progress information
+ * amongst worker ( and leader ) processes of a vacuum.
+ */
+typedef struct VacOneWorkerProgressInfo
+{
+	int                             leader_pid;
+	int                             indexes_processed;
+} VacOneWorkerProgressInfo;
+
+typedef struct VacWorkerProgressInfo
+{
+	int                     num_vacuums;    /* number of active VACUUMS with parallel workers */
+	int                     max_vacuums;    /* max number of VACUUMS with parallel workers */
+	slock_t			mutex;
+	VacOneWorkerProgressInfo vacuums[FLEXIBLE_ARRAY_MEMBER];
+} VacWorkerProgressInfo;
+
+static VacWorkerProgressInfo *vacworkerprogress;
+
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel, VacuumParams *params,
@@ -341,6 +361,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	vacuum_set_xid_limits(rel,
 						  params->freeze_min_age,
@@ -404,6 +425,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2000,21 +2025,34 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
-
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		/* Report that we are now vacuuming indexes */
+		pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+									PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/* For the non-parallel variant of a vacuum, the array position
+			 * of the index determines how many indexes are processed so far.
+			 * Add 1 to the posititon as this is 0-based array.
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
+
+			/* Advertise we are done vacuuming indexes */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2025,9 +2063,28 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
+		/* Report that we are now vacuuming indexes in parallel
+		 * and Outsource everything to parallel variant.
+		 */
+		pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+									PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL);
+
+		/*
+		 * For the parallel variant of a vacuum, we will be populating shared memory
+		 * for the index completion progress. This is done with a call to
+		 * vacuum_worker_update inside vacuumparallel.c.
+		 *
+		 * Make sure we are properly cleaning up this shared memory on failure
+		 * or we will end up with a leak in the slots.
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
 											vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2405,33 +2462,55 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
-
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		double		reltuples = vacrel->new_rel_tuples;
 		bool		estimated_count =
 		vacrel->tupcount_pages < vacrel->rel_pages;
 
+		/* Report that we are now cleaning up indexes */
+		pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+									PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* See the lazy_vacuum_all_indexes comments */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
+
+			/* Advertise we are done cleaning indexes */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
+		/* Report that we are now cleaning up indexes in parallel
+		 * and Outsource everything to parallel variant.
+		 */
+		pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+									PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL);
+
+		/* See the lazy_vacuum_all_indexes comments */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
 											vacrel->num_index_scans,
 											(vacrel->tupcount_pages < vacrel->rel_pages));
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 	}
+
 }
 
 /*
@@ -3205,3 +3284,136 @@ restore_vacuum_error_info(LVRelState *vacrel,
 	vacrel->offnum = saved_vacrel->offnum;
 	vacrel->phase = saved_vacrel->phase;
 }
+
+void
+vacuum_worker_end(int leader_pid)
+{
+	SpinLockAcquire(&vacworkerprogress->mutex);
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		VacOneWorkerProgressInfo *vac = &vacworkerprogress->vacuums[i];
+
+		if (vac->leader_pid == leader_pid)
+		{
+			*vac = vacworkerprogress->vacuums[vacworkerprogress->num_vacuums - 1];
+			vacworkerprogress->num_vacuums--;
+			SpinLockRelease(&vacworkerprogress->mutex);
+			break;
+		}
+	}
+	SpinLockRelease(&vacworkerprogress->mutex);
+}
+
+/*
+ * vacuum_worker_end wrapped as an on_shmem_exit callback function
+ */
+void
+vacuum_worker_end_callback(int code, Datum arg)
+{
+	vacuum_worker_end(DatumGetInt32(arg));
+}
+
+/*
+ * vacuum_worker_update sets the number of indexes processed so far
+ * in a parallel vacuum. This routine can be
+ * expanded to other progress tracking amongst parallel
+ * workers ( and leader ).
+ */
+void
+vacuum_worker_update(int leader_pid)
+{
+	VacOneWorkerProgressInfo *vac;
+
+	SpinLockAcquire(&vacworkerprogress->mutex);
+
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		int next_leader_pid;
+
+		vac = &vacworkerprogress->vacuums[i];
+
+		next_leader_pid = vac->leader_pid;
+
+		if (next_leader_pid == leader_pid)
+		{
+			vac->indexes_processed++;
+			SpinLockRelease(&vacworkerprogress->mutex);
+			return;
+		}
+	}
+
+	if (vacworkerprogress->num_vacuums >= vacworkerprogress->max_vacuums)
+	{
+		SpinLockRelease(&vacworkerprogress->mutex);
+		elog(ERROR, "out of vacuum worker progress slots");
+	}
+
+	vac = &vacworkerprogress->vacuums[vacworkerprogress->num_vacuums];
+	vac->leader_pid = leader_pid;
+	vac->indexes_processed = 1;
+	vacworkerprogress->num_vacuums++;
+	SpinLockRelease(&vacworkerprogress->mutex);
+}
+
+/*
+ * vacuum_progress_cb updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+vacuum_progress_cb(Datum *values, int offset)
+{
+	VacOneWorkerProgressInfo *vac;
+	int leader_pid = values[0];
+
+	/* If we are vacuuming in parallel, set the number of indexes vacuumed
+	 * from the shared memory counter.
+	 * */
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		int next_leader_pid;
+
+		vac = &vacworkerprogress->vacuums[i];
+
+		next_leader_pid = vac->leader_pid;
+
+		if (next_leader_pid == leader_pid)
+			values[PROGRESS_VACUUM_INDEXES_COMPLETED + offset] = vac->indexes_processed;
+	}
+}
+
+/*
+ * VacuumWorkerProgressShmemSize --- report amount of shared memory space needed
+ */
+Size
+VacuumWorkerProgressShmemSize(void)
+{
+	Size            size;
+
+	size = offsetof(VacWorkerProgressInfo, vacuums);
+	size = add_size(size, mul_size(GetMaxBackends(), sizeof(VacOneWorkerProgressInfo)));
+	return size;
+}
+
+/*
+ * VacuumWorkerProgressShmemInit --- initialize this module's shared memory
+ */
+void
+VacuumWorkerProgressShmemInit(void)
+{
+	bool            found;
+
+	vacworkerprogress = (VacWorkerProgressInfo *) ShmemInitStruct("Vacuum Worker Progress Stats",
+										VacuumWorkerProgressShmemSize(),
+										&found);
+
+	if (!IsUnderPostmaster)
+	{
+		/* Initialize shared memory area */
+		Assert(!found);
+
+		vacworkerprogress->max_vacuums = GetMaxBackends();
+		SpinLockInit(&vacworkerprogress->mutex);
+	}
+	else
+		Assert(found);
+}
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3cb69b1f87..0f91f66c73 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1123,13 +1123,53 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 4 THEN 'cleaning up indexes'
                       WHEN 5 THEN 'truncating heap'
                       WHEN 6 THEN 'performing final cleanup'
+                      WHEN 7 THEN 'vacuuming indexes'
+                      WHEN 8 THEN 'cleaning up indexes'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid,
+        S.datid,
+        S.datname,
+        S.indexrelid,
+        S.leader_pid,
+        CASE S.phase WHEN 7 THEN 'vacuuming indexes'
+                     WHEN 8 THEN 'cleaning up indexes'
+                     END AS phase,
+        S.tuples_removed
+    FROM (
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        WHERE S.param1 IN (2, 4, 7, 8) AND S.param10 > 0
+        UNION ALL
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM_PARALLEL') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        ) AS S
+    WHERE S.phase IN (2, 4, 7, 8) AND S.indexrelid > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..1332203047 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int		 leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -716,6 +721,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 		VacuumSharedCostBalance = NULL;
 		VacuumActiveNWorkers = NULL;
 	}
+
 }
 
 /*
@@ -840,13 +846,24 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			/* Report that we are now vacuuming indexes in parallel */
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+										PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			/* Report that we are now cleaning indexes in parallel */
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
+										PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
@@ -881,6 +898,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -965,6 +986,9 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL,
+								RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1036,6 +1060,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
+	pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index 9f26e41c46..8682578ac3 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -23,6 +23,7 @@
 #include "access/syncscan.h"
 #include "access/twophase.h"
 #include "commands/async.h"
+#include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -143,6 +144,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, VacuumWorkerProgressShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -293,6 +295,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	VacuumWorkerProgressShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 15cb17ace4..5dece71596 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,7 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/vacuum.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -452,6 +453,14 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 
 /*
  * Returns command progress information for the named command.
+ *
+ * A command type can optionally define a callback function
+ * which will derive Datum values rather than use values
+ * directly from the backends progress array.
+ *
+ * Derived values are useful to calculate values form multiple backends
+ * as is the case with parallel operations, in which progress values
+ * are calculated form multiple workers.
  */
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
@@ -466,6 +475,7 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	MemoryContext per_query_ctx;
 	MemoryContext oldcontext;
+	void (*callback)(Datum *, int) = NULL;
 
 	/* check to see if caller supports us returning a tuplestore */
 	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
@@ -483,7 +493,14 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 
 	/* Translate command name into command type code. */
 	if (pg_strcasecmp(cmd, "VACUUM") == 0)
+	{
 		cmdtype = PROGRESS_COMMAND_VACUUM;
+		callback = vacuum_progress_cb;
+	}
+	else if (pg_strcasecmp(cmd, "VACUUM_PARALLEL") == 0)
+	{
+		cmdtype = PROGRESS_COMMAND_VACUUM_PARALLEL;
+	}
 	else if (pg_strcasecmp(cmd, "ANALYZE") == 0)
 		cmdtype = PROGRESS_COMMAND_ANALYZE;
 	else if (pg_strcasecmp(cmd, "CLUSTER") == 0)
@@ -552,6 +569,9 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 				nulls[i + 3] = true;
 		}
 
+		if (callback)
+			callback(values, 3);
+
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..37659882ee 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,11 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED		8
+#define PROGRESS_VACUUM_INDEXRELID			9
+#define PROGRESS_VACUUM_TUPLES_REMOVED			10
+#define PROGRESS_VACUUM_LEADER_PID			11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
@@ -33,6 +38,8 @@
 #define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP		4
 #define PROGRESS_VACUUM_PHASE_TRUNCATE			5
 #define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP		6
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL	7
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL	8
 
 /* Progress parameters for analyze */
 #define PROGRESS_ANALYZE_PHASE						0
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 5d0bdfa427..119924694d 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -336,4 +336,12 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+/* in commands/vacuumparallel.c */
+extern Size VacuumWorkerProgressShmemSize(void);
+extern void VacuumWorkerProgressShmemInit(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_progress_cb(Datum *values, int offset);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..4651e45c40 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -23,6 +23,7 @@ typedef enum ProgressCommandType
 {
 	PROGRESS_COMMAND_INVALID,
 	PROGRESS_COMMAND_VACUUM,
+	PROGRESS_COMMAND_VACUUM_PARALLEL,
 	PROGRESS_COMMAND_ANALYZE,
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 1420288d67..35193dc6b7 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1995,6 +1995,8 @@ pg_stat_progress_vacuum| SELECT s.pid,
             WHEN 4 THEN 'cleaning up indexes'::text
             WHEN 5 THEN 'truncating heap'::text
             WHEN 6 THEN 'performing final cleanup'::text
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
             ELSE NULL::text
         END AS phase,
     s.param2 AS heap_blks_total,
@@ -2002,9 +2004,43 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    s.datname,
+    s.indexrelid,
+    s.leader_pid,
+        CASE s.phase
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.tuples_removed
+   FROM ( SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))
+          WHERE ((s_1.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s_1.param10 > 0))
+        UNION ALL
+         SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM_PARALLEL'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))) s
+  WHERE ((s.phase = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s.indexrelid > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

#31Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#30)
3 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

The change has been broken up as 3 separate patches.

0007-Expose-progress-for-the-vacuuming-indexes-and-cleani.patch - Introduces 2 new columns to pg_stat_progress_vacuum, indexes_total and indexes_processed. These 2 columns will provide progress on the index vacuuming/cleanup.
0001-Expose-the-index-being-processed-in-the-vacuuming-in.patch - Introduces a new view called pg_stat_prgoress_vacuum_index. This view tracks the index being vacuumed/cleaned and the total number of index tuples removed.
0001-Rename-index_vacuum_count-to-index_vacuum_cycle_coun.patch - Renames the existing index_vacuum_count to index_vacuum_cycle_count in pg_stat_progress_vacuum. Due to the other changes, it makes sense to include "cycle" in the column name to be crystal clear that the column refers to the index cycle count.

Thanks

On 2/10/22, 1:39 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

Attached is the latest version of the patch to deal with the changes in the recent commit aa64f23b02924724eafbd9eadbf26d85df30a12b

On 2/1/22, 2:32 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

After speaking with Nathan offline, A few changes have been made to the patch.

As mentioned earlier in the thread, tracking how many indexes are processed in PARALLEL vacuum mode is not very straightforward since only the workers or leader process have ability to inspect the Vacuum shared parallel state.

The latest version of the patch introduces a shared memory to track indexes vacuumed/cleaned by each worker ( or leader ) in a PARALLEL vacuum. In order to present this data in the pg_stat_progress_vacuum view, the value of the new column "indexes_processed" is retrieved from shared memory by pg_stat_get_progress_info. For non-parallel vacuums, the value of "indexes_processed" is retrieved from the backend progress array directly.

The patch also includes the changes to implement the new view pg_stat_progress_vacuum_index which exposes the index being vacuumed/cleaned up.

postgres=# \d+ pg_stat_progress_vacuum ;
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default | Storage | Description
--------------------+---------+-----------+----------+---------+----------+-------------
pid | integer | | | | plain |
datid | oid | | | | plain |
datname | name | | | | plain |
relid | oid | | | | plain |
phase | text | | | | extended |
heap_blks_total | bigint | | | | plain |
heap_blks_scanned | bigint | | | | plain |
heap_blks_vacuumed | bigint | | | | plain |
index_vacuum_count | bigint | | | | plain |
max_dead_tuples | bigint | | | | plain |
num_dead_tuples | bigint | | | | plain |
indexes_total | bigint | | | | plain | <<<-- new column
indexes_processed | bigint | | | | plain | <<<-- new column

<<<--- new view --->>>

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
leader_pid | bigint | | |
phase | text | | |
tuples_removed | bigint | | |

On 1/26/22, 8:07 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

Attached is the latest patch and associated documentation.

This version addresses the index_ordinal_position column confusion. Rather than displaying the index position, the pg_stat_progress_vacuum view now has 2 new column(s):
index_total - this column will show the total number of indexes to be vacuumed
index_complete_count - this column will show the total number of indexes processed so far. In order to deal with the parallel vacuums, the parallel_workers ( planned workers ) value had to be exposed and each backends performing an index vacuum/cleanup in parallel had to advertise the number of indexes it vacuumed/cleaned. The # of indexes vacuumed for the parallel cleanup can then be derived the pg_stat_progress_vacuum view.

postgres=# \d pg_stat_progress_vacuum
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default
----------------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
relid | oid | | |
phase | text | | |
heap_blks_total | bigint | | |
heap_blks_scanned | bigint | | |
heap_blks_vacuumed | bigint | | |
index_vacuum_count | bigint | | |
max_dead_tuples | bigint | | |
num_dead_tuples | bigint | | |
index_total | bigint | | |. <<<---------------------
index_complete_count | numeric | | |. <<<---------------------

The pg_stat_progress_vacuum_index view includes:

Indexrelid - the currently vacuumed index
Leader_pid - the pid of the leader process. NULL if the process is the leader or vacuum is not parallel
tuples_removed - the amount of indexes tuples removed. The user can use this column to see that the index vacuum has movement.

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
phase | text | | |
leader_pid | bigint | | |
tuples_removed | bigint | | |

On 1/12/22, 9:52 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

On 1/12/22, 1:28 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

To add, since a vacuum can utilize parallel worker processes + the main vacuum process to perform index vacuuming, it made sense to separate the backends doing index vacuum/cleanup in a separate view.
Besides what Nathan suggested, the only other clean option I can think of is to perhaps create a json column in pg_stat_progress_vacuum which will include all the new fields. My concern with this approach is that it will make usability, to flatten the json, difficult for users.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

The main purpose is to really show that the "index vacuum" phase is actually making progress. Note that for certain types of indexes, i.e. GIN/GIST the number of tuples_removed will end up exceeding the number of num_dead_tuples.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

Attachments:

0007-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchapplication/octet-stream; name=0007-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchDownload
From 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 17 Feb 2022 04:21:04 +0000
Subject: [PATCH 1/1] Expose progress for the "vacuuming indexes" and "cleaning
 up indexes" phase of a VACUUM operation.

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 doc/src/sgml/monitoring.sgml          |  22 +++
 src/backend/access/heap/vacuumlazy.c  | 207 +++++++++++++++++++++++++-
 src/backend/catalog/system_views.sql  |   3 +-
 src/backend/commands/vacuumparallel.c |   7 +
 src/backend/storage/ipc/ipci.c        |   3 +
 src/backend/utils/adt/pgstatfuncs.c   |  12 ++
 src/include/commands/progress.h       |   2 +
 src/include/commands/vacuum.h         |   8 +
 src/test/regress/expected/rules.out   |   4 +-
 9 files changed, 261 insertions(+), 7 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index bf7625d988..04440dfa88 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6278,6 +6278,28 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes processed in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase.
+       At the start of an index vacuum cycle, this value is set to <literal>0</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 242511a235..1198677bc0 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -245,6 +246,26 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/*
+ * Structs for tracking shared Progress information
+ * amongst worker ( and leader ) processes of a vacuum.
+ */
+typedef struct VacOneWorkerProgressInfo
+{
+	int                             leader_pid;
+	int                             indexes_processed;
+} VacOneWorkerProgressInfo;
+
+typedef struct VacWorkerProgressInfo
+{
+	int                     num_vacuums;    /* number of active VACUUMS with parallel workers */
+	int                     max_vacuums;    /* max number of VACUUMS with parallel workers */
+	slock_t			mutex;
+	VacOneWorkerProgressInfo vacuums[FLEXIBLE_ARRAY_MEMBER];
+} VacWorkerProgressInfo;
+
+static VacWorkerProgressInfo *vacworkerprogress;
+
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel, int nworkers);
@@ -420,6 +441,8 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2328,6 +2351,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/*
+			 * For the non-parallel variant of a vacuum, the array position
+			 * of the index determines how many indexes are processed so far.
+			 * Add 1 to the posititon as vacrel->nindexes is a 0-based array.
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2338,9 +2368,20 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * parallel_vacuum_bulkdel_all_indexes will call vacuum_worker_update
+		 * which updates shared memory for the index progress. To ensure shared
+		 * memory cleanup, do the work with PG_ENSURE_ERROR_CLEANUP.
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs,
+												vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2350,6 +2391,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			allindexes = false;
 	}
 
+	/*
+	 * We're done with index vacuuming.
+	 * Set the total number of indexes completed as the total number of indexes
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, vacrel->nindexes) ;
+
 	/*
 	 * We delete all LP_DEAD items from the first heap pass in all indexes on
 	 * each call here (except calls where we choose to do the failsafe). This
@@ -2675,15 +2722,34 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* See the lazy_vacuum_all_indexes comments */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * See the lazy_vacuum_all_indexes comments
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
 											vacrel->num_index_scans,
 											(vacrel->scanned_pages < vacrel->rel_pages));
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 	}
+
+	/*
+	 * We're done with index cleanup.
+	 * Set the total number of indexes completed as the total number of indexes
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, vacrel->nindexes) ;
+
 }
 
 /*
@@ -3464,3 +3530,134 @@ restore_vacuum_error_info(LVRelState *vacrel,
 	vacrel->offnum = saved_vacrel->offnum;
 	vacrel->phase = saved_vacrel->phase;
 }
+
+void
+vacuum_worker_end(int leader_pid)
+{
+	SpinLockAcquire(&vacworkerprogress->mutex);
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		VacOneWorkerProgressInfo *vac = &vacworkerprogress->vacuums[i];
+
+		if (vac->leader_pid == leader_pid)
+		{
+			*vac = vacworkerprogress->vacuums[vacworkerprogress->num_vacuums - 1];
+			vacworkerprogress->num_vacuums--;
+			SpinLockRelease(&vacworkerprogress->mutex);
+			break;
+		}
+	}
+	SpinLockRelease(&vacworkerprogress->mutex);
+}
+
+/*
+ * vacuum_worker_end wrapped as an on_shmem_exit callback function
+ */
+void
+vacuum_worker_end_callback(int code, Datum arg)
+{
+	vacuum_worker_end(DatumGetInt32(arg));
+}
+
+/*
+ * vacuum_worker_update sets the number of indexes processed so far
+ * in a parallel vacuum.
+ */
+void
+vacuum_worker_update(int leader_pid)
+{
+	VacOneWorkerProgressInfo *vac;
+
+	SpinLockAcquire(&vacworkerprogress->mutex);
+
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		int next_leader_pid;
+
+		vac = &vacworkerprogress->vacuums[i];
+
+		next_leader_pid = vac->leader_pid;
+
+		if (next_leader_pid == leader_pid)
+		{
+			vac->indexes_processed++;
+			SpinLockRelease(&vacworkerprogress->mutex);
+			return;
+		}
+	}
+
+	if (vacworkerprogress->num_vacuums >= vacworkerprogress->max_vacuums)
+	{
+		SpinLockRelease(&vacworkerprogress->mutex);
+		elog(ERROR, "out of vacuum worker progress slots");
+	}
+
+	vac = &vacworkerprogress->vacuums[vacworkerprogress->num_vacuums];
+	vac->leader_pid = leader_pid;
+	vac->indexes_processed = 1;
+	vacworkerprogress->num_vacuums++;
+	SpinLockRelease(&vacworkerprogress->mutex);
+}
+
+/*
+ * vacuum_progress_cb updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+vacuum_progress_cb(Datum *values, int offset)
+{
+	VacOneWorkerProgressInfo *vac;
+	int leader_pid = values[0];
+
+	/* If we are vacuuming in parallel, set the number of indexes vacuumed
+	 * from the shared memory counter.
+	 * */
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		int next_leader_pid;
+
+		vac = &vacworkerprogress->vacuums[i];
+
+		next_leader_pid = vac->leader_pid;
+
+		if (next_leader_pid == leader_pid)
+			values[PROGRESS_VACUUM_INDEXES_COMPLETED + offset] = vac->indexes_processed;
+	}
+}
+
+/*
+ * VacuumWorkerProgressShmemSize --- report amount of shared memory space needed
+ */
+Size
+VacuumWorkerProgressShmemSize(void)
+{
+	Size            size;
+
+	size = offsetof(VacWorkerProgressInfo, vacuums);
+	size = add_size(size, mul_size(GetMaxBackends(), sizeof(VacOneWorkerProgressInfo)));
+	return size;
+}
+
+/*
+ * VacuumWorkerProgressShmemInit --- initialize this module's shared memory
+ */
+void
+VacuumWorkerProgressShmemInit(void)
+{
+	bool            found;
+
+	vacworkerprogress = (VacWorkerProgressInfo *) ShmemInitStruct("Vacuum Worker Progress Stats",
+										VacuumWorkerProgressShmemSize(),
+										&found);
+
+	if (!IsUnderPostmaster)
+	{
+		/* Initialize shared memory area */
+		Assert(!found);
+
+		vacworkerprogress->max_vacuums = GetMaxBackends();
+		SpinLockInit(&vacworkerprogress->mutex);
+	}
+	else
+		Assert(found);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3cb69b1f87..eaa0508c0b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..9b465e12cc 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int		 leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..a4bd6a14a3 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -145,6 +146,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, VacuumWorkerProgressShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -296,6 +298,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	VacuumWorkerProgressShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 15cb17ace4..3d90b23ce4 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,7 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/vacuum.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -452,6 +453,10 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 
 /*
  * Returns command progress information for the named command.
+ *
+ * A command type can optionally define a callback function
+ * which will derive Datum values rather than use values
+ * directly from the backends progress array.
  */
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
@@ -466,6 +471,7 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	MemoryContext per_query_ctx;
 	MemoryContext oldcontext;
+	void (*callback)(Datum *, int) = NULL;
 
 	/* check to see if caller supports us returning a tuplestore */
 	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
@@ -483,7 +489,10 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 
 	/* Translate command name into command type code. */
 	if (pg_strcasecmp(cmd, "VACUUM") == 0)
+	{
 		cmdtype = PROGRESS_COMMAND_VACUUM;
+		callback = vacuum_progress_cb;
+	}
 	else if (pg_strcasecmp(cmd, "ANALYZE") == 0)
 		cmdtype = PROGRESS_COMMAND_ANALYZE;
 	else if (pg_strcasecmp(cmd, "CLUSTER") == 0)
@@ -552,6 +561,9 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 				nulls[i + 3] = true;
 		}
 
+		if (callback)
+			callback(values, 3);
+
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..e4f3cd9133 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index d64f6268f2..5642fae0cd 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -336,4 +336,12 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+/* in commands/vacuumparallel.c */
+extern Size VacuumWorkerProgressShmemSize(void);
+extern void VacuumWorkerProgressShmemInit(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_progress_cb(Datum *values, int offset);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 1420288d67..dc27b8614e 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

0001-Expose-the-index-being-processed-in-the-vacuuming-in.patchapplication/octet-stream; name=0001-Expose-the-index-being-processed-in-the-vacuuming-in.patchDownload
From 4bb70bc6dd3005d07b3c9b74d527e31b446b833f Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 17 Feb 2022 06:27:10 +0000
Subject: [PATCH 1/1] Expose the index being processed in the "vacuuming
 indexes" or "cleaning up indexes" phase of a VACUUM operation.

A new view called pg_stat_progress_vacuum_index to show the indexrelid
being vacuumed or cleaned during a vacuum. The view also shows the
number of tuples removed for the index during the vacuuming indexes phase.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 src/backend/access/gin/ginvacuum.c    |  3 ++
 src/backend/access/gist/gistvacuum.c  |  3 ++
 src/backend/access/hash/hash.c        |  1 +
 src/backend/access/heap/vacuumlazy.c  | 15 ++++++++++
 src/backend/access/nbtree/nbtree.c    |  1 +
 src/backend/access/spgist/spgvacuum.c |  4 +++
 src/backend/catalog/system_views.sql  | 41 +++++++++++++++++++++++++++
 src/backend/commands/vacuumparallel.c | 12 ++++++++
 src/backend/utils/adt/pgstatfuncs.c   |  2 ++
 src/include/commands/progress.h       | 17 +++++++----
 src/include/utils/backend_progress.h  |  1 +
 src/test/regress/expected/rules.out   | 36 +++++++++++++++++++++++
 12 files changed, 130 insertions(+), 6 deletions(-)

diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 514582972d..57dc9bc63b 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -370,6 +370,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -2347,6 +2348,9 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
@@ -2358,6 +2362,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			 */
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
 
+			/* Advertise we are done vacuuming indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2719,12 +2727,19 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned in non-parallel vacuum*/
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
 			/* See the lazy_vacuum_all_indexes comments */
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
+
+			/* Advertise we are done cleaning indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index eaa0508c0b..5bd6953996 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1123,6 +1123,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 4 THEN 'cleaning up indexes'
                       WHEN 5 THEN 'truncating heap'
                       WHEN 6 THEN 'performing final cleanup'
+                      WHEN 7 THEN 'vacuuming indexes'
+                      WHEN 8 THEN 'cleaning up indexes'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
@@ -1131,6 +1133,45 @@ CREATE VIEW pg_stat_progress_vacuum AS
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid,
+        S.datid,
+        S.datname,
+        S.indexrelid,
+        S.leader_pid,
+        CASE S.phase WHEN 2 THEN 'vacuuming indexes'
+					 WHEN 4 THEN 'cleaning up indexes'
+					 WHEN 7 THEN 'vacuuming indexes'
+                     WHEN 8 THEN 'cleaning up indexes'
+                     END AS phase,
+        S.tuples_removed
+    FROM (
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        WHERE S.param1 IN (2, 4, 7, 8) AND S.param10 > 0
+        UNION ALL
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM_PARALLEL') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        ) AS S
+    WHERE S.phase IN (2, 4, 7, 8) AND S.indexrelid > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 9b465e12cc..0c6eac8bb9 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -845,14 +845,19 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
@@ -888,6 +893,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -972,6 +981,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1041,6 +1052,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	error_context_stack = errcallback.previous;
 
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
+	pgstat_progress_end_command();
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
 }
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index e9b41da6b9..4813fae40e 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -493,6 +493,8 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		cmdtype = PROGRESS_COMMAND_VACUUM;
 		callback = vacuum_progress_cb;
 	}
+	else if (pg_strcasecmp(cmd, "VACUUM_PARALLEL") == 0)
+		cmdtype = PROGRESS_COMMAND_VACUUM_PARALLEL;
 	else if (pg_strcasecmp(cmd, "ANALYZE") == 0)
 		cmdtype = PROGRESS_COMMAND_ANALYZE;
 	else if (pg_strcasecmp(cmd, "CLUSTER") == 0)
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e4f3cd9133..c20466cc5f 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,14 +27,19 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_TOTAL_INDEXES			7
 #define PROGRESS_VACUUM_INDEXES_COMPLETED		8
+#define PROGRESS_VACUUM_INDEXRELID				9
+#define PROGRESS_VACUUM_TUPLES_REMOVED			10
+#define PROGRESS_VACUUM_LEADER_PID				11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
-#define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
-#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX		2
-#define PROGRESS_VACUUM_PHASE_VACUUM_HEAP		3
-#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP		4
-#define PROGRESS_VACUUM_PHASE_TRUNCATE			5
-#define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP		6
+#define PROGRESS_VACUUM_PHASE_SCAN_HEAP					1
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX				2
+#define PROGRESS_VACUUM_PHASE_VACUUM_HEAP				3
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP				4
+#define PROGRESS_VACUUM_PHASE_TRUNCATE					5
+#define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP				6
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL		7
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL	8
 
 /* Progress parameters for analyze */
 #define PROGRESS_ANALYZE_PHASE						0
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..4651e45c40 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -23,6 +23,7 @@ typedef enum ProgressCommandType
 {
 	PROGRESS_COMMAND_INVALID,
 	PROGRESS_COMMAND_VACUUM,
+	PROGRESS_COMMAND_VACUUM_PARALLEL,
 	PROGRESS_COMMAND_ANALYZE,
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index dc27b8614e..c230c6a441 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1995,6 +1995,8 @@ pg_stat_progress_vacuum| SELECT s.pid,
             WHEN 4 THEN 'cleaning up indexes'::text
             WHEN 5 THEN 'truncating heap'::text
             WHEN 6 THEN 'performing final cleanup'::text
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
             ELSE NULL::text
         END AS phase,
     s.param2 AS heap_blks_total,
@@ -2007,6 +2009,40 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    s.datname,
+    s.indexrelid,
+    s.leader_pid,
+        CASE s.phase
+            WHEN 2 THEN 'vacuuming indexes'::text
+            WHEN 4 THEN 'cleaning up indexes'::text
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.tuples_removed
+   FROM ( SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))
+          WHERE ((s_1.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s_1.param10 > 0))
+        UNION ALL
+         SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM_PARALLEL'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))) s
+  WHERE ((s.phase = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s.indexrelid > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

0001-Rename-index_vacuum_count-to-index_vacuum_cycle_coun.patchapplication/octet-stream; name=0001-Rename-index_vacuum_count-to-index_vacuum_cycle_coun.patchDownload
From f907d2e18743502a04e4c6ef878612cbcf99bf61 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 17 Feb 2022 13:39:38 +0000
Subject: [PATCH 1/1] Rename "index_vacuum_count" to "index_vacuum_cycle_count"
 in in pg_stat_pogress_vacuum.

Commit 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa introduces 2 new columns
to track index vacuum/cleanup progress. The columns have "index" in the
name. To make it clear what the existing "index_vacuum_count" column
refers to, which is the cycle count of the index vacuum, this change
renames the column to "index_vacuum_count"

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml         | 2 +-
 src/backend/catalog/system_views.sql | 2 +-
 src/test/regress/expected/rules.out  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 04440dfa88..7bf31f9686 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6252,7 +6252,7 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>index_vacuum_count</structfield> <type>bigint</type>
+       <structfield>index_vacuum_cycle_count</structfield> <type>bigint</type>
       </para>
       <para>
        Number of completed index vacuum cycles.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5bd6953996..0fa9c927e4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1127,7 +1127,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 8 THEN 'cleaning up indexes'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_cycle_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index c230c6a441..c08bcb8e74 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param2 AS heap_blks_total,
     s.param3 AS heap_blks_scanned,
     s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
+    s.param5 AS index_vacuum_cycle_count,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-- 
2.32.0

#32Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#31)
3 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

The change has been broken up as 3 separate patches.

0007-Expose-progress-for-the-vacuuming-indexes-and-cleani.patch - Introduces 2 new columns to pg_stat_progress_vacuum, indexes_total and indexes_processed. These 2 columns will provide progress on the index vacuuming/cleanup.
0001-Expose-the-index-being-processed-in-the-vacuuming-in.patch - Introduces a new view called pg_stat_prgoress_vacuum_index. This view tracks the index being vacuumed/cleaned and the total number of index tuples removed.
0001-Rename-index_vacuum_count-to-index_vacuum_cycle_coun.patch - Renames the existing index_vacuum_count to index_vacuum_cycle_count in pg_stat_progress_vacuum. Due to the other changes, it makes sense to include "cycle" in the column name to be crystal clear that the column refers to the index cycle count.

Thanks

Sending again with patch files renamed to ensure correct apply order.

On 2/10/22, 1:39 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

Attached is the latest version of the patch to deal with the changes in the recent commit aa64f23b02924724eafbd9eadbf26d85df30a12b

On 2/1/22, 2:32 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

After speaking with Nathan offline, A few changes have been made to the patch.

As mentioned earlier in the thread, tracking how many indexes are processed in PARALLEL vacuum mode is not very straightforward since only the workers or leader process have ability to inspect the Vacuum shared parallel state.

The latest version of the patch introduces a shared memory to track indexes vacuumed/cleaned by each worker ( or leader ) in a PARALLEL vacuum. In order to present this data in the pg_stat_progress_vacuum view, the value of the new column "indexes_processed" is retrieved from shared memory by pg_stat_get_progress_info. For non-parallel vacuums, the value of "indexes_processed" is retrieved from the backend progress array directly.

The patch also includes the changes to implement the new view pg_stat_progress_vacuum_index which exposes the index being vacuumed/cleaned up.

postgres=# \d+ pg_stat_progress_vacuum ;
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default | Storage | Description
--------------------+---------+-----------+----------+---------+----------+-------------
pid | integer | | | | plain |
datid | oid | | | | plain |
datname | name | | | | plain |
relid | oid | | | | plain |
phase | text | | | | extended |
heap_blks_total | bigint | | | | plain |
heap_blks_scanned | bigint | | | | plain |
heap_blks_vacuumed | bigint | | | | plain |
index_vacuum_count | bigint | | | | plain |
max_dead_tuples | bigint | | | | plain |
num_dead_tuples | bigint | | | | plain |
indexes_total | bigint | | | | plain | <<<-- new column
indexes_processed | bigint | | | | plain | <<<-- new column

<<<--- new view --->>>

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
leader_pid | bigint | | |
phase | text | | |
tuples_removed | bigint | | |

On 1/26/22, 8:07 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

Attached is the latest patch and associated documentation.

This version addresses the index_ordinal_position column confusion. Rather than displaying the index position, the pg_stat_progress_vacuum view now has 2 new column(s):
index_total - this column will show the total number of indexes to be vacuumed
index_complete_count - this column will show the total number of indexes processed so far. In order to deal with the parallel vacuums, the parallel_workers ( planned workers ) value had to be exposed and each backends performing an index vacuum/cleanup in parallel had to advertise the number of indexes it vacuumed/cleaned. The # of indexes vacuumed for the parallel cleanup can then be derived the pg_stat_progress_vacuum view.

postgres=# \d pg_stat_progress_vacuum
View "pg_catalog.pg_stat_progress_vacuum"
Column | Type | Collation | Nullable | Default
----------------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
relid | oid | | |
phase | text | | |
heap_blks_total | bigint | | |
heap_blks_scanned | bigint | | |
heap_blks_vacuumed | bigint | | |
index_vacuum_count | bigint | | |
max_dead_tuples | bigint | | |
num_dead_tuples | bigint | | |
index_total | bigint | | |. <<<---------------------
index_complete_count | numeric | | |. <<<---------------------

The pg_stat_progress_vacuum_index view includes:

Indexrelid - the currently vacuumed index
Leader_pid - the pid of the leader process. NULL if the process is the leader or vacuum is not parallel
tuples_removed - the amount of indexes tuples removed. The user can use this column to see that the index vacuum has movement.

postgres=# \d pg_stat_progress_vacuum_index
View "pg_catalog.pg_stat_progress_vacuum_index"
Column | Type | Collation | Nullable | Default
----------------+---------+-----------+----------+---------
pid | integer | | |
datid | oid | | |
datname | name | | |
indexrelid | bigint | | |
phase | text | | |
leader_pid | bigint | | |
tuples_removed | bigint | | |

On 1/12/22, 9:52 PM, "Imseih (AWS), Sami" <simseih@amazon.com> wrote:

On 1/12/22, 1:28 PM, "Bossart, Nathan" <bossartn@amazon.com> wrote:

On 1/11/22, 11:46 PM, "Masahiko Sawada" <sawada.mshk@gmail.com> wrote:

Regarding the new pg_stat_progress_vacuum_index view, why do we need
to have a separate view? Users will have to check two views. If this
view is expected to be used together with and joined to
pg_stat_progress_vacuum, why don't we provide one view that has full
information from the beginning? Especially, I think it's not useful
that the total number of indexes to vacuum (num_indexes_to_vacuum
column) and the current number of indexes that have been vacuumed
(index_ordinal_position column) are shown in separate views.

I suppose we could add all of the new columns to
pg_stat_progress_vacuum and just set columns to NULL as appropriate.
But is that really better than having a separate view?

To add, since a vacuum can utilize parallel worker processes + the main vacuum process to perform index vacuuming, it made sense to separate the backends doing index vacuum/cleanup in a separate view.
Besides what Nathan suggested, the only other clean option I can think of is to perhaps create a json column in pg_stat_progress_vacuum which will include all the new fields. My concern with this approach is that it will make usability, to flatten the json, difficult for users.

Also, I’m not sure how useful index_tuples_removed is; what can we
infer from this value (without a total number)?

I think the idea was that you can compare it against max_dead_tuples
and num_dead_tuples to get an estimate of the current cycle progress.
Otherwise, it just shows that progress is being made.

The main purpose is to really show that the "index vacuum" phase is actually making progress. Note that for certain types of indexes, i.e. GIN/GIST the number of tuples_removed will end up exceeding the number of num_dead_tuples.

Nathan

[0]: /messages/by-id/7874FB21-FAA5-49BD-8386-2866552656C7@amazon.com

Attachments:

0011-Rename-index_vacuum_count-to-index_vacuum_cycle_coun.patchapplication/octet-stream; name=0011-Rename-index_vacuum_count-to-index_vacuum_cycle_coun.patchDownload
From f907d2e18743502a04e4c6ef878612cbcf99bf61 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 17 Feb 2022 13:39:38 +0000
Subject: [PATCH 1/1] Rename "index_vacuum_count" to "index_vacuum_cycle_count"
 in in pg_stat_pogress_vacuum.

Commit 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa introduces 2 new columns
to track index vacuum/cleanup progress. The columns have "index" in the
name. To make it clear what the existing "index_vacuum_count" column
refers to, which is the cycle count of the index vacuum, this change
renames the column to "index_vacuum_count"

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml         | 2 +-
 src/backend/catalog/system_views.sql | 2 +-
 src/test/regress/expected/rules.out  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 04440dfa88..7bf31f9686 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6252,7 +6252,7 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>index_vacuum_count</structfield> <type>bigint</type>
+       <structfield>index_vacuum_cycle_count</structfield> <type>bigint</type>
       </para>
       <para>
        Number of completed index vacuum cycles.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5bd6953996..0fa9c927e4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1127,7 +1127,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 8 THEN 'cleaning up indexes'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_cycle_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index c230c6a441..c08bcb8e74 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param2 AS heap_blks_total,
     s.param3 AS heap_blks_scanned,
     s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
+    s.param5 AS index_vacuum_cycle_count,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-- 
2.32.0

0010-Expose-the-index-being-processed-in-the-vacuuming-in.patchapplication/octet-stream; name=0010-Expose-the-index-being-processed-in-the-vacuuming-in.patchDownload
From 6d38d09c72e917db1c078d93a5278efab5c771c2 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 17 Feb 2022 16:25:12 +0000
Subject: [PATCH 1/1] Expose the index being processed in the "vacuuming
 indexes" or "cleaning up indexes" phase of a VACUUM operation.

A new view called pg_stat_progress_vacuum_index to show the indexrelid
being vacuumed or cleaned during a vacuum. The view also shows the
number of tuples removed for the index during the vacuuming indexes phase.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 104 +++++++++++++++++++++++++-
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  15 ++++
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  41 ++++++++++
 src/backend/commands/vacuumparallel.c |  12 +++
 src/backend/utils/adt/pgstatfuncs.c   |   2 +
 src/include/commands/progress.h       |  17 +++--
 src/include/utils/backend_progress.h  |   1 +
 src/test/regress/expected/rules.out   |  36 +++++++++
 13 files changed, 233 insertions(+), 7 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 04440dfa88..85369acb39 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,14 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend processing an index during a
+       <command>VACUUM</command>, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6144,7 +6152,10 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    Whenever <command>VACUUM</command> is running, the
    <structname>pg_stat_progress_vacuum</structname> view will contain
    one row for each backend (including autovacuum worker processes) that is
-   currently vacuuming.  The tables below describe the information
+   currently vacuuming. The <structname>pg_stat_progress_vacuum_index</structname>
+   view will also contain one row for each backend processing an index
+   during the <command>VACUUM</command>. 
+   The tables below describe the information
    that will be reported and provide information about how to interpret it.
    Progress for <command>VACUUM FULL</command> commands is reported via
    <structname>pg_stat_progress_cluster</structname>
@@ -6304,6 +6315,97 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    </tgroup>
   </table>
 
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
+     </tbody>
+    </tgroup>
+   </table>
+
   <table id="vacuum-phases">
    <title>VACUUM Phases</title>
    <tgroup cols="2">
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 514582972d..57dc9bc63b 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -370,6 +370,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -2347,6 +2348,9 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
@@ -2358,6 +2362,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			 */
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
 
+			/* Advertise we are done vacuuming indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2719,12 +2727,19 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned in non-parallel vacuum*/
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
 			/* See the lazy_vacuum_all_indexes comments */
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
+
+			/* Advertise we are done cleaning indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index eaa0508c0b..5bd6953996 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1123,6 +1123,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 4 THEN 'cleaning up indexes'
                       WHEN 5 THEN 'truncating heap'
                       WHEN 6 THEN 'performing final cleanup'
+                      WHEN 7 THEN 'vacuuming indexes'
+                      WHEN 8 THEN 'cleaning up indexes'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
@@ -1131,6 +1133,45 @@ CREATE VIEW pg_stat_progress_vacuum AS
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid,
+        S.datid,
+        S.datname,
+        S.indexrelid,
+        S.leader_pid,
+        CASE S.phase WHEN 2 THEN 'vacuuming indexes'
+					 WHEN 4 THEN 'cleaning up indexes'
+					 WHEN 7 THEN 'vacuuming indexes'
+                     WHEN 8 THEN 'cleaning up indexes'
+                     END AS phase,
+        S.tuples_removed
+    FROM (
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        WHERE S.param1 IN (2, 4, 7, 8) AND S.param10 > 0
+        UNION ALL
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM_PARALLEL') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        ) AS S
+    WHERE S.phase IN (2, 4, 7, 8) AND S.indexrelid > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 9b465e12cc..0c6eac8bb9 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -845,14 +845,19 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
@@ -888,6 +893,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -972,6 +981,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1041,6 +1052,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	error_context_stack = errcallback.previous;
 
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
+	pgstat_progress_end_command();
 	table_close(rel, ShareUpdateExclusiveLock);
 	FreeAccessStrategy(pvs.bstrategy);
 }
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index e9b41da6b9..4813fae40e 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -493,6 +493,8 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		cmdtype = PROGRESS_COMMAND_VACUUM;
 		callback = vacuum_progress_cb;
 	}
+	else if (pg_strcasecmp(cmd, "VACUUM_PARALLEL") == 0)
+		cmdtype = PROGRESS_COMMAND_VACUUM_PARALLEL;
 	else if (pg_strcasecmp(cmd, "ANALYZE") == 0)
 		cmdtype = PROGRESS_COMMAND_ANALYZE;
 	else if (pg_strcasecmp(cmd, "CLUSTER") == 0)
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e4f3cd9133..c20466cc5f 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,14 +27,19 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_TOTAL_INDEXES			7
 #define PROGRESS_VACUUM_INDEXES_COMPLETED		8
+#define PROGRESS_VACUUM_INDEXRELID				9
+#define PROGRESS_VACUUM_TUPLES_REMOVED			10
+#define PROGRESS_VACUUM_LEADER_PID				11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
-#define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
-#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX		2
-#define PROGRESS_VACUUM_PHASE_VACUUM_HEAP		3
-#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP		4
-#define PROGRESS_VACUUM_PHASE_TRUNCATE			5
-#define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP		6
+#define PROGRESS_VACUUM_PHASE_SCAN_HEAP					1
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX				2
+#define PROGRESS_VACUUM_PHASE_VACUUM_HEAP				3
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP				4
+#define PROGRESS_VACUUM_PHASE_TRUNCATE					5
+#define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP				6
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL		7
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL	8
 
 /* Progress parameters for analyze */
 #define PROGRESS_ANALYZE_PHASE						0
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..4651e45c40 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -23,6 +23,7 @@ typedef enum ProgressCommandType
 {
 	PROGRESS_COMMAND_INVALID,
 	PROGRESS_COMMAND_VACUUM,
+	PROGRESS_COMMAND_VACUUM_PARALLEL,
 	PROGRESS_COMMAND_ANALYZE,
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index dc27b8614e..c230c6a441 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -1995,6 +1995,8 @@ pg_stat_progress_vacuum| SELECT s.pid,
             WHEN 4 THEN 'cleaning up indexes'::text
             WHEN 5 THEN 'truncating heap'::text
             WHEN 6 THEN 'performing final cleanup'::text
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
             ELSE NULL::text
         END AS phase,
     s.param2 AS heap_blks_total,
@@ -2007,6 +2009,40 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    s.datname,
+    s.indexrelid,
+    s.leader_pid,
+        CASE s.phase
+            WHEN 2 THEN 'vacuuming indexes'::text
+            WHEN 4 THEN 'cleaning up indexes'::text
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.tuples_removed
+   FROM ( SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))
+          WHERE ((s_1.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s_1.param10 > 0))
+        UNION ALL
+         SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM_PARALLEL'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))) s
+  WHERE ((s.phase = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s.indexrelid > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

0009-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchapplication/octet-stream; name=0009-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchDownload
From 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 17 Feb 2022 04:21:04 +0000
Subject: [PATCH 1/1] Expose progress for the "vacuuming indexes" and "cleaning
 up indexes" phase of a VACUUM operation.

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          |  22 +++
 src/backend/access/heap/vacuumlazy.c  | 207 +++++++++++++++++++++++++-
 src/backend/catalog/system_views.sql  |   3 +-
 src/backend/commands/vacuumparallel.c |   7 +
 src/backend/storage/ipc/ipci.c        |   3 +
 src/backend/utils/adt/pgstatfuncs.c   |  12 ++
 src/include/commands/progress.h       |   2 +
 src/include/commands/vacuum.h         |   8 +
 src/test/regress/expected/rules.out   |   4 +-
 9 files changed, 261 insertions(+), 7 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index bf7625d988..04440dfa88 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6278,6 +6278,28 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes processed in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase.
+       At the start of an index vacuum cycle, this value is set to <literal>0</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 242511a235..1198677bc0 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -245,6 +246,26 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/*
+ * Structs for tracking shared Progress information
+ * amongst worker ( and leader ) processes of a vacuum.
+ */
+typedef struct VacOneWorkerProgressInfo
+{
+	int                             leader_pid;
+	int                             indexes_processed;
+} VacOneWorkerProgressInfo;
+
+typedef struct VacWorkerProgressInfo
+{
+	int                     num_vacuums;    /* number of active VACUUMS with parallel workers */
+	int                     max_vacuums;    /* max number of VACUUMS with parallel workers */
+	slock_t			mutex;
+	VacOneWorkerProgressInfo vacuums[FLEXIBLE_ARRAY_MEMBER];
+} VacWorkerProgressInfo;
+
+static VacWorkerProgressInfo *vacworkerprogress;
+
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel, int nworkers);
@@ -420,6 +441,8 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2328,6 +2351,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/*
+			 * For the non-parallel variant of a vacuum, the array position
+			 * of the index determines how many indexes are processed so far.
+			 * Add 1 to the posititon as vacrel->nindexes is a 0-based array.
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2338,9 +2368,20 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * parallel_vacuum_bulkdel_all_indexes will call vacuum_worker_update
+		 * which updates shared memory for the index progress. To ensure shared
+		 * memory cleanup, do the work with PG_ENSURE_ERROR_CLEANUP.
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs,
+												vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2350,6 +2391,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			allindexes = false;
 	}
 
+	/*
+	 * We're done with index vacuuming.
+	 * Set the total number of indexes completed as the total number of indexes
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, vacrel->nindexes) ;
+
 	/*
 	 * We delete all LP_DEAD items from the first heap pass in all indexes on
 	 * each call here (except calls where we choose to do the failsafe). This
@@ -2675,15 +2722,34 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* See the lazy_vacuum_all_indexes comments */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * See the lazy_vacuum_all_indexes comments
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
 											vacrel->num_index_scans,
 											(vacrel->scanned_pages < vacrel->rel_pages));
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 	}
+
+	/*
+	 * We're done with index cleanup.
+	 * Set the total number of indexes completed as the total number of indexes
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, vacrel->nindexes) ;
+
 }
 
 /*
@@ -3464,3 +3530,134 @@ restore_vacuum_error_info(LVRelState *vacrel,
 	vacrel->offnum = saved_vacrel->offnum;
 	vacrel->phase = saved_vacrel->phase;
 }
+
+void
+vacuum_worker_end(int leader_pid)
+{
+	SpinLockAcquire(&vacworkerprogress->mutex);
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		VacOneWorkerProgressInfo *vac = &vacworkerprogress->vacuums[i];
+
+		if (vac->leader_pid == leader_pid)
+		{
+			*vac = vacworkerprogress->vacuums[vacworkerprogress->num_vacuums - 1];
+			vacworkerprogress->num_vacuums--;
+			SpinLockRelease(&vacworkerprogress->mutex);
+			break;
+		}
+	}
+	SpinLockRelease(&vacworkerprogress->mutex);
+}
+
+/*
+ * vacuum_worker_end wrapped as an on_shmem_exit callback function
+ */
+void
+vacuum_worker_end_callback(int code, Datum arg)
+{
+	vacuum_worker_end(DatumGetInt32(arg));
+}
+
+/*
+ * vacuum_worker_update sets the number of indexes processed so far
+ * in a parallel vacuum.
+ */
+void
+vacuum_worker_update(int leader_pid)
+{
+	VacOneWorkerProgressInfo *vac;
+
+	SpinLockAcquire(&vacworkerprogress->mutex);
+
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		int next_leader_pid;
+
+		vac = &vacworkerprogress->vacuums[i];
+
+		next_leader_pid = vac->leader_pid;
+
+		if (next_leader_pid == leader_pid)
+		{
+			vac->indexes_processed++;
+			SpinLockRelease(&vacworkerprogress->mutex);
+			return;
+		}
+	}
+
+	if (vacworkerprogress->num_vacuums >= vacworkerprogress->max_vacuums)
+	{
+		SpinLockRelease(&vacworkerprogress->mutex);
+		elog(ERROR, "out of vacuum worker progress slots");
+	}
+
+	vac = &vacworkerprogress->vacuums[vacworkerprogress->num_vacuums];
+	vac->leader_pid = leader_pid;
+	vac->indexes_processed = 1;
+	vacworkerprogress->num_vacuums++;
+	SpinLockRelease(&vacworkerprogress->mutex);
+}
+
+/*
+ * vacuum_progress_cb updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+vacuum_progress_cb(Datum *values, int offset)
+{
+	VacOneWorkerProgressInfo *vac;
+	int leader_pid = values[0];
+
+	/* If we are vacuuming in parallel, set the number of indexes vacuumed
+	 * from the shared memory counter.
+	 * */
+	for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+	{
+		int next_leader_pid;
+
+		vac = &vacworkerprogress->vacuums[i];
+
+		next_leader_pid = vac->leader_pid;
+
+		if (next_leader_pid == leader_pid)
+			values[PROGRESS_VACUUM_INDEXES_COMPLETED + offset] = vac->indexes_processed;
+	}
+}
+
+/*
+ * VacuumWorkerProgressShmemSize --- report amount of shared memory space needed
+ */
+Size
+VacuumWorkerProgressShmemSize(void)
+{
+	Size            size;
+
+	size = offsetof(VacWorkerProgressInfo, vacuums);
+	size = add_size(size, mul_size(GetMaxBackends(), sizeof(VacOneWorkerProgressInfo)));
+	return size;
+}
+
+/*
+ * VacuumWorkerProgressShmemInit --- initialize this module's shared memory
+ */
+void
+VacuumWorkerProgressShmemInit(void)
+{
+	bool            found;
+
+	vacworkerprogress = (VacWorkerProgressInfo *) ShmemInitStruct("Vacuum Worker Progress Stats",
+										VacuumWorkerProgressShmemSize(),
+										&found);
+
+	if (!IsUnderPostmaster)
+	{
+		/* Initialize shared memory area */
+		Assert(!found);
+
+		vacworkerprogress->max_vacuums = GetMaxBackends();
+		SpinLockInit(&vacworkerprogress->mutex);
+	}
+	else
+		Assert(found);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3cb69b1f87..eaa0508c0b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..9b465e12cc 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int		 leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..a4bd6a14a3 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -145,6 +146,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, VacuumWorkerProgressShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -296,6 +298,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	VacuumWorkerProgressShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 15cb17ace4..3d90b23ce4 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,7 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/vacuum.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -452,6 +453,10 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 
 /*
  * Returns command progress information for the named command.
+ *
+ * A command type can optionally define a callback function
+ * which will derive Datum values rather than use values
+ * directly from the backends progress array.
  */
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
@@ -466,6 +471,7 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 	ReturnSetInfo *rsinfo = (ReturnSetInfo *) fcinfo->resultinfo;
 	MemoryContext per_query_ctx;
 	MemoryContext oldcontext;
+	void (*callback)(Datum *, int) = NULL;
 
 	/* check to see if caller supports us returning a tuplestore */
 	if (rsinfo == NULL || !IsA(rsinfo, ReturnSetInfo))
@@ -483,7 +489,10 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 
 	/* Translate command name into command type code. */
 	if (pg_strcasecmp(cmd, "VACUUM") == 0)
+	{
 		cmdtype = PROGRESS_COMMAND_VACUUM;
+		callback = vacuum_progress_cb;
+	}
 	else if (pg_strcasecmp(cmd, "ANALYZE") == 0)
 		cmdtype = PROGRESS_COMMAND_ANALYZE;
 	else if (pg_strcasecmp(cmd, "CLUSTER") == 0)
@@ -552,6 +561,9 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 				nulls[i + 3] = true;
 		}
 
+		if (callback)
+			callback(values, 3);
+
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..e4f3cd9133 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index d64f6268f2..5642fae0cd 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -336,4 +336,12 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+/* in commands/vacuumparallel.c */
+extern Size VacuumWorkerProgressShmemSize(void);
+extern void VacuumWorkerProgressShmemInit(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_progress_cb(Datum *values, int offset);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 1420288d67..dc27b8614e 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

#33Nathan Bossart
nathandbossart@gmail.com
In reply to: Imseih (AWS), Sami (#32)
Re: Add index scan progress to pg_stat_progress_vacuum

On Mon, Feb 21, 2022 at 07:03:39PM +0000, Imseih (AWS), Sami wrote:

Sending again with patch files renamed to ensure correct apply order.

I haven't had a chance to test this too much, but I did look through the
patch set and have a couple of small comments.

+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
+       of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes processed in the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase.
+       At the start of an index vacuum cycle, this value is set to <literal>0</literal>.
+      </para></entry>
+     </row>

Will these be set to 0 for failsafe vacuums and vacuums with INDEX_CLEANUP
turned off?

+typedef struct VacWorkerProgressInfo
+{
+    int                     num_vacuums;    /* number of active VACUUMS with parallel workers */
+    int                     max_vacuums;    /* max number of VACUUMS with parallel workers */
+    slock_t         mutex;
+    VacOneWorkerProgressInfo vacuums[FLEXIBLE_ARRAY_MEMBER];
+} VacWorkerProgressInfo;

max_vacuums appears to just be a local copy of MaxBackends. Does this
information really need to be stored here? Also, is there a strong reason
for using a spinlock instead of an LWLock?

+void
+vacuum_worker_end(int leader_pid)
+{
+    SpinLockAcquire(&vacworkerprogress->mutex);
+    for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
+    {
+        VacOneWorkerProgressInfo *vac = &vacworkerprogress->vacuums[i];
+
+        if (vac->leader_pid == leader_pid)
+        {
+            *vac = vacworkerprogress->vacuums[vacworkerprogress->num_vacuums - 1];
+            vacworkerprogress->num_vacuums--;
+            SpinLockRelease(&vacworkerprogress->mutex);
+            break;
+        }
+    }
+    SpinLockRelease(&vacworkerprogress->mutex);
+}

I see this loop pattern in a couple of places, and it makes me wonder if
this information would fit more naturally in a hash table.

+ if (callback)
+ callback(values, 3);

Why does this need to be set up as a callback function? Could we just call
the function if cmdtype == PROGRESS_COMMAND_VACUUM? ISTM that is pretty
much all this is doing.

--
Nathan Bossart
Amazon Web Services: https://aws.amazon.com

#34Imseih (AWS), Sami
simseih@amazon.com
In reply to: Nathan Bossart (#33)
Re: Add index scan progress to pg_stat_progress_vacuum
    +     <row>
    +      <entry role="catalog_table_entry"><para role="column_definition">
    +       <structfield>indexes_total</structfield> <type>bigint</type>
    +      </para>
    +      <para>
    +       The number of indexes to be processed in the
    +       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase
    +       of the vacuum.
    +      </para></entry>
    +     </row>
    +
    +     <row>
    +      <entry role="catalog_table_entry"><para role="column_definition">
    +       <structfield>indexes_processed</structfield> <type>bigint</type>
    +      </para>
    +      <para>
    +       The number of indexes processed in the
    +       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal> phase.
    +       At the start of an index vacuum cycle, this value is set to <literal>0</literal>.
    +      </para></entry>
    +     </row>

Will these be set to 0 for failsafe vacuums and vacuums with INDEX_CLEANUP
turned off?

If the failsafe kicks in midway through a vacuum, the number indexes_total will not be reset to 0. If INDEX_CLEANUP is turned off, then the value will be 0 at the start of the vacuum.

    +typedef struct VacWorkerProgressInfo
    +{
    +    int                     num_vacuums;    /* number of active VACUUMS with parallel workers */
    +    int                     max_vacuums;    /* max number of VACUUMS with parallel workers */
    +    slock_t         mutex;
    +    VacOneWorkerProgressInfo vacuums[FLEXIBLE_ARRAY_MEMBER];
    +} VacWorkerProgressInfo;

max_vacuums appears to just be a local copy of MaxBackends. Does this
information really need to be stored here? Also, is there a strong reason
for using a spinlock instead of an LWLock?

First, The BTVacInfo code in backend/access/nbtree/nbtutils.c inspired this, so I wanted to follow this pattern. With that said, I do see max_vacuums being redundant here, and I am inclined to replace it with a MaxBackends() call.

Second, There is no strong reason to use spinlock here except I incorrectly assumed it will be better for this case. After reading more about this and reading up src/backend/storage/lmgr/README, an LWLock will be better.

    +void
    +vacuum_worker_end(int leader_pid)
    +{
    +    SpinLockAcquire(&vacworkerprogress->mutex);
    +    for (int i = 0; i < vacworkerprogress->num_vacuums; i++)
    +    {
    +        VacOneWorkerProgressInfo *vac = &vacworkerprogress->vacuums[i];
    +
    +        if (vac->leader_pid == leader_pid)
    +        {
    +            *vac = vacworkerprogress->vacuums[vacworkerprogress->num_vacuums - 1];
    +            vacworkerprogress->num_vacuums--;
    +            SpinLockRelease(&vacworkerprogress->mutex);
    +            break;
    +        }
    +    }
    +    SpinLockRelease(&vacworkerprogress->mutex);
    +}

I see this loop pattern in a couple of places, and it makes me wonder if
this information would fit more naturally in a hash table.

Followed the pattern in backend/access/nbtree/nbtutils.c for this as well. Using dynahash may make sense here if it simplifies the code. Will look.

+ if (callback)
+ callback(values, 3);

Why does this need to be set up as a callback function? Could we just call
the function if cmdtype == PROGRESS_COMMAND_VACUUM? ISTM that is pretty
much all this is doing.

The intention will be for the caller to set the callback early on in the function using the existing " if (pg_strcasecmp(cmd, "VACUUM") == 0), etc." statement. This way we avoid having to add another if/else block before tuplestore_putvalues is called.

--
Sami Imseih
Amazon Web Services

In reply to: Imseih (AWS), Sami (#34)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Feb 23, 2022 at 10:02 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

If the failsafe kicks in midway through a vacuum, the number indexes_total will not be reset to 0. If INDEX_CLEANUP is turned off, then the value will be 0 at the start of the vacuum.

The way that this works with num_index_scans is that we "round up"
when there has been non-zero work in lazy_vacuum_all_indexes(), but
not if the precheck in lazy_vacuum_all_indexes() fails. That seems
like a good model to generalize from here. Note that this makes
INDEX_CLEANUP=off affect num_index_scans in much the same way as a
VACUUM where the failsafe kicks in very early, during the initial heap
pass. That is, if the failsafe kicks in before we reach lazy_vacuum()
for the first time (which is not unlikely), or even in the
lazy_vacuum_all_indexes() precheck, then num_index_scans will remain
at 0, just like INDEX_CLEANUP=off.

The actual failsafe WARNING shows num_index_scans, possibly before it
gets incremented one last time (by "rounding up"). So it's reasonably
clear how this all works from that context (assuming that the
autovacuum logging stuff, which reports num_index_scans, outputs a
report for a table where the failsafe kicked in).

--
Peter Geoghegan

#36Nathan Bossart
nathandbossart@gmail.com
In reply to: Peter Geoghegan (#35)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Feb 23, 2022 at 10:41:36AM -0800, Peter Geoghegan wrote:

On Wed, Feb 23, 2022 at 10:02 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

If the failsafe kicks in midway through a vacuum, the number indexes_total will not be reset to 0. If INDEX_CLEANUP is turned off, then the value will be 0 at the start of the vacuum.

The way that this works with num_index_scans is that we "round up"
when there has been non-zero work in lazy_vacuum_all_indexes(), but
not if the precheck in lazy_vacuum_all_indexes() fails. That seems
like a good model to generalize from here. Note that this makes
INDEX_CLEANUP=off affect num_index_scans in much the same way as a
VACUUM where the failsafe kicks in very early, during the initial heap
pass. That is, if the failsafe kicks in before we reach lazy_vacuum()
for the first time (which is not unlikely), or even in the
lazy_vacuum_all_indexes() precheck, then num_index_scans will remain
at 0, just like INDEX_CLEANUP=off.

The actual failsafe WARNING shows num_index_scans, possibly before it
gets incremented one last time (by "rounding up"). So it's reasonably
clear how this all works from that context (assuming that the
autovacuum logging stuff, which reports num_index_scans, outputs a
report for a table where the failsafe kicked in).

I am confused. If failsafe kicks in during the middle of a vacuum, I
(perhaps naively) would expect indexes_total and indexes_processed to go to
zero, and I'd expect to no longer see the "vacuuming indexes" and "cleaning
up indexes" phases. Otherwise, how would I know that we are now skipping
indexes? Of course, you won't have any historical context about the index
work done before failsafe kicked in, but IMO it is misleading to still
include it in the progress view.

--
Nathan Bossart
Amazon Web Services: https://aws.amazon.com

#37Imseih (AWS), Sami
simseih@amazon.com
In reply to: Nathan Bossart (#36)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Feb 23, 2022 at 10:02 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

If the failsafe kicks in midway through a vacuum, the number indexes_total will not be reset to 0. If INDEX_CLEANUP is turned off, then the value will be 0 at the start of the vacuum.

The way that this works with num_index_scans is that we "round up"
when there has been non-zero work in lazy_vacuum_all_indexes(), but
not if the precheck in lazy_vacuum_all_indexes() fails. That seems
like a good model to generalize from here. Note that this makes
INDEX_CLEANUP=off affect num_index_scans in much the same way as a
VACUUM where the failsafe kicks in very early, during the initial heap
pass. That is, if the failsafe kicks in before we reach lazy_vacuum()
for the first time (which is not unlikely), or even in the
lazy_vacuum_all_indexes() precheck, then num_index_scans will remain
at 0, just like INDEX_CLEANUP=off.

The actual failsafe WARNING shows num_index_scans, possibly before it
gets incremented one last time (by "rounding up"). So it's reasonably
clear how this all works from that context (assuming that the
autovacuum logging stuff, which reports num_index_scans, outputs a
report for a table where the failsafe kicked in).

I am confused. If failsafe kicks in during the middle of a vacuum, I
(perhaps naively) would expect indexes_total and indexes_processed to go to
zero, and I'd expect to no longer see the "vacuuming indexes" and "cleaning
up indexes" phases. Otherwise, how would I know that we are now skipping
indexes? Of course, you won't have any historical context about the index
work done before failsafe kicked in, but IMO it is misleading to still
include it in the progress view.

Failsafe occurring in the middle of a vacuum and resetting "indexes_total" to 0 will be misleading. I am thinking that it is a better idea to expose only one column "indexes_remaining".

If index_cleanup is set to OFF, the values of indexes_remaining will be 0 at the start of the vacuum.
If failsafe kicks in during a vacuum in-progress, "indexes_remaining" will be calculated to 0.

This approach will provide a progress based on how many indexes remaining with no ambiguity.

--
Sami Imseih
Amazon Web Services

#38Imseih (AWS), Sami
simseih@amazon.com
In reply to: Nathan Bossart (#36)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

If the failsafe kicks in midway through a vacuum, the number indexes_total will not be reset to 0. If INDEX_CLEANUP is turned off, then the value will be 0 at the start of the vacuum.

The way that this works with num_index_scans is that we "round up"
when there has been non-zero work in lazy_vacuum_all_indexes(), but
not if the precheck in lazy_vacuum_all_indexes() fails. That seems
like a good model to generalize from here. Note that this makes
INDEX_CLEANUP=off affect num_index_scans in much the same way as a
VACUUM where the failsafe kicks in very early, during the initial heap
pass. That is, if the failsafe kicks in before we reach lazy_vacuum()
for the first time (which is not unlikely), or even in the
lazy_vacuum_all_indexes() precheck, then num_index_scans will remain
at 0, just like INDEX_CLEANUP=off.

The actual failsafe WARNING shows num_index_scans, possibly before it
gets incremented one last time (by "rounding up"). So it's reasonably
clear how this all works from that context (assuming that the
autovacuum logging stuff, which reports num_index_scans, outputs a
report for a table where the failsafe kicked in).

I am confused. If failsafe kicks in during the middle of a vacuum, I
(perhaps naively) would expect indexes_total and indexes_processed to go to
zero, and I'd expect to no longer see the "vacuuming indexes" and "cleaning
up indexes" phases. Otherwise, how would I know that we are now skipping
indexes? Of course, you won't have any historical context about the index
work done before failsafe kicked in, but IMO it is misleading to still
include it in the progress view.

After speaking with Nathan offline, the best forward is to reset indexes_total and indexes_processed to 0 after the start of "vacuuming indexes" or "cleaning up indexes" phase.
Also, if failsafe is triggered midway through a vacuum, the values for both indexes_total and indexes_processed is (re)set to 0.

Revision of the patch is attached.

Below is a test that shows the output.

-[ RECORD 1 ]------+------------------
pid | 4360
datid | 5
datname | postgres
relid | 16399
phase | vacuuming indexes
heap_blks_total | 401092
heap_blks_scanned | 211798
heap_blks_vacuumed | 158847
index_vacuum_count | 3
max_dead_tuples | 1747625
num_dead_tuples | 1747366
indexes_total | 8 <<<<--- index_vacuum_count is 3, indexes_total is 8 and indexes_processed so far is 1
indexes_processed | 1

-[ RECORD 1 ]------+--------------
pid | 4360
datid | 5
datname | postgres
relid | 16399
phase | scanning heap
heap_blks_total | 401092
heap_blks_scanned | 234590
heap_blks_vacuumed | 211797
index_vacuum_count | 4
max_dead_tuples | 1747625
num_dead_tuples | 752136
indexes_total | 0 <<<<--- index_vacuum_count is 4 and not in an index phase. indexes_total is 0 and indexes_processed so far is 0
indexes_processed | 0

-[ RECORD 1 ]------+------------------
pid | 4360
datid | 5
datname | postgres
relid | 16399
phase | vacuuming indexes
heap_blks_total | 401092
heap_blks_scanned | 264748
heap_blks_vacuumed | 211797
index_vacuum_count | 4
max_dead_tuples | 1747625
num_dead_tuples | 1747350
indexes_total | 8
indexes_processed | 6 <<<<--- index_vacuum_count is 4, indexes_total is 8 and indexes_processed so far is 6

--
Sami Imseih
Amazon Web Services

Attachments:

v2-0009-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchapplication/octet-stream; name=v2-0009-Expose-progress-for-the-vacuuming-indexes-and-cleani.patchDownload
From 996254bdbe792bbf1cabb914f3eab883978911a0 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 3 Mar 2022 04:37:48 +0000
Subject: [PATCH 1/1] Expose progress for the "vacuuming indexes" and "cleaning
 up indexes" phase of a VACUUM operation.

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml             |  24 ++++
 src/backend/access/heap/vacuumlazy.c     | 156 ++++++++++++++++++++++-
 src/backend/catalog/system_views.sql     |   3 +-
 src/backend/commands/vacuumparallel.c    |   7 +
 src/backend/storage/ipc/ipci.c           |   2 +
 src/backend/storage/lmgr/lwlocknames.txt |   1 +
 src/backend/utils/adt/pgstatfuncs.c      |  16 ++-
 src/include/commands/progress.h          |   5 +
 src/include/commands/vacuum.h            |   7 +
 src/test/regress/expected/rules.out      |   4 +-
 10 files changed, 215 insertions(+), 10 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 9fb62fec8e..140fbd5fb9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6227,6 +6227,30 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 40101e0cb8..e9c081ada6 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -244,6 +245,16 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/*
+ * Structs for tracking shared Progress information
+ * amongst worker ( and leader ) processes of a vacuum.
+ */
+typedef struct VacProgressEntry
+{
+	pid_t	leader_pid;
+	int		indexes_total;
+	int		indexes_processed;
+} VacProgressEntry;
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel, int nworkers);
@@ -290,6 +301,7 @@ static void update_vacuum_error_info(LVRelState *vacrel,
 static void restore_vacuum_error_info(LVRelState *vacrel,
 									  const LVSavedErrInfo *saved_vacrel);
 
+static HTAB *VacuumWorkerProgressHash;
 
 /*
  *	heap_vacuum_rel() -- perform VACUUM for one heap relation
@@ -2305,9 +2317,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	/* Report that we are now vacuuming indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_completed = 0;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2317,6 +2333,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2327,9 +2345,22 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * Outsource everything to parallel variant.
+		 *
+		 * * parallel_vacuum_bulkdel_all_indexes will call vacuum_worker_update
+		 * which updates shared memory for the index progress. To ensure shared
+		 * memory cleanup, do the work with PG_ENSURE_ERROR_CLEANUP.
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs,
+												vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2339,6 +2370,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			allindexes = false;
 	}
 
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 	/*
 	 * We delete all LP_DEAD items from the first heap pass in all indexes on
 	 * each call here (except calls where we choose to do the failsafe). This
@@ -2617,6 +2652,8 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2649,12 +2686,15 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 	/* Report that we are now cleaning up indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/* Advertise the number of indexes we are cleaning up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		double		reltuples = vacrel->new_rel_tuples;
 		bool		estimated_count =
 		vacrel->scanned_pages < vacrel->rel_pages;
+		int indexes_completed = 0;
 
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
@@ -2664,15 +2704,32 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
+
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * See the lazy_vacuum_all_indexes comments
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
 											vacrel->num_index_scans,
 											(vacrel->scanned_pages < vacrel->rel_pages));
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 	}
+
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 }
 
 /*
@@ -3453,3 +3510,92 @@ restore_vacuum_error_info(LVRelState *vacrel,
 	vacrel->offnum = saved_vacrel->offnum;
 	vacrel->phase = saved_vacrel->phase;
 }
+
+/*
+ * vacuum_worker_update --- sets the number of indexes processed so far
+ * in a parallel vacuum.
+ */
+void
+vacuum_worker_update(int leader_pid)
+{
+	VacProgressEntry *entry;
+	bool found;
+
+	LWLockAcquire(VacuumWorkerProgressLock, LW_EXCLUSIVE);
+
+	entry = (VacProgressEntry *) hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_ENTER_NULL, &found);
+
+	if (!entry)
+		elog(ERROR, "cannot allocate shared memory for vacuum worker progress");
+
+	if (!found)
+		entry->indexes_processed = 1;
+	else
+		entry->indexes_processed++;
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_end --- remove the leader_pid of a completed parallel vacuum
+ */
+void
+vacuum_worker_end(int leader_pid)
+{
+	LWLockAcquire(VacuumWorkerProgressLock, LW_EXCLUSIVE);
+
+	hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_REMOVE, NULL);
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_end wrapped as an on_shmem_exit callback function
+ */
+void
+vacuum_worker_end_callback(int code, Datum arg)
+{
+	vacuum_worker_end(DatumGetInt32(arg));
+}
+
+/*
+ * set_vaccum_worker_progress --- updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+set_vaccum_worker_progress(Datum *values)
+{
+	VacProgressEntry *entry;
+	int leader_pid = values[0];
+
+	LWLockAcquire(VacuumWorkerProgressLock, LW_SHARED);
+
+	entry = (VacProgressEntry *) hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_FIND, NULL);
+
+	if (entry != NULL)
+		values[PGSTAT_NUM_PROGRESS_COMMON + PROGRESS_VACUUM_INDEXES_COMPLETED] = entry->indexes_processed;
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_init --- initialize this module's shared memory hash
+ * to track the progress of a vacuum worker
+ */
+void
+vacuum_worker_init(void)
+{
+	HASHCTL     info;
+	long        max_table_size = GetMaxBackends();
+
+	VacuumWorkerProgressHash = NULL;
+
+	info.keysize = sizeof(pid_t);
+	info.entrysize = sizeof(VacProgressEntry);
+
+	VacuumWorkerProgressHash = ShmemInitHash("Vacuum Progress Hash",
+											 max_table_size,
+											 max_table_size,
+											 &info,
+											 HASH_ELEM | HASH_BLOBS);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 40b7bca5a9..04ce4a45d1 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..9b465e12cc 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int		 leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..83ecad4e4a 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -296,6 +297,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	vacuum_worker_init();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/storage/lmgr/lwlocknames.txt b/src/backend/storage/lmgr/lwlocknames.txt
index 6c7cf6c295..9a5fa0a0e0 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
+VacuumWorkerProgressLock			48
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index fd993d0d5f..f90c39d9b9 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,8 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
+#include "commands/vacuum.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -452,11 +454,15 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 
 /*
  * Returns command progress information for the named command.
+ *
+ * A command type can optionally define a callback function
+ * which will derive Datum values rather than use values
+ * directly from the backends progress array.
  */
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + 3
+#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + PGSTAT_NUM_PROGRESS_COMMON
 	int			num_backends = pgstat_fetch_stat_numbackends();
 	int			curr_backend;
 	char	   *cmd = text_to_cstring(PG_GETARG_TEXT_PP(0));
@@ -543,15 +549,19 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		{
 			values[2] = ObjectIdGetDatum(beentry->st_progress_command_target);
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				values[i + 3] = Int64GetDatum(beentry->st_progress_param[i]);
+				values[i + PGSTAT_NUM_PROGRESS_COMMON] = Int64GetDatum(beentry->st_progress_param[i]);
 		}
 		else
 		{
 			nulls[2] = true;
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				nulls[i + 3] = true;
+				nulls[i + PGSTAT_NUM_PROGRESS_COMMON] = true;
 		}
 
+		/* Call the command specific function to override datum values */
+		if (pg_strcasecmp(cmd, "VACUUM") == 0)
+			set_vaccum_worker_progress(values);
+
 		tuplestore_putvalues(tupstore, tupdesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..c0dd1d7ab5 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED       8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
@@ -151,4 +153,7 @@
 #define PROGRESS_COPY_TYPE_PIPE 3
 #define PROGRESS_COPY_TYPE_CALLBACK 4
 
+/* Number of common fields at the start of  progress views */
+#define PGSTAT_NUM_PROGRESS_COMMON 3
+
 #endif
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index d64f6268f2..8661516ae2 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -336,4 +336,11 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+/* in commands/vacuumparallel.c */
+extern void vacuum_worker_init(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+extern void set_vaccum_worker_progress(Datum *values);
+
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ac468568a1..d70a176514 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

#39Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#38)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Mar 3, 2022 at 2:08 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

If the failsafe kicks in midway through a vacuum, the number indexes_total will not be reset to 0. If INDEX_CLEANUP is turned off, then the value will be 0 at the start of the vacuum.

The way that this works with num_index_scans is that we "round up"
when there has been non-zero work in lazy_vacuum_all_indexes(), but
not if the precheck in lazy_vacuum_all_indexes() fails. That seems
like a good model to generalize from here. Note that this makes
INDEX_CLEANUP=off affect num_index_scans in much the same way as a
VACUUM where the failsafe kicks in very early, during the initial heap
pass. That is, if the failsafe kicks in before we reach lazy_vacuum()
for the first time (which is not unlikely), or even in the
lazy_vacuum_all_indexes() precheck, then num_index_scans will remain
at 0, just like INDEX_CLEANUP=off.

The actual failsafe WARNING shows num_index_scans, possibly before it
gets incremented one last time (by "rounding up"). So it's reasonably
clear how this all works from that context (assuming that the
autovacuum logging stuff, which reports num_index_scans, outputs a
report for a table where the failsafe kicked in).

I am confused. If failsafe kicks in during the middle of a vacuum, I
(perhaps naively) would expect indexes_total and indexes_processed to go to
zero, and I'd expect to no longer see the "vacuuming indexes" and "cleaning
up indexes" phases. Otherwise, how would I know that we are now skipping
indexes? Of course, you won't have any historical context about the index
work done before failsafe kicked in, but IMO it is misleading to still
include it in the progress view.

After speaking with Nathan offline, the best forward is to reset indexes_total and indexes_processed to 0 after the start of "vacuuming indexes" or "cleaning up indexes" phase.

+1

+/*
+ * vacuum_worker_init --- initialize this module's shared memory hash
+ * to track the progress of a vacuum worker
+ */
+void
+vacuum_worker_init(void)
+{
+       HASHCTL     info;
+       long        max_table_size = GetMaxBackends();
+
+       VacuumWorkerProgressHash = NULL;
+
+       info.keysize = sizeof(pid_t);
+       info.entrysize = sizeof(VacProgressEntry);
+
+       VacuumWorkerProgressHash = ShmemInitHash("Vacuum Progress Hash",
+
                  max_table_size,
+
                  max_table_size,
+
                  &info,
+
                  HASH_ELEM | HASH_BLOBS);
+}

It seems to me that creating a shmem hash with max_table_size entries
for parallel vacuum process tracking is too much. IIRC an old patch
had parallel vacuum workers advertise its progress and changed the
pg_stat_progress_vacuum view so that it aggregates the results
including workers' stats. I think it’s better than the current one.
Why did you change that?

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#40Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#39)
Re: Add index scan progress to pg_stat_progress_vacuum
+    +/*
+    + * vacuum_worker_init --- initialize this module's shared memory hash
+    + * to track the progress of a vacuum worker
+    + */
+    +void
+    +vacuum_worker_init(void)
+    +{
+    +       HASHCTL     info;
+    +       long        max_table_size = GetMaxBackends();
+    +
+    +       VacuumWorkerProgressHash = NULL;
+    +
+    +       info.keysize = sizeof(pid_t);
+    +       info.entrysize = sizeof(VacProgressEntry);
+    +
+    +       VacuumWorkerProgressHash = ShmemInitHash("Vacuum Progress Hash",
+    +
+                      max_table_size,
+    +
+                      max_table_size,
+    +
+                      &info,
+    +
+                      HASH_ELEM | HASH_BLOBS);
+    +}
+    It seems to me that creating a shmem hash with max_table_size entries
+    for parallel vacuum process tracking is too much. IIRC an old patch
+    had parallel vacuum workers advertise its progress and changed the
+    pg_stat_progress_vacuum view so that it aggregates the results
+    including workers' stats. I think it’s better than the current one.
+    Why did you change that?

+ Regards,

I was trying to avoid a shared memory to track completed indexes, but aggregating stats does not work with parallel vacuums. This is because a parallel worker will exit before the vacuum completes causing the aggregated total to be wrong.

For example

Leader_pid advertises it completed 2 indexes
Parallel worker advertises it completed 2 indexes

When aggregating we see 4 indexes completed.

After the parallel worker exits, the aggregation will show only 2 indexes completed.

--
Sami Imseih
Amazon Web Services

#41Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#40)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Mar 9, 2022 at 12:41 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

+    +/*
+    + * vacuum_worker_init --- initialize this module's shared memory hash
+    + * to track the progress of a vacuum worker
+    + */
+    +void
+    +vacuum_worker_init(void)
+    +{
+    +       HASHCTL     info;
+    +       long        max_table_size = GetMaxBackends();
+    +
+    +       VacuumWorkerProgressHash = NULL;
+    +
+    +       info.keysize = sizeof(pid_t);
+    +       info.entrysize = sizeof(VacProgressEntry);
+    +
+    +       VacuumWorkerProgressHash = ShmemInitHash("Vacuum Progress Hash",
+    +
+                      max_table_size,
+    +
+                      max_table_size,
+    +
+                      &info,
+    +
+                      HASH_ELEM | HASH_BLOBS);
+    +}
+    It seems to me that creating a shmem hash with max_table_size entries
+    for parallel vacuum process tracking is too much. IIRC an old patch
+    had parallel vacuum workers advertise its progress and changed the
+    pg_stat_progress_vacuum view so that it aggregates the results
+    including workers' stats. I think it’s better than the current one.
+    Why did you change that?

+ Regards,

I was trying to avoid a shared memory to track completed indexes, but aggregating stats does not work with parallel vacuums. This is because a parallel worker will exit before the vacuum completes causing the aggregated total to be wrong.

For example

Leader_pid advertises it completed 2 indexes
Parallel worker advertises it completed 2 indexes

When aggregating we see 4 indexes completed.

After the parallel worker exits, the aggregation will show only 2 indexes completed.

Indeed.

It might have already been discussed but other than using a new shmem
hash for parallel vacuum, I wonder if we can allow workers to change
the leader’s progress information. It would break the assumption that
the backend status entry is modified by its own backend, though. But
it might help for progress updates of other parallel operations too.
This essentially does the same thing as what the current patch does
but it doesn't require a new shmem hash.

Another idea I come up with is that the parallel vacuum leader checks
PVIndStats.status and updates how many indexes are processed to its
progress information. The leader can check it and update the progress
information before and after index vacuuming. And possibly we can add
a callback to the main loop of index AM's bulkdelete and vacuumcleanup
so that the leader can periodically make it up-to-date.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#42Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#41)
Re: Add index scan progress to pg_stat_progress_vacuum

Indeed.

It might have already been discussed but other than using a new shmem
hash for parallel vacuum, I wonder if we can allow workers to change
the leader’s progress information. It would break the assumption that
the backend status entry is modified by its own backend, though. But
it might help for progress updates of other parallel operations too.
This essentially does the same thing as what the current patch does
but it doesn't require a new shmem hash.

I experimented with this idea, but it did not work. The idea would have been to create a pgstat_progress_update function that takes the leader pid, however infrastructure does not exist to allow one backend to manipulate another backends backend status array.
pgstat_fetch_stat_beentry returns a local copy only.

Another idea I come up with is that the parallel vacuum leader checks
PVIndStats.status and updates how many indexes are processed to its
progress information. The leader can check it and update the progress
information before and after index vacuuming. And possibly we can add
a callback to the main loop of index AM's bulkdelete and vacuumcleanup
so that the leader can periodically make it up-to-date.

Regards,

The PVIndStats idea is also one I experimented with but it did not work. The reason being the backend checking the progress needs to do a shm_toc_lookup to access the data, but they are not prepared to do so.

I have not considered the callback in the index AM's bulkdelete and vacuumcleanup, but I can imagine this is not possible since a leader could be busy vacuuming rather than updating counters, but I may be misunderstanding the suggestion.

--
Sami Imseih
Amazon Web Services

#43Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#42)
3 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Attached is the latest revision of the patch(s). Renamed the patches correctly for Cfbot.

--
Sami Imseih
Amazon Web Services

Attachments:

v3-0001-Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v3-0001-Show-progress-for-index-vacuums.patchDownload
From fe37433fd4304ddf19be1f8c2f637f79d7fc2ebb Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Wed, 9 Mar 2022 01:54:08 +0000
Subject: [PATCH v3 1/3] Show progress for index vacuums

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml             |  24 ++++
 src/backend/access/heap/vacuumlazy.c     | 156 ++++++++++++++++++++++-
 src/backend/catalog/system_views.sql     |   3 +-
 src/backend/commands/vacuumparallel.c    |   7 +
 src/backend/storage/ipc/ipci.c           |   2 +
 src/backend/storage/lmgr/lwlocknames.txt |   1 +
 src/backend/utils/adt/pgstatfuncs.c      |  16 ++-
 src/include/commands/progress.h          |   5 +
 src/include/commands/vacuum.h            |   7 +
 src/test/regress/expected/rules.out      |   4 +-
 10 files changed, 215 insertions(+), 10 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 9fb62fec8e..140fbd5fb9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6227,6 +6227,30 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 40101e0cb8..4c91e673d1 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -244,6 +245,16 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/*
+ * Structs for tracking shared Progress information
+ * amongst worker ( and leader ) processes of a vacuum.
+ */
+typedef struct VacProgressEntry
+{
+	pid_t	leader_pid;
+	int		indexes_total;
+	int		indexes_processed;
+} VacProgressEntry;
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel, int nworkers);
@@ -290,6 +301,7 @@ static void update_vacuum_error_info(LVRelState *vacrel,
 static void restore_vacuum_error_info(LVRelState *vacrel,
 									  const LVSavedErrInfo *saved_vacrel);
 
+static HTAB *VacuumWorkerProgressHash;
 
 /*
  *	heap_vacuum_rel() -- perform VACUUM for one heap relation
@@ -2305,9 +2317,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	/* Report that we are now vacuuming indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_completed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2317,6 +2333,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2327,9 +2345,22 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * Outsource everything to parallel variant.
+		 *
+		 * * parallel_vacuum_bulkdel_all_indexes will call vacuum_worker_update
+		 * which updates shared memory for the index progress. To ensure shared
+		 * memory cleanup, do the work with PG_ENSURE_ERROR_CLEANUP.
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs,
+												vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2339,6 +2370,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			allindexes = false;
 	}
 
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 	/*
 	 * We delete all LP_DEAD items from the first heap pass in all indexes on
 	 * each call here (except calls where we choose to do the failsafe). This
@@ -2617,6 +2652,8 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2649,12 +2686,15 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 	/* Report that we are now cleaning up indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/* Advertise the number of indexes we are cleaning up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		double		reltuples = vacrel->new_rel_tuples;
 		bool		estimated_count =
 		vacrel->scanned_pages < vacrel->rel_pages;
+		int indexes_completed = 1;
 
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
@@ -2664,15 +2704,32 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
+
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * See the lazy_vacuum_all_indexes comments
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
 											vacrel->num_index_scans,
 											(vacrel->scanned_pages < vacrel->rel_pages));
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 	}
+
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 }
 
 /*
@@ -3453,3 +3510,92 @@ restore_vacuum_error_info(LVRelState *vacrel,
 	vacrel->offnum = saved_vacrel->offnum;
 	vacrel->phase = saved_vacrel->phase;
 }
+
+/*
+ * vacuum_worker_update --- sets the number of indexes processed so far
+ * in a parallel vacuum.
+ */
+void
+vacuum_worker_update(int leader_pid)
+{
+	VacProgressEntry *entry;
+	bool found;
+
+	LWLockAcquire(VacuumWorkerProgressLock, LW_EXCLUSIVE);
+
+	entry = (VacProgressEntry *) hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_ENTER_NULL, &found);
+
+	if (!entry)
+		elog(ERROR, "cannot allocate shared memory for vacuum worker progress");
+
+	if (!found)
+		entry->indexes_processed = 1;
+	else
+		entry->indexes_processed++;
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_end --- remove the leader_pid of a completed parallel vacuum
+ */
+void
+vacuum_worker_end(int leader_pid)
+{
+	LWLockAcquire(VacuumWorkerProgressLock, LW_EXCLUSIVE);
+
+	hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_REMOVE, NULL);
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_end wrapped as an on_shmem_exit callback function
+ */
+void
+vacuum_worker_end_callback(int code, Datum arg)
+{
+	vacuum_worker_end(DatumGetInt32(arg));
+}
+
+/*
+ * set_vaccum_worker_progress --- updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+set_vaccum_worker_progress(Datum *values)
+{
+	VacProgressEntry *entry;
+	int leader_pid = values[0];
+
+	LWLockAcquire(VacuumWorkerProgressLock, LW_SHARED);
+
+	entry = (VacProgressEntry *) hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_FIND, NULL);
+
+	if (entry != NULL)
+		values[PGSTAT_NUM_PROGRESS_COMMON + PROGRESS_VACUUM_INDEXES_COMPLETED] = entry->indexes_processed;
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_init --- initialize this module's shared memory hash
+ * to track the progress of a vacuum worker
+ */
+void
+vacuum_worker_init(void)
+{
+	HASHCTL     info;
+	long        max_table_size = GetMaxBackends();
+
+	VacuumWorkerProgressHash = NULL;
+
+	info.keysize = sizeof(pid_t);
+	info.entrysize = sizeof(VacProgressEntry);
+
+	VacuumWorkerProgressHash = ShmemInitHash("Vacuum Progress Hash",
+											 max_table_size,
+											 max_table_size,
+											 &info,
+											 HASH_ELEM | HASH_BLOBS);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 40b7bca5a9..04ce4a45d1 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..9b465e12cc 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int		 leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..83ecad4e4a 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -296,6 +297,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	vacuum_worker_init();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/storage/lmgr/lwlocknames.txt b/src/backend/storage/lmgr/lwlocknames.txt
index 6c7cf6c295..9a5fa0a0e0 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
+VacuumWorkerProgressLock			48
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index eff45b16f2..ad7327fbd9 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,8 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
+#include "commands/vacuum.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -452,11 +454,15 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 
 /*
  * Returns command progress information for the named command.
+ *
+ * A command type can optionally define a callback function
+ * which will derive Datum values rather than use values
+ * directly from the backends progress array.
  */
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + 3
+#define PG_STAT_GET_PROGRESS_COLS  PGSTAT_NUM_PROGRESS_PARAM + PGSTAT_NUM_PROGRESS_COMMON
 	int			num_backends = pgstat_fetch_stat_numbackends();
 	int			curr_backend;
 	char	   *cmd = text_to_cstring(PG_GETARG_TEXT_PP(0));
@@ -518,15 +524,19 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		{
 			values[2] = ObjectIdGetDatum(beentry->st_progress_command_target);
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				values[i + 3] = Int64GetDatum(beentry->st_progress_param[i]);
+				values[i + PGSTAT_NUM_PROGRESS_COMMON] = Int64GetDatum(beentry->st_progress_param[i]);
 		}
 		else
 		{
 			nulls[2] = true;
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				nulls[i + 3] = true;
+				nulls[i + PGSTAT_NUM_PROGRESS_COMMON] = true;
 		}
 
+		/* Call the command specific function to override datum values */
+		if (pg_strcasecmp(cmd, "VACUUM") == 0)
+			set_vaccum_worker_progress(values);
+
 		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..c0dd1d7ab5 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED       8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
@@ -151,4 +153,7 @@
 #define PROGRESS_COPY_TYPE_PIPE 3
 #define PROGRESS_COPY_TYPE_CALLBACK 4
 
+/* Number of common fields at the start of  progress views */
+#define PGSTAT_NUM_PROGRESS_COMMON 3
+
 #endif
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index d64f6268f2..8661516ae2 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -336,4 +336,11 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+/* in commands/vacuumparallel.c */
+extern void vacuum_worker_init(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+extern void set_vaccum_worker_progress(Datum *values);
+
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ac468568a1..d70a176514 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

v3-0002-Expose-indexes-being-processed-in-a-VACUUM-operat.patchapplication/octet-stream; name=v3-0002-Expose-indexes-being-processed-in-a-VACUUM-operat.patchDownload
From bb6561bf2ba6ba71e4409e2ad9daa034760c068c Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Wed, 9 Mar 2022 01:58:18 +0000
Subject: [PATCH v3 2/3] Expose indexes being processed in a VACUUM operation.

A new view called pg_stat_progress_vacuum_index to show the indexrelid
being vacuumed or cleaned during a vacuum. The view also shows the
number of tuples removed for the index during the vacuuming indexes phase.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 doc/src/sgml/monitoring.sgml          | 108 ++++++++++++++++++++++++++
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  15 +++-
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  39 ++++++++++
 src/backend/commands/vacuumparallel.c |  12 +++
 src/backend/utils/adt/pgstatfuncs.c   |   2 +
 src/include/commands/progress.h       |  17 ++--
 src/include/utils/backend_progress.h  |   1 +
 src/test/regress/expected/rules.out   |  34 ++++++++
 13 files changed, 233 insertions(+), 7 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 140fbd5fb9..3fdfc6a4ce 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend (including autovacuum worker processes) that is
+       currently performing the <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal> phase of the vacuum, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6255,6 +6264,105 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    </tgroup>
   </table>
 
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
+     </tbody>
+    </tgroup>
+   </table>
+
   <table id="vacuum-phases">
    <title>VACUUM Phases</title>
    <tgroup cols="2">
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 4c91e673d1..53a9c7f4d6 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -360,6 +360,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -2329,12 +2330,19 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
 
+			/* Advertise we are done vacuuming indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2348,7 +2356,7 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		/*
 		 * Outsource everything to parallel variant.
 		 *
-		 * * parallel_vacuum_bulkdel_all_indexes will call vacuum_worker_update
+		 * parallel_vacuum_bulkdel_all_indexes will call vacuum_worker_update
 		 * which updates shared memory for the index progress. To ensure shared
 		 * memory cleanup, do the work with PG_ENSURE_ERROR_CLEANUP.
 		 */
@@ -2701,12 +2709,17 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
 
+			/* Advertise we are done cleaning indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 04ce4a45d1..5bf8380ea4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1131,6 +1131,45 @@ CREATE VIEW pg_stat_progress_vacuum AS
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid,
+        S.datid,
+        S.datname,
+        S.indexrelid,
+        S.leader_pid,
+        CASE S.phase WHEN 2 THEN 'vacuuming indexes'
+                     WHEN 4 THEN 'cleaning up indexes'
+                     WHEN 7 THEN 'vacuuming indexes'
+                     WHEN 8 THEN 'cleaning up indexes'
+                     END AS phase,
+        S.tuples_removed
+    FROM (
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        WHERE S.param1 IN (2, 4, 7, 8) AND S.param10 > 0
+        UNION ALL
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM_PARALLEL') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        ) AS S
+    WHERE S.phase IN (2, 4, 7, 8) AND S.indexrelid > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 9b465e12cc..711fede2d9 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -845,14 +845,19 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
@@ -888,6 +893,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -972,6 +981,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1042,6 +1053,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
+	pgstat_progress_end_command();
 	FreeAccessStrategy(pvs.bstrategy);
 }
 
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index ad7327fbd9..e0e12570f5 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -472,6 +472,8 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 	/* Translate command name into command type code. */
 	if (pg_strcasecmp(cmd, "VACUUM") == 0)
 		cmdtype = PROGRESS_COMMAND_VACUUM;
+	else if (pg_strcasecmp(cmd, "VACUUM_PARALLEL") == 0)
+		cmdtype = PROGRESS_COMMAND_VACUUM_PARALLEL;
 	else if (pg_strcasecmp(cmd, "ANALYZE") == 0)
 		cmdtype = PROGRESS_COMMAND_ANALYZE;
 	else if (pg_strcasecmp(cmd, "CLUSTER") == 0)
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index c0dd1d7ab5..bcbf3279bd 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,14 +27,19 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_TOTAL_INDEXES			7
 #define PROGRESS_VACUUM_INDEXES_COMPLETED       8
+#define PROGRESS_VACUUM_INDEXRELID				9
+#define PROGRESS_VACUUM_TUPLES_REMOVED			10
+#define PROGRESS_VACUUM_LEADER_PID				11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
-#define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
-#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX		2
-#define PROGRESS_VACUUM_PHASE_VACUUM_HEAP		3
-#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP		4
-#define PROGRESS_VACUUM_PHASE_TRUNCATE			5
-#define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP		6
+#define PROGRESS_VACUUM_PHASE_SCAN_HEAP					1
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX				2
+#define PROGRESS_VACUUM_PHASE_VACUUM_HEAP				3
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP				4
+#define PROGRESS_VACUUM_PHASE_TRUNCATE					5
+#define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP				6
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL		7
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL	8
 
 /* Progress parameters for analyze */
 #define PROGRESS_ANALYZE_PHASE						0
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..4651e45c40 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -23,6 +23,7 @@ typedef enum ProgressCommandType
 {
 	PROGRESS_COMMAND_INVALID,
 	PROGRESS_COMMAND_VACUUM,
+	PROGRESS_COMMAND_VACUUM_PARALLEL,
 	PROGRESS_COMMAND_ANALYZE,
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d70a176514..709c7d9613 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2007,6 +2007,40 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    s.datname,
+    s.indexrelid,
+    s.leader_pid,
+        CASE s.phase
+            WHEN 2 THEN 'vacuuming indexes'::text
+            WHEN 4 THEN 'cleaning up indexes'::text
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.tuples_removed
+   FROM ( SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))
+          WHERE ((s_1.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s_1.param10 > 0))
+        UNION ALL
+         SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM_PARALLEL'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))) s
+  WHERE ((s.phase = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s.indexrelid > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

v3-0003-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchapplication/octet-stream; name=v3-0003-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchDownload
From 50be6ff7d719046e6fb855934e0b0120ede82627 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Wed, 9 Mar 2022 01:59:15 +0000
Subject: [PATCH v3 3/3] Rename "index_vacuum_count" in pg_stat_pogress_vacuum.

Commit 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa introduces 2 new columns
to track index vacuum/cleanup progress. The columns have "index" in the
name. To make it clear what the existing "index_vacuum_count" column
refers to, which is the cycle count of the index vacuum, this change
renames the column to "index_vacuum_count"

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml         | 2 +-
 src/backend/catalog/system_views.sql | 2 +-
 src/test/regress/expected/rules.out  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3fdfc6a4ce..263d155044 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6210,7 +6210,7 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>index_vacuum_count</structfield> <type>bigint</type>
+       <structfield>index_vacuum_cycle_count</structfield> <type>bigint</type>
       </para>
       <para>
        Number of completed index vacuum cycles.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5bf8380ea4..1bddc9867e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1125,7 +1125,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 6 THEN 'performing final cleanup'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_cycle_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 709c7d9613..bc3bac25b2 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2000,7 +2000,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param2 AS heap_blks_total,
     s.param3 AS heap_blks_scanned,
     s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
+    s.param5 AS index_vacuum_cycle_count,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-- 
2.32.0

#44Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#42)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Mar 9, 2022 at 11:35 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Indeed.

It might have already been discussed but other than using a new shmem
hash for parallel vacuum, I wonder if we can allow workers to change
the leader’s progress information. It would break the assumption that
the backend status entry is modified by its own backend, though. But
it might help for progress updates of other parallel operations too.
This essentially does the same thing as what the current patch does
but it doesn't require a new shmem hash.

I experimented with this idea, but it did not work. The idea would have been to create a pgstat_progress_update function that takes the leader pid, however infrastructure does not exist to allow one backend to manipulate another backends backend status array.
pgstat_fetch_stat_beentry returns a local copy only.

I think if it's a better approach we can do that including adding a
new infrastructure for it.

Another idea I come up with is that the parallel vacuum leader checks
PVIndStats.status and updates how many indexes are processed to its
progress information. The leader can check it and update the progress
information before and after index vacuuming. And possibly we can add
a callback to the main loop of index AM's bulkdelete and vacuumcleanup
so that the leader can periodically make it up-to-date.

Regards,

The PVIndStats idea is also one I experimented with but it did not work. The reason being the backend checking the progress needs to do a shm_toc_lookup to access the data, but they are not prepared to do so.

What I imagined is that the leader checks how many PVIndStats.status
is PARALLEL_INDVAC_STATUS_COMPLETED and updates the result to its
progress information as indexes_processed. That way, the backend
checking the progress can see it.

I have not considered the callback in the index AM's bulkdelete and vacuumcleanup, but I can imagine this is not possible since a leader could be busy vacuuming rather than updating counters, but I may be misunderstanding the suggestion.

Checking PVIndStats.status values is cheap. Probably the leader can
check it every 1GB index block, for example.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#45Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#44)
Re: Add index scan progress to pg_stat_progress_vacuum

I think if it's a better approach we can do that including adding a
new infrastructure for it.

+1 This is a beneficial idea, especially to other progress reporting, but I see this as a separate thread targeting the next major version.

#46Nathan Bossart
nathandbossart@gmail.com
In reply to: Imseih (AWS), Sami (#45)
Re: Add index scan progress to pg_stat_progress_vacuum

I took a look at the latest patch set.

+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>

Could we avoid resetting it to 0 unless INDEX_CLEANUP was turned off or
failsafe kicked in? It might be nice to know how many indexes the vacuum
intends to process. I don't feel too strongly about this, so if it would
add a lot of complexity, it might be okay as is.

BTreeShmemInit();
SyncScanShmemInit();
AsyncShmemInit();
+ vacuum_worker_init();

Don't we also need to add the size of the hash table to
CalculateShmemSize()?

+ * A command type can optionally define a callback function
+ * which will derive Datum values rather than use values
+ * directly from the backends progress array.

I think this can be removed.

--
Nathan Bossart
Amazon Web Services: https://aws.amazon.com

#47Imseih (AWS), Sami
simseih@amazon.com
In reply to: Nathan Bossart (#46)
Re: Add index scan progress to pg_stat_progress_vacuum

I took a look at the latest patch set.

+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>

Could we avoid resetting it to 0 unless INDEX_CLEANUP was turned off or
failsafe kicked in? It might be nice to know how many indexes the vacuum
intends to process. I don't feel too strongly about this, so if it would
add a lot of complexity, it might be okay as is.

Your suggestion is valid. On INDEX_CLEANUP it is set to 0 from the start and when failsafe kicks in it will be reset to 0. I Will remove the reset call for the common index vacuum path.

BTreeShmemInit();
SyncScanShmemInit();
AsyncShmemInit();
+ vacuum_worker_init();

Don't we also need to add the size of the hash table to
CalculateShmemSize()?

No, ShmemInitHash takes the min and max size of the hash and in turn calls ShmemInitStruct to setup the shared memory.

+ * A command type can optionally define a callback function
+ * which will derive Datum values rather than use values
+ * directly from the backends progress array.

I think this can be removed.

Good catch.

--
Sami Imseih
Amazon Web Services

#48Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#47)
Re: Add index scan progress to pg_stat_progress_vacuum

BTreeShmemInit();
SyncScanShmemInit();
AsyncShmemInit();
+ vacuum_worker_init();

Don't we also need to add the size of the hash table to
CalculateShmemSize()?

No, ShmemInitHash takes the min and max size of the hash and in turn calls ShmemInitStruct to setup the shared memory.

Sorry, I am wrong here. The size needs to be accounted for at startup.

--
Sami Imseih
Amazon Web Services

#49Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#48)
3 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Attached v4 which includes accounting for the hash size on startup, removal of the no longer needed comment in pgstatfuncs.c and a change in both code/docs to only reset the indexes_total to 0 when failsafe is triggered.
--
Sami Imseih
Amazon Web Services

Attachments:

v4-0001-Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v4-0001-Show-progress-for-index-vacuums.patchDownload
From 41ab6f0c1d6e43c9373527093424365a274ea020 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 10 Mar 2022 02:58:56 +0000
Subject: [PATCH v4 1/3] Show progress for index vacuums

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml             |  25 ++++
 src/backend/access/heap/vacuumlazy.c     | 170 ++++++++++++++++++++++-
 src/backend/catalog/system_views.sql     |   3 +-
 src/backend/commands/vacuumparallel.c    |   7 +
 src/backend/storage/ipc/ipci.c           |   3 +
 src/backend/storage/lmgr/lwlocknames.txt |   1 +
 src/backend/utils/adt/pgstatfuncs.c      |  12 +-
 src/include/commands/progress.h          |   5 +
 src/include/commands/vacuum.h            |   8 ++
 src/test/regress/expected/rules.out      |   4 +-
 10 files changed, 228 insertions(+), 10 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 9fb62fec8e..1acc741da9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6227,6 +6227,31 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when there are no indexes to process 
+       or when failsafe is triggered during the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 40101e0cb8..ccdd41ff55 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -244,6 +245,16 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/*
+ * Structs for tracking shared Progress information
+ * amongst worker ( and leader ) processes of a vacuum.
+ */
+typedef struct VacProgressEntry
+{
+	pid_t	leader_pid;
+	int		indexes_total;
+	int		indexes_processed;
+} VacProgressEntry;
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel, int nworkers);
@@ -290,6 +301,7 @@ static void update_vacuum_error_info(LVRelState *vacrel,
 static void restore_vacuum_error_info(LVRelState *vacrel,
 									  const LVSavedErrInfo *saved_vacrel);
 
+static HTAB *VacuumWorkerProgressHash;
 
 /*
  *	heap_vacuum_rel() -- perform VACUUM for one heap relation
@@ -2305,9 +2317,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	/* Report that we are now vacuuming indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_completed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2317,6 +2333,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2327,9 +2345,22 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * Outsource everything to parallel variant.
+		 *
+		 * parallel_vacuum_bulkdel_all_indexes will call vacuum_worker_update
+		 * which updates shared memory for the index progress. To ensure shared
+		 * memory cleanup, do the work with PG_ENSURE_ERROR_CLEANUP.
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs,
+												vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2339,6 +2370,9 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			allindexes = false;
 	}
 
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 	/*
 	 * We delete all LP_DEAD items from the first heap pass in all indexes on
 	 * each call here (except calls where we choose to do the failsafe). This
@@ -2617,6 +2651,8 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2649,12 +2685,15 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 	/* Report that we are now cleaning up indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/* Advertise the number of indexes we are cleaning up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		double		reltuples = vacrel->new_rel_tuples;
 		bool		estimated_count =
 		vacrel->scanned_pages < vacrel->rel_pages;
+		int indexes_completed = 1;
 
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
@@ -2664,15 +2703,29 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * See the lazy_vacuum_all_indexes comments
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
 											vacrel->num_index_scans,
 											(vacrel->scanned_pages < vacrel->rel_pages));
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 	}
+
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 }
 
 /*
@@ -3453,3 +3506,110 @@ restore_vacuum_error_info(LVRelState *vacrel,
 	vacrel->offnum = saved_vacrel->offnum;
 	vacrel->phase = saved_vacrel->phase;
 }
+
+/*
+ * vacuum_worker_update --- sets the number of indexes processed so far
+ * in a parallel vacuum.
+ */
+void
+vacuum_worker_update(int leader_pid)
+{
+	VacProgressEntry *entry;
+	bool found;
+
+	LWLockAcquire(VacuumWorkerProgressLock, LW_EXCLUSIVE);
+
+	entry = (VacProgressEntry *) hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_ENTER_NULL, &found);
+
+	if (!entry)
+		elog(ERROR, "cannot allocate shared memory for vacuum worker progress");
+
+	if (!found)
+		entry->indexes_processed = 1;
+	else
+		entry->indexes_processed++;
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_end --- remove the leader_pid of a completed parallel vacuum
+ */
+void
+vacuum_worker_end(int leader_pid)
+{
+	LWLockAcquire(VacuumWorkerProgressLock, LW_EXCLUSIVE);
+
+	hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_REMOVE, NULL);
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_end wrapped as an on_shmem_exit callback function
+ */
+void
+vacuum_worker_end_callback(int code, Datum arg)
+{
+	vacuum_worker_end(DatumGetInt32(arg));
+}
+
+/*
+ * set_vaccum_worker_progress --- updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+set_vaccum_worker_progress(Datum *values)
+{
+	VacProgressEntry *entry;
+	int leader_pid = values[0];
+
+	LWLockAcquire(VacuumWorkerProgressLock, LW_SHARED);
+
+	entry = (VacProgressEntry *) hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_FIND, NULL);
+
+	if (entry != NULL)
+		values[PGSTAT_NUM_PROGRESS_COMMON + PROGRESS_VACUUM_INDEXES_COMPLETED] = entry->indexes_processed;
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * VacuumWorkerProgressShmemInit --- initialize this module's shared memory hash
+ * to track the progress of a vacuum worker
+ */
+void
+VacuumWorkerProgressShmemInit(void)
+{
+	HASHCTL     info;
+	long        max_table_size = GetMaxBackends();
+
+	VacuumWorkerProgressHash = NULL;
+
+	info.keysize = sizeof(pid_t);
+	info.entrysize = sizeof(VacProgressEntry);
+
+	VacuumWorkerProgressHash = ShmemInitHash("Vacuum Progress Hash",
+											 max_table_size,
+											 max_table_size,
+											 &info,
+											 HASH_ELEM | HASH_BLOBS);
+}
+
+/*
+ * VacuumWorkerProgressShmemSize --- estimate shared-memory space for
+ * the worker progress table
+ */
+Size
+VacuumWorkerProgressShmemSize(void)
+{
+	Size        size = 0;
+	long        max_table_size;
+
+	/* vacuum worker progress hash table */
+	max_table_size = GetMaxBackends();
+	size = add_size(size, hash_estimate_size(max_table_size,
+											 sizeof(VacProgressEntry)));
+
+	return size;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 40b7bca5a9..04ce4a45d1 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..9b465e12cc 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int		 leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..a4bd6a14a3 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -145,6 +146,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, VacuumWorkerProgressShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -296,6 +298,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	VacuumWorkerProgressShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/storage/lmgr/lwlocknames.txt b/src/backend/storage/lmgr/lwlocknames.txt
index 6c7cf6c295..9a5fa0a0e0 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
+VacuumWorkerProgressLock			48
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index eff45b16f2..37696fb026 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,8 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
+#include "commands/vacuum.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -456,7 +458,7 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + 3
+#define PG_STAT_GET_PROGRESS_COLS  PGSTAT_NUM_PROGRESS_PARAM + PGSTAT_NUM_PROGRESS_COMMON
 	int			num_backends = pgstat_fetch_stat_numbackends();
 	int			curr_backend;
 	char	   *cmd = text_to_cstring(PG_GETARG_TEXT_PP(0));
@@ -518,15 +520,19 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		{
 			values[2] = ObjectIdGetDatum(beentry->st_progress_command_target);
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				values[i + 3] = Int64GetDatum(beentry->st_progress_param[i]);
+				values[i + PGSTAT_NUM_PROGRESS_COMMON] = Int64GetDatum(beentry->st_progress_param[i]);
 		}
 		else
 		{
 			nulls[2] = true;
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				nulls[i + 3] = true;
+				nulls[i + PGSTAT_NUM_PROGRESS_COMMON] = true;
 		}
 
+		/* Call the command specific function to override datum values */
+		if (pg_strcasecmp(cmd, "VACUUM") == 0)
+			set_vaccum_worker_progress(values);
+
 		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..c0dd1d7ab5 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED       8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
@@ -151,4 +153,7 @@
 #define PROGRESS_COPY_TYPE_PIPE 3
 #define PROGRESS_COPY_TYPE_CALLBACK 4
 
+/* Number of common fields at the start of  progress views */
+#define PGSTAT_NUM_PROGRESS_COMMON 3
+
 #endif
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index d64f6268f2..0ef360a43d 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -336,4 +336,12 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+/* in commands/vacuumparallel.c */
+extern void VacuumWorkerProgressShmemInit(void);
+extern Size VacuumWorkerProgressShmemSize(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+extern void set_vaccum_worker_progress(Datum *values);
+
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ac468568a1..d70a176514 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

v4-0002-Expose-indexes-being-processed-in-a-VACUUM-operat.patchapplication/octet-stream; name=v4-0002-Expose-indexes-being-processed-in-a-VACUUM-operat.patchDownload
From 40a1da09824ba97e6a34ed42f9aab07af5363d5f Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 10 Mar 2022 03:32:32 +0000
Subject: [PATCH v4 2/3] Expose indexes being processed in a VACUUM operation.

A new view called pg_stat_progress_vacuum_index to show the indexrelid
being vacuumed or cleaned during a vacuum. The view also shows the
number of tuples removed for the index during the vacuuming indexes phase.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 doc/src/sgml/monitoring.sgml          | 108 ++++++++++++++++++++++++++
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  14 ++++
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  39 ++++++++++
 src/backend/commands/vacuumparallel.c |  12 +++
 src/backend/utils/adt/pgstatfuncs.c   |   2 +
 src/include/commands/progress.h       |  17 ++--
 src/include/utils/backend_progress.h  |   1 +
 src/test/regress/expected/rules.out   |  34 ++++++++
 13 files changed, 233 insertions(+), 6 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1acc741da9..240d2438d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend (including autovacuum worker processes) that is
+       currently performing the <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal> phase of the vacuum, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6256,6 +6265,105 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    </tgroup>
   </table>
 
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
+     </tbody>
+    </tgroup>
+   </table>
+
   <table id="vacuum-phases">
    <title>VACUUM Phases</title>
    <tgroup cols="2">
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index ccdd41ff55..abf9109f7b 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -360,6 +360,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -2329,12 +2330,19 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
 
+			/* Advertise we are done vacuuming indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2700,11 +2708,17 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
+
+			/* Advertise we are done cleaning indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 04ce4a45d1..5bf8380ea4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1131,6 +1131,45 @@ CREATE VIEW pg_stat_progress_vacuum AS
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid,
+        S.datid,
+        S.datname,
+        S.indexrelid,
+        S.leader_pid,
+        CASE S.phase WHEN 2 THEN 'vacuuming indexes'
+                     WHEN 4 THEN 'cleaning up indexes'
+                     WHEN 7 THEN 'vacuuming indexes'
+                     WHEN 8 THEN 'cleaning up indexes'
+                     END AS phase,
+        S.tuples_removed
+    FROM (
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        WHERE S.param1 IN (2, 4, 7, 8) AND S.param10 > 0
+        UNION ALL
+        SELECT
+            S.pid AS pid,
+            S.datid AS datid,
+            D.datname AS datname,
+            S.param10 AS indexrelid,
+            S.param12 AS leader_pid,
+            S.param1 AS phase,
+            S.param11 AS tuples_removed
+        FROM pg_stat_get_progress_info('VACUUM_PARALLEL') AS S
+            LEFT JOIN pg_database D ON S.datid = D.oid
+        ) AS S
+    WHERE S.phase IN (2, 4, 7, 8) AND S.indexrelid > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 9b465e12cc..711fede2d9 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -845,14 +845,19 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
@@ -888,6 +893,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -972,6 +981,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1042,6 +1053,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
+	pgstat_progress_end_command();
 	FreeAccessStrategy(pvs.bstrategy);
 }
 
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 37696fb026..a1ef9aef49 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -468,6 +468,8 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 	/* Translate command name into command type code. */
 	if (pg_strcasecmp(cmd, "VACUUM") == 0)
 		cmdtype = PROGRESS_COMMAND_VACUUM;
+	else if (pg_strcasecmp(cmd, "VACUUM_PARALLEL") == 0)
+		cmdtype = PROGRESS_COMMAND_VACUUM_PARALLEL;
 	else if (pg_strcasecmp(cmd, "ANALYZE") == 0)
 		cmdtype = PROGRESS_COMMAND_ANALYZE;
 	else if (pg_strcasecmp(cmd, "CLUSTER") == 0)
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index c0dd1d7ab5..bcbf3279bd 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,14 +27,19 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_TOTAL_INDEXES			7
 #define PROGRESS_VACUUM_INDEXES_COMPLETED       8
+#define PROGRESS_VACUUM_INDEXRELID				9
+#define PROGRESS_VACUUM_TUPLES_REMOVED			10
+#define PROGRESS_VACUUM_LEADER_PID				11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
-#define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
-#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX		2
-#define PROGRESS_VACUUM_PHASE_VACUUM_HEAP		3
-#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP		4
-#define PROGRESS_VACUUM_PHASE_TRUNCATE			5
-#define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP		6
+#define PROGRESS_VACUUM_PHASE_SCAN_HEAP					1
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX				2
+#define PROGRESS_VACUUM_PHASE_VACUUM_HEAP				3
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP				4
+#define PROGRESS_VACUUM_PHASE_TRUNCATE					5
+#define PROGRESS_VACUUM_PHASE_FINAL_CLEANUP				6
+#define PROGRESS_VACUUM_PHASE_VACUUM_INDEX_PARALLEL		7
+#define PROGRESS_VACUUM_PHASE_INDEX_CLEANUP_PARALLEL	8
 
 /* Progress parameters for analyze */
 #define PROGRESS_ANALYZE_PHASE						0
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..4651e45c40 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -23,6 +23,7 @@ typedef enum ProgressCommandType
 {
 	PROGRESS_COMMAND_INVALID,
 	PROGRESS_COMMAND_VACUUM,
+	PROGRESS_COMMAND_VACUUM_PARALLEL,
 	PROGRESS_COMMAND_ANALYZE,
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d70a176514..709c7d9613 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2007,6 +2007,40 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    s.datname,
+    s.indexrelid,
+    s.leader_pid,
+        CASE s.phase
+            WHEN 2 THEN 'vacuuming indexes'::text
+            WHEN 4 THEN 'cleaning up indexes'::text
+            WHEN 7 THEN 'vacuuming indexes'::text
+            WHEN 8 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.tuples_removed
+   FROM ( SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))
+          WHERE ((s_1.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s_1.param10 > 0))
+        UNION ALL
+         SELECT s_1.pid,
+            s_1.datid,
+            d.datname,
+            s_1.param10 AS indexrelid,
+            s_1.param12 AS leader_pid,
+            s_1.param1 AS phase,
+            s_1.param11 AS tuples_removed
+           FROM (pg_stat_get_progress_info('VACUUM_PARALLEL'::text) s_1(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+             LEFT JOIN pg_database d ON ((s_1.datid = d.oid)))) s
+  WHERE ((s.phase = ANY (ARRAY[(2)::bigint, (4)::bigint, (7)::bigint, (8)::bigint])) AND (s.indexrelid > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

v4-0003-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchapplication/octet-stream; name=v4-0003-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchDownload
From 7aa0140b73b448b64852235e36e34e23b3f6557d Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 10 Mar 2022 03:33:31 +0000
Subject: [PATCH v4 3/3] Rename "index_vacuum_count" in pg_stat_pogress_vacuum.

Commit 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa introduces 2 new columns
to track index vacuum/cleanup progress. The columns have "index" in the
name. To make it clear what the existing "index_vacuum_count" column
refers to, which is the cycle count of the index vacuum, this change
renames the column to "index_vacuum_count"

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml         | 2 +-
 src/backend/catalog/system_views.sql | 2 +-
 src/test/regress/expected/rules.out  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 240d2438d5..1e5af0b400 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6210,7 +6210,7 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>index_vacuum_count</structfield> <type>bigint</type>
+       <structfield>index_vacuum_cycle_count</structfield> <type>bigint</type>
       </para>
       <para>
        Number of completed index vacuum cycles.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5bf8380ea4..1bddc9867e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1125,7 +1125,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 6 THEN 'performing final cleanup'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_cycle_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 709c7d9613..bc3bac25b2 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2000,7 +2000,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param2 AS heap_blks_total,
     s.param3 AS heap_blks_scanned,
     s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
+    s.param5 AS index_vacuum_cycle_count,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-- 
2.32.0

#50Nathan Bossart
nathandbossart@gmail.com
In reply to: Imseih (AWS), Sami (#49)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Mar 10, 2022 at 09:30:57PM +0000, Imseih (AWS), Sami wrote:

Attached v4 which includes accounting for the hash size on startup, removal of the no longer needed comment in pgstatfuncs.c and a change in both code/docs to only reset the indexes_total to 0 when failsafe is triggered.

Thanks for the new patch set.

+/*
+ * Structs for tracking shared Progress information
+ * amongst worker ( and leader ) processes of a vacuum.
+ */

nitpick: Can we remove the extra spaces in the parentheses?

+    if (entry != NULL)
+        values[PGSTAT_NUM_PROGRESS_COMMON + PROGRESS_VACUUM_INDEXES_COMPLETED] = entry->indexes_processed;

What does it mean if there isn't an entry in the map? Is this actually
expected, or should we ERROR instead?

+    /* vacuum worker progress hash table */
+    max_table_size = GetMaxBackends();
+    size = add_size(size, hash_estimate_size(max_table_size,
+                                             sizeof(VacProgressEntry)));

I think the number of entries should be shared between
VacuumWorkerProgressShmemInit() and VacuumWorkerProgressShmemSize().
Otherwise, we might update one and not the other.

+        /* Call the command specific function to override datum values */
+        if (pg_strcasecmp(cmd, "VACUUM") == 0)
+            set_vaccum_worker_progress(values);

I think we should elaborate a bit more in this comment. It's difficult to
follow what this is doing without referencing the comment above
set_vacuum_worker_progress().

IMO the patches are in decent shape, and this should likely be marked as
ready-for-committer in the near future. Before doing so, I think we should
check that Sawada-san is okay with moving the deeper infrastructure changes
to a separate threaḋ.

--
Nathan Bossart
Amazon Web Services: https://aws.amazon.com

#51Imseih (AWS), Sami
simseih@amazon.com
In reply to: Nathan Bossart (#46)
3 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

nitpick: Can we remove the extra spaces in the parentheses?

fixed

What does it mean if there isn't an entry in the map? Is this actually
expected, or should we ERROR instead?

I cleaned up the code here and added comments.

I think the number of entries should be shared between
VacuumWorkerProgressShmemInit() and VacuumWorkerProgressShmemSize().
Otherwise, we might update one and not the other.

Fixed

I think we should elaborate a bit more in this comment. It's difficult to
follow what this is doing without referencing the comment above
set_vacuum_worker_progress().

More comments added

I also simplified the 0002 patch as well.

--
Sami Imseih
Amazon Web Services

Attachments:

v5-0001-Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v5-0001-Show-progress-for-index-vacuums.patchDownload
From f5b0c2f62e45e78326b038d90d741ee6260e5f9e Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Sat, 12 Mar 2022 01:38:10 +0000
Subject: [PATCH 1/1] Show progress for index vacuums

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml             |  25 +++
 src/backend/access/heap/vacuumlazy.c     | 186 ++++++++++++++++++++++-
 src/backend/catalog/system_views.sql     |   3 +-
 src/backend/commands/vacuumparallel.c    |   7 +
 src/backend/storage/ipc/ipci.c           |   3 +
 src/backend/storage/lmgr/lwlocknames.txt |   1 +
 src/backend/utils/adt/pgstatfuncs.c      |  19 ++-
 src/include/commands/progress.h          |   5 +
 src/include/commands/vacuum.h            |   8 +
 src/test/regress/expected/rules.out      |   4 +-
 10 files changed, 251 insertions(+), 10 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 9fb62fec8e..1acc741da9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6227,6 +6227,31 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when there are no indexes to process 
+       or when failsafe is triggered during the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 40101e0cb8..3075f36c26 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -126,6 +127,11 @@
  */
 #define ParallelVacuumIsActive(vacrel) ((vacrel)->pvs != NULL)
 
+/*
+ * Set the size of the progress hash table
+ */
+#define VACUUM_WORKER_PROGRESS_NUM_ENTRIES (GetMaxBackends())
+
 /* Phases of vacuum during which we report error context. */
 typedef enum
 {
@@ -244,6 +250,16 @@ typedef struct LVSavedErrInfo
 	VacErrPhase phase;
 } LVSavedErrInfo;
 
+/*
+ * Structs for tracking shared Progress information
+ * amongst worker (and leader) processes of a vacuum.
+ */
+typedef struct VacProgressEntry
+{
+	pid_t	leader_pid;
+	int		indexes_total;
+	int		indexes_processed;
+} VacProgressEntry;
 
 /* non-export function prototypes */
 static void lazy_scan_heap(LVRelState *vacrel, int nworkers);
@@ -290,6 +306,7 @@ static void update_vacuum_error_info(LVRelState *vacrel,
 static void restore_vacuum_error_info(LVRelState *vacrel,
 									  const LVSavedErrInfo *saved_vacrel);
 
+static HTAB *VacuumWorkerProgressHash;
 
 /*
  *	heap_vacuum_rel() -- perform VACUUM for one heap relation
@@ -2305,9 +2322,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	/* Report that we are now vacuuming indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_completed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2317,6 +2338,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2327,9 +2350,22 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * Outsource everything to parallel variant.
+		 *
+		 * parallel_vacuum_bulkdel_all_indexes will call vacuum_worker_update
+		 * which updates shared memory for the index progress. To ensure shared
+		 * memory cleanup, do the work with PG_ENSURE_ERROR_CLEANUP.
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs,
+												vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2339,6 +2375,9 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			allindexes = false;
 	}
 
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 	/*
 	 * We delete all LP_DEAD items from the first heap pass in all indexes on
 	 * each call here (except calls where we choose to do the failsafe). This
@@ -2617,6 +2656,8 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2649,12 +2690,15 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 	/* Report that we are now cleaning up indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/* Advertise the number of indexes we are cleaning up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
 		double		reltuples = vacrel->new_rel_tuples;
 		bool		estimated_count =
 		vacrel->scanned_pages < vacrel->rel_pages;
+		int indexes_completed = 1;
 
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
@@ -2664,15 +2708,29 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * See the lazy_vacuum_all_indexes comments
+		 */
+		PG_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, vacrel->new_rel_tuples,
 											vacrel->num_index_scans,
 											(vacrel->scanned_pages < vacrel->rel_pages));
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(vacuum_worker_end_callback, Int32GetDatum(MyProcPid));
+		vacuum_worker_end(MyProcPid);
 	}
+
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 }
 
 /*
@@ -3453,3 +3511,121 @@ restore_vacuum_error_info(LVRelState *vacrel,
 	vacrel->offnum = saved_vacrel->offnum;
 	vacrel->phase = saved_vacrel->phase;
 }
+
+/*
+ * vacuum_worker_update --- sets the number of indexes processed so far
+ * in a parallel vacuum.
+ */
+void
+vacuum_worker_update(int leader_pid)
+{
+	VacProgressEntry *entry;
+	bool found;
+
+	LWLockAcquire(VacuumWorkerProgressLock, LW_EXCLUSIVE);
+
+	entry = (VacProgressEntry *) hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_ENTER_NULL, &found);
+
+	if (!entry)
+		elog(ERROR, "cannot allocate shared memory for vacuum worker progress");
+
+	if (!found)
+		entry->indexes_processed = 1;
+	else
+		entry->indexes_processed++;
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_end --- remove the leader_pid of a completed parallel vacuum
+ */
+void
+vacuum_worker_end(int leader_pid)
+{
+	LWLockAcquire(VacuumWorkerProgressLock, LW_EXCLUSIVE);
+
+	/*
+	 * Remove from hashtable. It should always be present,
+	 * but don't complain if it's not.
+	 */
+	hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_REMOVE, NULL);
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * vacuum_worker_end wrapped as an on_shmem_exit callback function
+ */
+void
+vacuum_worker_end_callback(int code, Datum arg)
+{
+	vacuum_worker_end(DatumGetInt32(arg));
+}
+
+/*
+ * set_vaccum_worker_progress --- updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+set_vaccum_worker_progress(Datum *values)
+{
+	VacProgressEntry *entry;
+	int leader_pid = values[0];
+
+	LWLockAcquire(VacuumWorkerProgressLock, LW_SHARED);
+
+	entry = (VacProgressEntry *) hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_FIND, NULL);
+
+	/*
+	 * If an entry is not found, it is because we looked at a pid that is not involved in parallel vacuum,
+	 * therefore release the read lock and break. For non-parallel vacuums, the indexes_completed is
+	 * set in lazy_vacuum_all_indexes.
+	 */
+	if (!entry)
+	{
+		LWLockRelease(VacuumWorkerProgressLock);
+		return;
+	}
+
+	values[PGSTAT_NUM_PROGRESS_COMMON + PROGRESS_VACUUM_INDEXES_COMPLETED] = entry->indexes_processed;
+
+	LWLockRelease(VacuumWorkerProgressLock);
+}
+
+/*
+ * VacuumWorkerProgressShmemInit --- initialize this module's shared memory hash
+ * to track the progress of a vacuum worker
+ */
+void
+VacuumWorkerProgressShmemInit(void)
+{
+	HASHCTL     info;
+
+	VacuumWorkerProgressHash = NULL;
+
+	info.keysize = sizeof(pid_t);
+	info.entrysize = sizeof(VacProgressEntry);
+
+	VacuumWorkerProgressHash = ShmemInitHash("Vacuum Progress Hash",
+											 VACUUM_WORKER_PROGRESS_NUM_ENTRIES,
+											 VACUUM_WORKER_PROGRESS_NUM_ENTRIES,
+											 &info,
+											 HASH_ELEM | HASH_BLOBS);
+}
+
+/*
+ * VacuumWorkerProgressShmemSize --- estimate shared-memory space for
+ * the worker progress table
+ */
+Size
+VacuumWorkerProgressShmemSize(void)
+{
+	Size        size = 0;
+
+	/* vacuum worker progress hash table */
+	size = add_size(size, hash_estimate_size(VACUUM_WORKER_PROGRESS_NUM_ENTRIES,
+											 sizeof(VacProgressEntry)));
+
+	return size;
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 40b7bca5a9..04ce4a45d1 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..9b465e12cc 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int		 leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..a4bd6a14a3 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -145,6 +146,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, VacuumWorkerProgressShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -296,6 +298,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	VacuumWorkerProgressShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/storage/lmgr/lwlocknames.txt b/src/backend/storage/lmgr/lwlocknames.txt
index 6c7cf6c295..9a5fa0a0e0 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
+VacuumWorkerProgressLock			48
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index eff45b16f2..5526005c47 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,8 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
+#include "commands/vacuum.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -456,7 +458,7 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + 3
+#define PG_STAT_GET_PROGRESS_COLS  PGSTAT_NUM_PROGRESS_PARAM + PGSTAT_NUM_PROGRESS_COMMON
 	int			num_backends = pgstat_fetch_stat_numbackends();
 	int			curr_backend;
 	char	   *cmd = text_to_cstring(PG_GETARG_TEXT_PP(0));
@@ -518,15 +520,26 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		{
 			values[2] = ObjectIdGetDatum(beentry->st_progress_command_target);
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				values[i + 3] = Int64GetDatum(beentry->st_progress_param[i]);
+				values[i + PGSTAT_NUM_PROGRESS_COMMON] = Int64GetDatum(beentry->st_progress_param[i]);
 		}
 		else
 		{
 			nulls[2] = true;
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				nulls[i + 3] = true;
+				nulls[i + PGSTAT_NUM_PROGRESS_COMMON] = true;
 		}
 
+		/*
+		 * Before putting values in a tuple store, call a command specific function
+		 * to populate datum values that are not set directly in the backend status array.
+		 */
+		if (pg_strcasecmp(cmd, "VACUUM") == 0)
+			/*
+			 * set_vaccum_worker_progress sets indexes_completed from vacuum progress
+			 * shared memory.
+			 */
+			set_vaccum_worker_progress(values);
+
 		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..c0dd1d7ab5 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED       8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
@@ -151,4 +153,7 @@
 #define PROGRESS_COPY_TYPE_PIPE 3
 #define PROGRESS_COPY_TYPE_CALLBACK 4
 
+/* Number of common fields at the start of  progress views */
+#define PGSTAT_NUM_PROGRESS_COMMON 3
+
 #endif
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index d64f6268f2..0ef360a43d 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -336,4 +336,12 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+/* in commands/vacuumparallel.c */
+extern void VacuumWorkerProgressShmemInit(void);
+extern Size VacuumWorkerProgressShmemSize(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+extern void set_vaccum_worker_progress(Datum *values);
+
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ac468568a1..d70a176514 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

v5-0002-Expose-indexes-being-processed-in-a-VACUUM-operat.patchapplication/octet-stream; name=v5-0002-Expose-indexes-being-processed-in-a-VACUUM-operat.patchDownload
From e139f9d59dd87ef7253a308ccaae77261494c884 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Sat, 12 Mar 2022 06:23:07 +0000
Subject: [PATCH 1/1] Expose indexes being processed in a VACUUM operation.

A new view called pg_stat_progress_vacuum_index to show the indexrelid
being vacuumed or cleaned during a vacuum. The view also shows the
number of tuples removed for the index during the vacuuming indexes phase.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 doc/src/sgml/monitoring.sgml          | 108 ++++++++++++++++++++++++++
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  14 ++++
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  18 ++++-
 src/backend/commands/vacuumparallel.c |  12 +++
 src/include/commands/progress.h       |   3 +
 src/test/regress/expected/rules.out   |  17 +++-
 11 files changed, 182 insertions(+), 2 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1acc741da9..240d2438d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend (including autovacuum worker processes) that is
+       currently performing the <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal> phase of the vacuum, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6256,6 +6265,105 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    </tgroup>
   </table>
 
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
+     </tbody>
+    </tgroup>
+   </table>
+
   <table id="vacuum-phases">
    <title>VACUUM Phases</title>
    <tgroup cols="2">
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 3075f36c26..e1dbb36533 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -365,6 +365,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -2334,12 +2335,19 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
 
+			/* Advertise we are done vacuuming indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2705,11 +2713,17 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed++);
+
+			/* Advertise we are done cleaning indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 04ce4a45d1..a9e30b2248 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1129,7 +1129,23 @@ CREATE VIEW pg_stat_progress_vacuum AS
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.pid = S.param12;
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid AS pid,
+        S.datid AS datid,
+        D.datname AS datname,
+        S.param10 AS indexrelid,
+        S.param12 AS leader_pid,
+        CASE S.param1 WHEN 2 THEN 'vacuuming indexes'
+                      WHEN 4 THEN 'cleaning up indexes'
+                      END AS phase,
+        S.param11 AS tuples_removed
+    FROM pg_stat_get_progress_info('VACUUM') AS S
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.param1 IN (2, 4) AND S.param10 > 0;
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 9b465e12cc..14ffb4df6c 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -845,13 +845,18 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
 			vacuum_worker_update(pvs->shared->leader_pid);
 			break;
@@ -888,6 +893,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -972,6 +981,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1042,6 +1053,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
+	pgstat_progress_end_command();
 	FreeAccessStrategy(pvs.bstrategy);
 }
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index c0dd1d7ab5..dcbb79072f 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,6 +27,9 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_TOTAL_INDEXES			7
 #define PROGRESS_VACUUM_INDEXES_COMPLETED       8
+#define PROGRESS_VACUUM_INDEXRELID				9
+#define PROGRESS_VACUUM_TUPLES_REMOVED			10
+#define PROGRESS_VACUUM_LEADER_PID				11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d70a176514..d2c66bf034 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2006,7 +2006,22 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param8 AS indexes_total,
     s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
-     LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE (s.pid = s.param12);
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    d.datname,
+    s.param10 AS indexrelid,
+    s.param12 AS leader_pid,
+        CASE s.param1
+            WHEN 2 THEN 'vacuuming indexes'::text
+            WHEN 4 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.param11 AS tuples_removed
+   FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s.param10 > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

v5-0003-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchapplication/octet-stream; name=v5-0003-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchDownload
From 7aa0140b73b448b64852235e36e34e23b3f6557d Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 10 Mar 2022 03:33:31 +0000
Subject: [PATCH v4 3/3] Rename "index_vacuum_count" in pg_stat_pogress_vacuum.

Commit 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa introduces 2 new columns
to track index vacuum/cleanup progress. The columns have "index" in the
name. To make it clear what the existing "index_vacuum_count" column
refers to, which is the cycle count of the index vacuum, this change
renames the column to "index_vacuum_count"

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml         | 2 +-
 src/backend/catalog/system_views.sql | 2 +-
 src/test/regress/expected/rules.out  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 240d2438d5..1e5af0b400 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6210,7 +6210,7 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>index_vacuum_count</structfield> <type>bigint</type>
+       <structfield>index_vacuum_cycle_count</structfield> <type>bigint</type>
       </para>
       <para>
        Number of completed index vacuum cycles.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5bf8380ea4..1bddc9867e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1125,7 +1125,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 6 THEN 'performing final cleanup'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_cycle_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 709c7d9613..bc3bac25b2 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2000,7 +2000,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param2 AS heap_blks_total,
     s.param3 AS heap_blks_scanned,
     s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
+    s.param5 AS index_vacuum_cycle_count,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-- 
2.32.0

#52Nathan Bossart
nathandbossart@gmail.com
In reply to: Imseih (AWS), Sami (#51)
Re: Add index scan progress to pg_stat_progress_vacuum

On Sat, Mar 12, 2022 at 07:00:06AM +0000, Imseih (AWS), Sami wrote:

nitpick: Can we remove the extra spaces in the parentheses?

fixed

What does it mean if there isn't an entry in the map? Is this actually
expected, or should we ERROR instead?

I cleaned up the code here and added comments.

I think the number of entries should be shared between
VacuumWorkerProgressShmemInit() and VacuumWorkerProgressShmemSize().
Otherwise, we might update one and not the other.

Fixed

I think we should elaborate a bit more in this comment. It's difficult to
follow what this is doing without referencing the comment above
set_vacuum_worker_progress().

More comments added

I also simplified the 0002 patch as well.

These patches look pretty good to me. Barring additional feedback, I
intend to mark this as ready-for-committer early next week.

--
Nathan Bossart
Amazon Web Services: https://aws.amazon.com

#53Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#51)
Re: Add index scan progress to pg_stat_progress_vacuum

On Sat, Mar 12, 2022 at 4:00 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

nitpick: Can we remove the extra spaces in the parentheses?

fixed

What does it mean if there isn't an entry in the map? Is this actually
expected, or should we ERROR instead?

I cleaned up the code here and added comments.

I think the number of entries should be shared between
VacuumWorkerProgressShmemInit() and VacuumWorkerProgressShmemSize().
Otherwise, we might update one and not the other.

Fixed

I think we should elaborate a bit more in this comment. It's difficult to
follow what this is doing without referencing the comment above
set_vacuum_worker_progress().

More comments added

I also simplified the 0002 patch as well.

I'm still unsure the current design of 0001 patch is better than other
approaches we’ve discussed. Even users who don't use parallel vacuum
are forced to allocate shared memory for index vacuum progress, with
GetMaxBackends() entries from the beginning. Also, it’s likely to
extend the progress tracking feature for other parallel operations in
the future but I think the current design is not extensible. If we
want to do that, we will end up creating similar things for each of
them or re-creating index vacuum progress tracking feature while
creating a common infra. It might not be a problem as of now but I'm
concerned that introducing a feature that is not extensible and forces
users to allocate additional shmem might be a blocker in the future.
Looking at the precedent example, When we introduce the progress
tracking feature, we implemented it in an extensible way. On the other
hand, others in this thread seem to agree with this approach, so I'd
like to leave it to committers.

Anyway, here are some comments on v5-0001 patch:

+/* in commands/vacuumparallel.c */
+extern void VacuumWorkerProgressShmemInit(void);
+extern Size VacuumWorkerProgressShmemSize(void);
+extern void vacuum_worker_end(int leader_pid);
+extern void vacuum_worker_update(int leader_pid);
+extern void vacuum_worker_end_callback(int code, Datum arg);
+extern void set_vaccum_worker_progress(Datum *values);

These functions' body is not in vacuumparallel.c. As the comment says,
I think these functions should be implemented in vacuumparallel.c.

---
+/*
+ * set_vaccum_worker_progress --- updates the number of indexes that have been
+ * vacuumed or cleaned up in a parallel vacuum.
+ */
+void
+set_vaccum_worker_progress(Datum *values)

s/vaccum/vacuum/

---
+void
+set_vaccum_worker_progress(Datum *values)
+{
+        VacProgressEntry *entry;
+        int leader_pid = values[0];

I thik we should use DatumGetInt32().

---
+        entry = (VacProgressEntry *)
hash_search(VacuumWorkerProgressHash, &leader_pid, HASH_ENTER_NULL,
&found);
+
+        if (!entry)
+                elog(ERROR, "cannot allocate shared memory for vacuum
worker progress");

Since we raise an error in case of out of memory, I think we can use
HASH_ENTER instead of HASH_ENTER_NULL. Or do we want to emit a
detailed error message here?

---
+       VacuumWorkerProgressHash = NULL;

This line is not necessary.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#54Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#53)
4 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

I'm still unsure the current design of 0001 patch is better than other
approaches we’ve discussed. Even users who don't use parallel vacuum
are forced to allocate shared memory for index vacuum progress, with
GetMaxBackends() entries from the beginning. Also, it’s likely to
extend the progress tracking feature for other parallel operations in
the future but I think the current design is not extensible. If we
want to do that, we will end up creating similar things for each of
them or re-creating index vacuum progress tracking feature while
creating a common infra. It might not be a problem as of now but I'm
concerned that introducing a feature that is not extensible and forces
users to allocate additional shmem might be a blocker in the future.
Looking at the precedent example, When we introduce the progress
tracking feature, we implemented it in an extensible way. On the other
hand, others in this thread seem to agree with this approach, so I'd
like to leave it to committers.

Thanks for the review!

I think you make strong arguments as to why we need to take a different approach now than later.

Flaws with current patch set:

1. GetMaxBackends() is a really heavy-handed overallocation of a shared memory serving a very specific purpose.
2. Going with the approach of a vacuum specific hash breaks the design of progress which is meant to be extensible.
3. Even if we go with this current approach as an interim solution, it will be a real pain in the future.

With that said, v7 introduces the new infrastructure. 0001 includes the new infrastructure and 0002 takes advantage of this.

This approach is the following:

1. Introduces a new API called pgstat_progress_update_param_parallel along with some others support functions. This new infrastructure is in backend_progress.c

2. There is still a shared memory involved, but the size is capped to " max_worker_processes" which is the max to how many parallel workers can be doing work at any given time. The shared memory hash includes a st_progress_param array just like the Backend Status array.

typedef struct ProgressParallelEntry
{
pid_t leader_pid;
int64 st_progress_param[PGSTAT_NUM_PROGRESS_PARAM];
} ProgressParallelEntry;

3. The progress update function is "pgstat_progress_update_param_parallel" and will aggregate totals reported for a specific progress parameter

For example , it can be called lie below. In the case below, PROGRESS_VACUUM_INDEXES_COMPLETED is incremented by 1 in the shared memory entry shared by the workers and leader.

case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_COMPLETED, 1); <<-----
break;

4. pg_stat_get_progress_info will call a function called pgstat_progress_set_parallel which will set the parameter value to the total from the shared memory hash.

I believe this approach gives proper infrastructure for future use-cases of workers reporting progress -and- does not do the heavy-handed shared memory allocation.

--
Sami Imseih
Amazon Web Services

Attachments:

v7-0001-Add-infrastructure-for-parallel-progress-reportin.patchapplication/octet-stream; name=v7-0001-Add-infrastructure-for-parallel-progress-reportin.patchDownload
From 5314444d550893cf41a780c7a5cf63274e579dc7 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 15:55:57 +0000
Subject: [PATCH v7 1/1] Add infrastructure for parallel progress reporting

Infrastructure to allow a parallel worker to report
progress. In a PARALLEL command, the workers and
leader can report progress using a new pgstat_progress
API.

The progress is maintaned in a shared memory hash
table for which progress values are aggregated for
all processes involved in the command.

pg_stat_get_progress_info reads from the shared memory
hash to report the aggregated data to the caller.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 src/backend/storage/ipc/ipci.c                |   3 +
 src/backend/storage/lmgr/lwlocknames.txt      |   1 +
 src/backend/utils/activity/backend_progress.c | 166 ++++++++++++++++++
 src/backend/utils/adt/pgstatfuncs.c           |  10 +-
 src/include/commands/progress.h               |   4 +
 src/include/utils/backend_progress.h          |  11 +-
 6 files changed, 191 insertions(+), 4 deletions(-)

diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..ccb9262b97 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/progress.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -145,6 +146,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, ProgressParallelShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -296,6 +298,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	ProgressParallelShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/storage/lmgr/lwlocknames.txt b/src/backend/storage/lmgr/lwlocknames.txt
index 6c7cf6c295..4212dea7f3 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
+ProgressParallelLock				48
diff --git a/src/backend/utils/activity/backend_progress.c b/src/backend/utils/activity/backend_progress.c
index f29199725b..de69aaa8ad 100644
--- a/src/backend/utils/activity/backend_progress.c
+++ b/src/backend/utils/activity/backend_progress.c
@@ -10,10 +10,30 @@
  */
 #include "postgres.h"
 
+#include "commands/progress.h"
 #include "port/atomics.h"		/* for memory barriers */
+#include "storage/ipc.h"
+#include "storage/lmgr.h"
+#include "storage/shmem.h"
 #include "utils/backend_progress.h"
 #include "utils/backend_status.h"
 
+/*
+ * Structs for parallel progress tracking.
+ *
+ * The parallel workers and leader report progress
+ * into a hash entry with a key of the leader pid.
+ */
+typedef struct ProgressParallelEntry
+{
+	pid_t   leader_pid;
+	int64 	st_progress_param[PGSTAT_NUM_PROGRESS_PARAM];
+} ProgressParallelEntry;
+
+static HTAB *ProgressParallelHash;
+
+/* We can only have as many parallel progress entries as max_parallel_workers */
+#define PROGRESS_PARALLEL_NUM_ENTRIES max_worker_processes
 
 /*-----------
  * pgstat_progress_start_command() -
@@ -110,3 +130,149 @@ pgstat_progress_end_command(void)
 	beentry->st_progress_command_target = InvalidOid;
 	PGSTAT_END_WRITE_ACTIVITY(beentry);
 }
+
+/*-----------
+ * ProgressParallelShmemInit() -
+ *
+ * Initialize the parallel progress hash.
+ *-----------
+ */
+void
+ProgressParallelShmemInit(void)
+{
+	HASHCTL     info;
+
+	info.keysize = sizeof(pid_t);
+	info.entrysize = sizeof(ProgressParallelEntry);
+
+	ProgressParallelHash = ShmemInitHash("Parallel Progress hash",
+									   PROGRESS_PARALLEL_NUM_ENTRIES,
+									   PROGRESS_PARALLEL_NUM_ENTRIES,
+									   &info,
+									   HASH_ELEM | HASH_BLOBS);
+}
+
+/*-----------
+ * ProgressParallelShmemSize() -
+ *
+ * Calculate the size of the parallel progress hash.
+ *-----------
+ */
+Size
+ProgressParallelShmemSize(void)
+{
+   Size        size = 0;
+
+   /* parallel progress hash table */
+   size = add_size(size, hash_estimate_size(PROGRESS_PARALLEL_NUM_ENTRIES,
+											sizeof(ProgressParallelEntry)));
+
+   return size;
+}
+
+/*-----------
+ * pgstat_progress_update_param_parallel() -
+ *
+ * Update the index'th member in then st_progress_param[] of the
+ * parallel progress hash table.
+ *-----------
+ */
+void
+pgstat_progress_update_param_parallel(int leader_pid, int index, int64 val)
+{
+	ProgressParallelEntry *entry;
+	bool found;
+
+	LWLockAcquire(ProgressParallelLock, LW_EXCLUSIVE);
+
+	entry = (ProgressParallelEntry *) hash_search(ProgressParallelHash, &leader_pid, HASH_ENTER, &found);
+
+	/*
+	 * If the entry is not found, set the value for the index'th member,
+	 * else increment the current value of the index'th member.
+	 */
+	if (!found)
+		entry->st_progress_param[index] = val;
+	else
+		entry->st_progress_param[index] += val;
+
+	LWLockRelease(ProgressParallelLock);
+}
+
+/*-----------
+ * pgstat_progress_end_parallel() -
+ *
+ * This removes an entry with from the parallel progress
+ * hash table.
+ *-----------
+ */
+void
+pgstat_progress_end_parallel(int leader_pid)
+{
+	LWLockAcquire(ProgressParallelLock, LW_EXCLUSIVE);
+
+	/*
+	* Remove from hashtable. It should always be present,
+	* but don't complain if it's not.
+	*/
+	hash_search(ProgressParallelHash, &leader_pid, HASH_REMOVE, NULL);
+
+	LWLockRelease(ProgressParallelLock);
+}
+
+/*-----------
+ * pgstat_progress_end_parallel_callback() -
+ *
+ * PG_ENSURE_ERROR_CLEANUP callback. The caller is responsible
+ * for ensuring cleanup when invoking pgstat_progress_update_param_parallel.
+ *-----------
+ */
+void
+pgstat_progress_end_parallel_callback(int code, Datum arg)
+{
+	pgstat_progress_end_parallel(DatumGetInt32(arg));
+}
+
+/*-----------
+ * pgstat_progress_set_parallel() -
+ *
+ * This routine is called by pg_stat_get_progress_info
+ * to update the datum with values from the parallel progress
+ * hash.
+ *-----------
+ */
+void
+pgstat_progress_set_parallel(Datum *values)
+{
+	ProgressParallelEntry *entry;
+	/* First element of the datum is always the pid */
+	int leader_pid = values[0];
+
+	LWLockAcquire(ProgressParallelLock, LW_SHARED);
+
+	entry = (ProgressParallelEntry *) hash_search(ProgressParallelHash, &leader_pid, HASH_FIND, NULL);
+
+	/*
+	 * If an entry is not found, it is because we looked at a pid that is not involved in a parallel command,
+	 * therefore release the read lock and break.
+	 */
+	if (!entry)
+	{
+		LWLockRelease(ProgressParallelLock);
+		return;
+	}
+
+	for (int i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
+	{
+		int64 val = entry->st_progress_param[i];
+
+		/*
+		 * We only care about hash entry members that have been updated by
+		 * parallel workers ( or leader ). This is true if the member's value > 0.
+		 */
+		if (val > 0)
+			values[i + PGSTAT_NUM_PROGRESS_COMMON] = val;
+	}
+
+	LWLockRelease(ProgressParallelLock);
+}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index eff45b16f2..1856e9c3b6 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,7 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -456,7 +457,7 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + 3
+#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + PGSTAT_NUM_PROGRESS_COMMON
 	int			num_backends = pgstat_fetch_stat_numbackends();
 	int			curr_backend;
 	char	   *cmd = text_to_cstring(PG_GETARG_TEXT_PP(0));
@@ -518,15 +519,18 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		{
 			values[2] = ObjectIdGetDatum(beentry->st_progress_command_target);
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				values[i + 3] = Int64GetDatum(beentry->st_progress_param[i]);
+				values[i + PGSTAT_NUM_PROGRESS_COMMON] = Int64GetDatum(beentry->st_progress_param[i]);
 		}
 		else
 		{
 			nulls[2] = true;
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				nulls[i + 3] = true;
+				nulls[i + PGSTAT_NUM_PROGRESS_COMMON] = true;
 		}
 
+		/* Before returning the datum, set the fields from parallel progress tracking */
+		pgstat_progress_set_parallel(values);
+
 		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
 	}
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..cd3122c344 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
@@ -151,4 +153,6 @@
 #define PROGRESS_COPY_TYPE_PIPE 3
 #define PROGRESS_COPY_TYPE_CALLBACK 4
 
+/* Number of common fields at the start of progress views */
+#define PGSTAT_NUM_PROGRESS_COMMON 3
 #endif
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..291d9413c7 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -39,6 +39,15 @@ extern void pgstat_progress_update_param(int index, int64 val);
 extern void pgstat_progress_update_multi_param(int nparam, const int *index,
 											   const int64 *val);
 extern void pgstat_progress_end_command(void);
-
+/* -----------
+ * Routines for parallel command progress reporting
+ * -----------
+ */
+extern void ProgressParallelShmemInit(void);
+extern Size ProgressParallelShmemSize(void);
+extern void pgstat_progress_update_param_parallel(int leader_pid, int index, int64 val);
+extern void pgstat_progress_end_parallel(int leader_pid);
+extern void pgstat_progress_end_parallel_callback(int code, Datum arg);
+extern void pgstat_progress_set_parallel(Datum *values);
 
 #endif							/* BACKEND_PROGRESS_H */
-- 
2.32.0

v7-0002-Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v7-0002-Show-progress-for-index-vacuums.patchDownload
From 5b49df7ce65893c19008376c4cbab7e9fc5eca33 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 14:07:31 +0000
Subject: [PATCH v7 2/4] Show progress for index vacuums

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 25 ++++++++++++
 src/backend/access/heap/vacuumlazy.c  | 59 +++++++++++++++++++++++----
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c |  7 ++++
 src/test/regress/expected/rules.out   |  4 +-
 5 files changed, 89 insertions(+), 9 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 9fb62fec8e..1acc741da9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6227,6 +6227,31 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when there are no indexes to process 
+       or when failsafe is triggered during the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 87ab7775ae..65e4440dc3 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -2313,9 +2314,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	/* Report that we are now vacuuming indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_processed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2325,6 +2330,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_processed++);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2335,9 +2342,21 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * To ensure cleanup of the progress worker hash entry,
+		 * wrap parallel_vacuum_bulkdel_all_indexes in a
+		 * PG_ENSURE_ERROR_CLEANUP
+		 *
+		 */
+		PG_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		pgstat_progress_end_parallel(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2345,6 +2364,9 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		 */
 		if (lazy_check_wraparound_failsafe(vacrel))
 			allindexes = false;
+
+		/* reset index progress */
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 	}
 
 	/*
@@ -2628,6 +2650,8 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2664,9 +2688,13 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 	/* Report that we are now cleaning up indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/* Advertise the number of indexes we are cleaning up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_processed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2675,15 +2703,32 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_processed++);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, reltuples,
-											vacrel->num_index_scans,
-											estimated_count);
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * To ensure cleanup of the paralle progress hash entry,
+		 * wrap parallel_vacuum_bulkdel_all_indexes in a
+		 * PG_ENSURE_ERROR_CLEANUP
+		 *
+		 */
+		PG_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, reltuples,
+												vacrel->num_index_scans,
+												estimated_count);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		pgstat_progress_end_parallel(MyProcPid);
 	}
+
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 }
 
 /*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index bb1ac30cd1..35ea25026f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..b491728425 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int      leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_COMPLETED, 1);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_COMPLETED, 1);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ac468568a1..d70a176514 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

v7-0003-Expose-indexes-being-processed-in-a-VACUUM-operat.patchapplication/octet-stream; name=v7-0003-Expose-indexes-being-processed-in-a-VACUUM-operat.patchDownload
From 7a5244805d4694a74f073a89b2e808310810bbd1 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 14:26:20 +0000
Subject: [PATCH v7 3/4] Expose indexes being processed in a VACUUM operation.

A new view called pg_stat_progress_vacuum_index to show the indexrelid
being vacuumed or cleaned during a vacuum. The view also shows the
number of tuples removed for the index during the vacuuming indexes phase.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 doc/src/sgml/monitoring.sgml          | 108 ++++++++++++++++++++++++++
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  14 ++++
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  18 ++++-
 src/backend/commands/vacuumparallel.c |  12 +++
 src/include/commands/progress.h       |   3 +
 src/test/regress/expected/rules.out   |  17 +++-
 11 files changed, 182 insertions(+), 2 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1acc741da9..240d2438d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend (including autovacuum worker processes) that is
+       currently performing the <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal> phase of the vacuum, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6256,6 +6265,105 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    </tgroup>
   </table>
 
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
+     </tbody>
+    </tgroup>
+   </table>
+
   <table id="vacuum-phases">
    <title>VACUUM Phases</title>
    <tgroup cols="2">
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 65e4440dc3..f33faabbe9 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -348,6 +348,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -2326,12 +2327,19 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_processed++);
 
+			/* Advertise we are done vacuuming indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2700,11 +2708,17 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_processed++);
+
+			/* Advertise we are done cleaning indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 35ea25026f..88375aad0a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1129,7 +1129,23 @@ CREATE VIEW pg_stat_progress_vacuum AS
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.pid = S.param12;
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid AS pid,
+        S.datid AS datid,
+        D.datname AS datname,
+        S.param10 AS indexrelid,
+        S.param12 AS leader_pid,
+        CASE S.param1 WHEN 2 THEN 'vacuuming indexes'
+                      WHEN 4 THEN 'cleaning up indexes'
+                      END AS phase,
+        S.param11 AS tuples_removed
+    FROM pg_stat_get_progress_info('VACUUM') AS S
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.param1 IN (2, 4) AND S.param10 > 0;
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index b491728425..a8a1f221e5 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -845,13 +845,18 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
 			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_COMPLETED, 1);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
 			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_COMPLETED, 1);
 			break;
@@ -888,6 +893,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -972,6 +981,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1042,6 +1053,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
+	pgstat_progress_end_command();
 	FreeAccessStrategy(pvs.bstrategy);
 }
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index cd3122c344..879691b37d 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,6 +27,9 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_TOTAL_INDEXES			7
 #define PROGRESS_VACUUM_INDEXES_COMPLETED		8
+#define PROGRESS_VACUUM_INDEXRELID				9
+#define PROGRESS_VACUUM_TUPLES_REMOVED			10
+#define PROGRESS_VACUUM_LEADER_PID				11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d70a176514..d2c66bf034 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2006,7 +2006,22 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param8 AS indexes_total,
     s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
-     LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE (s.pid = s.param12);
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    d.datname,
+    s.param10 AS indexrelid,
+    s.param12 AS leader_pid,
+        CASE s.param1
+            WHEN 2 THEN 'vacuuming indexes'::text
+            WHEN 4 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.param11 AS tuples_removed
+   FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s.param10 > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

v7-0004-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchapplication/octet-stream; name=v7-0004-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchDownload
From c2f2074b595219d2063822eae1ea0ca87a5bd205 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 14:27:57 +0000
Subject: [PATCH v7 4/4] Rename "index_vacuum_count" in pg_stat_pogress_vacuum.

Commit 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa introduces 2 new columns
to track index vacuum/cleanup progress. The columns have "index" in the
name. To make it clear what the existing "index_vacuum_count" column
refers to, which is the cycle count of the index vacuum, this change
renames the column to "index_vacuum_count"

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml         | 2 +-
 src/backend/catalog/system_views.sql | 2 +-
 src/test/regress/expected/rules.out  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 240d2438d5..1e5af0b400 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6210,7 +6210,7 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>index_vacuum_count</structfield> <type>bigint</type>
+       <structfield>index_vacuum_cycle_count</structfield> <type>bigint</type>
       </para>
       <para>
        Number of completed index vacuum cycles.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 88375aad0a..639ad00a71 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1125,7 +1125,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 6 THEN 'performing final cleanup'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_cycle_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d2c66bf034..7a6bf3af27 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2000,7 +2000,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param2 AS heap_blks_total,
     s.param3 AS heap_blks_scanned,
     s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
+    s.param5 AS index_vacuum_cycle_count,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-- 
2.32.0

#55Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#54)
4 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Attaching a v8 due to naming convention fixes and one slight change in where index_processed is set after all indexes are vacuumed.

s/indexes_completed/indexes_processed/

--
Sami Imseih
Amazon Web Services

Attachments:

v8-0001-Add-infrastructure-for-parallel-progress-reportin.patchapplication/octet-stream; name=v8-0001-Add-infrastructure-for-parallel-progress-reportin.patchDownload
From e67da55a69f03da0725b68add27feabb0a08dbe5 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 21:32:44 +0000
Subject: [PATCH v8 1/4] Add infrastructure for parallel progress reporting

Infrastructure to allow a parallel worker to report
progress. In a PARALLEL command, the workers and
leader can report progress using a new pgstat_progress
API.

The progress is maintaned in a shared memory hash
table for which progress values are aggregated for
all processes involved in the command.

pg_stat_get_progress_info reads from the shared memory
hash to report the aggregated data to the caller.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 src/backend/storage/ipc/ipci.c                |   3 +
 src/backend/storage/lmgr/lwlocknames.txt      |   1 +
 src/backend/utils/activity/backend_progress.c | 166 ++++++++++++++++++
 src/backend/utils/adt/pgstatfuncs.c           |  10 +-
 src/include/utils/backend_progress.h          |  12 +-
 5 files changed, 188 insertions(+), 4 deletions(-)

diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..ccb9262b97 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/progress.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -145,6 +146,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, ProgressParallelShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -296,6 +298,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	ProgressParallelShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/storage/lmgr/lwlocknames.txt b/src/backend/storage/lmgr/lwlocknames.txt
index 6c7cf6c295..4212dea7f3 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
+ProgressParallelLock				48
diff --git a/src/backend/utils/activity/backend_progress.c b/src/backend/utils/activity/backend_progress.c
index f29199725b..de69aaa8ad 100644
--- a/src/backend/utils/activity/backend_progress.c
+++ b/src/backend/utils/activity/backend_progress.c
@@ -10,10 +10,30 @@
  */
 #include "postgres.h"
 
+#include "commands/progress.h"
 #include "port/atomics.h"		/* for memory barriers */
+#include "storage/ipc.h"
+#include "storage/lmgr.h"
+#include "storage/shmem.h"
 #include "utils/backend_progress.h"
 #include "utils/backend_status.h"
 
+/*
+ * Structs for parallel progress tracking.
+ *
+ * The parallel workers and leader report progress
+ * into a hash entry with a key of the leader pid.
+ */
+typedef struct ProgressParallelEntry
+{
+	pid_t   leader_pid;
+	int64 	st_progress_param[PGSTAT_NUM_PROGRESS_PARAM];
+} ProgressParallelEntry;
+
+static HTAB *ProgressParallelHash;
+
+/* We can only have as many parallel progress entries as max_parallel_workers */
+#define PROGRESS_PARALLEL_NUM_ENTRIES max_worker_processes
 
 /*-----------
  * pgstat_progress_start_command() -
@@ -110,3 +130,149 @@ pgstat_progress_end_command(void)
 	beentry->st_progress_command_target = InvalidOid;
 	PGSTAT_END_WRITE_ACTIVITY(beentry);
 }
+
+/*-----------
+ * ProgressParallelShmemInit() -
+ *
+ * Initialize the parallel progress hash.
+ *-----------
+ */
+void
+ProgressParallelShmemInit(void)
+{
+	HASHCTL     info;
+
+	info.keysize = sizeof(pid_t);
+	info.entrysize = sizeof(ProgressParallelEntry);
+
+	ProgressParallelHash = ShmemInitHash("Parallel Progress hash",
+									   PROGRESS_PARALLEL_NUM_ENTRIES,
+									   PROGRESS_PARALLEL_NUM_ENTRIES,
+									   &info,
+									   HASH_ELEM | HASH_BLOBS);
+}
+
+/*-----------
+ * ProgressParallelShmemSize() -
+ *
+ * Calculate the size of the parallel progress hash.
+ *-----------
+ */
+Size
+ProgressParallelShmemSize(void)
+{
+   Size        size = 0;
+
+   /* parallel progress hash table */
+   size = add_size(size, hash_estimate_size(PROGRESS_PARALLEL_NUM_ENTRIES,
+											sizeof(ProgressParallelEntry)));
+
+   return size;
+}
+
+/*-----------
+ * pgstat_progress_update_param_parallel() -
+ *
+ * Update the index'th member in then st_progress_param[] of the
+ * parallel progress hash table.
+ *-----------
+ */
+void
+pgstat_progress_update_param_parallel(int leader_pid, int index, int64 val)
+{
+	ProgressParallelEntry *entry;
+	bool found;
+
+	LWLockAcquire(ProgressParallelLock, LW_EXCLUSIVE);
+
+	entry = (ProgressParallelEntry *) hash_search(ProgressParallelHash, &leader_pid, HASH_ENTER, &found);
+
+	/*
+	 * If the entry is not found, set the value for the index'th member,
+	 * else increment the current value of the index'th member.
+	 */
+	if (!found)
+		entry->st_progress_param[index] = val;
+	else
+		entry->st_progress_param[index] += val;
+
+	LWLockRelease(ProgressParallelLock);
+}
+
+/*-----------
+ * pgstat_progress_end_parallel() -
+ *
+ * This removes an entry with from the parallel progress
+ * hash table.
+ *-----------
+ */
+void
+pgstat_progress_end_parallel(int leader_pid)
+{
+	LWLockAcquire(ProgressParallelLock, LW_EXCLUSIVE);
+
+	/*
+	* Remove from hashtable. It should always be present,
+	* but don't complain if it's not.
+	*/
+	hash_search(ProgressParallelHash, &leader_pid, HASH_REMOVE, NULL);
+
+	LWLockRelease(ProgressParallelLock);
+}
+
+/*-----------
+ * pgstat_progress_end_parallel_callback() -
+ *
+ * PG_ENSURE_ERROR_CLEANUP callback. The caller is responsible
+ * for ensuring cleanup when invoking pgstat_progress_update_param_parallel.
+ *-----------
+ */
+void
+pgstat_progress_end_parallel_callback(int code, Datum arg)
+{
+	pgstat_progress_end_parallel(DatumGetInt32(arg));
+}
+
+/*-----------
+ * pgstat_progress_set_parallel() -
+ *
+ * This routine is called by pg_stat_get_progress_info
+ * to update the datum with values from the parallel progress
+ * hash.
+ *-----------
+ */
+void
+pgstat_progress_set_parallel(Datum *values)
+{
+	ProgressParallelEntry *entry;
+	/* First element of the datum is always the pid */
+	int leader_pid = values[0];
+
+	LWLockAcquire(ProgressParallelLock, LW_SHARED);
+
+	entry = (ProgressParallelEntry *) hash_search(ProgressParallelHash, &leader_pid, HASH_FIND, NULL);
+
+	/*
+	 * If an entry is not found, it is because we looked at a pid that is not involved in a parallel command,
+	 * therefore release the read lock and break.
+	 */
+	if (!entry)
+	{
+		LWLockRelease(ProgressParallelLock);
+		return;
+	}
+
+	for (int i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
+	{
+		int64 val = entry->st_progress_param[i];
+
+		/*
+		 * We only care about hash entry members that have been updated by
+		 * parallel workers ( or leader ). This is true if the member's value > 0.
+		 */
+		if (val > 0)
+			values[i + PGSTAT_NUM_PROGRESS_COMMON] = val;
+	}
+
+	LWLockRelease(ProgressParallelLock);
+}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index eff45b16f2..1856e9c3b6 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,7 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -456,7 +457,7 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + 3
+#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + PGSTAT_NUM_PROGRESS_COMMON
 	int			num_backends = pgstat_fetch_stat_numbackends();
 	int			curr_backend;
 	char	   *cmd = text_to_cstring(PG_GETARG_TEXT_PP(0));
@@ -518,15 +519,18 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		{
 			values[2] = ObjectIdGetDatum(beentry->st_progress_command_target);
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				values[i + 3] = Int64GetDatum(beentry->st_progress_param[i]);
+				values[i + PGSTAT_NUM_PROGRESS_COMMON] = Int64GetDatum(beentry->st_progress_param[i]);
 		}
 		else
 		{
 			nulls[2] = true;
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				nulls[i + 3] = true;
+				nulls[i + PGSTAT_NUM_PROGRESS_COMMON] = true;
 		}
 
+		/* Before returning the datum, set the fields from parallel progress tracking */
+		pgstat_progress_set_parallel(values);
+
 		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
 	}
 
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..936eb97b64 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -31,7 +31,7 @@ typedef enum ProgressCommandType
 } ProgressCommandType;
 
 #define PGSTAT_NUM_PROGRESS_PARAM	20
-
+#define PGSTAT_NUM_PROGRESS_COMMON	3
 
 extern void pgstat_progress_start_command(ProgressCommandType cmdtype,
 										  Oid relid);
@@ -40,5 +40,15 @@ extern void pgstat_progress_update_multi_param(int nparam, const int *index,
 											   const int64 *val);
 extern void pgstat_progress_end_command(void);
 
+/* -----------
+ * Routines for parallel command progress reporting
+ * -----------
+ */
+extern void ProgressParallelShmemInit(void);
+extern Size ProgressParallelShmemSize(void);
+extern void pgstat_progress_update_param_parallel(int leader_pid, int index, int64 val);
+extern void pgstat_progress_end_parallel(int leader_pid);
+extern void pgstat_progress_end_parallel_callback(int code, Datum arg);
+extern void pgstat_progress_set_parallel(Datum *values);
 
 #endif							/* BACKEND_PROGRESS_H */
-- 
2.32.0

v8-0002-Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v8-0002-Show-progress-for-index-vacuums.patchDownload
From 56f9db40e4d1cfa3464198c841bef2a0127563f5 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 21:55:43 +0000
Subject: [PATCH v8 2/4] Show progress for index vacuums

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 25 +++++++++++
 src/backend/access/heap/vacuumlazy.c  | 60 +++++++++++++++++++++++----
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c |  7 ++++
 src/include/commands/progress.h       |  2 +
 src/test/regress/expected/rules.out   |  4 +-
 6 files changed, 92 insertions(+), 9 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 9fb62fec8e..1acc741da9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6227,6 +6227,31 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when there are no indexes to process 
+       or when failsafe is triggered during the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 87ab7775ae..047a5db494 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -2313,9 +2314,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	/* Report that we are now vacuuming indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_processed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2325,6 +2330,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, indexes_processed++);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2335,9 +2342,21 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * To ensure cleanup of the progress worker hash entry,
+		 * wrap parallel_vacuum_bulkdel_all_indexes in a
+		 * PG_ENSURE_ERROR_CLEANUP
+		 *
+		 */
+		PG_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		pgstat_progress_end_parallel(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2345,8 +2364,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		 */
 		if (lazy_check_wraparound_failsafe(vacrel))
 			allindexes = false;
+
 	}
 
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 0);
+
 	/*
 	 * We delete all LP_DEAD items from the first heap pass in all indexes on
 	 * each call here (except calls where we choose to do the failsafe). This
@@ -2628,6 +2651,8 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2664,9 +2689,13 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 	/* Report that we are now cleaning up indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/* Advertise the number of indexes we are cleaning up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_processed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2675,15 +2704,32 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, indexes_processed++);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, reltuples,
-											vacrel->num_index_scans,
-											estimated_count);
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * To ensure cleanup of the paralle progress hash entry,
+		 * wrap parallel_vacuum_bulkdel_all_indexes in a
+		 * PG_ENSURE_ERROR_CLEANUP
+		 *
+		 */
+		PG_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, reltuples,
+												vacrel->num_index_scans,
+												estimated_count);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		pgstat_progress_end_parallel(MyProcPid);
 	}
+
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 0);
 }
 
 /*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index bb1ac30cd1..35ea25026f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..f08eb837e7 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int      leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..54ee60caad 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES           7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ac468568a1..d70a176514 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

v8-0003-Expose-indexes-being-processed-in-a-VACUUM-operat.patchapplication/octet-stream; name=v8-0003-Expose-indexes-being-processed-in-a-VACUUM-operat.patchDownload
From b44359eb937b2f02878ba80fd56b94ef6130ac10 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 22:07:09 +0000
Subject: [PATCH v8 3/4] Expose indexes being processed in a VACUUM operation.

A new view called pg_stat_progress_vacuum_index to show the indexrelid
being vacuumed or cleaned during a vacuum. The view also shows the
number of tuples removed for the index during the vacuuming indexes phase.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 doc/src/sgml/monitoring.sgml          | 108 ++++++++++++++++++++++++++
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  14 ++++
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  18 ++++-
 src/backend/commands/vacuumparallel.c |  12 +++
 src/include/commands/progress.h       |   3 +
 src/test/regress/expected/rules.out   |  17 +++-
 11 files changed, 182 insertions(+), 2 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1acc741da9..240d2438d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend (including autovacuum worker processes) that is
+       currently performing the <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal> phase of the vacuum, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6256,6 +6265,105 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    </tgroup>
   </table>
 
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
+     </tbody>
+    </tgroup>
+   </table>
+
   <table id="vacuum-phases">
    <title>VACUUM Phases</title>
    <tgroup cols="2">
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 047a5db494..1cc3fae20c 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -348,6 +348,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -2326,12 +2327,19 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, indexes_processed++);
 
+			/* Advertise we are done vacuuming indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2701,11 +2709,17 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, indexes_processed++);
+
+			/* Advertise we are done cleaning indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 35ea25026f..88375aad0a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1129,7 +1129,23 @@ CREATE VIEW pg_stat_progress_vacuum AS
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.pid = S.param12;
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid AS pid,
+        S.datid AS datid,
+        D.datname AS datname,
+        S.param10 AS indexrelid,
+        S.param12 AS leader_pid,
+        CASE S.param1 WHEN 2 THEN 'vacuuming indexes'
+                      WHEN 4 THEN 'cleaning up indexes'
+                      END AS phase,
+        S.param11 AS tuples_removed
+    FROM pg_stat_get_progress_info('VACUUM') AS S
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.param1 IN (2, 4) AND S.param10 > 0;
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f08eb837e7..7d47565e05 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -845,13 +845,18 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
 			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
 			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 			break;
@@ -888,6 +893,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -972,6 +981,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1042,6 +1053,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
+	pgstat_progress_end_command();
 	FreeAccessStrategy(pvs.bstrategy);
 }
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index 54ee60caad..7e632feb7d 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,6 +27,9 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_TOTAL_INDEXES           7
 #define PROGRESS_VACUUM_INDEXES_PROCESSED		8
+#define PROGRESS_VACUUM_INDEXRELID              9
+#define PROGRESS_VACUUM_TUPLES_REMOVED          10
+#define PROGRESS_VACUUM_LEADER_PID              11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d70a176514..d2c66bf034 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2006,7 +2006,22 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param8 AS indexes_total,
     s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
-     LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE (s.pid = s.param12);
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    d.datname,
+    s.param10 AS indexrelid,
+    s.param12 AS leader_pid,
+        CASE s.param1
+            WHEN 2 THEN 'vacuuming indexes'::text
+            WHEN 4 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.param11 AS tuples_removed
+   FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s.param10 > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

v8-0004-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchapplication/octet-stream; name=v8-0004-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchDownload
From 5303ab5d4d1f11a1a0fe62f28f74b13e2df9e3a2 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 22:08:04 +0000
Subject: [PATCH v8 4/4] Rename "index_vacuum_count" in pg_stat_pogress_vacuum.

Commit 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa introduces 2 new columns
to track index vacuum/cleanup progress. The columns have "index" in the
name. To make it clear what the existing "index_vacuum_count" column
refers to, which is the cycle count of the index vacuum, this change
renames the column to "index_vacuum_count"

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml         | 2 +-
 src/backend/catalog/system_views.sql | 2 +-
 src/test/regress/expected/rules.out  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 240d2438d5..1e5af0b400 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6210,7 +6210,7 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>index_vacuum_count</structfield> <type>bigint</type>
+       <structfield>index_vacuum_cycle_count</structfield> <type>bigint</type>
       </para>
       <para>
        Number of completed index vacuum cycles.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 88375aad0a..639ad00a71 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1125,7 +1125,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 6 THEN 'performing final cleanup'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_cycle_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d2c66bf034..7a6bf3af27 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2000,7 +2000,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param2 AS heap_blks_total,
     s.param3 AS heap_blks_scanned,
     s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
+    s.param5 AS index_vacuum_cycle_count,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-- 
2.32.0

#56Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#55)
4 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Spoke to Nathan offline and fixed some more comments/nitpicks in the patch.

Attachments:

v9-0001-Add-infrastructure-for-parallel-progress-reporting.patchapplication/octet-stream; name=v9-0001-Add-infrastructure-for-parallel-progress-reporting.patchDownload
From 85c47dfb3bb72f764b9052e74a7282c19ebd9898 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Wed, 16 Mar 2022 20:39:52 +0000
Subject: [PATCH 1/1] Add infrastructure for parallel progress reporting

Infrastructure to allow a parallel worker to report
progress. In a PARALLEL command, the workers and
leader can report progress using a new pgstat_progress
API.

The progress is maintaned in a shared memory hash
table for which progress values are aggregated for
all processes involved in the command.

pg_stat_get_progress_info reads from the shared memory
hash to report the aggregated data to the caller.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228%40amazon.com
---
 src/backend/storage/ipc/ipci.c                |   3 +
 src/backend/storage/lmgr/lwlocknames.txt      |   1 +
 src/backend/utils/activity/backend_progress.c | 166 ++++++++++++++++++
 src/backend/utils/adt/pgstatfuncs.c           |  10 +-
 src/include/utils/backend_progress.h          |  12 +-
 5 files changed, 188 insertions(+), 4 deletions(-)

diff --git a/src/backend/storage/ipc/ipci.c b/src/backend/storage/ipc/ipci.c
index cd4ebe2fc5..ccb9262b97 100644
--- a/src/backend/storage/ipc/ipci.c
+++ b/src/backend/storage/ipc/ipci.c
@@ -24,6 +24,7 @@
 #include "access/twophase.h"
 #include "access/xlogrecovery.h"
 #include "commands/async.h"
+#include "commands/progress.h"
 #include "miscadmin.h"
 #include "pgstat.h"
 #include "postmaster/autovacuum.h"
@@ -145,6 +146,7 @@ CalculateShmemSize(int *num_semaphores)
 	size = add_size(size, BTreeShmemSize());
 	size = add_size(size, SyncScanShmemSize());
 	size = add_size(size, AsyncShmemSize());
+	size = add_size(size, ProgressParallelShmemSize());
 #ifdef EXEC_BACKEND
 	size = add_size(size, ShmemBackendArraySize());
 #endif
@@ -296,6 +298,7 @@ CreateSharedMemoryAndSemaphores(void)
 	BTreeShmemInit();
 	SyncScanShmemInit();
 	AsyncShmemInit();
+	ProgressParallelShmemInit();
 
 #ifdef EXEC_BACKEND
 
diff --git a/src/backend/storage/lmgr/lwlocknames.txt b/src/backend/storage/lmgr/lwlocknames.txt
index 6c7cf6c295..4212dea7f3 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
+ProgressParallelLock				48
diff --git a/src/backend/utils/activity/backend_progress.c b/src/backend/utils/activity/backend_progress.c
index f29199725b..afb191da85 100644
--- a/src/backend/utils/activity/backend_progress.c
+++ b/src/backend/utils/activity/backend_progress.c
@@ -10,10 +10,30 @@
  */
 #include "postgres.h"
 
+#include "commands/progress.h"
 #include "port/atomics.h"		/* for memory barriers */
+#include "storage/ipc.h"
+#include "storage/lmgr.h"
+#include "storage/shmem.h"
 #include "utils/backend_progress.h"
 #include "utils/backend_status.h"
 
+/*
+ * Structs for parallel progress tracking.
+ *
+ * The parallel workers and leader report progress
+ * into a hash entry with a key of the leader pid.
+ */
+typedef struct ProgressParallelEntry
+{
+	pid_t   leader_pid;
+	int64 	st_progress_param[PGSTAT_NUM_PROGRESS_PARAM];
+} ProgressParallelEntry;
+
+static HTAB *ProgressParallelHash;
+
+/* We can only have as many parallel progress entries as max_worker_processes */
+#define PROGRESS_PARALLEL_NUM_ENTRIES max_worker_processes
 
 /*-----------
  * pgstat_progress_start_command() -
@@ -110,3 +130,149 @@ pgstat_progress_end_command(void)
 	beentry->st_progress_command_target = InvalidOid;
 	PGSTAT_END_WRITE_ACTIVITY(beentry);
 }
+
+/*-----------
+ * ProgressParallelShmemInit() -
+ *
+ * Initialize the parallel progress hash.
+ *-----------
+ */
+void
+ProgressParallelShmemInit(void)
+{
+	HASHCTL     info;
+
+	info.keysize = sizeof(pid_t);
+	info.entrysize = sizeof(ProgressParallelEntry);
+
+	ProgressParallelHash = ShmemInitHash("Parallel Progress hash",
+									   PROGRESS_PARALLEL_NUM_ENTRIES,
+									   PROGRESS_PARALLEL_NUM_ENTRIES,
+									   &info,
+									   HASH_ELEM | HASH_BLOBS);
+}
+
+/*-----------
+ * ProgressParallelShmemSize() -
+ *
+ * Calculate the size of the parallel progress hash.
+ *-----------
+ */
+Size
+ProgressParallelShmemSize(void)
+{
+   Size        size = 0;
+
+   /* parallel progress hash table */
+   size = add_size(size, hash_estimate_size(PROGRESS_PARALLEL_NUM_ENTRIES,
+											sizeof(ProgressParallelEntry)));
+
+   return size;
+}
+
+/*-----------
+ * pgstat_progress_update_param_parallel() -
+ *
+ * Update the index'th member in the st_progress_param[] of the
+ * parallel progress hash table.
+ *-----------
+ */
+void
+pgstat_progress_update_param_parallel(int leader_pid, int index, int64 val)
+{
+	ProgressParallelEntry *entry;
+	bool found;
+
+	LWLockAcquire(ProgressParallelLock, LW_EXCLUSIVE);
+
+	entry = (ProgressParallelEntry *) hash_search(ProgressParallelHash, &leader_pid, HASH_ENTER, &found);
+
+	/*
+	 * If the entry is not found, set the value for the index'th member,
+	 * else increment the current value of the index'th member.
+	 */
+	if (!found)
+		entry->st_progress_param[index] = val;
+	else
+		entry->st_progress_param[index] += val;
+
+	LWLockRelease(ProgressParallelLock);
+}
+
+/*-----------
+ * pgstat_progress_end_parallel() -
+ *
+ * This removes an entry with from the parallel progress
+ * hash table.
+ *-----------
+ */
+void
+pgstat_progress_end_parallel(int leader_pid)
+{
+	LWLockAcquire(ProgressParallelLock, LW_EXCLUSIVE);
+
+	/*
+	* Remove from hashtable. It should always be present,
+	* but don't complain if it's not.
+	*/
+	hash_search(ProgressParallelHash, &leader_pid, HASH_REMOVE, NULL);
+
+	LWLockRelease(ProgressParallelLock);
+}
+
+/*-----------
+ * pgstat_progress_end_parallel_callback() -
+ *
+ * PG_ENSURE_ERROR_CLEANUP callback. The caller is responsible
+ * for ensuring cleanup when invoking pgstat_progress_update_param_parallel.
+ *-----------
+ */
+void
+pgstat_progress_end_parallel_callback(int code, Datum arg)
+{
+	pgstat_progress_end_parallel(DatumGetInt32(arg));
+}
+
+/*-----------
+ * pgstat_progress_set_parallel() -
+ *
+ * This routine is called by pg_stat_get_progress_info
+ * to update the datum with values from the parallel progress
+ * hash.
+ *-----------
+ */
+void
+pgstat_progress_set_parallel(Datum *values)
+{
+	ProgressParallelEntry *entry;
+	/* First element of the datum is always the pid */
+	int leader_pid = values[0];
+
+	LWLockAcquire(ProgressParallelLock, LW_SHARED);
+
+	entry = (ProgressParallelEntry *) hash_search(ProgressParallelHash, &leader_pid, HASH_FIND, NULL);
+
+	/*
+	 * If an entry is not found, it is because we looked at a pid that is not involved in a parallel command,
+	 * therefore release the read lock and return.
+	 */
+	if (!entry)
+	{
+		LWLockRelease(ProgressParallelLock);
+		return;
+	}
+
+	for (int i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
+	{
+		int64 val = entry->st_progress_param[i];
+
+		/*
+		 * We only care about hash entry members that have been updated by
+		 * parallel workers ( or leader ). This is true if the member's value > 0.
+		 */
+		if (val > 0)
+			values[i + PGSTAT_NUM_PROGRESS_COMMON] = val;
+	}
+
+	LWLockRelease(ProgressParallelLock);
+}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index eff45b16f2..1856e9c3b6 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -18,6 +18,7 @@
 #include "access/xlog.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -456,7 +457,7 @@ pg_stat_get_backend_idset(PG_FUNCTION_ARGS)
 Datum
 pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 {
-#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + 3
+#define PG_STAT_GET_PROGRESS_COLS	PGSTAT_NUM_PROGRESS_PARAM + PGSTAT_NUM_PROGRESS_COMMON
 	int			num_backends = pgstat_fetch_stat_numbackends();
 	int			curr_backend;
 	char	   *cmd = text_to_cstring(PG_GETARG_TEXT_PP(0));
@@ -518,15 +519,18 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		{
 			values[2] = ObjectIdGetDatum(beentry->st_progress_command_target);
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				values[i + 3] = Int64GetDatum(beentry->st_progress_param[i]);
+				values[i + PGSTAT_NUM_PROGRESS_COMMON] = Int64GetDatum(beentry->st_progress_param[i]);
 		}
 		else
 		{
 			nulls[2] = true;
 			for (i = 0; i < PGSTAT_NUM_PROGRESS_PARAM; i++)
-				nulls[i + 3] = true;
+				nulls[i + PGSTAT_NUM_PROGRESS_COMMON] = true;
 		}
 
+		/* Before returning the datum, set the fields from parallel progress tracking */
+		pgstat_progress_set_parallel(values);
+
 		tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc, values, nulls);
 	}
 
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..936eb97b64 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -31,7 +31,7 @@ typedef enum ProgressCommandType
 } ProgressCommandType;
 
 #define PGSTAT_NUM_PROGRESS_PARAM	20
-
+#define PGSTAT_NUM_PROGRESS_COMMON	3
 
 extern void pgstat_progress_start_command(ProgressCommandType cmdtype,
 										  Oid relid);
@@ -40,5 +40,15 @@ extern void pgstat_progress_update_multi_param(int nparam, const int *index,
 											   const int64 *val);
 extern void pgstat_progress_end_command(void);
 
+/* -----------
+ * Routines for parallel command progress reporting
+ * -----------
+ */
+extern void ProgressParallelShmemInit(void);
+extern Size ProgressParallelShmemSize(void);
+extern void pgstat_progress_update_param_parallel(int leader_pid, int index, int64 val);
+extern void pgstat_progress_end_parallel(int leader_pid);
+extern void pgstat_progress_end_parallel_callback(int code, Datum arg);
+extern void pgstat_progress_set_parallel(Datum *values);
 
 #endif							/* BACKEND_PROGRESS_H */
-- 
2.32.0

v9-0002-Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v9-0002-Show-progress-for-index-vacuums.patchDownload
From ff0810c22f392c6d994e84b8b9f2518bfb0f58bb Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Wed, 16 Mar 2022 21:11:28 +0000
Subject: [PATCH 1/1] Show progress for index vacuums

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 25 +++++++++++
 src/backend/access/heap/vacuumlazy.c  | 60 +++++++++++++++++++++++----
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c |  7 ++++
 src/include/commands/progress.h       |  2 +
 src/test/regress/expected/rules.out   |  4 +-
 6 files changed, 92 insertions(+), 9 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 35b2923c5e..322f156d70 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6236,6 +6236,31 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when there are no indexes to process 
+       or when failsafe is triggered during the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 87ab7775ae..0018271fa6 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -58,6 +58,7 @@
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/freespace.h"
+#include "storage/ipc.h"
 #include "storage/lmgr.h"
 #include "tcop/tcopprot.h"
 #include "utils/lsyscache.h"
@@ -2313,9 +2314,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	/* Report that we are now vacuuming indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/* Advertise the number of indexes we are vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_processed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2325,6 +2330,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, indexes_processed++);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2335,9 +2342,21 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
-											vacrel->num_index_scans);
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * To ensure cleanup of the progress worker hash entry,
+		 * wrap parallel_vacuum_bulkdel_all_indexes in a
+		 * PG_ENSURE_ERROR_CLEANUP
+		 *
+		 */
+		PG_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_bulkdel_all_indexes(vacrel->pvs, vacrel->old_live_tuples,
+												vacrel->num_index_scans);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		pgstat_progress_end_parallel(MyProcPid);
 
 		/*
 		 * Do a postcheck to consider applying wraparound failsafe now.  Note
@@ -2345,8 +2364,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		 */
 		if (lazy_check_wraparound_failsafe(vacrel))
 			allindexes = false;
+
 	}
 
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 0);
+
 	/*
 	 * We delete all LP_DEAD items from the first heap pass in all indexes on
 	 * each call here (except calls where we choose to do the failsafe). This
@@ -2628,6 +2651,8 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2664,9 +2689,13 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 	/* Report that we are now cleaning up indexes */
 	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
 								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/* Advertise the number of indexes we are cleaning up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_TOTAL_INDEXES, vacrel->nindexes);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
+		int indexes_processed = 1;
+
 		for (int idx = 0; idx < vacrel->nindexes; idx++)
 		{
 			Relation	indrel = vacrel->indrels[idx];
@@ -2675,15 +2704,32 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, indexes_processed++);
 		}
 	}
 	else
 	{
-		/* Outsource everything to parallel variant */
-		parallel_vacuum_cleanup_all_indexes(vacrel->pvs, reltuples,
-											vacrel->num_index_scans,
-											estimated_count);
+		/*
+		 * Outsource everything to parallel variant
+		 *
+		 * To ensure cleanup of the parallel progress hash entry,
+		 * wrap parallel_vacuum_bulkdel_all_indexes in a
+		 * PG_ENSURE_ERROR_CLEANUP
+		 *
+		 */
+		PG_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		{
+			parallel_vacuum_cleanup_all_indexes(vacrel->pvs, reltuples,
+												vacrel->num_index_scans,
+												estimated_count);
+		}
+		PG_END_ENSURE_ERROR_CLEANUP(pgstat_progress_end_parallel_callback, Int32GetDatum(MyProcPid));
+		pgstat_progress_end_parallel(MyProcPid);
 	}
+
+	/* reset index progress */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 0);
 }
 
 /*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index bb1ac30cd1..35ea25026f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1126,7 +1126,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 974a29e7a9..f08eb837e7 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -29,6 +29,7 @@
 #include "access/amapi.h"
 #include "access/table.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -101,6 +102,9 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/* Leader PID of the vacuum */
+	int      leader_pid;
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -357,6 +361,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 		(nindexes_mwm > 0) ?
 		maintenance_work_mem / Min(parallel_workers, nindexes_mwm) :
 		maintenance_work_mem;
+	shared->leader_pid = MyProcPid;
 
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
@@ -844,9 +849,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
+			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
+			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 			break;
 		default:
 			elog(ERROR, "unexpected parallel vacuum index status %d for index \"%s\"",
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..54ee60caad 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_TOTAL_INDEXES           7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ac468568a1..d70a176514 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2002,7 +2002,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_replication| SELECT s.pid,
-- 
2.32.0

v9-0003-Expose-indexes-being-processed-in-a-VACUUM-operat.patchapplication/octet-stream; name=v9-0003-Expose-indexes-being-processed-in-a-VACUUM-operat.patchDownload
From b44359eb937b2f02878ba80fd56b94ef6130ac10 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 22:07:09 +0000
Subject: [PATCH v8 3/4] Expose indexes being processed in a VACUUM operation.

A new view called pg_stat_progress_vacuum_index to show the indexrelid
being vacuumed or cleaned during a vacuum. The view also shows the
number of tuples removed for the index during the vacuuming indexes phase.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
---
 doc/src/sgml/monitoring.sgml          | 108 ++++++++++++++++++++++++++
 src/backend/access/gin/ginvacuum.c    |   3 +
 src/backend/access/gist/gistvacuum.c  |   3 +
 src/backend/access/hash/hash.c        |   1 +
 src/backend/access/heap/vacuumlazy.c  |  14 ++++
 src/backend/access/nbtree/nbtree.c    |   1 +
 src/backend/access/spgist/spgvacuum.c |   4 +
 src/backend/catalog/system_views.sql  |  18 ++++-
 src/backend/commands/vacuumparallel.c |  12 +++
 src/include/commands/progress.h       |   3 +
 src/test/regress/expected/rules.out   |  17 +++-
 11 files changed, 182 insertions(+), 2 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1acc741da9..240d2438d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -379,6 +379,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary></indexterm></entry>
+      <entry>One row for each backend (including autovacuum worker processes) that is
+       currently performing the <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal> phase of the vacuum, showing current progress.
+       See <xref linkend='vacuum-progress-reporting'/>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6256,6 +6265,105 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
    </tgroup>
   </table>
 
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) that is
+   currently performing the <literal>vacuuming indexes</literal> or
+   <literal>cleaning up indexes</literal> phase of the vacuum.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-index-view" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>datname</structfield> <type>name</type>
+      </para>
+      <para>
+       Name of the database to which this backend is connected.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>oid</type>
+      </para>
+      <para>
+       OID of the index being processed in the ongoing phase of the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the parallel vacuum leader, if this process is a
+       parallel vacuum worker.  <literal>NULL</literal> if this process is a
+       parallel vacuum leader or does not participate in parallel vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>phase</structfield> <type>text</type>
+      </para>
+      <para>
+       Current processing phase of a vacuum. Only the
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+       phase will be listed in this view. See <xref linkend="vacuum-phases"/>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>tuples_removed</structfield> <type>oid</type>
+      </para>
+      <para>
+       The number of index tuples removed by the <literal>vacuuming indexes</literal> phase.
+       This field is <literal>0</literal> during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
+     </tbody>
+    </tgroup>
+   </table>
+
   <table id="vacuum-phases">
    <title>VACUUM Phases</title>
    <tgroup cols="2">
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..1d5d003780 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,8 +17,10 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -60,6 +62,7 @@ ginVacuumItemPointers(GinVacuumState *gvs, ItemPointerData *items,
 		if (gvs->callback(items + i, gvs->callback_state))
 		{
 			gvs->result->tuples_removed += 1;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, gvs->result->tuples_removed);
 			if (!tmpitems)
 			{
 				/*
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index aac4afab8f..8a0f23388b 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,9 +17,11 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "utils/memutils.h"
@@ -375,6 +377,7 @@ restart:
 			END_CRIT_SECTION();
 
 			vstate->stats->tuples_removed += ntodelete;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, vstate->stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 		}
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index a259a301fa..23dacee52e 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -632,6 +632,7 @@ loop_top:
 	stats->estimated_count = false;
 	stats->num_index_tuples = num_index_tuples;
 	stats->tuples_removed += tuples_removed;
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 	/* hashvacuumcleanup will fill in num_pages */
 
 	return stats;
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 047a5db494..1cc3fae20c 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -348,6 +348,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -2326,12 +2327,19 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being vacuumed in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, indexes_processed++);
 
+			/* Advertise we are done vacuuming indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2701,11 +2709,17 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* Advertise the index being cleaned in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED, indexes_processed++);
+
+			/* Advertise we are done cleaning indexes in non-parallel vacuum */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index c9b4964c1e..09edf49082 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -1273,6 +1273,7 @@ backtrack:
 								nupdatable);
 
 			stats->tuples_removed += nhtidsdead;
+			pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, stats->tuples_removed);
 			/* must recompute maxoff */
 			maxoff = PageGetMaxOffsetNumber(page);
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..db73f8ef59 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,8 +21,10 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
+#include "pgstat.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
@@ -160,6 +162,7 @@ vacuumLeafPage(spgBulkDeleteState *bds, Relation index, Buffer buffer,
 				bds->stats->tuples_removed += 1;
 				deletable[i] = true;
 				nDeletable++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
@@ -430,6 +433,7 @@ vacuumLeafRoot(spgBulkDeleteState *bds, Relation index, Buffer buffer)
 				bds->stats->tuples_removed += 1;
 				toDelete[xlrec.nDelete] = i;
 				xlrec.nDelete++;
+				pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, bds->stats->tuples_removed);
 			}
 			else
 			{
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 35ea25026f..88375aad0a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1129,7 +1129,23 @@ CREATE VIEW pg_stat_progress_vacuum AS
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
-        LEFT JOIN pg_database D ON S.datid = D.oid;
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.pid = S.param12;
+
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid AS pid,
+        S.datid AS datid,
+        D.datname AS datname,
+        S.param10 AS indexrelid,
+        S.param12 AS leader_pid,
+        CASE S.param1 WHEN 2 THEN 'vacuuming indexes'
+                      WHEN 4 THEN 'cleaning up indexes'
+                      END AS phase,
+        S.param11 AS tuples_removed
+    FROM pg_stat_get_progress_info('VACUUM') AS S
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.param1 IN (2, 4) AND S.param10 > 0;
 
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f08eb837e7..7d47565e05 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -845,13 +845,18 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
 
+	/* Advertise the index we are cleaning or vacuuming */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, RelationGetRelid(indrel));
+
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
 			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
 			pgstat_progress_update_param_parallel(pvs->shared->leader_pid, PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 			break;
@@ -888,6 +893,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	 */
 	indstats->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 
+	/* Advertise we are no longer vacuuming/cleaning an index */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXRELID, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_TUPLES_REMOVED, 0);
+
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
@@ -972,6 +981,8 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 	 * workers.
 	 */
 	rel = table_open(shared->relid, ShareUpdateExclusiveLock);
+	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM, RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, shared->leader_pid);
 
 	/*
 	 * Open all indexes. indrels are sorted in order by OID, which should be
@@ -1042,6 +1053,7 @@ parallel_vacuum_main(dsm_segment *seg, shm_toc *toc)
 
 	vac_close_indexes(nindexes, indrels, RowExclusiveLock);
 	table_close(rel, ShareUpdateExclusiveLock);
+	pgstat_progress_end_command();
 	FreeAccessStrategy(pvs.bstrategy);
 }
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index 54ee60caad..7e632feb7d 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,6 +27,9 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_TOTAL_INDEXES           7
 #define PROGRESS_VACUUM_INDEXES_PROCESSED		8
+#define PROGRESS_VACUUM_INDEXRELID              9
+#define PROGRESS_VACUUM_TUPLES_REMOVED          10
+#define PROGRESS_VACUUM_LEADER_PID              11
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d70a176514..d2c66bf034 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2006,7 +2006,22 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param8 AS indexes_total,
     s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
-     LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE (s.pid = s.param12);
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.datid,
+    d.datname,
+    s.param10 AS indexrelid,
+    s.param12 AS leader_pid,
+        CASE s.param1
+            WHEN 2 THEN 'vacuuming indexes'::text
+            WHEN 4 THEN 'cleaning up indexes'::text
+            ELSE NULL::text
+        END AS phase,
+    s.param11 AS tuples_removed
+   FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s.param10 > 0));
 pg_stat_replication| SELECT s.pid,
     s.usesysid,
     u.rolname AS usename,
-- 
2.32.0

v9-0004-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchapplication/octet-stream; name=v9-0004-Rename-index_vacuum_count-in-pg_stat_pogress_vacu.patchDownload
From 5303ab5d4d1f11a1a0fe62f28f74b13e2df9e3a2 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Mon, 14 Mar 2022 22:08:04 +0000
Subject: [PATCH v8 4/4] Rename "index_vacuum_count" in pg_stat_pogress_vacuum.

Commit 76d8a1bfd5207d28a4e9fe98a0e1ea7c096d70aa introduces 2 new columns
to track index vacuum/cleanup progress. The columns have "index" in the
name. To make it clear what the existing "index_vacuum_count" column
refers to, which is the cycle count of the index vacuum, this change
renames the column to "index_vacuum_count"

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Justin Pryzby
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml         | 2 +-
 src/backend/catalog/system_views.sql | 2 +-
 src/test/regress/expected/rules.out  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 240d2438d5..1e5af0b400 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6210,7 +6210,7 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>index_vacuum_count</structfield> <type>bigint</type>
+       <structfield>index_vacuum_cycle_count</structfield> <type>bigint</type>
       </para>
       <para>
        Number of completed index vacuum cycles.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 88375aad0a..639ad00a71 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1125,7 +1125,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       WHEN 6 THEN 'performing final cleanup'
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
-        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
+        S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_cycle_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
         S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index d2c66bf034..7a6bf3af27 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2000,7 +2000,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param2 AS heap_blks_total,
     s.param3 AS heap_blks_scanned,
     s.param4 AS heap_blks_vacuumed,
-    s.param5 AS index_vacuum_count,
+    s.param5 AS index_vacuum_cycle_count,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-- 
2.32.0

#57Nathan Bossart
nathandbossart@gmail.com
In reply to: Imseih (AWS), Sami (#56)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Mar 16, 2022 at 09:47:49PM +0000, Imseih (AWS), Sami wrote:

Spoke to Nathan offline and fixed some more comments/nitpicks in the patch.

I don't have any substantial comments for v9, so I think this can be marked
as ready-for-committer. However, we probably should first see whether
Sawada-san has any comments on the revised approach.

--
Nathan Bossart
Amazon Web Services: https://aws.amazon.com

#58Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#54)
Re: Add index scan progress to pg_stat_progress_vacuum

Sorry for the late reply.

On Tue, Mar 15, 2022 at 1:20 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

I'm still unsure the current design of 0001 patch is better than other
approaches we’ve discussed. Even users who don't use parallel vacuum
are forced to allocate shared memory for index vacuum progress, with
GetMaxBackends() entries from the beginning. Also, it’s likely to
extend the progress tracking feature for other parallel operations in
the future but I think the current design is not extensible. If we
want to do that, we will end up creating similar things for each of
them or re-creating index vacuum progress tracking feature while
creating a common infra. It might not be a problem as of now but I'm
concerned that introducing a feature that is not extensible and forces
users to allocate additional shmem might be a blocker in the future.
Looking at the precedent example, When we introduce the progress
tracking feature, we implemented it in an extensible way. On the other
hand, others in this thread seem to agree with this approach, so I'd
like to leave it to committers.

Thanks for the review!

I think you make strong arguments as to why we need to take a different approach now than later.

Flaws with current patch set:

1. GetMaxBackends() is a really heavy-handed overallocation of a shared memory serving a very specific purpose.
2. Going with the approach of a vacuum specific hash breaks the design of progress which is meant to be extensible.
3. Even if we go with this current approach as an interim solution, it will be a real pain in the future.

With that said, v7 introduces the new infrastructure. 0001 includes the new infrastructure and 0002 takes advantage of this.

This approach is the following:

1. Introduces a new API called pgstat_progress_update_param_parallel along with some others support functions. This new infrastructure is in backend_progress.c

2. There is still a shared memory involved, but the size is capped to " max_worker_processes" which is the max to how many parallel workers can be doing work at any given time. The shared memory hash includes a st_progress_param array just like the Backend Status array.

I think that there is a corner case where a parallel operation could
not perform due to the lack of a free shared hash entry, because there
is a window between a parallel worker exiting and the leader
deallocating the hash table entry.

BTW have we discussed another idea I mentioned before that we have the
leader process periodically check the number of completed indexes and
advertise it in its progress information? I'm not sure which one is
better but this idea would require only changes of vacuum code and
probably simpler than the current idea.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#59Andres Freund
andres@anarazel.de
In reply to: Imseih (AWS), Sami (#56)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

On 2022-03-16 21:47:49 +0000, Imseih (AWS), Sami wrote:

From 85c47dfb3bb72f764b9052e74a7282c19ebd9898 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Wed, 16 Mar 2022 20:39:52 +0000
Subject: [PATCH 1/1] Add infrastructure for parallel progress reporting

Infrastructure to allow a parallel worker to report
progress. In a PARALLEL command, the workers and
leader can report progress using a new pgstat_progress
API.

What happens if we run out of memory for hashtable entries?

+void
+pgstat_progress_update_param_parallel(int leader_pid, int index, int64 val)
+{
+	ProgressParallelEntry *entry;
+	bool found;
+
+	LWLockAcquire(ProgressParallelLock, LW_EXCLUSIVE);
+
+	entry = (ProgressParallelEntry *) hash_search(ProgressParallelHash, &leader_pid, HASH_ENTER, &found);
+
+	/*
+	 * If the entry is not found, set the value for the index'th member,
+	 * else increment the current value of the index'th member.
+	 */
+	if (!found)
+		entry->st_progress_param[index] = val;
+	else
+		entry->st_progress_param[index] += val;
+
+	LWLockRelease(ProgressParallelLock);
+}

I think that's an absolute no-go. Adding locking to progress reporting,
particularly a single central lwlock, is going to *vastly* increase the
overhead incurred by progress reporting.

Greetings,

Andres Freund

#60Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#58)
Re: Add index scan progress to pg_stat_progress_vacuum

BTW have we discussed another idea I mentioned before that we have the
leader process periodically check the number of completed indexes and
advertise it in its progress information? I'm not sure which one is
better but this idea would require only changes of vacuum code and
probably simpler than the current idea.

Regards,

If I understand correctly, to accomplish this we will need to have the leader
check the number of indexes completed In the ambukdelete or amvacuumcleanup
callbacks. These routines do not know about PVIndStats, and they are called
by both parallel and non-parallel vacuums.

From what I can see, PVIndstats will need to be passed down to these routines
or pass a NULL for non-parallel vacuums.

Sami

#61Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#60)
Re: Add index scan progress to pg_stat_progress_vacuum

On Tue, Mar 22, 2022 at 4:27 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

BTW have we discussed another idea I mentioned before that we have the
leader process periodically check the number of completed indexes and
advertise it in its progress information? I'm not sure which one is
better but this idea would require only changes of vacuum code and
probably simpler than the current idea.

Regards,

If I understand correctly, to accomplish this we will need to have the leader
check the number of indexes completed In the ambukdelete or amvacuumcleanup
callbacks. These routines do not know about PVIndStats, and they are called
by both parallel and non-parallel vacuums.

From what I can see, PVIndstats will need to be passed down to these routines
or pass a NULL for non-parallel vacuums.

Can the leader pass a callback that checks PVIndStats to ambulkdelete
an amvacuumcleanup callbacks? I think that in the passed callback, the
leader checks if the number of processed indexes and updates its
progress information if the current progress needs to be updated.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#62Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#61)
Re: Add index scan progress to pg_stat_progress_vacuum

Can the leader pass a callback that checks PVIndStats to ambulkdelete
an amvacuumcleanup callbacks? I think that in the passed callback, the
leader checks if the number of processed indexes and updates its
progress information if the current progress needs to be updated.

Thanks for the suggestion.

I looked at this option a but today and found that passing the callback
will also require signature changes to the ambulkdelete and
amvacuumcleanup routines.

This will also require us to check after x pages have been
scanned inside vacuumscan and vacuumcleanup. After x pages
the callback can then update the leaders progress.
I am not sure if adding additional complexity to the scan/cleanup path
is justified for what this patch is attempting to do.

There will also be a lag of the leader updating the progress as it
must scan x amount of pages before updating. Obviously, the more
Pages to the scan, the longer the lag will be.

Would like to hear your thoughts on the above.

Regards,

Sami Imseih
Amazon Web Services.

#63Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#62)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Mar 23, 2022 at 6:57 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Can the leader pass a callback that checks PVIndStats to ambulkdelete
an amvacuumcleanup callbacks? I think that in the passed callback, the
leader checks if the number of processed indexes and updates its
progress information if the current progress needs to be updated.

Thanks for the suggestion.

I looked at this option a but today and found that passing the callback
will also require signature changes to the ambulkdelete and
amvacuumcleanup routines.

I think it would not be a critical problem since it's a new feature.

This will also require us to check after x pages have been
scanned inside vacuumscan and vacuumcleanup. After x pages
the callback can then update the leaders progress.
I am not sure if adding additional complexity to the scan/cleanup path
is justified for what this patch is attempting to do.

There will also be a lag of the leader updating the progress as it
must scan x amount of pages before updating. Obviously, the more
Pages to the scan, the longer the lag will be.

Fair points.

On the other hand, the approach of the current patch requires more
memory for progress tracking, which could fail, e.g., due to running
out of hashtable entries. I think that it would be worse that the
parallel operation failed to start due to not being able to track the
progress than the above concerns you mentioned such as introducing
additional complexity and a possible lag of progress updates. So if we
go with the current approach, I think we need to make sure enough (and
not too many) hash table entries.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#64Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#63)
Re: Add index scan progress to pg_stat_progress_vacuum

Sorry for the late reply.

additional complexity and a possible lag of progress updates. So if we
go with the current approach, I think we need to make sure enough (and
not too many) hash table entries.

The hash table can be set 4 times the size of
max_worker_processes which should give more than
enough padding.
Note that max_parallel_maintenance_workers
is what should be used, but since it's dynamic, it cannot
be used to determine the size of shared memory.

Regards,

---
Sami Imseih
Amazon Web Services

#65Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#64)
Re: Add index scan progress to pg_stat_progress_vacuum

I think that's an absolute no-go. Adding locking to progress reporting,
particularly a single central lwlock, is going to *vastly* increase the
overhead incurred by progress reporting.

Sorry for the late reply.

The usage of the shared memory will be limited
to PARALLEL maintenance operations. For now,
it will only be populated for parallel vacuums.
Autovacuum for example will not be required to
populate this shared memory.

Regards,

---
Sami Imseih
Amazon Web Services

#66Andres Freund
andres@anarazel.de
In reply to: Imseih (AWS), Sami (#65)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

On 2022-03-29 12:25:52 +0000, Imseih (AWS), Sami wrote:

I think that's an absolute no-go. Adding locking to progress reporting,
particularly a single central lwlock, is going to *vastly* increase the
overhead incurred by progress reporting.

Sorry for the late reply.

The usage of the shared memory will be limited
to PARALLEL maintenance operations. For now,
it will only be populated for parallel vacuums.
Autovacuum for example will not be required to
populate this shared memory.

I nevertheless think that's not acceptable. The whole premise of the progress
reporting infrastructure is to be low overhead. It's OK to require locking to
initialize parallel progress reporting, it's definitely not ok to require
locking to report progress.

Leaving the locking aside, doing a hashtable lookup for each progress report
is pretty expensive.

Why isn't the obvious thing to do here to provide a way to associate workers
with their leaders in shared memory, but to use the existing progress fields
to report progress? Then, when querying progress, the leader and workers
progress fields can be combined to show the overall progress?

Greetings,

Andres Freund

#67Imseih (AWS), Sami
simseih@amazon.com
In reply to: Andres Freund (#66)
Re: Add index scan progress to pg_stat_progress_vacuum

I nevertheless think that's not acceptable. The whole premise of the progress
reporting infrastructure is to be low overhead. It's OK to require locking to
initialize parallel progress reporting, it's definitely not ok to require
locking to report progress.

Fair point.

Why isn't the obvious thing to do here to provide a way to associate workers
with their leaders in shared memory, but to use the existing progress fields
to report progress? Then, when querying progress, the leader and workers
progress fields can be combined to show the overall progress?

The original intent was this, however the workers
can exit before the command completes and the
worker progress data will be lost.
This is why the shared memory was introduced.
This allows the worker progress to persist for the duration
of the command.

Regards,

Sami Imseih
Amazon Web Services.

#68Robert Haas
robertmhaas@gmail.com
In reply to: Imseih (AWS), Sami (#67)
Re: Add index scan progress to pg_stat_progress_vacuum

On Tue, Apr 5, 2022 at 12:42 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Why isn't the obvious thing to do here to provide a way to associate workers
with their leaders in shared memory, but to use the existing progress fields
to report progress? Then, when querying progress, the leader and workers
progress fields can be combined to show the overall progress?

The original intent was this, however the workers
can exit before the command completes and the
worker progress data will be lost.
This is why the shared memory was introduced.
This allows the worker progress to persist for the duration
of the command.

At the beginning of a parallel operation, we allocate a chunk of
dynamic shared memory which persists even after some or all workers
have exited. It's only torn down at the end of the parallel operation.
That seems like the appropriate place to be storing any kind of data
that needs to be propagated between parallel workers. The current
patch uses the main shared memory segment, which seems unacceptable to
me.

--
Robert Haas
EDB: http://www.enterprisedb.com

#69Andres Freund
andres@anarazel.de
In reply to: Imseih (AWS), Sami (#67)
Re: Add index scan progress to pg_stat_progress_vacuum

On 2022-04-05 16:42:28 +0000, Imseih (AWS), Sami wrote:

Why isn't the obvious thing to do here to provide a way to associate workers
with their leaders in shared memory, but to use the existing progress fields
to report progress? Then, when querying progress, the leader and workers
progress fields can be combined to show the overall progress?

The original intent was this, however the workers
can exit before the command completes and the
worker progress data will be lost.

Can't the progress data trivially be inferred by the fact that the worker
completed?

#70Imseih (AWS), Sami
simseih@amazon.com
In reply to: Andres Freund (#69)
Re: Add index scan progress to pg_stat_progress_vacuum

Can't the progress data trivially be inferred by the fact that the worker
completed?

Yes, at some point, this idea was experimented with in
0004-Expose-progress-for-the-vacuuming-indexes-cleanup-ph.patch.
This patch did the calculation in system_views.sql

However, the view is complex and there could be some edge
cases with inferring the values that lead to wrong values being reported.

Regards,

Sami Imseih
Amazon Web Services

#71Imseih (AWS), Sami
simseih@amazon.com
In reply to: Robert Haas (#68)
Re: Add index scan progress to pg_stat_progress_vacuum

At the beginning of a parallel operation, we allocate a chunk of>
dynamic shared memory which persists even after some or all workers
have exited. It's only torn down at the end of the parallel operation.
That seems like the appropriate place to be storing any kind of data
that needs to be propagated between parallel workers. The current
patch uses the main shared memory segment, which seems unacceptable to
me.

Correct, DSM does track shared data. However only participating
processes in the parallel vacuum can attach and lookup this data.

The purpose of the main shared memory is to allow a process that
Is querying the progress views to retrieve the information.

Regards,

Sami Imseih
Amazon Web Services

#72Robert Haas
robertmhaas@gmail.com
In reply to: Imseih (AWS), Sami (#71)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Apr 6, 2022 at 5:22 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

At the beginning of a parallel operation, we allocate a chunk of>
dynamic shared memory which persists even after some or all workers
have exited. It's only torn down at the end of the parallel operation.
That seems like the appropriate place to be storing any kind of data
that needs to be propagated between parallel workers. The current
patch uses the main shared memory segment, which seems unacceptable to
me.

Correct, DSM does track shared data. However only participating
processes in the parallel vacuum can attach and lookup this data.

The purpose of the main shared memory is to allow a process that
Is querying the progress views to retrieve the information.

Sure, but I think that you should likely be doing what Andres
recommended before:

# Why isn't the obvious thing to do here to provide a way to associate workers
# with their leaders in shared memory, but to use the existing progress fields
# to report progress? Then, when querying progress, the leader and workers
# progress fields can be combined to show the overall progress?

That is, I am imagining that you would want to use DSM to propagate
data from workers back to the leader and then have the leader report
the data using the existing progress-reporting facilities. Now, if we
really need a whole row from each worker that doesn't work, but why do
we need that?

--
Robert Haas
EDB: http://www.enterprisedb.com

#73Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Robert Haas (#72)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Apr 7, 2022 at 10:20 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Wed, Apr 6, 2022 at 5:22 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

At the beginning of a parallel operation, we allocate a chunk of>
dynamic shared memory which persists even after some or all workers
have exited. It's only torn down at the end of the parallel operation.
That seems like the appropriate place to be storing any kind of data
that needs to be propagated between parallel workers. The current
patch uses the main shared memory segment, which seems unacceptable to
me.

Correct, DSM does track shared data. However only participating
processes in the parallel vacuum can attach and lookup this data.

The purpose of the main shared memory is to allow a process that
Is querying the progress views to retrieve the information.

Sure, but I think that you should likely be doing what Andres
recommended before:

# Why isn't the obvious thing to do here to provide a way to associate workers
# with their leaders in shared memory, but to use the existing progress fields
# to report progress? Then, when querying progress, the leader and workers
# progress fields can be combined to show the overall progress?

That is, I am imagining that you would want to use DSM to propagate
data from workers back to the leader and then have the leader report
the data using the existing progress-reporting facilities. Now, if we
really need a whole row from each worker that doesn't work, but why do
we need that?

+1

I also proposed the same idea before[1]/messages/by-id/CAD21AoBW6SMJ96CNoMeu+f_BR4jmatPcfVA016FdD2hkLDsaTA@mail.gmail.com. The leader can know how many
indexes are processed so far by checking PVIndStats.status allocated
on DSM for each index. We can have the leader check it and update the
progress information before and after vacuuming one index. If we want
to update the progress information more timely, probably we can pass a
callback function to ambulkdelete and amvacuumcleanup so that the
leader can do that periodically, e.g., every 1000 blocks, while
vacuuming an index.

Regards,

[1]: /messages/by-id/CAD21AoBW6SMJ96CNoMeu+f_BR4jmatPcfVA016FdD2hkLDsaTA@mail.gmail.com

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#74Greg Stark
stark@mit.edu
In reply to: Masahiko Sawada (#73)
Re: Add index scan progress to pg_stat_progress_vacuum

It looks like this patch got feedback from Andres and Robert with some
significant design change recommendations. I'm marking the patch
Returned with Feedback. Feel free to add it back to a future
commitfest when a new version is ready.

#75Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#71)
2 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Taking the above feedback, I modified the patches
and I believe this approach should be acceptable.

For now, I also reduced the scope to only exposing
Indexes_total and indexes_completed in
pg_stat_progress_vacuum. I will create a new CF entry
for the new view pg_stat_progress_vacuum_index.

V10-0001: This patch adds a callback to ParallelContext
that could be called by the leader in vacuumparallel.c
and more importantly while the leader is waiting
for the parallel workers to complete inside.

This ensures that the leader is continuously polling and
reporting completed indexes for the life of the PARALLEL
VACUUM. This covers cases where the leader completes
vacuuming before the workers complete.

V10-0002: This implements the indexes_total and
indexes_completed columns in pg_stat_progress_vacuum.

This work is now tracked in the next commitfest:
https://commitfest.postgresql.org/38/3617/

Regards,

Sami Imseih
Amazon Web Services

Attachments:

v10-0001-Add-progress-reporting-callback-to-ParallelConte.patchapplication/octet-stream; name=v10-0001-Add-progress-reporting-callback-to-ParallelConte.patchDownload
From 94fc3655b55cff3346a8afdf0bac2ee9b26fd035 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 14 Apr 2022 00:06:14 +0000
Subject: [PATCH v10 1/2] Add progress reporting callback to ParallelContext

The purpose of supporting a progress reporting
callback in ParallelContext is to allow for the
leader process to report progress while waiting
for workers to complete.

The first use-case for this is to report index
progress in pg_stat_progress_vacuum.
---
 src/backend/access/transam/parallel.c | 16 ++++++++++++++++
 src/include/access/parallel.h         |  5 +++++
 2 files changed, 21 insertions(+)

diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index df0cd77558..bfe3275d8b 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -774,6 +774,22 @@ WaitForParallelWorkersToFinish(ParallelContext *pcxt)
 		 */
 		CHECK_FOR_INTERRUPTS();
 
+		/*
+		 * We call the parallel progress callback while
+		 * waiting for the parallel workers to finish.
+		 * This is to ensure that the leader keeps
+		 * updating progress when waiting for
+		 * parallel workers to finish.
+		 *
+		 * We must ensure that pcxt->parallel_progress_callback
+		 * is set before calling as not all parallel
+		 * operations will set a callback.
+		 */
+		if (pcxt->parallel_progress_callback)
+		{
+			pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+		}
+
 		for (i = 0; i < pcxt->nworkers_launched; ++i)
 		{
 			/*
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 983841d45e..53a3d13ec0 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -20,6 +20,9 @@
 #include "storage/shm_mq.h"
 #include "storage/shm_toc.h"
 
+/* progress callback definition */
+typedef void (*ParallelProgressCallback) (void *parallel_progress_callback_state);
+
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);
 
 typedef struct ParallelWorkerInfo
@@ -46,6 +49,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	ParallelProgressCallback parallel_progress_callback;
+	void            *parallel_progress_callback_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
-- 
2.32.0

v10-0002-Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v10-0002-Show-progress-for-index-vacuums.patchDownload
From e5f70a1853c8be8b02144420f84763ccfcee9b13 Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)" <simseih@amazon.com>
Date: Thu, 14 Apr 2022 00:47:42 +0000
Subject: [PATCH 2/2] Show progress for index vacuums

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 25 ++++++++++++++++
 src/backend/access/heap/vacuumlazy.c  | 19 ++++++++++++
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 42 +++++++++++++++++++++++++++
 src/include/commands/progress.h       |  2 ++
 src/test/regress/expected/rules.out   |  4 ++-
 6 files changed, 93 insertions(+), 2 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 180798a6b9..c3e60e57d9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6396,6 +6396,31 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the 
+       <literal>vacuuming indexes</literal> 
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when there are no indexes to process 
+       or when failsafe is triggered during the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 788db569b2..587aa668f9 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -415,6 +415,9 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+	/* Report the number of indexes to vacuum/cleanup */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_TOTAL, vacrel->nindexes);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2327,6 +2330,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
 
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
@@ -2361,6 +2366,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		   vacrel->dead_items->num_items == vacrel->lpdead_items);
 	Assert(allindexes || vacrel->failsafe_active);
 
+	/* Report that we're done vacuuming indexes */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_TOTAL, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 	/*
 	 * Increase and report the number of index scans.
 	 *
@@ -2625,6 +2634,10 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Report that we're no longer vacuuming indexes */
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						get_database_name(MyDatabaseId),
@@ -2671,6 +2684,8 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+			/* Report the number of indexes cleaned */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
 		}
 	}
 	else
@@ -2680,6 +2695,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Report that we're done cleaning indexes */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_TOTAL, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 }
 
 /*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 0fc614e32c..bbd2d6352b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1155,7 +1155,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index bbf3b69c57..238c49d7d1 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -213,6 +214,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_vacuum_progress_callback(void *arg);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -288,6 +290,10 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	shm_toc_estimate_chunk(&pcxt->estimator, est_dead_items_len);
 	shm_toc_estimate_keys(&pcxt->estimator, 1);
 
+	/* Setup the Parallel Progress Callback */
+	pvs->pcxt->parallel_progress_callback = parallel_vacuum_progress_callback;
+	pvs->pcxt->parallel_progress_callback_arg = pvs;
+
 	/*
 	 * Estimate space for BufferUsage and WalUsage --
 	 * PARALLEL_VACUUM_KEY_BUFFER_USAGE and PARALLEL_VACUUM_KEY_WAL_USAGE.
@@ -856,6 +862,18 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 				 RelationGetRelationName(indrel));
 	}
 
+	/*
+	 * If we are the leader, update index vacuum progress.
+	 *
+	 * For convenience, rely on the parallel_progress_callback
+	 * which calculates and reports the number of indexes
+	 * vacuumed/cleaned up in parallel vacuum mode.
+	 * */
+	if (!IsParallelWorker())
+	{
+		pvs->pcxt->parallel_progress_callback(pvs->pcxt->parallel_progress_callback_arg);
+	}
+
 	/*
 	 * Copy the index bulk-deletion result returned from ambulkdelete and
 	 * amvacuumcleanup to the DSM segment if it's the first cycle because they
@@ -1071,3 +1089,27 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Callback to report index vacuum progress.
+ * Vacuum Progress should only be reported
+ * to the leader process.
+ */
+static void
+parallel_vacuum_progress_callback(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+	int indexes_completed = 0;
+
+	Assert(!IsParallelWorker());
+
+	for (int i = 0; i < pvs->nindexes; i++)
+	{
+		PVIndStats *indstats = &(pvs->indstats[i]);
+
+		if (indstats->status == PARALLEL_INDVAC_STATUS_COMPLETED)
+			indexes_completed++;
+	}
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed);
+}
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..0d1724ee75 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 21effe8315..53ff7151c8 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2008,7 +2008,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.0

#76Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#75)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Apr 14, 2022 at 10:32 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Taking the above feedback, I modified the patches
and I believe this approach should be acceptable.

For now, I also reduced the scope to only exposing
Indexes_total and indexes_completed in
pg_stat_progress_vacuum. I will create a new CF entry
for the new view pg_stat_progress_vacuum_index.

V10-0001: This patch adds a callback to ParallelContext
that could be called by the leader in vacuumparallel.c
and more importantly while the leader is waiting
for the parallel workers to complete inside.

Thank you for updating the patch! The new design looks much better to me.

 typedef struct ParallelWorkerInfo
@@ -46,6 +49,8 @@ typedef struct ParallelContext
    ParallelWorkerInfo *worker;
    int         nknown_attached_workers;
    bool       *known_attached_workers;
+   ParallelProgressCallback parallel_progress_callback;
+   void            *parallel_progress_callback_arg;
 } ParallelContext;

I think we can pass the progress update function to
WaitForParallelWorkersToFinish(), which seems simpler. And we can call
the function after updating the index status to
PARALLEL_INDVAC_STATUS_COMPLETED.

BTW, currently we don't need a lock for touching index status since
each worker touches different indexes. But after this patch, the
leader will touch all index status, do we need a lock for that?

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#77Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#76)
Re: Add index scan progress to pg_stat_progress_vacuum

Thank you for the feedback!

I think we can pass the progress update function to
WaitForParallelWorkersToFinish(), which seems simpler. And we can call

Directly passing the callback to WaitForParallelWorkersToFinish
will require us to modify the function signature.

To me, it seemed simpler and touches less code to have
the caller set the callback in the ParallelContext.

the function after updating the index status to
PARALLEL_INDVAC_STATUS_COMPLETED.

I also like this better. Will make the change.

BTW, currently we don't need a lock for touching index status since
each worker touches different indexes. But after this patch, the
leader will touch all index status, do we need a lock for that?

I do not think locking is needed here. The leader and workers
will continue to touch different indexes to update the status.

However, if the process is a leader, it will call the function
which will go through indstats and count how many
Indexes have a status of PARALLEL_INDVAC_STATUS_COMPLETED.
This value is then reported to the leaders backend only.

Regards,

Sami Imseih
Amazon Web Services

#78Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#77)
2 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

the function after updating the index status to
PARALLEL_INDVAC_STATUS_COMPLETED.

I also like this better. Will make the change.

I updated the patch. The progress function is called after
updating index status to PARALLEL_INDVAC_STATUS_COMPLETED.

I believe all comments have been addressed at this point.

Regards,

Sami Imseih
Amazon Web Services

Attachments:

v11-0001-Add-progress-reporting-callback-to-ParallelConte.patchapplication/octet-stream; name=v11-0001-Add-progress-reporting-callback-to-ParallelConte.patchDownload
From 19b1a9b4814fdf411a34a0ca86ce224079c857cc Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Thu, 26 May 2022 07:45:26 -0500
Subject: [PATCH v11 1/2] Add progress reporting callback to ParallelContext

The purpose of supporting a progress reporting
callback in ParallelContext is to allow for the
leader process to report progress while waiting
for workers to complete.

The first use-case for this is to report index
progress in pg_stat_progress_vacuum.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 src/backend/access/transam/parallel.c | 16 ++++++++++++++++
 src/include/access/parallel.h         |  5 +++++
 2 files changed, 21 insertions(+)

diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index df0cd77..bfe3275 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -774,6 +774,22 @@ WaitForParallelWorkersToFinish(ParallelContext *pcxt)
 		 */
 		CHECK_FOR_INTERRUPTS();
 
+		/*
+		 * We call the parallel progress callback while
+		 * waiting for the parallel workers to finish.
+		 * This is to ensure that the leader keeps
+		 * updating progress when waiting for
+		 * parallel workers to finish.
+		 *
+		 * We must ensure that pcxt->parallel_progress_callback
+		 * is set before calling as not all parallel
+		 * operations will set a callback.
+		 */
+		if (pcxt->parallel_progress_callback)
+		{
+			pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+		}
+
 		for (i = 0; i < pcxt->nworkers_launched; ++i)
 		{
 			/*
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 983841d..53a3d13 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -20,6 +20,9 @@
 #include "storage/shm_mq.h"
 #include "storage/shm_toc.h"
 
+/* progress callback definition */
+typedef void (*ParallelProgressCallback) (void *parallel_progress_callback_state);
+
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);
 
 typedef struct ParallelWorkerInfo
@@ -46,6 +49,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	ParallelProgressCallback parallel_progress_callback;
+	void            *parallel_progress_callback_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
-- 
2.32.1 (Apple Git-133)

v11-0002-Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v11-0002-Show-progress-for-index-vacuums.patchDownload
From 5d0c7f7f761316d1089b010bc750cf36803b53e3 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Thu, 26 May 2022 08:02:11 -0500
Subject: [PATCH v11 2/2] Show progress for index vacuums

Add 2 new columns to pg_stat_progress_vacuum. The columns are
indexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so
far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 25 ++++++++++++++++++
 src/backend/access/heap/vacuumlazy.c  | 19 ++++++++++++++
 src/backend/catalog/system_views.sql  |  3 ++-
 src/backend/commands/vacuumparallel.c | 37 +++++++++++++++++++++++++++
 src/include/commands/progress.h       |  2 ++
 src/test/regress/expected/rules.out   |  4 ++-
 6 files changed, 88 insertions(+), 2 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 56d9b37..0599384 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6396,6 +6396,31 @@ SELECT pg_stat_get_backend_pid(s.backendid) AS pid,
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes to be processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when there are no indexes to process
+       or when failsafe is triggered during the vacuum.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       The number of indexes already processed in the
+       <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase. It is set to
+       <literal>0</literal> when vacuum is not in any of these phases.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index b802ed2..f01341e 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -418,6 +418,9 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+	/* Report the number of indexes to vacuum/cleanup */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_TOTAL, vacrel->nindexes);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2327,6 +2330,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
 
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
@@ -2361,6 +2366,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		   vacrel->dead_items->num_items == vacrel->lpdead_items);
 	Assert(allindexes || vacrel->failsafe_active);
 
+	/* Report that we're done vacuuming indexes */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_TOTAL, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 	/*
 	 * Increase and report the number of index scans.
 	 *
@@ -2625,6 +2634,10 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Report that we're no longer vacuuming indexes */
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						get_database_name(MyDatabaseId),
@@ -2671,6 +2684,8 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+			/* Report the number of indexes cleaned */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, idx + 1);
 		}
 	}
 	else
@@ -2680,6 +2695,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Report that we're done cleaning up indexes */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_TOTAL, 0);
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, 0);
 }
 
 /*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index fedaed5..0c8261b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1163,7 +1163,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 1753da6..ad61a54 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -213,6 +214,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_vacuum_progress_callback(void *arg);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -288,6 +290,10 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	shm_toc_estimate_chunk(&pcxt->estimator, est_dead_items_len);
 	shm_toc_estimate_keys(&pcxt->estimator, 1);
 
+	/* Setup the Parallel Progress Callback */
+	pvs->pcxt->parallel_progress_callback = parallel_vacuum_progress_callback;
+	pvs->pcxt->parallel_progress_callback_arg = pvs;
+
 	/*
 	 * Estimate space for BufferUsage and WalUsage --
 	 * PARALLEL_VACUUM_KEY_BUFFER_USAGE and PARALLEL_VACUUM_KEY_WAL_USAGE.
@@ -885,6 +891,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 
 	/* Reset error traceback information */
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
+
+	/*
+	 * If we are the leader, update index vacuum progress.
+	 */
+	if (!IsParallelWorker())
+		pvs->pcxt->parallel_progress_callback(pvs->pcxt->parallel_progress_callback_arg);
+
 	pfree(pvs->indname);
 	pvs->indname = NULL;
 }
@@ -1071,3 +1084,27 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Callback to report index vacuum progress.
+ * Vacuum Progress should only be reported
+ * to the leader process.
+ */
+static void
+parallel_vacuum_progress_callback(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+	int indexes_completed = 0;
+
+	Assert(!IsParallelWorker());
+
+	for (int i = 0; i < pvs->nindexes; i++)
+	{
+		PVIndStats *indstats = &(pvs->indstats[i]);
+
+		if (indstats->status == PARALLEL_INDVAC_STATUS_COMPLETED)
+			indexes_completed++;
+	}
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_COMPLETED, indexes_completed);
+}
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938c..0d1724e 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_COMPLETED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fc3cde3..9c3442f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2017,7 +2017,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

#79Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#77)
Re: Add index scan progress to pg_stat_progress_vacuum

On Fri, May 6, 2022 at 4:26 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Thank you for the feedback!

I think we can pass the progress update function to
WaitForParallelWorkersToFinish(), which seems simpler. And we can call

Directly passing the callback to WaitForParallelWorkersToFinish
will require us to modify the function signature.

To me, it seemed simpler and touches less code to have
the caller set the callback in the ParallelContext.

Okay, but if we do that, I think we should add comments about when
it's used. The callback is used only when
WaitForParallelWorkersToFinish(), but not when
WaitForParallelWorkersToExit().

Another idea I came up with is that we can wait for all index vacuums
to finish while checking and updating the progress information, and
then calls WaitForParallelWorkersToFinish after confirming all index
status became COMPLETED. That way, we don’t need to change the
parallel query infrastructure. What do you think?

the function after updating the index status to
PARALLEL_INDVAC_STATUS_COMPLETED.

I also like this better. Will make the change.

BTW, currently we don't need a lock for touching index status since
each worker touches different indexes. But after this patch, the
leader will touch all index status, do we need a lock for that?

I do not think locking is needed here. The leader and workers
will continue to touch different indexes to update the status.

However, if the process is a leader, it will call the function
which will go through indstats and count how many
Indexes have a status of PARALLEL_INDVAC_STATUS_COMPLETED.
This value is then reported to the leaders backend only.

I was concerned that the leader process could report the wrong
progress if updating and checking index status happen concurrently.
But I think it should be fine since we can read PVIndVacStatus
atomically.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#80Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#79)
Re: Add index scan progress to pg_stat_progress_vacuum

Another idea I came up with is that we can wait for all index vacuums
to finish while checking and updating the progress information, and
then calls WaitForParallelWorkersToFinish after confirming all index
status became COMPLETED. That way, we don’t need to change the
parallel query infrastructure. What do you think?

Thinking about this a bit more, the idea of using
WaitForParallelWorkersToFinish
Will not work if you have a leader worker that is
stuck on a large index. The progress will not be updated
until the leader completes. Even if the parallel workers
finish.

What are your thought about piggybacking on the
vacuum_delay_point to update progress. The leader can
perhaps keep a counter to update progress every few thousand
calls to vacuum_delay_point.

This goes back to your original idea to keep updating progress
while scanning the indexes.

/*
* vacuum_delay_point --- check for interrupts and cost-based delay.
*
* This should be called in each major loop of VACUUM processing,
* typically once per page processed.
*/
void
vacuum_delay_point(void)
{

---
Sami Imseih
Amazon Web Services

#81Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#80)
Re: Add index scan progress to pg_stat_progress_vacuum

On Fri, May 27, 2022 at 10:52 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Another idea I came up with is that we can wait for all index vacuums
to finish while checking and updating the progress information, and
then calls WaitForParallelWorkersToFinish after confirming all index
status became COMPLETED. That way, we don’t need to change the
parallel query infrastructure. What do you think?

Thinking about this a bit more, the idea of using
WaitForParallelWorkersToFinish
Will not work if you have a leader worker that is
stuck on a large index. The progress will not be updated
until the leader completes. Even if the parallel workers
finish.

Right.

What are your thought about piggybacking on the
vacuum_delay_point to update progress. The leader can
perhaps keep a counter to update progress every few thousand
calls to vacuum_delay_point.

This goes back to your original idea to keep updating progress
while scanning the indexes.

I think we can have the leader process wait for all index statuses to
become COMPLETED before WaitForParallelWorkersToFinish(). While
waiting for it, the leader can update its progress information. After
the leader confirmed all index statuses became COMPLETED, it can wait
for the workers to finish by WaitForParallelWorkersToFinish().

Regarding waiting in vacuum_delay_point, it might be a side effect as
it’s called every page and used not only by vacuum such as analyze,
but it seems to be worth trying.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#82Robert Haas
robertmhaas@gmail.com
In reply to: Masahiko Sawada (#79)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, May 26, 2022 at 11:43 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:

Another idea I came up with is that we can wait for all index vacuums
to finish while checking and updating the progress information, and
then calls WaitForParallelWorkersToFinish after confirming all index
status became COMPLETED. That way, we don’t need to change the
parallel query infrastructure. What do you think?

+1 from me. It doesn't seem to me that we should need to add something
like parallel_vacuum_progress_callback in order to solve this problem,
because the parallel index vacuum code could just do the waiting
itself, as you propose here.

The question Sami asks him his reply is a good one, though -- who is
to say that the leader only needs to update progress at the end, once
it's finished the index it's handling locally? There will need to be a
callback system of some kind to allow the leader to update progress as
other workers finish, even if the leader is still working. I am not
too sure that the idea of using the vacuum delay points is the best
plan. I think we should try to avoid piggybacking on such general
infrastructure if we can, and instead look for a way to tie this to
something that is specific to parallel vacuum. However, I haven't
studied the problem so I'm not sure whether there's a reasonable way
to do that.

--
Robert Haas
EDB: http://www.enterprisedb.com

#83Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Robert Haas (#82)
Re: Add index scan progress to pg_stat_progress_vacuum

On Mon, Jun 6, 2022 at 11:42 PM Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, May 26, 2022 at 11:43 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:

Another idea I came up with is that we can wait for all index vacuums
to finish while checking and updating the progress information, and
then calls WaitForParallelWorkersToFinish after confirming all index
status became COMPLETED. That way, we don’t need to change the
parallel query infrastructure. What do you think?

+1 from me. It doesn't seem to me that we should need to add something
like parallel_vacuum_progress_callback in order to solve this problem,
because the parallel index vacuum code could just do the waiting
itself, as you propose here.

The question Sami asks him his reply is a good one, though -- who is
to say that the leader only needs to update progress at the end, once
it's finished the index it's handling locally? There will need to be a
callback system of some kind to allow the leader to update progress as
other workers finish, even if the leader is still working. I am not
too sure that the idea of using the vacuum delay points is the best
plan. I think we should try to avoid piggybacking on such general
infrastructure if we can, and instead look for a way to tie this to
something that is specific to parallel vacuum. However, I haven't
studied the problem so I'm not sure whether there's a reasonable way
to do that.

One idea would be to add a flag, say report_parallel_vacuum_progress,
to IndexVacuumInfo struct and expect index AM to check and update the
parallel index vacuum progress, say every 1GB blocks processed. The
flag is true only when the leader process is vacuuming an index.

Regards,

--
Masahiko Sawada
EDB: https://www.enterprisedb.com/

#84Jacob Champion
jchampion@timescale.com
In reply to: Masahiko Sawada (#83)
Re: Add index scan progress to pg_stat_progress_vacuum

This entry has been waiting on author input for a while (our current
threshold is roughly two weeks), so I've marked it Returned with
Feedback.

Once you think the patchset is ready for review again, you (or any
interested party) can resurrect the patch entry by visiting

https://commitfest.postgresql.org/38/3617/

and changing the status to "Needs Review", and then changing the
status again to "Move to next CF". (Don't forget the second step;
hopefully we will have streamlined this in the near future!)

Thanks,
--Jacob

#85Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#51)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

One idea would be to add a flag, say report_parallel_vacuum_progress,
to IndexVacuumInfo struct and expect index AM to check and update the
parallel index vacuum progress, say every 1GB blocks processed. The
flag is true only when the leader process is vacuuming an index.

Regards,

Sorry for the long delay on this. I have taken the approach as suggested
by Sawada-san and Robert and attached is v12.

1. The patch introduces a new counter in the same shared memory already
used by the parallel leader and workers to keep track of the number
of indexes completed. This way there is no reason to loop through
the index status every time we want to get the status of indexes completed.

2. A new function in vacuumparallel.c will be used to update
the progress of indexes completed by reading from the
counter created in point #1.

3. The function is called during the vacuum_delay_point as a
matter of convenience, since this is called in all major vacuum
loops. The function will only do something if the caller
sets a boolean to report progress. Doing so will also ensure
progress is being reported in case the parallel workers completed
before the leader.

4. Rather than adding any complexity to WaitForParallelWorkersToFinish
and introducing a new callback, vacuumparallel.c will wait until
the number of vacuum workers is 0 and then call
WaitForParallelWorkersToFinish as it does currently.

5. Went back to the idea of adding a new view called pg_stat_progress_vacuum_index
which is accomplished by adding a new type called VACUUM_PARALLEL in progress.h

Thanks,

Sami Imseih
Amazon Web Servies (AWS)

FYI: the above message was originally sent yesterday but
was created under a separate thread. Please ignore this
thread[1]/messages/by-id/4CD97E17-B9E4-421E-9A53-4317C90EFF35@amazon.com

[1]: /messages/by-id/4CD97E17-B9E4-421E-9A53-4317C90EFF35@amazon.com

Attachments:

v12-0001--Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v12-0001--Show-progress-for-index-vacuums.patchDownload
From fd394f0bf01406f850206a6c4a81ff187a685a69 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Mon, 10 Oct 2022 11:22:25 -0500
Subject: [PATCH v12 1/1] Add 2 new columns to pg_stat_progress_vacuum. The
 columns are indexes_total as the total indexes to be vacuumed or cleaned and
 indexes_processed as the number of indexes vacuumed or cleaned up so far.

Also, introduce a new view called pg_stat_progress_vacuum_index that
exposes the current index being vacuumed.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          |  92 ++++++++++++++++++++++
 doc/src/sgml/ref/vacuum.sgml          |   8 +-
 src/backend/access/heap/vacuumlazy.c  |  44 ++++++++++-
 src/backend/catalog/system_views.sql  |  20 ++++-
 src/backend/commands/vacuum.c         |   6 ++
 src/backend/commands/vacuumparallel.c | 106 +++++++++++++++++++++++++-
 src/backend/utils/adt/pgstatfuncs.c   |   2 +
 src/include/commands/progress.h       |   4 +
 src/include/commands/vacuum.h         |   2 +
 src/include/utils/backend_progress.h  |   7 +-
 src/test/regress/expected/rules.out   |  17 ++++-
 11 files changed, 301 insertions(+), 7 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 342b20ebeb..473c76e6e8 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -392,6 +392,15 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       </entry>
      </row>
 
+     <row>
+      <entry><structname>pg_stat_progress_vacuum_index</structname><indexterm><primary>pg_stat_progress_vacuum_index</primary>
+       </indexterm>
+      </entry>
+      <entry>One row for each backend (including autovacuum worker processes) performing the <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal> phase of a <command>VACUUM</command>.
+      </entry>
+     </row>
+
      <row>
       <entry><structname>pg_stat_progress_cluster</structname><indexterm><primary>pg_stat_progress_cluster</primary></indexterm></entry>
       <entry>One row for each backend running
@@ -6414,6 +6423,89 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum or
+       vacuum failsafe is triggered. See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
+    </tbody>
+   </tgroup>
+  </table>
+
+  <indexterm>
+   <primary>pg_stat_progress_vacuum_index</primary>
+  </indexterm>
+
+  <para>
+   Whenever <command>VACUUM</command> is running, the
+   <structname>pg_stat_progress_vacuum_index</structname> view will contain
+   one row for each backend (including autovacuum worker processes) performing
+   the <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>
+   phase of a <command>VACUUM</command>.
+  </para>
+
+  <table id="pg-stat-progress-vacuum-view_index" xreflabel="pg_stat_progress_vacuum_index">
+   <title><structname>pg_stat_progress_vacuum_index</structname> View</title>
+   <tgroup cols="1">
+    <thead>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       Column Type
+      </para>
+      <para>
+       Description
+      </para></entry>
+     </row>
+    </thead>
+
+    <tbody>
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of backend.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>leader_pid</structfield> <type>integer</type>
+      </para>
+      <para>
+       Process ID of the leader backend in a parallel <command>VACUUM</command>. This value
+       will match the <structfield>pid</structfield> value whenever the leader
+       is processing an index or the <command>VACUUM</command> is not using parallel.
+       This field can be joined to <structfield>pid</structfield>
+       of <structfield>pg_stat_progress_vacuum</structfield> to get more details about
+       the <command>VACUUM</command>.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexrelid</structfield> <type>integer</type>
+      </para>
+      <para>
+       OID of the index being processed in the current vacuum phase.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/doc/src/sgml/ref/vacuum.sgml b/doc/src/sgml/ref/vacuum.sgml
index c582021d29..0c08d9ac6d 100644
--- a/doc/src/sgml/ref/vacuum.sgml
+++ b/doc/src/sgml/ref/vacuum.sgml
@@ -411,7 +411,13 @@ VACUUM [ FULL ] [ FREEZE ] [ VERBOSE ] [ ANALYZE ] [ <replaceable class="paramet
    <para>
     Each backend running <command>VACUUM</command> without the
     <literal>FULL</literal> option will report its progress in the
-    <structname>pg_stat_progress_vacuum</structname> view. Backends running
+    <structname>pg_stat_progress_vacuum</structname> view.
+    <structname>pg_stat_progress_vacuum</structname> view. Whenever a
+    <command>VACUUM</command> is in the <literal>vacuuming indexes</literal>
+    or <literal>cleaning up indexes</literal> phase,
+    the current index being processed is reported in
+    <structname>pg_stat_progress_vacuum_index</structname>.
+    <command>VACUUM FULL</command> will report their progress in the Backends running
     <command>VACUUM FULL</command> will instead report their progress in the
     <structname>pg_stat_progress_cluster</structname> view. See
     <xref linkend="vacuum-progress-reporting"/> and
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index dfbe37472f..b057d18dd9 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -350,8 +350,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 		}
 	}
 
+	/* start the vacuum progress command and report the leader pid. */
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
+	pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, MyProcPid);
 
 	/*
 	 * Get OldestXmin cutoff, which is used to determine which deleted tuples
@@ -420,6 +422,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+
+	/* report number of indexes to vacuum */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2337,10 +2343,21 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* report the index relid being vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/*
+			 * Done vacuuming an index.
+			 * Increment the indexes completed and reset the index relid to 0
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2384,6 +2401,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
 
+	/*
+	 * Reset the indexes completed at this point.
+	 * If we end up in another index vacuum cycle, we will
+	 * start counting from the start.
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
+
 	return allindexes;
 }
 
@@ -2633,10 +2657,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation
+		 *
+		 * Also, report to progress.h that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2681,9 +2712,20 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/*+ report the index relid being cleaned */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/*
+			 * Done cleaning an index.
+			 * Increment the indexes completed and reset the index relid to 0
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 55f7ec79e0..5b5e7b4080 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1162,10 +1162,28 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param10 AS indexes_total, S.param11 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
+CREATE VIEW pg_stat_progress_vacuum_index AS
+    SELECT
+        S.pid AS pid,
+        S.param9 AS leader_pid,
+        S.param8 AS indexrelid
+    FROM pg_stat_get_progress_info('VACUUM') AS S
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.param1 in (2, 4) AND S.param8 > 0
+    UNION ALL
+    SELECT
+        S.pid AS pid,
+        S.param9 AS leader_pid,
+        S.param8 AS indexrelid
+    FROM pg_stat_get_progress_info('VACUUM_PARALLEL') AS S
+        LEFT JOIN pg_database D ON S.datid = D.oid
+    WHERE S.param1 in (2, 4) AND S.param8 > 0;
+
 CREATE VIEW pg_stat_progress_cluster AS
     SELECT
         S.pid AS pid,
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index 7ccde07de9..19a2692704 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -2173,12 +2173,18 @@ vac_close_indexes(int nindexes, Relation *Irel, LOCKMODE lockmode)
  *
  * This should be called in each major loop of VACUUM processing,
  * typically once per page processed.
+ *
+ * NOTE: For convenience, parallel_vacuum_progress_report, is called
+ * here so the leader can report the number of indexes vacuumed in
+ * while inside all the major VACUUM loops.
  */
 void
 vacuum_delay_point(void)
 {
 	double		msec = 0;
 
+	parallel_vacuum_progress_report();
+
 	/* Always check for interrupts */
 	CHECK_FOR_INTERRUPTS();
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f26d796e52..65a71cbac9 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,20 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 idx_completed_progress;
+
+	/* track the leader pid of a parallel vacuum */
+	int leader_pid;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -214,6 +229,9 @@ static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_inde
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
 
+static pg_atomic_uint32 *index_vacuum_completed = NULL;
+static bool report_parallel_vacuum_progress = false;
+
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
  * shared memory state.
@@ -364,6 +382,9 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->idx_completed_progress), 0);
+
+	shared->leader_pid = MyProcPid;
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +639,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing counter ( index proress counter also ) */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->idx_completed_progress), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -657,6 +679,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 			/* Enable shared cost balance for leader backend */
 			VacuumSharedCostBalance = &(pvs->shared->cost_balance);
 			VacuumActiveNWorkers = &(pvs->shared->active_nworkers);
+
+			/*
+			 * If we are launching a parallel vacuum/cleanup,
+			 * setup the tracking.
+			 */
+			report_parallel_vacuum_progress = true;
+			index_vacuum_completed = &(pvs->shared->idx_completed_progress);
 		}
 
 		if (vacuum)
@@ -682,13 +711,36 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	 */
 	parallel_vacuum_process_safe_indexes(pvs);
 
+	/*
+	 * In case the leader completes vacuuming all
+	 * its indexes before the parallel workers do,
+	 * it can spin here waiting for the number of
+	 * active workers to complete while reporting
+	 * the progress of indexes vacuumed.
+	 *
+	 */
+	if (VacuumActiveNWorkers)
+	{
+		while (pg_atomic_read_u32(VacuumActiveNWorkers) > 0)
+		{
+			parallel_vacuum_progress_report();
+		}
+	}
+
 	/*
 	 * Next, accumulate buffer and WAL usage.  (This must wait for the workers
 	 * to finish, or we might get incomplete data.)
 	 */
 	if (nworkers > 0)
 	{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * Wait for all vacuum workers to finish
+		 *
+		 * We must do this even if we know we don't
+		 * have anymore active workers. See the
+		 * WaitForParallelWorkersToFinish commentary
+		 * as to why.
+		 */
 		WaitForParallelWorkersToFinish(pvs->pcxt);
 
 		for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)
@@ -719,6 +771,14 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 		VacuumSharedCostBalance = NULL;
 		VacuumActiveNWorkers = NULL;
 	}
+
+	/*
+	 * Disabled index vacuum progress reporting.
+	 * If we havee another index vacuum cycle, the
+	 * progress reporting will be re-enabled then.
+	 */
+	index_vacuum_completed = NULL;
+	report_parallel_vacuum_progress = false;
 }
 
 /*
@@ -823,6 +883,21 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	IndexBulkDeleteResult *istat = NULL;
 	IndexBulkDeleteResult *istat_res;
 	IndexVacuumInfo ivinfo;
+	Oid indrelid = RelationGetRelid(indrel);
+
+	/*
+	 * If we are a parallel worker, start a PROGRESS_COMMAND_VACUUM_PARALLEL
+	 * command for progress reporting, and set the leader pid.
+	 */
+	if (IsParallelWorker())
+	{
+		pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL,
+									  pvs->shared->relid);
+		pgstat_progress_update_param(PROGRESS_VACUUM_LEADER_PID, pvs->shared->leader_pid);
+	}
+
+	/* report the index being vacuumed or cleaned up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, indrelid);
 
 	/*
 	 * Update the pointer to the corresponding bulk-deletion result if someone
@@ -846,9 +921,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
 			break;
 		default:
@@ -888,6 +965,17 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Reset the index relid begin vacuumed and incremebts the
+	 * index vacuum counter.
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
+	pg_atomic_add_fetch_u32(&(pvs->shared->idx_completed_progress), 1);
+
+	/* if we are a parallel worker, end the command */
+	if (IsParallelWorker())
+		pgstat_progress_end_command();
 }
 
 /*
@@ -1072,3 +1160,17 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read the number of indexes vacuumed from the shared counter
+ * and report it to progress.h
+ */
+void
+parallel_vacuum_progress_report(void)
+{
+	if (IsParallelWorker() || !report_parallel_vacuum_progress)
+		return;
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+								 pg_atomic_read_u32(index_vacuum_completed));
+}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index eadd8464ff..1584a9bd7d 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -484,6 +484,8 @@ pg_stat_get_progress_info(PG_FUNCTION_ARGS)
 		cmdtype = PROGRESS_COMMAND_BASEBACKUP;
 	else if (pg_strcasecmp(cmd, "COPY") == 0)
 		cmdtype = PROGRESS_COMMAND_COPY;
+	else if (pg_strcasecmp(cmd, "VACUUM_PARALLEL") == 0)
+		cmdtype = PROGRESS_COMMAND_VACUUM_PARALLEL;
 	else
 		ereport(ERROR,
 				(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..410eec217b 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,10 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDRELID				7
+#define PROGRESS_VACUUM_LEADER_PID				8
+#define PROGRESS_VACUUM_INDEX_TOTAL             9
+#define PROGRESS_VACUUM_INDEX_COMPLETED         10
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 5d816ba7f4..96901d7234 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -338,4 +338,6 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+extern void parallel_vacuum_progress_report(void);
+
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..74d9dfb4c7 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -17,6 +17,10 @@
 
 /* ----------
  * Command type for progress reporting purposes
+ *
+ * Note: PROGRESS_COMMAND_VACUUM_PARALLEL is not
+ * a command per se, but this type is used to track
+ * parallel vacuum workers progress.
  * ----------
  */
 typedef enum ProgressCommandType
@@ -27,7 +31,8 @@ typedef enum ProgressCommandType
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
 	PROGRESS_COMMAND_BASEBACKUP,
-	PROGRESS_COMMAND_COPY
+	PROGRESS_COMMAND_COPY,
+	PROGRESS_COMMAND_VACUUM_PARALLEL
 } ProgressCommandType;
 
 #define PGSTAT_NUM_PROGRESS_PARAM	20
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 9dd137415e..d7d6148d52 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2013,9 +2013,24 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param10 AS indexes_total,
+    s.param11 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
+pg_stat_progress_vacuum_index| SELECT s.pid,
+    s.param9 AS leader_pid,
+    s.param8 AS indexrelid
+   FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s.param8 > 0))
+UNION ALL
+ SELECT s.pid,
+    s.param9 AS leader_pid,
+    s.param8 AS indexrelid
+   FROM (pg_stat_get_progress_info('VACUUM_PARALLEL'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
+     LEFT JOIN pg_database d ON ((s.datid = d.oid)))
+  WHERE ((s.param1 = ANY (ARRAY[(2)::bigint, (4)::bigint])) AND (s.param8 > 0));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
     s.prefetch,
     s.hit,
-- 
2.32.1 (Apple Git-133)

#86Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#85)
Re: Add index scan progress to pg_stat_progress_vacuum

On Tue, Oct 11, 2022 at 10:50 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

One idea would be to add a flag, say report_parallel_vacuum_progress,
to IndexVacuumInfo struct and expect index AM to check and update the
parallel index vacuum progress, say every 1GB blocks processed. The
flag is true only when the leader process is vacuuming an index.

Regards,

Sorry for the long delay on this. I have taken the approach as suggested
by Sawada-san and Robert and attached is v12.

Thank you for updating the patch!

1. The patch introduces a new counter in the same shared memory already
used by the parallel leader and workers to keep track of the number
of indexes completed. This way there is no reason to loop through
the index status every time we want to get the status of indexes completed.

While it seems to be a good idea to have the atomic counter for the
number of indexes completed, I think we should not use the global
variable referencing the counter as follow:

+static pg_atomic_uint32 *index_vacuum_completed = NULL;
:
+void
+parallel_vacuum_progress_report(void)
+{
+   if (IsParallelWorker() || !report_parallel_vacuum_progress)
+       return;
+
+   pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+                                pg_atomic_read_u32(index_vacuum_completed));
+}

I think we can pass ParallelVacuumState (or PVIndStats) to the
reporting function so that it can check the counter and report the
progress.

2. A new function in vacuumparallel.c will be used to update
the progress of indexes completed by reading from the
counter created in point #1.

3. The function is called during the vacuum_delay_point as a
matter of convenience, since this is called in all major vacuum
loops. The function will only do something if the caller
sets a boolean to report progress. Doing so will also ensure
progress is being reported in case the parallel workers completed
before the leader.

Robert pointed out:

---
I am not too sure that the idea of using the vacuum delay points is the best
plan. I think we should try to avoid piggybacking on such general
infrastructure if we can, and instead look for a way to tie this to
something that is specific to parallel vacuum.
---

I agree with this part.

Instead, I think we can add a boolean and the pointer for
ParallelVacuumState to IndexVacuumInfo. If the boolean is true, an
index AM can call the reporting function with the pointer to
ParallelVacuumState while scanning index blocks, for example, for
every 1GB. The boolean can be true only for the leader process.

4. Rather than adding any complexity to WaitForParallelWorkersToFinish
and introducing a new callback, vacuumparallel.c will wait until
the number of vacuum workers is 0 and then call
WaitForParallelWorkersToFinish as it does currently.

Agreed, but with the following change, the leader process waits in a
busy loop, which should not be acceptable:

+   if (VacuumActiveNWorkers)
+   {
+       while (pg_atomic_read_u32(VacuumActiveNWorkers) > 0)
+       {
+           parallel_vacuum_progress_report();
+       }
+   }
+

Also, I think it's better to check whether idx_completed_progress
equals to the number indexes instead.

5. Went back to the idea of adding a new view called pg_stat_progress_vacuum_index
which is accomplished by adding a new type called VACUUM_PARALLEL in progress.h

Probably, we can devide the patch into two patches. One for adding the
new statistics of the number of vacuumed indexes to
pg_stat_progress_vacuum and another one for adding new statistics view
pg_stat_progress_vacuum_index.

Regards,

--
Masahiko Sawada
PostgreSQL Contributors Team
RDS Open Source Databases
Amazon Web Services: https://aws.amazon.com

#87Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#86)
Re: Add index scan progress to pg_stat_progress_vacuum

Thank you for the feedback!

While it seems to be a good idea to have the atomic counter for the
number of indexes completed, I think we should not use the global
variable referencing the counter as follow:

+static pg_atomic_uint32 *index_vacuum_completed = NULL;
:
+void
+parallel_vacuum_progress_report(void)
+{
+   if (IsParallelWorker() || !report_parallel_vacuum_progress)
+       return;
+
+   pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+                                pg_atomic_read_u32(index_vacuum_completed));
+}

I think we can pass ParallelVacuumState (or PVIndStats) to the
reporting function so that it can check the counter and report the
progress.

Yes, that makes sense.

---
I am not too sure that the idea of using the vacuum delay points is the best
plan. I think we should try to avoid piggybacking on such general
infrastructure if we can, and instead look for a way to tie this to
something that is specific to parallel vacuum.
---

Adding the call to vacuum_delay_point made sense since it's
called in all major vacuum scans. While it is also called
by analyze, it will only do anything if the caller sets a flag
to report_parallel_vacuum_progress.

Instead, I think we can add a boolean and the pointer for
ParallelVacuumState to IndexVacuumInfo. If the boolean is true, an
index AM can call the reporting function with the pointer to
ParallelVacuumState while scanning index blocks, for example, for
every 1GB. The boolean can be true only for the leader process.

Will doing this every 1GB be necessary? Considering the function
will not do much more than update progress from the value
stored in index_vacuum_completed?

Agreed, but with the following change, the leader process waits in a
busy loop, which should not be acceptable:

Good point.

Also, I think it's better to check whether idx_completed_progress

equals to the number indexes instead.

Good point

5. Went back to the idea of adding a new view called pg_stat_progress_vacuum_index
which is accomplished by adding a new type called VACUUM_PARALLEL in progress.h

Probably, we can devide the patch into two patches. One for adding the

Makes sense.

Thanks

Sami Imseih
Amazon Web Services (AWS)

#88Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#87)
2 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Attached is v13-0001--Show-progress-for-index-vacuums.patch which addresses
the latest comments. The main changes are:

1/ Call the parallel_vacuum_progress_report inside the AMs rather than vacuum_delay_point.

2/ A Boolean when set to True in vacuumparallel.c will be used to determine if calling
parallel_vacuum_progress_report is necessary.

3/ Removed global varilable from vacuumparallel.c

4/ Went back to calling parallel_vacuum_progress_report inside
WaitForParallelWorkersToFinish to cover the case when a
leader is waiting for parallel workers to finish.

5/ I did not see a need to only report progress after 1GB as it's a fairly cheap call to update
progress.

6/ v1-0001-Function-to-return-currently-vacuumed-or-cleaned-ind.patch is a separate patch
for exposing the index relid being vacuumed by a backend.

Thanks

Sami Imseih
Amazon Web Services (AWS)

Attachments:

v13-0001--Show-progress-for-index-vacuums.patchapplication/octet-stream; name=v13-0001--Show-progress-for-index-vacuums.patchDownload
From e080c28d09567cb3dd974c0855487c717dad0a57 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Tue, 1 Nov 2022 11:50:14 -0500
Subject: [PATCH 1/1] Add 2 new columns to pg_stat_progress_vacuum. The columns
 are indexes_total as the total indexes to be vacuumed or cleaned and
 indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c              |  6 +++
 doc/src/sgml/monitoring.sgml          | 21 +++++++++
 src/backend/access/brin/brin.c        |  9 ++--
 src/backend/access/gin/ginvacuum.c    |  9 ++++
 src/backend/access/gist/gistvacuum.c  |  5 +++
 src/backend/access/hash/hash.c        |  8 +++-
 src/backend/access/hash/hashpage.c    |  4 +-
 src/backend/access/heap/vacuumlazy.c  | 40 ++++++++++++++++-
 src/backend/access/nbtree/nbtree.c    |  3 ++
 src/backend/access/spgist/spgvacuum.c |  3 ++
 src/backend/access/transam/parallel.c |  5 +++
 src/backend/catalog/index.c           |  1 +
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 63 ++++++++++++++++++++++++++-
 src/include/access/genam.h            |  3 ++
 src/include/access/hash.h             |  3 +-
 src/include/access/parallel.h         |  2 +
 src/include/commands/progress.h       |  2 +
 src/include/commands/vacuum.h         |  2 +
 src/test/regress/expected/rules.out   |  4 +-
 20 files changed, 184 insertions(+), 12 deletions(-)

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 91fae5b0c0..19ae819cfe 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -136,6 +136,9 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			GenericXLogAbort(gxlogState);
 		}
 		UnlockReleaseBuffer(buffer);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	/*
@@ -209,6 +212,9 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		}
 
 		UnlockReleaseBuffer(buffer);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	IndexFreeSpaceMapVacuum(info->index);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5d622d514..c61443f3d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6455,6 +6455,27 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum or
+       vacuum failsafe is triggered. See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 20b7d65b94..bd96ecf41e 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -78,7 +78,7 @@ static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRang
 static void form_and_insert_tuple(BrinBuildState *state);
 static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
 						 BrinTuple *b);
-static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
+static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, IndexVacuumInfo *info);
 static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
 								BrinMemTuple *dtup, Datum *values, bool *nulls);
 static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
@@ -952,7 +952,7 @@ brinvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 	heapRel = table_open(IndexGetRelation(RelationGetRelid(info->index), false),
 						 AccessShareLock);
 
-	brin_vacuum_scan(info->index, info->strategy);
+	brin_vacuum_scan(info->index, info->strategy, info);
 
 	brinsummarize(info->index, heapRel, BRIN_ALL_BLOCKRANGES, false,
 				  &stats->num_index_tuples, &stats->num_index_tuples);
@@ -1659,7 +1659,7 @@ union_tuples(BrinDesc *bdesc, BrinMemTuple *a, BrinTuple *b)
  * and such.
  */
 static void
-brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
+brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, IndexVacuumInfo *info)
 {
 	BlockNumber nblocks;
 	BlockNumber blkno;
@@ -1681,6 +1681,9 @@ brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
 		brin_page_cleanup(idxrel, buf);
 
 		ReleaseBuffer(buf);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	/*
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..3d5e4600dc 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -633,6 +633,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 		UnlockReleaseBuffer(buffer);
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	/* right now we found leftmost page in entry's BTree */
@@ -677,6 +680,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_EXCLUSIVE);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	MemoryContextDelete(gvs.tmpCxt);
@@ -775,6 +781,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		}
 
 		UnlockReleaseBuffer(buffer);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	/* Update the metapage with accurate page and entry counts */
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0aa6e58a62..da3a0540d0 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -223,7 +223,12 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+
+			if (info->report_parallel_progress)
+				info->parallel_progress_callback(info->parallel_progress_arg);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index c361509d68..14790adf61 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -550,7 +550,7 @@ loop_top:
 						  cachedmetap->hashm_highmask,
 						  cachedmetap->hashm_lowmask, &tuples_removed,
 						  &num_index_tuples, split_cleanup,
-						  callback, callback_state);
+						  callback, callback_state, info);
 
 		_hash_dropbuf(rel, bucket_buf);
 
@@ -686,7 +686,8 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 				  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 				  double *tuples_removed, double *num_index_tuples,
 				  bool split_cleanup,
-				  IndexBulkDeleteCallback callback, void *callback_state)
+				  IndexBulkDeleteCallback callback, void *callback_state,
+				  IndexVacuumInfo *info)
 {
 	BlockNumber blkno;
 	Buffer		buf;
@@ -775,6 +776,9 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 				if (num_index_tuples)
 					*num_index_tuples += 1;
 			}
+
+			if (info && info->report_parallel_progress)
+				info->parallel_progress_callback(info->parallel_progress_arg);
 		}
 
 		/* retain the pin on primary bucket page till end of bucket scan */
diff --git a/src/backend/access/hash/hashpage.c b/src/backend/access/hash/hashpage.c
index d2edcd4617..e9ce1a1110 100644
--- a/src/backend/access/hash/hashpage.c
+++ b/src/backend/access/hash/hashpage.c
@@ -759,7 +759,7 @@ restart_expand:
 
 		hashbucketcleanup(rel, old_bucket, buf_oblkno, start_oblkno, NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL);
 
 		_hash_dropbuf(rel, buf_oblkno);
 
@@ -1327,7 +1327,7 @@ _hash_splitbucket(Relation rel,
 		hashbucketcleanup(rel, obucket, bucket_obuf,
 						  BufferGetBlockNumber(bucket_obuf), NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL);
 	}
 	else
 	{
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index dfbe37472f..120a2b18d7 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -420,6 +420,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+
+	/* report number of indexes to vacuum */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2341,6 +2345,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/*
+			 * Done vacuuming an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2384,6 +2394,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
 
+	/*
+	 * Reset the indexes completed at this point.
+	 * If we end up in another index vacuum cycle, we will
+	 * start counting from the start.
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
+
 	return allindexes;
 }
 
@@ -2633,10 +2650,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation
+		 *
+		 * Also, report to progress.h that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2684,6 +2708,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/*
+			 * Done cleaning an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2693,6 +2723,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/*
+	 * Reset the indexes completed at this point.
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
+
 }
 
 /*
@@ -2718,6 +2754,7 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
@@ -2766,6 +2803,7 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index b52eca8f38..9904a9eb0c 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,9 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+
+			if (info->report_parallel_progress)
+				info->parallel_progress_callback(info->parallel_progress_arg);
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..b1ab491980 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -843,6 +843,9 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+
+			if (bds->info->report_parallel_progress)
+				bds->info->parallel_progress_callback(bds->info->parallel_progress_arg);
 		}
 	}
 
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index ee0985c7ed..0b4bde8297 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -185,6 +185,8 @@ CreateParallelContext(const char *library_name, const char *function_name,
 	pcxt->library_name = pstrdup(library_name);
 	pcxt->function_name = pstrdup(function_name);
 	pcxt->error_context_stack = error_context_stack;
+	pcxt->parallel_progress_callback = NULL;
+	pcxt->parallel_progress_arg = NULL;
 	shm_toc_initialize_estimator(&pcxt->estimator);
 	dlist_push_head(&pcxt_list, &pcxt->node);
 
@@ -785,6 +787,9 @@ WaitForParallelWorkersToFinish(ParallelContext *pcxt)
 		 */
 		CHECK_FOR_INTERRUPTS();
 
+		if (pcxt->parallel_progress_callback)
+			pcxt->parallel_progress_callback(pcxt->parallel_progress_arg);
+
 		for (i = 0; i < pcxt->nworkers_launched; ++i)
 		{
 			/*
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 61f1d3926a..11b32129a7 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,6 +3348,7 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2d8104b090..c37b20b91b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f26d796e52..85e2de5d54 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 idx_completed_progress;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -273,6 +285,9 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	Assert(pcxt->nworkers > 0);
 	pvs->pcxt = pcxt;
 
+	pvs->pcxt->parallel_progress_callback = parallel_vacuum_progress_report;
+	pvs->pcxt->parallel_progress_arg = pvs;
+
 	/* Estimate size for index vacuum stats -- PARALLEL_VACUUM_KEY_INDEX_STATS */
 	est_indstats_len = mul_size(sizeof(PVIndStats), nindexes);
 	shm_toc_estimate_chunk(&pcxt->estimator, est_indstats_len);
@@ -364,6 +379,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->idx_completed_progress), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +634,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing counter ( index progress counter also ) */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->idx_completed_progress), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -834,10 +851,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.estimated_count = pvs->shared->estimated_count;
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
+	ivinfo.parallel_progress_callback = parallel_vacuum_progress_report;
+	ivinfo.parallel_progress_arg = pvs;
 
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
@@ -846,9 +866,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
 			break;
 		default:
@@ -888,6 +910,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Update the number of indexes completed.
+	 */
+	pg_atomic_add_fetch_u32(&(pvs->shared->idx_completed_progress), 1);
 }
 
 /*
@@ -1072,3 +1099,37 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read the number of indexes vacuumed from the shared counter
+ * and report it to progress.h
+ *
+ * This function is only called by the leader process of
+ * a parallel vacuum.
+ *
+ * If report_parallel_progress is set to true,
+ * this function is called from major loops inside the
+ * ambulkdelete and amvacuumcleanup.
+ *
+ * This function is also called inside WaitForParallelWorkersToFinish
+ * when report_parallel_progress is set to true.
+ *
+ * The reason for calling this function in both vacuum AM's
+ * and WaitForParallelWorkersToFinish is to ensure that
+ * parallel vacuum progress is constantly being reported
+ * if the leader process is either waiting for parallel
+ * proceses to finish (WaitForParallelWorkersToFinish) or
+ * the leader process is the last to finish and still
+ * inside the vacuum AM's.
+ */
+void
+parallel_vacuum_progress_report(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *) arg;
+
+	if (IsParallelWorker())
+		return;
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+								 pg_atomic_read_u32(&(pvs->shared->idx_completed_progress)));
+}
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index e1c4fdbd03..6d5fc189f4 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -46,10 +46,13 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
 	BufferAccessStrategy strategy;	/* access strategy for reads */
+	void       (*parallel_progress_callback)(void *arg);
+	void       *parallel_progress_arg;
 } IndexVacuumInfo;
 
 /*
diff --git a/src/include/access/hash.h b/src/include/access/hash.h
index da372841c4..14fb75a4ad 100644
--- a/src/include/access/hash.h
+++ b/src/include/access/hash.h
@@ -480,6 +480,7 @@ extern void hashbucketcleanup(Relation rel, Bucket cur_bucket,
 							  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 							  double *tuples_removed, double *num_index_tuples,
 							  bool split_cleanup,
-							  IndexBulkDeleteCallback callback, void *callback_state);
+							  IndexBulkDeleteCallback callback, void *callback_state,
+							  IndexVacuumInfo *info);
 
 #endif							/* HASH_H */
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 1ec8e33af4..77e9c0a517 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -46,6 +46,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	void       (*parallel_progress_callback)(void *arg);
+	void       *parallel_progress_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..0e97c6d4ef 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 5d816ba7f4..9510d028d3 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -338,4 +338,6 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+extern void parallel_vacuum_progress_report(void *arg);
+
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 624d0e5aae..e0bf81247f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2018,7 +2018,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

v1-0001-Function-to-return-currently-vacuumed-or-cleaned-ind.patchapplication/octet-stream; name=v1-0001-Function-to-return-currently-vacuumed-or-cleaned-ind.patchDownload
From 03f716bb9b1a79884655d87a6b3986a622aa4285 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Wed, 2 Nov 2022 10:36:10 -0500
Subject: [PATCH 1/1] Function to return currently vacuumed ( or cleaned )
 index

pg_stat_get_vacuum_index takes a PID as an argument and
returns the index relid of the currently vacuumed index.
NULL is returned if the backend is not involved in an index vacuum.
---
 doc/src/sgml/func.sgml                | 13 +++++++
 doc/src/sgml/monitoring.sgml          |  8 +++--
 src/backend/access/heap/vacuumlazy.c  |  9 +++++
 src/backend/commands/vacuumparallel.c | 23 ++++++++++++
 src/backend/utils/adt/pgstatfuncs.c   | 52 +++++++++++++++++++++++++++
 src/include/catalog/pg_proc.dat       |  4 +++
 src/include/commands/progress.h       |  1 +
 src/include/utils/backend_progress.h  |  7 +++-
 8 files changed, 114 insertions(+), 3 deletions(-)

diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 6e0425cb3d..39841354de 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -22670,6 +22670,19 @@ SELECT * FROM pg_ls_dir('.') WITH ORDINALITY AS t(ls,n);
         parsing the text version.
        </para></entry>
       </row>
+
+      <row>
+       <entry role="func_table_entry"><para role="func_signature">
+        <indexterm>
+         <primary>pg_stat_get_vacuum_index</primary>
+        </indexterm>
+        <function>pg_stat_get_vacuum_index</function> ( <type>integer</type> )
+        <returnvalue>oid</returnvalue>
+       </para>
+       <para>
+        Returns the relid of an index current being vacuumed or cleaned by the backend.
+       </para></entry>
+      </row>
      </tbody>
     </tgroup>
    </table>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index c61443f3d5..8d947d8c3a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6517,7 +6517,9 @@ FROM pg_stat_get_backend_idset() AS backendid;
        has been completely scanned.  It may happen multiple times per vacuum
        if <xref linkend="guc-maintenance-work-mem"/> (or, in the case of autovacuum,
        <xref linkend="guc-autovacuum-work-mem"/> if set) is insufficient to store
-       the number of dead tuples found.
+       the number of dead tuples found. While in this phase, the function
+       <function>pg_stat_get_vacuum_index()</function> will return the relid
+       of the index being vacuumed.
      </entry>
     </row>
     <row>
@@ -6536,7 +6538,9 @@ FROM pg_stat_get_backend_idset() AS backendid;
      <entry>
        <command>VACUUM</command> is currently cleaning up indexes.  This occurs after
        the heap has been completely scanned and all vacuuming of the indexes
-       and the heap has been completed.
+       and the heap has been completed. While in this phase, the function
+       <function>pg_stat_get_vacuum_index()</function> will return the relid
+       of the index being cleaned.
      </entry>
     </row>
     <row>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 120a2b18d7..8cd8e34f88 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -350,6 +350,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 		}
 	}
 
+	/* start the vacuum progress command and report to progress.h */
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
 
@@ -2341,6 +2342,9 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* report the index relid being vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
@@ -2350,6 +2354,7 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			 */
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
 										 idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
 
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
@@ -2705,6 +2710,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* report the index relid being cleaned */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
@@ -2714,6 +2722,7 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			 */
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
 										 idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 85e2de5d54..f2f51e86eb 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -840,6 +840,20 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	IndexBulkDeleteResult *istat = NULL;
 	IndexBulkDeleteResult *istat_res;
 	IndexVacuumInfo ivinfo;
+	Oid indrelid = RelationGetRelid(indrel);
+
+	/*
+	 * If we are a parallel worker, start a PROGRESS_COMMAND_VACUUM_PARALLEL
+	 * command for progress reporting and report the index being vacuumed.
+	 */
+	if (IsParallelWorker())
+	{
+		pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL,
+									  pvs->shared->relid);
+	}
+
+	/* report the index being vacuumed or cleaned up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, indrelid);
 
 	/*
 	 * Update the pointer to the corresponding bulk-deletion result if someone
@@ -911,10 +925,19 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pfree(pvs->indname);
 	pvs->indname = NULL;
 
+	/*
+	 * Reset the index relid begin vacuumed/cleaned
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
+
 	/*
 	 * Update the number of indexes completed.
 	 */
 	pg_atomic_add_fetch_u32(&(pvs->shared->idx_completed_progress), 1);
+
+	/* if we are a parallel worker, end the command */
+	if (IsParallelWorker())
+		pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 96bffc0f2a..b08c15e7aa 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -19,6 +19,7 @@
 #include "access/xlogprefetcher.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -2383,3 +2384,54 @@ pg_stat_have_stats(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(pgstat_have_entry(kind, dboid, objoid));
 }
+
+/*
+ * For a given PID, return the relid of the index
+ * currently being vacuumed.
+ */
+Datum
+pg_stat_get_vacuum_index(PG_FUNCTION_ARGS)
+{
+	int			pid = PG_GETARG_INT32(0);
+	int			num_backends = pgstat_fetch_stat_numbackends();
+	int			curr_backend;
+	Oid 		indrelid = 0;
+	ProgressCommandType cmdtype_vacuum = PROGRESS_COMMAND_VACUUM;
+	ProgressCommandType cmdtype_vacuum_parallel = PROGRESS_COMMAND_VACUUM_PARALLEL;
+
+	/* 1-based index */
+	for (curr_backend = 1; curr_backend <= num_backends; curr_backend++)
+	{
+		LocalPgBackendStatus *local_beentry;
+		PgBackendStatus *beentry;
+
+		local_beentry = pgstat_fetch_stat_local_beentry(curr_backend);
+		beentry = &local_beentry->backendStatus;
+
+		/*
+		 * If a backend has a VACUUM or VACUUM_PARALLEL progress command type
+		 * and the procpid matches the given PID, return the index relid id
+		 * that is current being vacuumed. In some cases, the index relid could
+		 * be a '0' if we got here at thhe end of a vacuum or when no indexes
+		 * are being vacuumed. In this case return NULL.
+		 */
+		if	(
+				(beentry->st_progress_command == cmdtype_vacuum
+					|| beentry->st_progress_command == cmdtype_vacuum_parallel) &&
+				 beentry->st_procpid == pid
+			)
+			{
+				indrelid = Int64GetDatum(beentry->st_progress_param[PROGRESS_VACUUM_INDRELID]);
+
+				if (indrelid > 0)
+					PG_RETURN_OID(indrelid);
+				else
+					PG_RETURN_NULL();
+			}
+	}
+
+	/*
+	 * No matching PID performing a VACUUM operation, return NULL.
+	 */
+	PG_RETURN_NULL();
+}
\ No newline at end of file
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 20f5aa56ea..dd5a32172a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11827,5 +11827,9 @@
   proname => 'brin_minmax_multi_summary_send', provolatile => 's',
   prorettype => 'bytea', proargtypes => 'pg_brin_minmax_multi_summary',
   prosrc => 'brin_minmax_multi_summary_send' },
+{ oid => '4642',
+  descr => 'get the relid of the index being vacuumed by a backend',
+  proname => 'pg_stat_get_vacuum_index', provolatile => 'v', prorettype => 'int8',
+  proargtypes => 'int4', prosrc => 'pg_stat_get_vacuum_index' },
 
 ]
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index 0e97c6d4ef..bc2b7d1e95 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,6 +27,7 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_INDEX_TOTAL             7
 #define PROGRESS_VACUUM_INDEX_COMPLETED         8
+#define PROGRESS_VACUUM_INDRELID                9
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..74d9dfb4c7 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -17,6 +17,10 @@
 
 /* ----------
  * Command type for progress reporting purposes
+ *
+ * Note: PROGRESS_COMMAND_VACUUM_PARALLEL is not
+ * a command per se, but this type is used to track
+ * parallel vacuum workers progress.
  * ----------
  */
 typedef enum ProgressCommandType
@@ -27,7 +31,8 @@ typedef enum ProgressCommandType
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
 	PROGRESS_COMMAND_BASEBACKUP,
-	PROGRESS_COMMAND_COPY
+	PROGRESS_COMMAND_COPY,
+	PROGRESS_COMMAND_VACUUM_PARALLEL
 } ProgressCommandType;
 
 #define PGSTAT_NUM_PROGRESS_PARAM	20
-- 
2.32.1 (Apple Git-133)

#89Ian Lawrence Barwick
barwick@gmail.com
In reply to: Imseih (AWS), Sami (#88)
Re: Add index scan progress to pg_stat_progress_vacuum

2022年11月3日(木) 1:52 Imseih (AWS), Sami <simseih@amazon.com>:

Attached is v13-0001--Show-progress-for-index-vacuums.patch which addresses
the latest comments. The main changes are:

1/ Call the parallel_vacuum_progress_report inside the AMs rather than vacuum_delay_point.

2/ A Boolean when set to True in vacuumparallel.c will be used to determine if calling
parallel_vacuum_progress_report is necessary.

3/ Removed global varilable from vacuumparallel.c

4/ Went back to calling parallel_vacuum_progress_report inside
WaitForParallelWorkersToFinish to cover the case when a
leader is waiting for parallel workers to finish.

5/ I did not see a need to only report progress after 1GB as it's a fairly cheap call to update
progress.

6/ v1-0001-Function-to-return-currently-vacuumed-or-cleaned-ind.patch is a separate patch
for exposing the index relid being vacuumed by a backend.

This entry was marked "Needs review" in the CommitFest app but cfbot
reports the patch [1]this patch: v1-0001-Function-to-return-currently-vacuumed-or-cleaned-ind.patch no longer applies.

[1]: this patch: v1-0001-Function-to-return-currently-vacuumed-or-cleaned-ind.patch
v1-0001-Function-to-return-currently-vacuumed-or-cleaned-ind.patch

We've marked it as "Waiting on Author". As CommitFest 2022-11 is
currently underway, this would be an excellent time update the patch.

Once you think the patchset is ready for review again, you (or any
interested party) can move the patch entry forward by visiting

https://commitfest.postgresql.org/40/3617/

and changing the status to "Needs review".

Thanks

Ian Barwick

#90Imseih (AWS), Sami
simseih@amazon.com
In reply to: Imseih (AWS), Sami (#88)
2 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Resubmitting patches with proper format.

Thanks

Sami Imseih
Amazon Web Services (AWS)

Attachments:

v14-0002-Function-to-return-currently-vacuumed-or-cleaned.patchapplication/octet-stream; name=v14-0002-Function-to-return-currently-vacuumed-or-cleaned.patchDownload
From 78a437946d5ccc70db98da9cc0fa0ae413afa625 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Fri, 4 Nov 2022 07:52:02 -0500
Subject: [PATCH v14 2/2] Function to return currently vacuumed ( or cleaned )
 index.

pg_stat_get_vacuum_index takes a PID as an argument and
returns the index relid of the currently vacuumed index.
NULL is returned if the backend is not involved in an index vacuum.
---
 doc/src/sgml/func.sgml                | 13 +++++++
 doc/src/sgml/monitoring.sgml          |  8 +++--
 src/backend/access/heap/vacuumlazy.c  |  9 +++++
 src/backend/commands/vacuumparallel.c | 23 ++++++++++++
 src/backend/utils/adt/pgstatfuncs.c   | 52 +++++++++++++++++++++++++++
 src/include/catalog/pg_proc.dat       |  4 +++
 src/include/commands/progress.h       |  1 +
 src/include/utils/backend_progress.h  |  7 +++-
 8 files changed, 114 insertions(+), 3 deletions(-)
  22.2% doc/src/sgml/
  11.2% src/backend/access/heap/
  16.7% src/backend/commands/
  37.1% src/backend/utils/adt/
   5.7% src/include/catalog/
   5.4% src/include/utils/

diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 6e0425cb3d..39841354de 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -22670,6 +22670,19 @@ SELECT * FROM pg_ls_dir('.') WITH ORDINALITY AS t(ls,n);
         parsing the text version.
        </para></entry>
       </row>
+
+      <row>
+       <entry role="func_table_entry"><para role="func_signature">
+        <indexterm>
+         <primary>pg_stat_get_vacuum_index</primary>
+        </indexterm>
+        <function>pg_stat_get_vacuum_index</function> ( <type>integer</type> )
+        <returnvalue>oid</returnvalue>
+       </para>
+       <para>
+        Returns the relid of an index current being vacuumed or cleaned by the backend.
+       </para></entry>
+      </row>
      </tbody>
     </tgroup>
    </table>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index c61443f3d5..8d947d8c3a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6517,7 +6517,9 @@ FROM pg_stat_get_backend_idset() AS backendid;
        has been completely scanned.  It may happen multiple times per vacuum
        if <xref linkend="guc-maintenance-work-mem"/> (or, in the case of autovacuum,
        <xref linkend="guc-autovacuum-work-mem"/> if set) is insufficient to store
-       the number of dead tuples found.
+       the number of dead tuples found. While in this phase, the function
+       <function>pg_stat_get_vacuum_index()</function> will return the relid
+       of the index being vacuumed.
      </entry>
     </row>
     <row>
@@ -6536,7 +6538,9 @@ FROM pg_stat_get_backend_idset() AS backendid;
      <entry>
        <command>VACUUM</command> is currently cleaning up indexes.  This occurs after
        the heap has been completely scanned and all vacuuming of the indexes
-       and the heap has been completed.
+       and the heap has been completed. While in this phase, the function
+       <function>pg_stat_get_vacuum_index()</function> will return the relid
+       of the index being cleaned.
      </entry>
     </row>
     <row>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 120a2b18d7..8cd8e34f88 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -350,6 +350,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 		}
 	}
 
+	/* start the vacuum progress command and report to progress.h */
 	pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM,
 								  RelationGetRelid(rel));
 
@@ -2341,6 +2342,9 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* report the index relid being vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
@@ -2350,6 +2354,7 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 			 */
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
 										 idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
 
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
@@ -2705,6 +2710,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			Relation	indrel = vacrel->indrels[idx];
 			IndexBulkDeleteResult *istat = vacrel->indstats[idx];
 
+			/* report the index relid being cleaned */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, RelationGetRelid(indrel));
+
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
@@ -2714,6 +2722,7 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			 */
 			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
 										 idx + 1);
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
 		}
 	}
 	else
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 85e2de5d54..f2f51e86eb 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -840,6 +840,20 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	IndexBulkDeleteResult *istat = NULL;
 	IndexBulkDeleteResult *istat_res;
 	IndexVacuumInfo ivinfo;
+	Oid indrelid = RelationGetRelid(indrel);
+
+	/*
+	 * If we are a parallel worker, start a PROGRESS_COMMAND_VACUUM_PARALLEL
+	 * command for progress reporting and report the index being vacuumed.
+	 */
+	if (IsParallelWorker())
+	{
+		pgstat_progress_start_command(PROGRESS_COMMAND_VACUUM_PARALLEL,
+									  pvs->shared->relid);
+	}
+
+	/* report the index being vacuumed or cleaned up */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, indrelid);
 
 	/*
 	 * Update the pointer to the corresponding bulk-deletion result if someone
@@ -911,10 +925,19 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pfree(pvs->indname);
 	pvs->indname = NULL;
 
+	/*
+	 * Reset the index relid begin vacuumed/cleaned
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDRELID, 0);
+
 	/*
 	 * Update the number of indexes completed.
 	 */
 	pg_atomic_add_fetch_u32(&(pvs->shared->idx_completed_progress), 1);
+
+	/* if we are a parallel worker, end the command */
+	if (IsParallelWorker())
+		pgstat_progress_end_command();
 }
 
 /*
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 96bffc0f2a..82eedae1f0 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -19,6 +19,7 @@
 #include "access/xlogprefetcher.h"
 #include "catalog/pg_authid.h"
 #include "catalog/pg_type.h"
+#include "commands/progress.h"
 #include "common/ip.h"
 #include "funcapi.h"
 #include "miscadmin.h"
@@ -2383,3 +2384,54 @@ pg_stat_have_stats(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(pgstat_have_entry(kind, dboid, objoid));
 }
+
+/*
+ * For a given PID, return the relid of the index
+ * currently being vacuumed.
+ */
+Datum
+pg_stat_get_vacuum_index(PG_FUNCTION_ARGS)
+{
+	int			pid = PG_GETARG_INT32(0);
+	int			num_backends = pgstat_fetch_stat_numbackends();
+	int			curr_backend;
+	Oid 		indrelid = 0;
+	ProgressCommandType cmdtype_vacuum = PROGRESS_COMMAND_VACUUM;
+	ProgressCommandType cmdtype_vacuum_parallel = PROGRESS_COMMAND_VACUUM_PARALLEL;
+
+	/* 1-based index */
+	for (curr_backend = 1; curr_backend <= num_backends; curr_backend++)
+	{
+		LocalPgBackendStatus *local_beentry;
+		PgBackendStatus *beentry;
+
+		local_beentry = pgstat_fetch_stat_local_beentry(curr_backend);
+		beentry = &local_beentry->backendStatus;
+
+		/*
+		 * If a backend has a VACUUM or VACUUM_PARALLEL progress command type
+		 * and the procpid matches the given PID, return the index relid
+		 * that is current being vacuumed. In some cases, the index relid could
+		 * be a '0' if we got here at the end of a vacuum or when no indexes
+		 * are being vacuumed. In this case return NULL.
+		 */
+		if	(
+				(beentry->st_progress_command == cmdtype_vacuum
+					|| beentry->st_progress_command == cmdtype_vacuum_parallel) &&
+				 beentry->st_procpid == pid
+			)
+			{
+				indrelid = Int64GetDatum(beentry->st_progress_param[PROGRESS_VACUUM_INDRELID]);
+
+				if (indrelid > 0)
+					PG_RETURN_OID(indrelid);
+				else
+					PG_RETURN_NULL();
+			}
+	}
+
+	/*
+	 * No matching PID performing a VACUUM operation, return NULL.
+	 */
+	PG_RETURN_NULL();
+}
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 20f5aa56ea..dd5a32172a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11827,5 +11827,9 @@
   proname => 'brin_minmax_multi_summary_send', provolatile => 's',
   prorettype => 'bytea', proargtypes => 'pg_brin_minmax_multi_summary',
   prosrc => 'brin_minmax_multi_summary_send' },
+{ oid => '4642',
+  descr => 'get the relid of the index being vacuumed by a backend',
+  proname => 'pg_stat_get_vacuum_index', provolatile => 'v', prorettype => 'int8',
+  proargtypes => 'int4', prosrc => 'pg_stat_get_vacuum_index' },
 
 ]
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index 0e97c6d4ef..bc2b7d1e95 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -27,6 +27,7 @@
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_INDEX_TOTAL             7
 #define PROGRESS_VACUUM_INDEX_COMPLETED         8
+#define PROGRESS_VACUUM_INDRELID                9
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index 47bf8029b0..74d9dfb4c7 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -17,6 +17,10 @@
 
 /* ----------
  * Command type for progress reporting purposes
+ *
+ * Note: PROGRESS_COMMAND_VACUUM_PARALLEL is not
+ * a command per se, but this type is used to track
+ * parallel vacuum workers progress.
  * ----------
  */
 typedef enum ProgressCommandType
@@ -27,7 +31,8 @@ typedef enum ProgressCommandType
 	PROGRESS_COMMAND_CLUSTER,
 	PROGRESS_COMMAND_CREATE_INDEX,
 	PROGRESS_COMMAND_BASEBACKUP,
-	PROGRESS_COMMAND_COPY
+	PROGRESS_COMMAND_COPY,
+	PROGRESS_COMMAND_VACUUM_PARALLEL
 } ProgressCommandType;
 
 #define PGSTAT_NUM_PROGRESS_PARAM	20
-- 
2.32.1 (Apple Git-133)

v14-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v14-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From d0dba3c28dbf8915db4df731edb445e5064f97bb Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Fri, 4 Nov 2022 07:48:42 -0500
Subject: [PATCH v14 1/2] Add 2 new columns to pg_stat_progress_vacuum. The
 columns are indexes_total as the total indexes to be vacuumed or cleaned and
 indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c              |  6 +++
 doc/src/sgml/monitoring.sgml          | 21 +++++++++
 src/backend/access/brin/brin.c        |  9 ++--
 src/backend/access/gin/ginvacuum.c    |  9 ++++
 src/backend/access/gist/gistvacuum.c  |  5 +++
 src/backend/access/hash/hash.c        |  8 +++-
 src/backend/access/hash/hashpage.c    |  4 +-
 src/backend/access/heap/vacuumlazy.c  | 40 ++++++++++++++++-
 src/backend/access/nbtree/nbtree.c    |  3 ++
 src/backend/access/spgist/spgvacuum.c |  3 ++
 src/backend/access/transam/parallel.c |  5 +++
 src/backend/catalog/index.c           |  1 +
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 63 ++++++++++++++++++++++++++-
 src/include/access/genam.h            |  3 ++
 src/include/access/hash.h             |  3 +-
 src/include/access/parallel.h         |  2 +
 src/include/commands/progress.h       |  2 +
 src/include/commands/vacuum.h         |  2 +
 src/test/regress/expected/rules.out   |  4 +-
 20 files changed, 184 insertions(+), 12 deletions(-)
  10.5% doc/src/sgml/
   5.8% src/backend/access/brin/
   4.4% src/backend/access/gin/
   6.2% src/backend/access/hash/
  17.7% src/backend/access/heap/
   7.5% src/backend/access/
  32.8% src/backend/commands/
   6.1% src/include/access/
   5.6% src/

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 91fae5b0c0..19ae819cfe 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -136,6 +136,9 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			GenericXLogAbort(gxlogState);
 		}
 		UnlockReleaseBuffer(buffer);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	/*
@@ -209,6 +212,9 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		}
 
 		UnlockReleaseBuffer(buffer);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	IndexFreeSpaceMapVacuum(info->index);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5d622d514..c61443f3d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6455,6 +6455,27 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum or
+       vacuum failsafe is triggered. See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 20b7d65b94..bd96ecf41e 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -78,7 +78,7 @@ static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRang
 static void form_and_insert_tuple(BrinBuildState *state);
 static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
 						 BrinTuple *b);
-static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
+static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, IndexVacuumInfo *info);
 static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
 								BrinMemTuple *dtup, Datum *values, bool *nulls);
 static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
@@ -952,7 +952,7 @@ brinvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 	heapRel = table_open(IndexGetRelation(RelationGetRelid(info->index), false),
 						 AccessShareLock);
 
-	brin_vacuum_scan(info->index, info->strategy);
+	brin_vacuum_scan(info->index, info->strategy, info);
 
 	brinsummarize(info->index, heapRel, BRIN_ALL_BLOCKRANGES, false,
 				  &stats->num_index_tuples, &stats->num_index_tuples);
@@ -1659,7 +1659,7 @@ union_tuples(BrinDesc *bdesc, BrinMemTuple *a, BrinTuple *b)
  * and such.
  */
 static void
-brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
+brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy, IndexVacuumInfo *info)
 {
 	BlockNumber nblocks;
 	BlockNumber blkno;
@@ -1681,6 +1681,9 @@ brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
 		brin_page_cleanup(idxrel, buf);
 
 		ReleaseBuffer(buf);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	/*
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..3d5e4600dc 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -633,6 +633,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 		UnlockReleaseBuffer(buffer);
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	/* right now we found leftmost page in entry's BTree */
@@ -677,6 +680,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_EXCLUSIVE);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	MemoryContextDelete(gvs.tmpCxt);
@@ -775,6 +781,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		}
 
 		UnlockReleaseBuffer(buffer);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
 	}
 
 	/* Update the metapage with accurate page and entry counts */
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0aa6e58a62..da3a0540d0 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -223,7 +223,12 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+
+			if (info->report_parallel_progress)
+				info->parallel_progress_callback(info->parallel_progress_arg);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index c361509d68..14790adf61 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -550,7 +550,7 @@ loop_top:
 						  cachedmetap->hashm_highmask,
 						  cachedmetap->hashm_lowmask, &tuples_removed,
 						  &num_index_tuples, split_cleanup,
-						  callback, callback_state);
+						  callback, callback_state, info);
 
 		_hash_dropbuf(rel, bucket_buf);
 
@@ -686,7 +686,8 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 				  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 				  double *tuples_removed, double *num_index_tuples,
 				  bool split_cleanup,
-				  IndexBulkDeleteCallback callback, void *callback_state)
+				  IndexBulkDeleteCallback callback, void *callback_state,
+				  IndexVacuumInfo *info)
 {
 	BlockNumber blkno;
 	Buffer		buf;
@@ -775,6 +776,9 @@ hashbucketcleanup(Relation rel, Bucket cur_bucket, Buffer bucket_buf,
 				if (num_index_tuples)
 					*num_index_tuples += 1;
 			}
+
+			if (info && info->report_parallel_progress)
+				info->parallel_progress_callback(info->parallel_progress_arg);
 		}
 
 		/* retain the pin on primary bucket page till end of bucket scan */
diff --git a/src/backend/access/hash/hashpage.c b/src/backend/access/hash/hashpage.c
index d2edcd4617..e9ce1a1110 100644
--- a/src/backend/access/hash/hashpage.c
+++ b/src/backend/access/hash/hashpage.c
@@ -759,7 +759,7 @@ restart_expand:
 
 		hashbucketcleanup(rel, old_bucket, buf_oblkno, start_oblkno, NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL);
 
 		_hash_dropbuf(rel, buf_oblkno);
 
@@ -1327,7 +1327,7 @@ _hash_splitbucket(Relation rel,
 		hashbucketcleanup(rel, obucket, bucket_obuf,
 						  BufferGetBlockNumber(bucket_obuf), NULL,
 						  maxbucket, highmask, lowmask, NULL, NULL, true,
-						  NULL, NULL);
+						  NULL, NULL, NULL);
 	}
 	else
 	{
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index dfbe37472f..120a2b18d7 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -420,6 +420,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+
+	/* report number of indexes to vacuum */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2341,6 +2345,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/*
+			 * Done vacuuming an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2384,6 +2394,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
 
+	/*
+	 * Reset the indexes completed at this point.
+	 * If we end up in another index vacuum cycle, we will
+	 * start counting from the start.
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
+
 	return allindexes;
 }
 
@@ -2633,10 +2650,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation
+		 *
+		 * Also, report to progress.h that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2684,6 +2708,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/*
+			 * Done cleaning an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2693,6 +2723,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/*
+	 * Reset the indexes completed at this point.
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
+
 }
 
 /*
@@ -2718,6 +2754,7 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
@@ -2766,6 +2803,7 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index b52eca8f38..9904a9eb0c 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,9 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+
+			if (info->report_parallel_progress)
+				info->parallel_progress_callback(info->parallel_progress_arg);
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..b1ab491980 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -843,6 +843,9 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+
+			if (bds->info->report_parallel_progress)
+				bds->info->parallel_progress_callback(bds->info->parallel_progress_arg);
 		}
 	}
 
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index ee0985c7ed..0b4bde8297 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -185,6 +185,8 @@ CreateParallelContext(const char *library_name, const char *function_name,
 	pcxt->library_name = pstrdup(library_name);
 	pcxt->function_name = pstrdup(function_name);
 	pcxt->error_context_stack = error_context_stack;
+	pcxt->parallel_progress_callback = NULL;
+	pcxt->parallel_progress_arg = NULL;
 	shm_toc_initialize_estimator(&pcxt->estimator);
 	dlist_push_head(&pcxt_list, &pcxt->node);
 
@@ -785,6 +787,9 @@ WaitForParallelWorkersToFinish(ParallelContext *pcxt)
 		 */
 		CHECK_FOR_INTERRUPTS();
 
+		if (pcxt->parallel_progress_callback)
+			pcxt->parallel_progress_callback(pcxt->parallel_progress_arg);
+
 		for (i = 0; i < pcxt->nworkers_launched; ++i)
 		{
 			/*
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 61f1d3926a..11b32129a7 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,6 +3348,7 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2d8104b090..c37b20b91b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f26d796e52..85e2de5d54 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 idx_completed_progress;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -273,6 +285,9 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	Assert(pcxt->nworkers > 0);
 	pvs->pcxt = pcxt;
 
+	pvs->pcxt->parallel_progress_callback = parallel_vacuum_progress_report;
+	pvs->pcxt->parallel_progress_arg = pvs;
+
 	/* Estimate size for index vacuum stats -- PARALLEL_VACUUM_KEY_INDEX_STATS */
 	est_indstats_len = mul_size(sizeof(PVIndStats), nindexes);
 	shm_toc_estimate_chunk(&pcxt->estimator, est_indstats_len);
@@ -364,6 +379,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->idx_completed_progress), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +634,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing counter ( index progress counter also ) */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->idx_completed_progress), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -834,10 +851,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.estimated_count = pvs->shared->estimated_count;
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
+	ivinfo.parallel_progress_callback = parallel_vacuum_progress_report;
+	ivinfo.parallel_progress_arg = pvs;
 
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
@@ -846,9 +866,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	switch (indstats->status)
 	{
 		case PARALLEL_INDVAC_STATUS_NEED_BULKDELETE:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
 			istat_res = vac_bulkdel_one_index(&ivinfo, istat, pvs->dead_items);
 			break;
 		case PARALLEL_INDVAC_STATUS_NEED_CLEANUP:
+			pgstat_progress_update_param(PROGRESS_VACUUM_PHASE, PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
 			istat_res = vac_cleanup_one_index(&ivinfo, istat);
 			break;
 		default:
@@ -888,6 +910,11 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Update the number of indexes completed.
+	 */
+	pg_atomic_add_fetch_u32(&(pvs->shared->idx_completed_progress), 1);
 }
 
 /*
@@ -1072,3 +1099,37 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read the number of indexes vacuumed from the shared counter
+ * and report it to progress.h
+ *
+ * This function is only called by the leader process of
+ * a parallel vacuum.
+ *
+ * If report_parallel_progress is set to true,
+ * this function is called from major loops inside the
+ * ambulkdelete and amvacuumcleanup.
+ *
+ * This function is also called inside WaitForParallelWorkersToFinish
+ * when report_parallel_progress is set to true.
+ *
+ * The reason for calling this function in both vacuum AM's
+ * and WaitForParallelWorkersToFinish is to ensure that
+ * parallel vacuum progress is constantly being reported
+ * if the leader process is either waiting for parallel
+ * proceses to finish (WaitForParallelWorkersToFinish) or
+ * the leader process is the last to finish and still
+ * inside the vacuum AM's.
+ */
+void
+parallel_vacuum_progress_report(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *) arg;
+
+	if (IsParallelWorker())
+		return;
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+								 pg_atomic_read_u32(&(pvs->shared->idx_completed_progress)));
+}
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index e1c4fdbd03..6d5fc189f4 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -46,10 +46,13 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
 	BufferAccessStrategy strategy;	/* access strategy for reads */
+	void       (*parallel_progress_callback)(void *arg);
+	void       *parallel_progress_arg;
 } IndexVacuumInfo;
 
 /*
diff --git a/src/include/access/hash.h b/src/include/access/hash.h
index da372841c4..14fb75a4ad 100644
--- a/src/include/access/hash.h
+++ b/src/include/access/hash.h
@@ -480,6 +480,7 @@ extern void hashbucketcleanup(Relation rel, Bucket cur_bucket,
 							  uint32 maxbucket, uint32 highmask, uint32 lowmask,
 							  double *tuples_removed, double *num_index_tuples,
 							  bool split_cleanup,
-							  IndexBulkDeleteCallback callback, void *callback_state);
+							  IndexBulkDeleteCallback callback, void *callback_state,
+							  IndexVacuumInfo *info);
 
 #endif							/* HASH_H */
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 1ec8e33af4..77e9c0a517 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -46,6 +46,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	void       (*parallel_progress_callback)(void *arg);
+	void       *parallel_progress_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..0e97c6d4ef 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 5d816ba7f4..9510d028d3 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -338,4 +338,6 @@ extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
 
+extern void parallel_vacuum_progress_report(void *arg);
+
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 624d0e5aae..e0bf81247f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2018,7 +2018,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

#91Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#88)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Nov 3, 2022 at 1:52 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Attached is v13-0001--Show-progress-for-index-vacuums.patch which addresses
the latest comments.

Thank you for updating the patch!

4/ Went back to calling parallel_vacuum_progress_report inside
WaitForParallelWorkersToFinish to cover the case when a
leader is waiting for parallel workers to finish.

I don't think we need to modify WaitForParallelWorkersToFinish to
cover that case. Instead, I think the leader process can execute a new
function. The function will be like WaitForParallelWorkersToFinish()
but simpler; it just updates the progress information if necessary and
then checks if idx_completed_progress is equal to the number of
indexes to vacuum. If yes, return from the function and call
WaitForParallelWorkersToFinish() to wait for all workers to finish.
Otherwise, it naps by using WaitLatch() and does this loop again.

---
@@ -46,6 +46,8 @@ typedef struct ParallelContext
         ParallelWorkerInfo *worker;
         int                    nknown_attached_workers;
         bool      *known_attached_workers;
+        void       (*parallel_progress_callback)(void *arg);
+        void       *parallel_progress_arg;
 } ParallelContext;

With the above change I suggested, I think we won't need to have a
callback function in ParallelContext. Instead, I think we can have
index-AMs call parallel_vacuum_report() if report_parallel_progress is
true.

Regards,

--
Masahiko Sawada

Amazon Web Services: https://aws.amazon.com

#92Andres Freund
andres@anarazel.de
In reply to: Imseih (AWS), Sami (#90)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

On 2022-11-04 13:27:34 +0000, Imseih (AWS), Sami wrote:

diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..3d5e4600dc 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -633,6 +633,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
UnlockReleaseBuffer(buffer);
buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
RBM_NORMAL, info->strategy);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
}

/* right now we found leftmost page in entry's BTree */

I don't think any of these progress callbacks should be done while pinning a
buffer and ...

@@ -677,6 +680,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
RBM_NORMAL, info->strategy);
LockBuffer(buffer, GIN_EXCLUSIVE);
+
+		if (info->report_parallel_progress)
+			info->parallel_progress_callback(info->parallel_progress_arg);
}

MemoryContextDelete(gvs.tmpCxt);

... definitely not while holding a buffer lock.

I also don't understand why info->parallel_progress_callback exists? It's only
set to parallel_vacuum_progress_report(). Why make this stuff more expensive
than it has to already be?

+parallel_vacuum_progress_report(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *) arg;
+
+	if (IsParallelWorker())
+		return;
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+								 pg_atomic_read_u32(&(pvs->shared->idx_completed_progress)));
+}

So each of the places that call this need to make an additional external
function call for each page, just to find that there's nothing to do or to
make yet another indirect function call. This should probably a static inline
function.

This is called, for every single page, just to read the number of indexes
completed by workers? A number that barely ever changes?

This seems all wrong.

Greetings,

Andres Freund

#93Imseih (AWS), Sami
simseih@amazon.com
In reply to: Andres Freund (#92)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

I don't think any of these progress callbacks should be done while pinning a
buffer and ...

Good point.

I also don't understand why info->parallel_progress_callback exists? It's only
set to parallel_vacuum_progress_report(). Why make this stuff more expensive
than it has to already be?

Agree. Modified the patch to avoid the callback .

So each of the places that call this need to make an additional external
function call for each page, just to find that there's nothing to do or to
make yet another indirect function call. This should probably a static inline
function.

Even better to just remove a function call altogether.

This is called, for every single page, just to read the number of indexes
completed by workers? A number that barely ever changes?

I will take the initial suggestion by Sawada-san to update the progress
every 1GB of blocks scanned.

Also, It sems to me that we don't need to track progress in brin indexes,
Since it is rare, if ever, this type of index will go through very heavy
block scans. In fact, I noticed the vacuum AMs for brin don't invoke the
vacuum_delay_point at all.

The attached patch addresses the feedback.

Regards,

Sami Imseih
Amazon Web Services (AWS)

Attachments:

v15-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v15-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From c59171a3ecee410380e9f62bfc2d678243842a2c Mon Sep 17 00:00:00 2001
From: "Sami Imseih (AWS)"
 <simseih@dev-dsk-simseih-1e-d2cff277.us-east-1.amazon.com>
Date: Fri, 11 Nov 2022 18:40:14 +0000
Subject: [PATCH 1/1] Add 2 new columns to pg_stat_progress_vacuum. The columns
 are indexes_total as the total indexes to be vacuumed or cleaned and
 indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c              |  9 ++++
 doc/src/sgml/monitoring.sgml          | 21 +++++++++
 src/backend/access/gin/ginvacuum.c    | 10 ++++
 src/backend/access/gist/gistvacuum.c  |  7 +++
 src/backend/access/hash/hash.c        |  7 +++
 src/backend/access/heap/vacuumlazy.c  | 40 +++++++++++++++-
 src/backend/access/nbtree/nbtree.c    |  3 ++
 src/backend/access/spgist/spgvacuum.c |  6 +++
 src/backend/catalog/index.c           |  2 +
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 66 +++++++++++++++++++++++++--
 src/include/access/genam.h            |  2 +
 src/include/commands/progress.h       |  2 +
 src/include/commands/vacuum.h         |  6 +++
 src/test/regress/expected/rules.out   |  4 +-
 15 files changed, 182 insertions(+), 6 deletions(-)

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 91fae5b0c0..92a557518f 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -15,12 +15,14 @@
 #include "access/genam.h"
 #include "bloom.h"
 #include "catalog/storage.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
+#include "utils/backend_progress.h"
 
 
 /*
@@ -62,6 +64,10 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 				   *itupEnd;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 pg_atomic_read_u32(&(info->idx_completed_progress)));
+
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
@@ -192,6 +198,9 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		Page		page;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 pg_atomic_read_u32(&(info->idx_completed_progress)));
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5d622d514..c61443f3d5 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6455,6 +6455,27 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum or
+       vacuum failsafe is triggered. See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..623ee78afa 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,12 +17,14 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "storage/predicate.h"
+#include "utils/backend_progress.h"
 #include "utils/memutils.h"
 
 struct GinVacuumState
@@ -665,6 +667,10 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 pg_atomic_read_u32(&(info->idx_completed_progress)));
+
 		for (i = 0; i < nRoot; i++)
 		{
 			ginVacuumPostingTree(&gvs, rootOfPostingTree[i]);
@@ -751,6 +757,10 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 pg_atomic_read_u32(&(info->idx_completed_progress)));
+
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_SHARE);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0aa6e58a62..45902e7aa9 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,11 +17,13 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
+#include "utils/backend_progress.h"
 #include "utils/memutils.h"
 
 /* Working state needed by gistbulkdelete */
@@ -223,7 +225,12 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+											 pg_atomic_read_u32(&(info->idx_completed_progress)));
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index c361509d68..798c93b6cb 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -505,6 +505,13 @@ loop_top:
 
 		blkno = bucket_blkno;
 
+		/*
+		 * For hash indexes, we report parallel vacuum progress
+		 * for every bucket.
+		 */
+		if (info->report_parallel_progress)
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 pg_atomic_read_u32(&(info->idx_completed_progress)));
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
 		 * wait concurrent scans before deleting the dead tuples.
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index dfbe37472f..120a2b18d7 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -420,6 +420,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+
+	/* report number of indexes to vacuum */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -2341,6 +2345,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/*
+			 * Done vacuuming an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2384,6 +2394,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
 
+	/*
+	 * Reset the indexes completed at this point.
+	 * If we end up in another index vacuum cycle, we will
+	 * start counting from the start.
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
+
 	return allindexes;
 }
 
@@ -2633,10 +2650,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation
+		 *
+		 * Also, report to progress.h that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2684,6 +2708,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/*
+			 * Done cleaning an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2693,6 +2723,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/*
+	 * Reset the indexes completed at this point.
+	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
+
 }
 
 /*
@@ -2718,6 +2754,7 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
@@ -2766,6 +2803,7 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index b52eca8f38..374a712952 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,9 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+			if (info->report_parallel_progress && (scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+											 pg_atomic_read_u32(&(info->idx_completed_progress)));
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 0049630532..1f13bc28ce 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,11 +21,13 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
+#include "utils/backend_progress.h"
 #include "utils/snapmgr.h"
 
 
@@ -843,6 +845,10 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+			/* report parallel vacuum progress */
+			if (bds->info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+											 pg_atomic_read_u32(&(bds->info->idx_completed_progress)));
 		}
 	}
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 61f1d3926a..3728750569 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,10 +3348,12 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
 	ivinfo.strategy = NULL;
+	pg_atomic_init_u32(&(ivinfo.idx_completed_progress), 0);
 
 	/*
 	 * Encode TIDs as int8 values for the sort, rather than directly sorting
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2d8104b090..c37b20b91b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f26d796e52..c3579af169 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 idx_completed_progress;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -213,6 +225,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_wait_for_workers_to_finish(ParallelVacuumState *pvs);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -364,6 +377,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->idx_completed_progress), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +632,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing counter ( index progress counter also ) */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->idx_completed_progress), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -688,7 +703,21 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	 */
 	if (nworkers > 0)
 	{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * To wait for parallel workers to finish,
+		 * first call parallel_wait_for_workers_to_finish
+		 * which is responsible for reporting the
+		 * number of indexes completed.
+		 *
+		 * Afterwards, WaitForParallelWorkersToFinish is called
+		 * to do the real work of waiting for parallel workers
+		 * to finish.
+		 *
+		 * Note: Both routines will acquire a WaitLatch in their
+		 * respective loops.
+		 */
+		parallel_wait_for_workers_to_finish(pvs);
+
 		WaitForParallelWorkersToFinish(pvs->pcxt);
 
 		for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)
@@ -838,7 +867,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.estimated_count = pvs->shared->estimated_count;
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
-
+	ivinfo.idx_completed_progress = pvs->shared->idx_completed_progress;
+	/* Only the leader should report parallel vacuum progress */
+	if (!IsParallelWorker())
+		ivinfo.report_parallel_progress = true;
+	else
+		ivinfo.report_parallel_progress = false;
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
@@ -857,6 +891,9 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 				 RelationGetRelationName(indrel));
 	}
 
+	if (ivinfo.report_parallel_progress)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_read_u32(&(ivinfo.idx_completed_progress)));
 	/*
 	 * Copy the index bulk-deletion result returned from ambulkdelete and
 	 * amvacuumcleanup to the DSM segment if it's the first cycle because they
@@ -888,6 +925,9 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/* Update the number of indexes completed. */
+	pg_atomic_add_fetch_u32(&(pvs->shared->idx_completed_progress), 1);
 }
 
 /*
@@ -1072,3 +1112,23 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Check if we are done vacuuming indexes and report
+ * progress.
+ *
+ * We nap using with a WaitLatch to avoid a busy loop.
+ */
+void
+parallel_wait_for_workers_to_finish(ParallelVacuumState *pvs)
+{
+	while (pg_atomic_read_u32(&(pvs->shared->idx_completed_progress)) < pvs->nindexes)
+	{
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_read_u32(&(pvs->shared->idx_completed_progress)));
+
+		(void) WaitLatch(MyLatch, WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, -1,
+						 WAIT_EVENT_PARALLEL_FINISH);
+		ResetLatch(MyLatch);
+	}
+}
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index e1c4fdbd03..bf69c93d31 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -46,10 +46,12 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
 	BufferAccessStrategy strategy;	/* access strategy for reads */
+	pg_atomic_uint32 idx_completed_progress;
 } IndexVacuumInfo;
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..0e97c6d4ef 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 5d816ba7f4..34cf36c425 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -64,6 +64,12 @@
 /* value for checking vacuum flags */
 #define VACUUM_OPTION_MAX_VALID_VALUE		((1 << 3) - 1)
 
+/*
+ * Parallel Index vacuum progress is reported every 1GB of blocks
+ * scanned.
+ */
+#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (((uint64) 1024 * 1024 * 1024) / BLCKSZ))
+
 /* Abstract type for parallel vacuum state */
 typedef struct ParallelVacuumState ParallelVacuumState;
 
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 624d0e5aae..e0bf81247f 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2018,7 +2018,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.37.1

#94Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#93)
Re: Add index scan progress to pg_stat_progress_vacuum

On Sat, Nov 12, 2022 at 4:10 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

I don't think any of these progress callbacks should be done while pinning a
buffer and ...

Good point.

I also don't understand why info->parallel_progress_callback exists? It's only
set to parallel_vacuum_progress_report(). Why make this stuff more expensive
than it has to already be?

Agree. Modified the patch to avoid the callback .

So each of the places that call this need to make an additional external
function call for each page, just to find that there's nothing to do or to
make yet another indirect function call. This should probably a static inline
function.

Even better to just remove a function call altogether.

This is called, for every single page, just to read the number of indexes
completed by workers? A number that barely ever changes?

I will take the initial suggestion by Sawada-san to update the progress
every 1GB of blocks scanned.

Also, It sems to me that we don't need to track progress in brin indexes,
Since it is rare, if ever, this type of index will go through very heavy
block scans. In fact, I noticed the vacuum AMs for brin don't invoke the
vacuum_delay_point at all.

The attached patch addresses the feedback.

Thank you for updating the patch! Here are review comments on v15 patch:

+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum or
+       vacuum failsafe is triggered.

I think that indexes_total should be 0 also when INDEX_CLEANUP is off.

---
+        /*
+         * Reset the indexes completed at this point.
+         * If we end up in another index vacuum cycle, we will
+         * start counting from the start.
+         */
+        pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);

I think we don't need to reset it at the end of index vacuuming. There
is a small window before switching to the next phase. If we reset this
value while showing "index vacuuming" phase, the user might get
confused. Instead, we can reset it at the beginning of the index
vacuuming.

---
+void
+parallel_wait_for_workers_to_finish(ParallelVacuumState *pvs)
+{
+        while (pg_atomic_read_u32(&(pvs->shared->idx_completed_progress))
< pvs->nindexes)
+        {
+                pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+
   pg_atomic_read_u32(&(pvs->shared->idx_completed_progress)));
+
+                (void) WaitLatch(MyLatch, WL_LATCH_SET |
WL_EXIT_ON_PM_DEATH, -1,
+                                                 WAIT_EVENT_PARALLEL_FINISH);
+                ResetLatch(MyLatch);
+        }
+}

We should add CHECK_FOR_INTERRUPTS() at the beginning of the loop to
make the wait interruptible.

I think it would be better to update the counter only when the value
has been increased.

I think we should set a timeout, say 1 sec, to WaitLatch so that it
can periodically check the progress.

Probably it's better to have a new wait event for this wait in order
to distinguish wait for workers to complete index vacuum from the wait
for workers to exit.

---
@@ -838,7 +867,12 @@
parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
indrel,
         ivinfo.estimated_count = pvs->shared->estimated_count;
         ivinfo.num_heap_tuples = pvs->shared->reltuples;
         ivinfo.strategy = pvs->bstrategy;
-
+        ivinfo.idx_completed_progress = pvs->shared->idx_completed_progress;

and

@@ -998,6 +998,9 @@ btvacuumscan(IndexVacuumInfo *info,
IndexBulkDeleteResult *stats,
if (info->report_progress)

pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,

                   scanblkno);
+                        if (info->report_parallel_progress &&
(scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+
pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+

pg_atomic_read_u32(&(info->idx_completed_progress)));
}

I think this doesn't work, since ivinfo.idx_completed is in the
backend-local memory. Instead, I think we can have a function in
vacuumparallel.c that updates the progress. Then we can have index AM
call this function.

---
+        if (!IsParallelWorker())
+                ivinfo.report_parallel_progress = true;
+        else
+                ivinfo.report_parallel_progress = false;

We can do like:

ivinfo.report_parallel_progress = !IsParallelWorker();

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#95Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#94)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

I think that indexes_total should be 0 also when INDEX_CLEANUP is off.

Patch updated for handling of INDEX_CLEANUP = off, with an update to
the documentation as well.

I think we don't need to reset it at the end of index vacuuming. There
is a small window before switching to the next phase. If we reset this
value while showing "index vacuuming" phase, the user might get
confused. Instead, we can reset it at the beginning of the index
vacuuming.

No, I think the way it's currently done is correct. We want to reset the number
of indexes completed before we increase the num_index_scans ( index vacuum cycle ).
This ensures that when we enter a new index cycle, the number of indexes completed
are already reset. The 2 fields that matter here is how many indexes are vacuumed in the
currently cycle and this is called out in the documentation as such.

We should add CHECK_FOR_INTERRUPTS() at the beginning of the loop to
make the wait interruptible.

Done.

I think it would be better to update the counter only when the value
has been increased.

Done. Did so by checking the progress value from the beentry directly
in the function. We can do a more generalized

I think we should set a timeout, say 1 sec, to WaitLatch so that it
can periodically check the progress.

Done.

Probably it's better to have a new wait event for this wait in order
to distinguish wait for workers to complete index vacuum from the wait
for workers to exit.

I was trying to avoid introducing a new wait event, but thinking about it,
I agree with your suggestion.

I created a new ParallelVacuumFinish wait event and documentation
for the wait event.

I think this doesn't work, since ivinfo.idx_completed is in the
backend-local memory. Instead, I think we can have a function in
vacuumparallel.c that updates the progress. Then we can have index AM
call this function.

Yeah, you're absolutely correct.

To make this work correctly, I did something similar to VacuumActiveNWorkers
and introduced an extern variable called ParallelVacuumProgress.
This variable points to pvs->shared->idx_completed_progress.

The index AMs then call parallel_vacuum_update_progress which
Is responsible for updating the progress with the current value
of ParallelVacuumProgress.

ParallelVacuumProgress is also set to NULL at the end of every index cycle.

ivinfo.report_parallel_progress = !IsParallelWorker();

Done

Regards,

Sami Imseih
Amazon Web Services (AWS)

Attachments:

v16-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v16-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From db03efc9f21fa8cd9597cff146bbcce0becdf86a Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Mon, 28 Nov 2022 17:47:04 -0600
Subject: [PATCH 1/1] Add 2 new columns to pg_stat_progress_vacuum. The columns
 are indexes_total as the total indexes to be vacuumed or cleaned and
 indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c                |   7 ++
 doc/src/sgml/monitoring.sgml            |  26 ++++++
 src/backend/access/gin/ginvacuum.c      |   8 ++
 src/backend/access/gist/gistvacuum.c    |   6 ++
 src/backend/access/hash/hash.c          |   6 ++
 src/backend/access/heap/vacuumlazy.c    |  33 ++++++-
 src/backend/access/nbtree/nbtree.c      |   2 +
 src/backend/access/spgist/spgvacuum.c   |   5 ++
 src/backend/catalog/index.c             |   1 +
 src/backend/catalog/system_views.sql    |   3 +-
 src/backend/commands/vacuum.c           |   2 +
 src/backend/commands/vacuumparallel.c   | 110 +++++++++++++++++++++++-
 src/backend/utils/activity/wait_event.c |   3 +
 src/include/access/genam.h              |   1 +
 src/include/commands/progress.h         |   2 +
 src/include/commands/vacuum.h           |   9 ++
 src/include/utils/wait_event.h          |   3 +-
 src/test/regress/expected/rules.out     |   4 +-
 18 files changed, 223 insertions(+), 8 deletions(-)

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 91fae5b..023fc7e 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -15,12 +15,14 @@
 #include "access/genam.h"
 #include "bloom.h"
 #include "catalog/storage.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
+#include "utils/backend_progress.h"
 
 
 /*
@@ -62,6 +64,9 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 				   *itupEnd;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress();
+
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
@@ -192,6 +197,8 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		Page		page;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress();
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5579b8b..477dfe2 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1759,6 +1759,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish computing.</entry>
+     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -6815,6 +6819,28 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum, <literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6..e681164 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -17,12 +17,14 @@
 #include "access/gin_private.h"
 #include "access/ginxlog.h"
 #include "access/xloginsert.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "postmaster/autovacuum.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
 #include "storage/predicate.h"
+#include "utils/backend_progress.h"
 #include "utils/memutils.h"
 
 struct GinVacuumState
@@ -665,6 +667,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress();
+
 		for (i = 0; i < nRoot; i++)
 		{
 			ginVacuumPostingTree(&gvs, rootOfPostingTree[i]);
@@ -751,6 +756,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress();
+
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_SHARE);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0aa6e58..cb3deb1 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -17,11 +17,13 @@
 #include "access/genam.h"
 #include "access/gist_private.h"
 #include "access/transam.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "lib/integerset.h"
 #include "miscadmin.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
+#include "utils/backend_progress.h"
 #include "utils/memutils.h"
 
 /* Working state needed by gistbulkdelete */
@@ -223,7 +225,11 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress();
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 77fd147..d807c9f 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -505,6 +505,12 @@ loop_top:
 
 		blkno = bucket_blkno;
 
+		/*
+		 * For hash indexes, we report parallel vacuum progress
+		 * for every bucket.
+		 */
+		if (info->report_parallel_progress)
+			parallel_vacuum_update_progress();
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
 		 * wait concurrent scans before deleting the dead tuples.
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index d59711b..f11a1b1 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -415,6 +415,7 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 	vacrel->rel = rel;
 	vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
 					 &vacrel->indrels);
+
 	if (instrument && vacrel->nindexes > 0)
 	{
 		/* Copy index names used by instrumentation (not error reporting) */
@@ -459,6 +460,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 		Assert(params->index_cleanup == VACOPTVALUE_AUTO);
 	}
 
+	/* report number of indexes to vacuum, if we are told to cleanup indexes */
+	if (vacrel->do_index_cleanup)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	vacrel->bstrategy = bstrategy;
 	vacrel->relfrozenxid = rel->rd_rel->relfrozenxid;
 	vacrel->relminmxid = rel->rd_rel->relminmxid;
@@ -2301,6 +2306,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/*
+			 * Done vacuuming an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2335,11 +2346,14 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Reset and report the number of indexes scanned.
+	 * Also, increase and report the number of index
+	 * scans.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 	vacrel->num_index_scans++;
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
@@ -2593,10 +2607,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation
+		 *
+		 * Also, report to progress.h that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2644,6 +2665,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/*
+			 * Done cleaning an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2678,6 +2705,7 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
@@ -2726,6 +2754,7 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index b52eca8..39349bb 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,8 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+			if (info->report_parallel_progress && (scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress();
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index ad90b21..0589683 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -21,11 +21,13 @@
 #include "access/transam.h"
 #include "access/xloginsert.h"
 #include "catalog/storage_xlog.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
+#include "utils/backend_progress.h"
 #include "utils/snapmgr.h"
 
 
@@ -843,6 +845,9 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+			/* report parallel vacuum progress */
+			if (bds->info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress();
 		}
 	}
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 61f1d39..11b3212 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,6 +3348,7 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2d8104b..c37b20b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index a6d5ed1..2da956f 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -75,6 +75,8 @@ int			vacuum_multixact_failsafe_age;
 static MemoryContext vac_context = NULL;
 static BufferAccessStrategy vac_strategy;
 
+/* Shared parameter to track vacuum parallel progress */
+pg_atomic_uint32 *ParallelVacuumProgress = NULL;
 
 /*
  * Variables for cost-based parallel vacuum.  See comments atop
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f26d796..b5b80c9 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -50,6 +51,8 @@
 #define PARALLEL_VACUUM_KEY_WAL_USAGE		5
 #define PARALLEL_VACUUM_KEY_INDEX_STATS		6
 
+#define PARALLEL_VACUUM_PROGRESS_TIMEOUT	1000
+
 /*
  * Shared information among parallel workers.  So this is allocated in the DSM
  * segment.
@@ -103,6 +106,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 idx_completed_progress;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -213,6 +227,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_wait_for_workers_to_finish(ParallelVacuumState *pvs);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -364,6 +379,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->idx_completed_progress), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +634,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing counter ( index progress counter also ) */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->idx_completed_progress), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -645,6 +662,14 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 
 		LaunchParallelWorkers(pvs->pcxt);
 
+		/*
+		 * Set the shared parallel vacuum progress. This will be used
+		 * to periodically update progress.h with completed indexes
+		 * in a parallel vacuum. See comments in parallel_vacuum_update_progress
+		 * for more details.
+		 */
+		ParallelVacuumProgress = &(pvs->shared->idx_completed_progress);
+
 		if (pvs->pcxt->nworkers_launched > 0)
 		{
 			/*
@@ -688,7 +713,21 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	 */
 	if (nworkers > 0)
 	{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * To wait for parallel workers to finish,
+		 * first call parallel_wait_for_workers_to_finish
+		 * which is responsible for reporting the
+		 * number of indexes completed.
+		 *
+		 * Afterwards, WaitForParallelWorkersToFinish is called
+		 * to do the real work of waiting for parallel workers
+		 * to finish.
+		 *
+		 * Note: Both routines will acquire a WaitLatch in their
+		 * respective loops.
+		 */
+		parallel_wait_for_workers_to_finish(pvs);
+
 		WaitForParallelWorkersToFinish(pvs->pcxt);
 
 		for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)
@@ -710,6 +749,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 		indstats->status = PARALLEL_INDVAC_STATUS_INITIAL;
 	}
 
+	/* Reset parallel progress */
+	ParallelVacuumProgress = NULL;
+
 	/*
 	 * Carry the shared balance value to heap scan and disable shared costing
 	 */
@@ -838,7 +880,8 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.estimated_count = pvs->shared->estimated_count;
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
-
+	/* Only the leader should report parallel vacuum progress */
+	ivinfo.report_parallel_progress = !IsParallelWorker();
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
@@ -857,6 +900,9 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 				 RelationGetRelationName(indrel));
 	}
 
+	if (ivinfo.report_parallel_progress)
+		parallel_vacuum_update_progress();
+
 	/*
 	 * Copy the index bulk-deletion result returned from ambulkdelete and
 	 * amvacuumcleanup to the DSM segment if it's the first cycle because they
@@ -888,6 +934,9 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/* Update the number of indexes completed. */
+	pg_atomic_add_fetch_u32(&(pvs->shared->idx_completed_progress), 1);
 }
 
 /*
@@ -1072,3 +1121,58 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read the shared ParallelVacuumProgress and update progress.h
+ * with indexes vacuumed so far. This function is called periodically
+ * by index AMs as well as parallel_vacuum_process_one_index.
+ *
+ * To avoid unnecessarily updating progress, we check the progress
+ * values from the backend entry and only update if the value
+ * of completed indexes increases.
+ *
+ * Note: This function should be used by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(void)
+{
+	volatile PgBackendStatus *beentry = MyBEEntry;
+
+	Assert(!IsParallelWorker);
+
+	if (beentry && ParallelVacuumProgress)
+	{
+		int parallel_vacuum_current_value = beentry->st_progress_param[PROGRESS_VACUUM_INDEX_COMPLETED];
+		int parallel_vacuum_new_value = pg_atomic_read_u32(ParallelVacuumProgress);
+
+		if (parallel_vacuum_new_value > parallel_vacuum_current_value)
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, parallel_vacuum_new_value);
+	}
+}
+
+/*
+ * Check if we are done vacuuming indexes and report
+ * progress.
+ *
+ * We nap using with a WaitLatch to avoid a busy loop.
+ *
+ * Note: This function should be used by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_wait_for_workers_to_finish(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker);
+
+	while (pg_atomic_read_u32(ParallelVacuumProgress) < pvs->nindexes)
+	{
+		CHECK_FOR_INTERRUPTS();
+
+		parallel_vacuum_update_progress();
+
+		(void) WaitLatch(MyLatch, WL_TIMEOUT | WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, PARALLEL_VACUUM_PROGRESS_TIMEOUT,
+						 WAIT_EVENT_PARALLEL_VACUUM_FINISH);
+		ResetLatch(MyLatch);
+	}
+}
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index b2abd75..eeba2be 100644
--- a/src/backend/utils/activity/wait_event.c
+++ b/src/backend/utils/activity/wait_event.c
@@ -460,6 +460,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_XACT_GROUP_UPDATE:
 			event_name = "XactGroupUpdate";
 			break;
+		case WAIT_EVENT_PARALLEL_VACUUM_FINISH:
+			event_name = "ParallelVacuumFinish";
+			break;
 			/* no default case, so that compiler will warn */
 	}
 
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index e1c4fdb..7474734 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -46,6 +46,7 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938c..0e97c6d 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 4e4bc26..5a6b454 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -64,6 +64,12 @@
 /* value for checking vacuum flags */
 #define VACUUM_OPTION_MAX_VALID_VALUE		((1 << 3) - 1)
 
+/*
+ * Parallel Index vacuum progress is reported every 1GB of blocks
+ * scanned.
+ */
+#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (((uint64) 1024 * 1024 * 1024) / BLCKSZ))
+
 /* Abstract type for parallel vacuum state */
 typedef struct ParallelVacuumState ParallelVacuumState;
 
@@ -259,6 +265,8 @@ extern PGDLLIMPORT int vacuum_multixact_freeze_table_age;
 extern PGDLLIMPORT int vacuum_failsafe_age;
 extern PGDLLIMPORT int vacuum_multixact_failsafe_age;
 
+extern PGDLLIMPORT pg_atomic_uint32 *ParallelVacuumProgress;
+
 /* Variables for cost-based parallel vacuum */
 extern PGDLLIMPORT pg_atomic_uint32 *VacuumSharedCostBalance;
 extern PGDLLIMPORT pg_atomic_uint32 *VacuumActiveNWorkers;
@@ -333,5 +341,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(void);
 
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index 0b2100b..95e9fef 100644
--- a/src/include/utils/wait_event.h
+++ b/src/include/utils/wait_event.h
@@ -128,7 +128,8 @@ typedef enum
 	WAIT_EVENT_SYNC_REP,
 	WAIT_EVENT_WAL_RECEIVER_EXIT,
 	WAIT_EVENT_WAL_RECEIVER_WAIT_START,
-	WAIT_EVENT_XACT_GROUP_UPDATE
+	WAIT_EVENT_XACT_GROUP_UPDATE,
+	WAIT_EVENT_PARALLEL_VACUUM_FINISH
 } WaitEventIPC;
 
 /* ----------
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 37c1c86..896043e 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2021,7 +2021,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

#96Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#95)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

Thank you for updating the patch!

On Tue, Nov 29, 2022 at 8:57 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

I think that indexes_total should be 0 also when INDEX_CLEANUP is off.

Patch updated for handling of INDEX_CLEANUP = off, with an update to
the documentation as well.

I think we don't need to reset it at the end of index vacuuming. There
is a small window before switching to the next phase. If we reset this
value while showing "index vacuuming" phase, the user might get
confused. Instead, we can reset it at the beginning of the index
vacuuming.

No, I think the way it's currently done is correct. We want to reset the number
of indexes completed before we increase the num_index_scans ( index vacuum cycle ).
This ensures that when we enter a new index cycle, the number of indexes completed
are already reset. The 2 fields that matter here is how many indexes are vacuumed in the
currently cycle and this is called out in the documentation as such.

Agreed.

Here are comments on v16 patch.

--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -15,12 +15,14 @@
 #include "access/genam.h"
 #include "bloom.h"
 #include "catalog/storage.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "miscadmin.h"
 #include "postmaster/autovacuum.h"
 #include "storage/bufmgr.h"
 #include "storage/indexfsm.h"
 #include "storage/lmgr.h"
+#include "utils/backend_progress.h"

I think we don't need to include them here. Probably the same is true
for other index AMs.

---
                vacuum_delay_point();
+               if (info->report_parallel_progress && (blkno %
REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+                       parallel_vacuum_update_progress();
+

buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,

There is an extra new line.

---
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish computing.</entry>
+     </row>

How about "Waiting for parallel vacuum workers to finish index vacuum"?

---
vacrel->rel = rel;
vac_open_indexes(vacrel->rel, RowExclusiveLock, &vacrel->nindexes,
&vacrel->indrels);
+
if (instrument && vacrel->nindexes > 0)
{
/* Copy index names used by instrumentation (not error reporting) */

This added line is not necessary.

---
         /* Counter for vacuuming and cleanup */
         pg_atomic_uint32 idx;
+
+        /*
+         * Counter for vacuuming and cleanup progress reporting.
+         * This value is used to report index vacuum/cleanup progress
+         * in parallel_vacuum_progress_report. We keep this
+         * counter to avoid having to loop through
+         * ParallelVacuumState->indstats to determine the number
+         * of indexes completed.
+         */
+        pg_atomic_uint32 idx_completed_progress;

I think the name of idx_completed_progress is very confusing. Since
the idx of PVShared refers to the current index in the pvs->indstats[]
whereas idx_completed_progress is the number of vacuumed indexes. How
about "nindexes_completed"?

---
+                /*
+                 * Set the shared parallel vacuum progress. This will be used
+                 * to periodically update progress.h with completed indexes
+                 * in a parallel vacuum. See comments in
parallel_vacuum_update_progress
+                 * for more details.
+                 */
+                ParallelVacuumProgress =
&(pvs->shared->idx_completed_progress);
+

Since we pass pvs to parallel_wait_for_workers_to_finish(), we don't
need to have ParallelVacuumProgress. I see
parallel_vacuum_update_progress() uses this value but I think it's
better to pass ParallelVacuumState to via IndexVacuumInfo.

---
+                /*
+                 * To wait for parallel workers to finish,
+                 * first call parallel_wait_for_workers_to_finish
+                 * which is responsible for reporting the
+                 * number of indexes completed.
+                 *
+                 * Afterwards, WaitForParallelWorkersToFinish is called
+                 * to do the real work of waiting for parallel workers
+                 * to finish.
+                 *
+                 * Note: Both routines will acquire a WaitLatch in their
+                 * respective loops.
+                 */

How about something like:

Wait for all indexes to be vacuumed while updating the parallel vacuum
index progress. And then wait for all workers to finish.

---
RelationGetRelationName(indrel));
}

+        if (ivinfo.report_parallel_progress)
+                parallel_vacuum_update_progress();
+

I think it's better to update the progress info after updating
pvs->shared->idx_completed_progress.

---
+/*
+ * Check if we are done vacuuming indexes and report
+ * progress.

How about "Waiting for all indexes to be vacuumed while updating the
parallel index vacuum progress"?

+ *
+ * We nap using with a WaitLatch to avoid a busy loop.
+ *
+ * Note: This function should be used by the leader process only,
+ * and it's up to the caller to ensure this.
+ */

I think these comments are not necessary.

+void
+parallel_wait_for_workers_to_finish(ParallelVacuumState *pvs)

How about "parallel_vacuum_wait_to_finish"?

---
+/*
+ * Read the shared ParallelVacuumProgress and update progress.h
+ * with indexes vacuumed so far. This function is called periodically
+ * by index AMs as well as parallel_vacuum_process_one_index.
+ *
+ * To avoid unnecessarily updating progress, we check the progress
+ * values from the backend entry and only update if the value
+ * of completed indexes increases.
+ *
+ * Note: This function should be used by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(void)
+{
+        volatile PgBackendStatus *beentry = MyBEEntry;
+
+        Assert(!IsParallelWorker);
+
+        if (beentry && ParallelVacuumProgress)
+        {
+                int parallel_vacuum_current_value =
beentry->st_progress_param[PROGRESS_VACUUM_INDEX_COMPLETED];
+                int parallel_vacuum_new_value =
pg_atomic_read_u32(ParallelVacuumProgress);
+
+                if (parallel_vacuum_new_value > parallel_vacuum_current_value)
+
pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
parallel_vacuum_new_value);
+        }
+}

parallel_vacuum_update_progress() is typically called every 1GB so I
think we don't need to worry about unnecessary update. Also, I think
this code doesn't work when pgstat_track_activities is false. Instead,
I think that in parallel_wait_for_workers_to_finish(), we can check
the value of pvs->nindexes_completed and update the progress if there
is an update or it's first time.

---
+                (void) WaitLatch(MyLatch, WL_TIMEOUT | WL_LATCH_SET |
WL_EXIT_ON_PM_DEATH, PARALLEL_VACUUM_PROGRESS_TIMEOUT,
+
WAIT_EVENT_PARALLEL_VACUUM_FINISH);
+                ResetLatch(MyLatch);

I think we don't necessarily need to use
PARALLEL_VACUUM_PROGRESS_TIMEOUT here. Probably we can use 1000L
instead. If we want to use PARALLEL_VACUUM_PROGRESS_TIMEOUT, we need
comments for that:

+#define PARALLEL_VACUUM_PROGRESS_TIMEOUT 1000

---
-        WAIT_EVENT_XACT_GROUP_UPDATE
+        WAIT_EVENT_XACT_GROUP_UPDATE,
+        WAIT_EVENT_PARALLEL_VACUUM_FINISH
 } WaitEventIPC;

Enums of WaitEventIPC should be defined in alphabetical order.

---
cfbot fails.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#97Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#96)
Re: Add index scan progress to pg_stat_progress_vacuum

Thanks for the feedback. I agree with the feedback, except
for

need to have ParallelVacuumProgress. I see
parallel_vacuum_update_progress() uses this value but I think it's
better to pass ParallelVacuumState to via IndexVacuumInfo.

I was trying to avoid passing a pointer to
ParallelVacuumState in IndexVacuuminfo.

ParallelVacuumProgress is implemented in the same
way as VacuumSharedCostBalance and
VacuumActiveNWorkers. See vacuum.h

These values are reset at the start of a parallel vacuum cycle
and reset at the end of an index vacuum cycle.

This seems like a better approach and less invasive.
What would be a reason not to go with this approach?

parallel_vacuum_update_progress() is typically called every 1GB so I
think we don't need to worry about unnecessary update. Also, I think
this code doesn't work when pgstat_track_activities is false. Instead,
I think that in parallel_wait_for_workers_to_finish(), we can check
the value of pvs->nindexes_completed and update the progress if there
is an update or it's first time.

I agree that we don’t need to worry about unnecessary updates
in parallel_vacuum_update_progress since we are calling
every 1GB. I also don't think we should do anything additional
in parallel_wait_for_workers_to_finish since here we are only
updating every 1 second.

Thanks,

Sami Imseih
Amazon Web Services

#98Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#97)
Re: Add index scan progress to pg_stat_progress_vacuum

On Tue, Dec 13, 2022 at 1:40 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Thanks for the feedback. I agree with the feedback, except
for

need to have ParallelVacuumProgress. I see
parallel_vacuum_update_progress() uses this value but I think it's
better to pass ParallelVacuumState to via IndexVacuumInfo.

I was trying to avoid passing a pointer to
ParallelVacuumState in IndexVacuuminfo.

ParallelVacuumProgress is implemented in the same
way as VacuumSharedCostBalance and
VacuumActiveNWorkers. See vacuum.h

These values are reset at the start of a parallel vacuum cycle
and reset at the end of an index vacuum cycle.

This seems like a better approach and less invasive.
What would be a reason not to go with this approach?

First of all, I don't think we need to declare ParallelVacuumProgress
in vacuum.c since it's set and used only in vacuumparallel.c. But I
don't even think it's a good idea to declare it in vacuumparallel.c as
a static variable. The primary reason is that it adds things we need
to care about. For example, what if we raise an error during index
vacuum? The transaction aborts but ParallelVacuumProgress still refers
to something old. Suppose further that the next parallel vacuum
doesn't launch any workers, the leader process would still end up
accessing the old value pointed by ParallelVacuumProgress, which
causes a SEGV. So we need to reset it anyway at the beginning of the
parallel vacuum. It's easy to fix at this time but once the parallel
vacuum code gets more complex, it could forget to care about it.

IMO VacuumSharedCostBalance and VacuumActiveNWorkers have a different
story. They are set in vacuumparallel.c and are used in vacuum.c for
vacuum delay. If they weren't global variables, we would need to pass
them to every function that could eventually call the vacuum delay
function. So it makes sense to me to have them as global variables.On
the other hand, for ParallelVacuumProgress, it's a common pattern that
ambulkdelete(), amvacuumcleanup() or a common index scan routine like
btvacuumscan() checks the progress. I don't think index AM needs to
pass the value down to many of its functions. So it makes sense to me
to pass it via IndexVacuumInfo.

Having said that, I'd like to hear opinions also from other hackers, I
might be wrong and it's more invasive as you pointed out.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#99Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#98)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

First of all, I don't think we need to declare ParallelVacuumProgress
in vacuum.c since it's set and used only in vacuumparallel.c. But I
don't even think it's a good idea to declare it in vacuumparallel.c as
a static variable. The primary reason is that it adds things we need
to care about. For example, what if we raise an error during index
vacuum? The transaction aborts but ParallelVacuumProgress still refers
to something old. Suppose further that the next parallel vacuum
doesn't launch any workers, the leader process would still end up
accessing the old value pointed by ParallelVacuumProgress, which
causes a SEGV. So we need to reset it anyway at the beginning of the
parallel vacuum. It's easy to fix at this time but once the parallel
vacuum code gets more complex, it could forget to care about it.

IMO VacuumSharedCostBalance and VacuumActiveNWorkers have a different
story. They are set in vacuumparallel.c and are used in vacuum.c for
vacuum delay. If they weren't global variables, we would need to pass
them to every function that could eventually call the vacuum delay
function. So it makes sense to me to have them as global variables.On
the other hand, for ParallelVacuumProgress, it's a common pattern that
ambulkdelete(), amvacuumcleanup() or a common index scan routine like
btvacuumscan() checks the progress. I don't think index AM needs to
pass the value down to many of its functions. So it makes sense to me
to pass it via IndexVacuumInfo.

Thanks for the detailed explanation and especially clearing up
my understanding of VacuumSharedCostBalance and VacuumActiveNWorker.

I do now think that passing ParallelVacuumState in IndexVacuumInfo is
a more optimal choice.

Attached version addresses the above and the previous comments.

Thanks

Sami Imseih
Amazon Web Services (AWS)

Attachments:

v17-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v17-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From 1fce3b7657c589cbf0fa049fd41470c7d713c492 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Tue, 13 Dec 2022 23:02:24 -0600
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c                |  4 ++
 doc/src/sgml/monitoring.sgml            | 26 +++++++++
 src/backend/access/gin/ginvacuum.c      |  6 ++
 src/backend/access/gist/gistvacuum.c    |  4 ++
 src/backend/access/hash/hash.c          |  6 ++
 src/backend/access/heap/vacuumlazy.c    | 34 +++++++++++-
 src/backend/access/nbtree/nbtree.c      |  2 +
 src/backend/access/spgist/spgvacuum.c   |  3 +
 src/backend/catalog/index.c             |  2 +
 src/backend/catalog/system_views.sql    |  3 +-
 src/backend/commands/analyze.c          |  1 +
 src/backend/commands/vacuumparallel.c   | 74 ++++++++++++++++++++++++-
 src/backend/utils/activity/wait_event.c |  3 +
 src/include/access/genam.h              |  5 +-
 src/include/commands/progress.h         |  2 +
 src/include/commands/vacuum.h           |  7 +++
 src/include/utils/wait_event.h          |  1 +
 src/test/regress/expected/rules.out     |  4 +-
 18 files changed, 180 insertions(+), 7 deletions(-)

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 91fae5b0c0..005858ce9f 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -62,6 +62,8 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 				   *itupEnd;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
@@ -192,6 +194,8 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		Page		page;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 11a8ebe5ec..22bc3bdd4d 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1760,6 +1760,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index vacuum.</entry>
+     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -6817,6 +6821,28 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum, <literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..0af63ebe48 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -665,6 +665,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		for (i = 0; i < nRoot; i++)
 		{
 			ginVacuumPostingTree(&gvs, rootOfPostingTree[i]);
@@ -751,6 +754,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_SHARE);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0aa6e58a62..d99c92c0ae 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -223,7 +223,11 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 77fd147f68..e8c6df1d5f 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -505,6 +505,12 @@ loop_top:
 
 		blkno = bucket_blkno;
 
+		/*
+		 * For hash indexes, we report parallel vacuum progress
+		 * for every bucket.
+		 */
+		if (info->report_parallel_progress)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
 		 * wait concurrent scans before deleting the dead tuples.
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index d59711b7ec..81d1b7260a 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -459,6 +459,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 		Assert(params->index_cleanup == VACOPTVALUE_AUTO);
 	}
 
+	/* report number of indexes to vacuum, if we are told to cleanup indexes */
+	if (vacrel->do_index_cleanup)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	vacrel->bstrategy = bstrategy;
 	vacrel->relfrozenxid = rel->rd_rel->relfrozenxid;
 	vacrel->relminmxid = rel->rd_rel->relminmxid;
@@ -2301,6 +2305,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 				lazy_vacuum_one_index(indrel, istat, vacrel->old_live_tuples,
 									  vacrel);
 
+			/*
+			 * Done vacuuming an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2335,11 +2345,14 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Reset and report the number of indexes scanned.
+	 * Also, increase and report the number of index
+	 * scans.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 	vacrel->num_index_scans++;
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
@@ -2593,10 +2606,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation
+		 *
+		 * Also, report to progress.h that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2644,6 +2664,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/*
+			 * Done cleaning an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2678,10 +2704,12 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
@@ -2726,11 +2754,13 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index b52eca8f38..30d4677808 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,8 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+			if (info->report_parallel_progress && (scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index ad90b213b9..e64134a4d2 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -843,6 +843,9 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+			/* report parallel vacuum progress */
+			if (bds->info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(bds->info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 61f1d3926a..72f6a05e28 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,10 +3348,12 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
 	ivinfo.strategy = NULL;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Encode TIDs as int8 values for the sort, rather than directly sorting
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2d8104b090..c37b20b91b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index da1f0f043b..15c0eea75c 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -712,6 +712,7 @@ do_analyze_rel(Relation onerel, VacuumParams *params,
 			ivinfo.message_level = elevel;
 			ivinfo.num_heap_tuples = onerel->rd_rel->reltuples;
 			ivinfo.strategy = vac_strategy;
+			ivinfo.parallel_vacuum_state = NULL;
 
 			stats = index_vacuum_cleanup(&ivinfo, NULL);
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f26d796e52..c87a7f023f 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 nindexes_completed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -213,6 +225,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -364,6 +377,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_completed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +632,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing counter ( index progress counter also ) */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_completed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -688,7 +703,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	 */
 	if (nworkers > 0)
 	{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * Wait for all indexes to be vacuumed while
+		 * updating the parallel vacuum index progress,
+		 * and then wait for all workers to finish.
+		 */
+		parallel_vacuum_wait_to_finish(pvs);
+
 		WaitForParallelWorkersToFinish(pvs->pcxt);
 
 		for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)
@@ -839,9 +860,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
 
+	/* Only the leader should report parallel vacuum progress */
+	ivinfo.report_parallel_progress = !IsParallelWorker();
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
+	/* Pass ParallelVacuumState to IndexVacuumInfo for progress reporting */
+	ivinfo.parallel_vacuum_state = pvs;
 
 	switch (indstats->status)
 	{
@@ -888,6 +913,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/* Update the number of indexes completed. */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1);
+
+	if (ivinfo.report_parallel_progress)
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1103,42 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read pvs->shared->nindexes_completed and update progress.h
+ * with indexes vacuumed so far. This function is called periodically
+ * by index AMs as well as parallel_vacuum_process_one_index.
+ *
+ * Note: This function should be used by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker);
+
+	if (pvs)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_read_u32(&pvs->shared->nindexes_completed));
+}
+
+/*
+ * Waiting for all indexes to be vacuumed while updating the
+ * parallel index vacuum progress.
+ */
+void
+parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker);
+
+	while (pg_atomic_read_u32(&pvs->shared->nindexes_completed) < pvs->nindexes)
+	{
+		CHECK_FOR_INTERRUPTS();
+
+		parallel_vacuum_update_progress(pvs);
+
+		(void) WaitLatch(MyLatch, WL_TIMEOUT | WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 1000L,
+						 WAIT_EVENT_PARALLEL_VACUUM_FINISH);
+		ResetLatch(MyLatch);
+	}
+}
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index b2abd75ddb..eeba2bea5a 100644
--- a/src/backend/utils/activity/wait_event.c
+++ b/src/backend/utils/activity/wait_event.c
@@ -460,6 +460,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_XACT_GROUP_UPDATE:
 			event_name = "XactGroupUpdate";
 			break;
+		case WAIT_EVENT_PARALLEL_VACUUM_FINISH:
+			event_name = "ParallelVacuumFinish";
+			break;
 			/* no default case, so that compiler will warn */
 	}
 
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index e1c4fdbd03..8b0db91874 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -21,8 +21,9 @@
 #include "utils/relcache.h"
 #include "utils/snapshot.h"
 
-/* We don't want this file to depend on execnodes.h. */
+/* We don't want this file to depend on execnodes.h or vacuum.h. */
 struct IndexInfo;
+struct ParallelVacuumState;
 
 /*
  * Struct for statistics returned by ambuild
@@ -46,10 +47,12 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
 	BufferAccessStrategy strategy;	/* access strategy for reads */
+	struct ParallelVacuumState *parallel_vacuum_state; /* access parallel vacuum state for progress reporting */
 } IndexVacuumInfo;
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..0e97c6d4ef 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 4e4bc26a8b..fbd51786e0 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -64,6 +64,12 @@
 /* value for checking vacuum flags */
 #define VACUUM_OPTION_MAX_VALID_VALUE		((1 << 3) - 1)
 
+/*
+ * Parallel Index vacuum progress is reported every 1GB of blocks
+ * scanned.
+ */
+#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (((uint64) 1024 * 1024 * 1024) / BLCKSZ))
+
 /* Abstract type for parallel vacuum state */
 typedef struct ParallelVacuumState ParallelVacuumState;
 
@@ -333,5 +339,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(ParallelVacuumState *pvs);
 
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index 0b2100be4a..3a15460e98 100644
--- a/src/include/utils/wait_event.h
+++ b/src/include/utils/wait_event.h
@@ -114,6 +114,7 @@ typedef enum
 	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_PARALLEL_CREATE_INDEX_SCAN,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_VACUUM_FINISH,
 	WAIT_EVENT_PROCARRAY_GROUP_UPDATE,
 	WAIT_EVENT_PROC_SIGNAL_BARRIER,
 	WAIT_EVENT_PROMOTE,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fb9f936d43..df6f230715 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2021,7 +2021,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

#100Nathan Bossart
nathandbossart@gmail.com
In reply to: Imseih (AWS), Sami (#99)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Dec 14, 2022 at 05:09:46AM +0000, Imseih (AWS), Sami wrote:

Attached version addresses the above and the previous comments.

cfbot is complaining that this patch no longer applies. Sami, would you
mind rebasing it?

--
Nathan Bossart
Amazon Web Services: https://aws.amazon.com

#101Imseih (AWS), Sami
simseih@amazon.com
In reply to: Nathan Bossart (#100)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

cfbot is complaining that this patch no longer applies. Sami, would you
mind rebasing it?

Rebased patch attached.

--
Sami Imseih
Amazon Web Services: https://aws.amazon.com

Attachments:

v18-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v18-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From 64d185360f38f40846bbff973a87e0238c03bfa4 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Sun, 1 Jan 2023 21:17:30 -0600
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c                |  4 ++
 doc/src/sgml/monitoring.sgml            | 26 +++++++++
 src/backend/access/gin/ginvacuum.c      |  6 ++
 src/backend/access/gist/gistvacuum.c    |  4 ++
 src/backend/access/hash/hash.c          |  6 ++
 src/backend/access/heap/vacuumlazy.c    | 34 +++++++++++-
 src/backend/access/nbtree/nbtree.c      |  2 +
 src/backend/access/spgist/spgvacuum.c   |  3 +
 src/backend/catalog/index.c             |  2 +
 src/backend/catalog/system_views.sql    |  3 +-
 src/backend/commands/analyze.c          |  1 +
 src/backend/commands/vacuumparallel.c   | 74 ++++++++++++++++++++++++-
 src/backend/utils/activity/wait_event.c |  3 +
 src/include/access/genam.h              |  5 +-
 src/include/commands/progress.h         |  2 +
 src/include/commands/vacuum.h           |  7 +++
 src/include/utils/wait_event.h          |  1 +
 src/test/regress/expected/rules.out     |  4 +-
 18 files changed, 180 insertions(+), 7 deletions(-)

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 91fae5b0c0..005858ce9f 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -62,6 +62,8 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 				   *itupEnd;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
@@ -192,6 +194,8 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		Page		page;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 363b183e5f..73fb36f697 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1760,6 +1760,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index vacuum.</entry>
+     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -6835,6 +6839,28 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum, <literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index b4fa5f6bf8..0af63ebe48 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -665,6 +665,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		for (i = 0; i < nRoot; i++)
 		{
 			ginVacuumPostingTree(&gvs, rootOfPostingTree[i]);
@@ -751,6 +754,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_SHARE);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 0aa6e58a62..d99c92c0ae 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -223,7 +223,11 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 77fd147f68..e8c6df1d5f 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -505,6 +505,12 @@ loop_top:
 
 		blkno = bucket_blkno;
 
+		/*
+		 * For hash indexes, we report parallel vacuum progress
+		 * for every bucket.
+		 */
+		if (info->report_parallel_progress)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
 		 * wait concurrent scans before deleting the dead tuples.
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 9923994b50..001d6d4e2f 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -411,6 +411,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 		Assert(params->index_cleanup == VACOPTVALUE_AUTO);
 	}
 
+	/* report number of indexes to vacuum, if we are told to cleanup indexes */
+	if (vacrel->do_index_cleanup)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	/* Initialize page counters explicitly (be tidy) */
 	vacrel->scanned_pages = 0;
 	vacrel->removed_pages = 0;
@@ -2323,6 +2327,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/*
+			 * Done vacuuming an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2357,11 +2367,14 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Reset and report the number of indexes scanned.
+	 * Also, increase and report the number of index
+	 * scans.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 	vacrel->num_index_scans++;
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
@@ -2611,10 +2624,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation
+		 *
+		 * Also, report to progress.h that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2662,6 +2682,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/*
+			 * Done cleaning an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2696,10 +2722,12 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
@@ -2744,11 +2772,13 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index b52eca8f38..30d4677808 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,8 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+			if (info->report_parallel_progress && (scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index ad90b213b9..e64134a4d2 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -843,6 +843,9 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+			/* report parallel vacuum progress */
+			if (bds->info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(bds->info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 61f1d3926a..72f6a05e28 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,10 +3348,12 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
 	ivinfo.strategy = NULL;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Encode TIDs as int8 values for the sort, rather than directly sorting
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2d8104b090..c37b20b91b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index da1f0f043b..15c0eea75c 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -712,6 +712,7 @@ do_analyze_rel(Relation onerel, VacuumParams *params,
 			ivinfo.message_level = elevel;
 			ivinfo.num_heap_tuples = onerel->rd_rel->reltuples;
 			ivinfo.strategy = vac_strategy;
+			ivinfo.parallel_vacuum_state = NULL;
 
 			stats = index_vacuum_cleanup(&ivinfo, NULL);
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index f26d796e52..c87a7f023f 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 nindexes_completed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -213,6 +225,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -364,6 +377,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_completed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +632,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing counter ( index progress counter also ) */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_completed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -688,7 +703,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	 */
 	if (nworkers > 0)
 	{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * Wait for all indexes to be vacuumed while
+		 * updating the parallel vacuum index progress,
+		 * and then wait for all workers to finish.
+		 */
+		parallel_vacuum_wait_to_finish(pvs);
+
 		WaitForParallelWorkersToFinish(pvs->pcxt);
 
 		for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)
@@ -839,9 +860,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
 
+	/* Only the leader should report parallel vacuum progress */
+	ivinfo.report_parallel_progress = !IsParallelWorker();
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
+	/* Pass ParallelVacuumState to IndexVacuumInfo for progress reporting */
+	ivinfo.parallel_vacuum_state = pvs;
 
 	switch (indstats->status)
 	{
@@ -888,6 +913,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/* Update the number of indexes completed. */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1);
+
+	if (ivinfo.report_parallel_progress)
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1103,42 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read pvs->shared->nindexes_completed and update progress.h
+ * with indexes vacuumed so far. This function is called periodically
+ * by index AMs as well as parallel_vacuum_process_one_index.
+ *
+ * Note: This function should be used by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker);
+
+	if (pvs)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_read_u32(&pvs->shared->nindexes_completed));
+}
+
+/*
+ * Waiting for all indexes to be vacuumed while updating the
+ * parallel index vacuum progress.
+ */
+void
+parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker);
+
+	while (pg_atomic_read_u32(&pvs->shared->nindexes_completed) < pvs->nindexes)
+	{
+		CHECK_FOR_INTERRUPTS();
+
+		parallel_vacuum_update_progress(pvs);
+
+		(void) WaitLatch(MyLatch, WL_TIMEOUT | WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 1000L,
+						 WAIT_EVENT_PARALLEL_VACUUM_FINISH);
+		ResetLatch(MyLatch);
+	}
+}
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index b2abd75ddb..eeba2bea5a 100644
--- a/src/backend/utils/activity/wait_event.c
+++ b/src/backend/utils/activity/wait_event.c
@@ -460,6 +460,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_XACT_GROUP_UPDATE:
 			event_name = "XactGroupUpdate";
 			break;
+		case WAIT_EVENT_PARALLEL_VACUUM_FINISH:
+			event_name = "ParallelVacuumFinish";
+			break;
 			/* no default case, so that compiler will warn */
 	}
 
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index e1c4fdbd03..8b0db91874 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -21,8 +21,9 @@
 #include "utils/relcache.h"
 #include "utils/snapshot.h"
 
-/* We don't want this file to depend on execnodes.h. */
+/* We don't want this file to depend on execnodes.h or vacuum.h. */
 struct IndexInfo;
+struct ParallelVacuumState;
 
 /*
  * Struct for statistics returned by ambuild
@@ -46,10 +47,12 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
 	BufferAccessStrategy strategy;	/* access strategy for reads */
+	struct ParallelVacuumState *parallel_vacuum_state; /* access parallel vacuum state for progress reporting */
 } IndexVacuumInfo;
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index a28938caf4..0e97c6d4ef 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 2f274f2bec..7b8e07e953 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -64,6 +64,12 @@
 /* value for checking vacuum flags */
 #define VACUUM_OPTION_MAX_VALID_VALUE		((1 << 3) - 1)
 
+/*
+ * Parallel Index vacuum progress is reported every 1GB of blocks
+ * scanned.
+ */
+#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (((uint64) 1024 * 1024 * 1024) / BLCKSZ))
+
 /* Abstract type for parallel vacuum state */
 typedef struct ParallelVacuumState ParallelVacuumState;
 
@@ -368,5 +374,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(ParallelVacuumState *pvs);
 
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index 0b2100be4a..3a15460e98 100644
--- a/src/include/utils/wait_event.h
+++ b/src/include/utils/wait_event.h
@@ -114,6 +114,7 @@ typedef enum
 	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_PARALLEL_CREATE_INDEX_SCAN,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_VACUUM_FINISH,
 	WAIT_EVENT_PROCARRAY_GROUP_UPDATE,
 	WAIT_EVENT_PROC_SIGNAL_BARRIER,
 	WAIT_EVENT_PROMOTE,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fb9f936d43..df6f230715 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2021,7 +2021,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

#102vignesh C
vignesh21@gmail.com
In reply to: Imseih (AWS), Sami (#101)
Re: Add index scan progress to pg_stat_progress_vacuum

On Mon, 2 Jan 2023 at 10:04, Imseih (AWS), Sami <simseih@amazon.com> wrote:

cfbot is complaining that this patch no longer applies. Sami, would you
mind rebasing it?

Rebased patch attached.

CFBot shows some compilation errors as in [1]https://cirrus-ci.com/task/4557389261701120, please post an updated
version for the same:
[07:01:58.889] In file included from ../../../src/include/postgres.h:47,
[07:01:58.889] from vacuumparallel.c:27:
[07:01:58.889] vacuumparallel.c: In function ‘parallel_vacuum_update_progress’:
[07:01:58.889] vacuumparallel.c:1118:10: error: ‘IsParallelWorker’
undeclared (first use in this function); did you mean
‘ParallelWorkerMain’?
[07:01:58.889] 1118 | Assert(!IsParallelWorker);
[07:01:58.889] | ^~~~~~~~~~~~~~~~
[07:01:58.889] ../../../src/include/c.h:859:9: note: in definition of
macro ‘Assert’
[07:01:58.889] 859 | if (!(condition)) \
[07:01:58.889] | ^~~~~~~~~
[07:01:58.889] vacuumparallel.c:1118:10: note: each undeclared
identifier is reported only once for each function it appears in
[07:01:58.889] 1118 | Assert(!IsParallelWorker);
[07:01:58.889] | ^~~~~~~~~~~~~~~~

[1]: https://cirrus-ci.com/task/4557389261701120

Regards,
Vignesh

#103Imseih (AWS), Sami
simseih@amazon.com
In reply to: vignesh C (#102)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

cirrus-ci.com/task/4557389261701120

I earlier compiled without building with --enable-cassert,
which is why the compilation errors did not produce on my
buid.

Fixed in v19.

Thanks

--
Sami Imseih
Amazon Web Services: https://aws.amazon.com

Attachments:

v19-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v19-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From 42919ec064ec82e7afd86e64c3e1c1b75623778c Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Tue, 3 Jan 2023 10:16:57 -0600
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c                |  4 ++
 doc/src/sgml/monitoring.sgml            | 26 +++++++++
 src/backend/access/gin/ginvacuum.c      |  6 ++
 src/backend/access/gist/gistvacuum.c    |  4 ++
 src/backend/access/hash/hash.c          |  6 ++
 src/backend/access/heap/vacuumlazy.c    | 34 +++++++++++-
 src/backend/access/nbtree/nbtree.c      |  2 +
 src/backend/access/spgist/spgvacuum.c   |  3 +
 src/backend/catalog/index.c             |  2 +
 src/backend/catalog/system_views.sql    |  3 +-
 src/backend/commands/analyze.c          |  1 +
 src/backend/commands/vacuumparallel.c   | 74 ++++++++++++++++++++++++-
 src/backend/utils/activity/wait_event.c |  3 +
 src/include/access/genam.h              |  5 +-
 src/include/commands/progress.h         |  2 +
 src/include/commands/vacuum.h           |  7 +++
 src/include/utils/wait_event.h          |  1 +
 src/test/regress/expected/rules.out     |  4 +-
 18 files changed, 180 insertions(+), 7 deletions(-)

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 2340d49e00..731c16782d 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -62,6 +62,8 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 				   *itupEnd;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
@@ -192,6 +194,8 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		Page		page;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5bcba0fdec..76c9ed9691 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1767,6 +1767,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index vacuum.</entry>
+     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -6842,6 +6846,28 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum, <literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index e5d310d836..195065290a 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -665,6 +665,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		for (i = 0; i < nRoot; i++)
 		{
 			ginVacuumPostingTree(&gvs, rootOfPostingTree[i]);
@@ -751,6 +754,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_SHARE);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 3f60d3274d..8783c766ce 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -223,7 +223,11 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index eb258337d6..2426dcd968 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -505,6 +505,12 @@ loop_top:
 
 		blkno = bucket_blkno;
 
+		/*
+		 * For hash indexes, we report parallel vacuum progress
+		 * for every bucket.
+		 */
+		if (info->report_parallel_progress)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
 		 * wait concurrent scans before deleting the dead tuples.
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 06fd15405f..9d56d2be4e 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -411,6 +411,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 		Assert(params->index_cleanup == VACOPTVALUE_AUTO);
 	}
 
+	/* report number of indexes to vacuum, if we are told to cleanup indexes */
+	if (vacrel->do_index_cleanup)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	/* Initialize page counters explicitly (be tidy) */
 	vacrel->scanned_pages = 0;
 	vacrel->removed_pages = 0;
@@ -2325,6 +2329,12 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/*
+			 * Done vacuuming an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2359,11 +2369,14 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Reset and report the number of indexes scanned.
+	 * Also, increase and report the number of index
+	 * scans.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 	vacrel->num_index_scans++;
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
@@ -2613,10 +2626,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation
+		 *
+		 * Also, report to progress.h that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2664,6 +2684,12 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/*
+			 * Done cleaning an index. Increment the indexes completed
+			 */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2698,10 +2724,12 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
@@ -2746,11 +2774,13 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 1cc88da032..eb69409c6a 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,8 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+			if (info->report_parallel_progress && (scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 3adb18f2d8..b28036a830 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -843,6 +843,9 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+			/* report parallel vacuum progress */
+			if (bds->info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(bds->info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index e6579f2979..c734b2d4ce 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,10 +3348,12 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
 	ivinfo.strategy = NULL;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Encode TIDs as int8 values for the sort, rather than directly sorting
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 447c9b970f..0ab864185f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index c86e690980..84c1d2f3c7 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -712,6 +712,7 @@ do_analyze_rel(Relation onerel, VacuumParams *params,
 			ivinfo.message_level = elevel;
 			ivinfo.num_heap_tuples = onerel->rd_rel->reltuples;
 			ivinfo.strategy = vac_strategy;
+			ivinfo.parallel_vacuum_state = NULL;
 
 			stats = index_vacuum_cleanup(&ivinfo, NULL);
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index bcd40c80a1..3cf429481a 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 nindexes_completed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -213,6 +225,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -364,6 +377,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_completed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +632,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing counter ( index progress counter also ) */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_completed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -688,7 +703,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	 */
 	if (nworkers > 0)
 	{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * Wait for all indexes to be vacuumed while
+		 * updating the parallel vacuum index progress,
+		 * and then wait for all workers to finish.
+		 */
+		parallel_vacuum_wait_to_finish(pvs);
+
 		WaitForParallelWorkersToFinish(pvs->pcxt);
 
 		for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)
@@ -839,9 +860,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
 
+	/* Only the leader should report parallel vacuum progress */
+	ivinfo.report_parallel_progress = !IsParallelWorker();
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
+	/* Pass ParallelVacuumState to IndexVacuumInfo for progress reporting */
+	ivinfo.parallel_vacuum_state = pvs;
 
 	switch (indstats->status)
 	{
@@ -888,6 +913,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/* Update the number of indexes completed. */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1);
+
+	if (ivinfo.report_parallel_progress)
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1103,42 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read pvs->shared->nindexes_completed and update progress.h
+ * with indexes vacuumed so far. This function is called periodically
+ * by index AMs as well as parallel_vacuum_process_one_index.
+ *
+ * Note: This function should be used by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker());
+
+	if (pvs)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_read_u32(&pvs->shared->nindexes_completed));
+}
+
+/*
+ * Waiting for all indexes to be vacuumed while updating the
+ * parallel index vacuum progress.
+ */
+void
+parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker());
+
+	while (pg_atomic_read_u32(&pvs->shared->nindexes_completed) < pvs->nindexes)
+	{
+		CHECK_FOR_INTERRUPTS();
+
+		parallel_vacuum_update_progress(pvs);
+
+		(void) WaitLatch(MyLatch, WL_TIMEOUT | WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 1000L,
+						 WAIT_EVENT_PARALLEL_VACUUM_FINISH);
+		ResetLatch(MyLatch);
+	}
+}
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index f9574e800f..a7d4b8534a 100644
--- a/src/backend/utils/activity/wait_event.c
+++ b/src/backend/utils/activity/wait_event.c
@@ -460,6 +460,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_XACT_GROUP_UPDATE:
 			event_name = "XactGroupUpdate";
 			break;
+		case WAIT_EVENT_PARALLEL_VACUUM_FINISH:
+			event_name = "ParallelVacuumFinish";
+			break;
 			/* no default case, so that compiler will warn */
 	}
 
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index 83dbee0fe6..6e0f12cef0 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -21,8 +21,9 @@
 #include "utils/relcache.h"
 #include "utils/snapshot.h"
 
-/* We don't want this file to depend on execnodes.h. */
+/* We don't want this file to depend on execnodes.h or vacuum.h. */
 struct IndexInfo;
+struct ParallelVacuumState;
 
 /*
  * Struct for statistics returned by ambuild
@@ -46,10 +47,12 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
 	BufferAccessStrategy strategy;	/* access strategy for reads */
+	struct ParallelVacuumState *parallel_vacuum_state; /* access parallel vacuum state for progress reporting */
 } IndexVacuumInfo;
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..6b8b609a4f 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 5efb942368..84a2216119 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -64,6 +64,12 @@
 /* value for checking vacuum flags */
 #define VACUUM_OPTION_MAX_VALID_VALUE		((1 << 3) - 1)
 
+/*
+ * Parallel Index vacuum progress is reported every 1GB of blocks
+ * scanned.
+ */
+#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (((uint64) 1024 * 1024 * 1024) / BLCKSZ))
+
 /* Abstract type for parallel vacuum state */
 typedef struct ParallelVacuumState ParallelVacuumState;
 
@@ -368,5 +374,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(ParallelVacuumState *pvs);
 
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index f53254ad1f..46930fe7f5 100644
--- a/src/include/utils/wait_event.h
+++ b/src/include/utils/wait_event.h
@@ -114,6 +114,7 @@ typedef enum
 	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_PARALLEL_CREATE_INDEX_SCAN,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_VACUUM_FINISH,
 	WAIT_EVENT_PROCARRAY_GROUP_UPDATE,
 	WAIT_EVENT_PROC_SIGNAL_BARRIER,
 	WAIT_EVENT_PROMOTE,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fb9f936d43..df6f230715 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2021,7 +2021,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

#104Drouvot, Bertrand
bertranddrouvot.pg@gmail.com
In reply to: Imseih (AWS), Sami (#103)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

On 1/3/23 5:46 PM, Imseih (AWS), Sami wrote:

cirrus-ci.com/task/4557389261701120

I earlier compiled without building with --enable-cassert,
which is why the compilation errors did not produce on my
buid.

Fixed in v19.

Thanks

Thanks for the updated patch!

Some comments about it:

+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that wil be vacuumed. This value will be

Typo: wil

+       /* report number of indexes to vacuum, if we are told to cleanup indexes */
+       if (vacrel->do_index_cleanup)
+               pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);

"Report" instead? (to looks like the surrounding code)

+                       /*
+                        * Done vacuuming an index. Increment the indexes completed
+                        */
+                       pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+                                                                                idx + 1);

"Increment the indexes completed." (dot at the end) instead?

-        * Increase and report the number of index scans.
+        * Reset and report the number of indexes scanned.
+        * Also, increase and report the number of index
+        * scans.

What about "Reset and report zero as the number of indexes scanned."? (just to make clear we
don't want to report the value as it was prior to the reset)

-               /* Disable index vacuuming, index cleanup, and heap rel truncation */
+               /*
+                * Disable index vacuuming, index cleanup, and heap rel truncation
+                *

The new "Disable index vacuuming, index cleanup, and heap rel truncation" needs a dot at the end.

+                * Also, report to progress.h that we are no longer tracking
+                * index vacuum/cleanup.
+                */

"Also, report that we are" instead?

+                       /*
+                        * Done cleaning an index. Increment the indexes completed
+                        */

Needs a dot at the end.

-       /* Reset the parallel index processing counter */
+       /* Reset the parallel index processing counter ( index progress counter also ) */

"Reset the parallel index processing and progress counters" instead?

+       /* Update the number of indexes completed. */
+       pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1);

Remove the dot at the end? (to looks like the surrounding code)

+
+/*
+ * Read pvs->shared->nindexes_completed and update progress.h
+ * with indexes vacuumed so far. This function is called periodically

"Read pvs->shared->nindexes_completed and report the number of indexes vacuumed so far" instead?

+ * Note: This function should be used by the leader process only,

"called" instead of "used"?

                 case WAIT_EVENT_XACT_GROUP_UPDATE:
                         event_name = "XactGroupUpdate";
                         break;
+               case WAIT_EVENT_PARALLEL_VACUUM_FINISH:
+                       event_name = "ParallelVacuumFinish";
+                       break;
                         /* no default case, so that compiler will warn */

It seems to me that the case ordering should follow the alphabetical order (that's how it is done currently without the patch).

+#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (((uint64) 1024 * 1024 * 1024) / BLCKSZ))

It seems to me that "#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (1024 * 1024 * 1024 / BLCKSZ))" would be fine too.

Regards,

--
Bertrand Drouvot
PostgreSQL Contributors Team
RDS Open Source Databases
Amazon Web Services: https://aws.amazon.com

#105Imseih (AWS), Sami
simseih@amazon.com
In reply to: Drouvot, Bertrand (#104)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Thanks for the review!

Addressed the comments.

"Increment the indexes completed." (dot at the end) instead?

Used the commenting format being used in other places in this
file with an inclusion of a double-dash. i.,e.
/* Wraparound emergency -- end current index scan */

It seems to me that "#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (1024 * 1024 * 1024 / BLCKSZ))" would be fine too.

I kept this the same as it matches what we are doing in other places such
as FAILSAFE_EVERY_PAGES

v20 attached.

Regards,

--
Sami Imseih
Amazon Web Services: https://aws.amazon.com

Attachments:

v20-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v20-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From 4336a7fe6bd9d554bbca4c87001b5d6b753b7aa0 Mon Sep 17 00:00:00 2001
From: EC2 Default User <ec2-user@ip-172-31-94-175.ec2.internal>
Date: Wed, 4 Jan 2023 17:06:09 +0000
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c                |  4 ++
 doc/src/sgml/monitoring.sgml            | 26 +++++++++
 src/backend/access/gin/ginvacuum.c      |  6 ++
 src/backend/access/gist/gistvacuum.c    |  4 ++
 src/backend/access/hash/hash.c          |  6 ++
 src/backend/access/heap/vacuumlazy.c    | 30 +++++++++-
 src/backend/access/nbtree/nbtree.c      |  2 +
 src/backend/access/spgist/spgvacuum.c   |  3 +
 src/backend/catalog/index.c             |  2 +
 src/backend/catalog/system_views.sql    |  3 +-
 src/backend/commands/analyze.c          |  1 +
 src/backend/commands/vacuumparallel.c   | 74 ++++++++++++++++++++++++-
 src/backend/utils/activity/wait_event.c |  3 +
 src/include/access/genam.h              |  5 +-
 src/include/commands/progress.h         |  2 +
 src/include/commands/vacuum.h           |  7 +++
 src/include/utils/wait_event.h          |  1 +
 src/test/regress/expected/rules.out     |  4 +-
 18 files changed, 176 insertions(+), 7 deletions(-)

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 2340d49e00..731c16782d 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -62,6 +62,8 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 				   *itupEnd;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
@@ -192,6 +194,8 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		Page		page;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5bcba0fdec..df468d4424 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1767,6 +1767,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index vacuum.</entry>
+     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -6842,6 +6846,28 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that will be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum, <literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index e5d310d836..195065290a 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -665,6 +665,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		for (i = 0; i < nRoot; i++)
 		{
 			ginVacuumPostingTree(&gvs, rootOfPostingTree[i]);
@@ -751,6 +754,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_SHARE);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 3f60d3274d..8783c766ce 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -223,7 +223,11 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index eb258337d6..2426dcd968 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -505,6 +505,12 @@ loop_top:
 
 		blkno = bucket_blkno;
 
+		/*
+		 * For hash indexes, we report parallel vacuum progress
+		 * for every bucket.
+		 */
+		if (info->report_parallel_progress)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
 		 * wait concurrent scans before deleting the dead tuples.
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index a42e881da3..e65f43fbe9 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -413,6 +413,10 @@ heap_vacuum_rel(Relation rel, VacuumParams *params,
 		Assert(params->index_cleanup == VACOPTVALUE_AUTO);
 	}
 
+	/* Report the number of indexes to vacuum, if we are told to cleanup indexes */
+	if (vacrel->do_index_cleanup)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, vacrel->nindexes);
+
 	/* Initialize page counters explicitly (be tidy) */
 	vacrel->scanned_pages = 0;
 	vacrel->removed_pages = 0;
@@ -2321,6 +2325,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Done vacuuming an index -- increment the indexes completed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2355,11 +2363,14 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Reset and report zero as the number of indexes scanned.
+	 * Also, increase and report the number of index
+	 * scans.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 	vacrel->num_index_scans++;
 	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
 								 vacrel->num_index_scans);
@@ -2609,10 +2620,17 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation.
+		 *
+		 * Also, report that we are no longer tracking
+		 * index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_TOTAL, 0);
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED, 0);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2658,6 +2676,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Done cleaning an index -- increment the indexes completed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2692,10 +2714,12 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
@@ -2740,11 +2764,13 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 1cc88da032..eb69409c6a 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,8 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+			if (info->report_parallel_progress && (scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 3adb18f2d8..b28036a830 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -843,6 +843,9 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+			/* report parallel vacuum progress */
+			if (bds->info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(bds->info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index e6579f2979..c734b2d4ce 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,10 +3348,12 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
 	ivinfo.strategy = NULL;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Encode TIDs as int8 values for the sort, rather than directly sorting
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 447c9b970f..0ab864185f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index c86e690980..84c1d2f3c7 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -712,6 +712,7 @@ do_analyze_rel(Relation onerel, VacuumParams *params,
 			ivinfo.message_level = elevel;
 			ivinfo.num_heap_tuples = onerel->rd_rel->reltuples;
 			ivinfo.strategy = vac_strategy;
+			ivinfo.parallel_vacuum_state = NULL;
 
 			stats = index_vacuum_cleanup(&ivinfo, NULL);
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index bcd40c80a1..8e2be7d15b 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 nindexes_completed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -213,6 +225,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -364,6 +377,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_completed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +632,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_completed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -688,7 +703,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	 */
 	if (nworkers > 0)
 	{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * Wait for all indexes to be vacuumed while
+		 * updating the parallel vacuum index progress,
+		 * and then wait for all workers to finish.
+		 */
+		parallel_vacuum_wait_to_finish(pvs);
+
 		WaitForParallelWorkersToFinish(pvs->pcxt);
 
 		for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)
@@ -839,9 +860,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
 
+	/* Only the leader should report parallel vacuum progress */
+	ivinfo.report_parallel_progress = !IsParallelWorker();
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
+	/* Pass ParallelVacuumState to IndexVacuumInfo for progress reporting */
+	ivinfo.parallel_vacuum_state = pvs;
 
 	switch (indstats->status)
 	{
@@ -888,6 +913,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/* Update the number of indexes completed */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1);
+
+	if (ivinfo.report_parallel_progress)
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1103,42 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read pvs->shared->nindexes_completed and report the number of indexes
+ * vacuumed so far. This function is called periodically
+ * by index AMs as well as parallel_vacuum_process_one_index.
+ *
+ * Note: This function should be called by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker());
+
+	if (pvs)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_read_u32(&pvs->shared->nindexes_completed));
+}
+
+/*
+ * Waiting for all indexes to be vacuumed while updating the
+ * parallel index vacuum progress.
+ */
+void
+parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker());
+
+	while (pg_atomic_read_u32(&pvs->shared->nindexes_completed) < pvs->nindexes)
+	{
+		CHECK_FOR_INTERRUPTS();
+
+		parallel_vacuum_update_progress(pvs);
+
+		(void) WaitLatch(MyLatch, WL_TIMEOUT | WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 1000L,
+						 WAIT_EVENT_PARALLEL_VACUUM_FINISH);
+		ResetLatch(MyLatch);
+	}
+}
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index f9574e800f..1a06428f3c 100644
--- a/src/backend/utils/activity/wait_event.c
+++ b/src/backend/utils/activity/wait_event.c
@@ -415,6 +415,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_VACUUM_FINISH:
+			event_name = "ParallelVacuumFinish";
+			break;
 		case WAIT_EVENT_PROCARRAY_GROUP_UPDATE:
 			event_name = "ProcArrayGroupUpdate";
 			break;
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index 83dbee0fe6..6e0f12cef0 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -21,8 +21,9 @@
 #include "utils/relcache.h"
 #include "utils/snapshot.h"
 
-/* We don't want this file to depend on execnodes.h. */
+/* We don't want this file to depend on execnodes.h or vacuum.h. */
 struct IndexInfo;
+struct ParallelVacuumState;
 
 /*
  * Struct for statistics returned by ambuild
@@ -46,10 +47,12 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
 	BufferAccessStrategy strategy;	/* access strategy for reads */
+	struct ParallelVacuumState *parallel_vacuum_state; /* access parallel vacuum state for progress reporting */
 } IndexVacuumInfo;
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..6b8b609a4f 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 5efb942368..84a2216119 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -64,6 +64,12 @@
 /* value for checking vacuum flags */
 #define VACUUM_OPTION_MAX_VALID_VALUE		((1 << 3) - 1)
 
+/*
+ * Parallel Index vacuum progress is reported every 1GB of blocks
+ * scanned.
+ */
+#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (((uint64) 1024 * 1024 * 1024) / BLCKSZ))
+
 /* Abstract type for parallel vacuum state */
 typedef struct ParallelVacuumState ParallelVacuumState;
 
@@ -368,5 +374,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(ParallelVacuumState *pvs);
 
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index f53254ad1f..46930fe7f5 100644
--- a/src/include/utils/wait_event.h
+++ b/src/include/utils/wait_event.h
@@ -114,6 +114,7 @@ typedef enum
 	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_PARALLEL_CREATE_INDEX_SCAN,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_VACUUM_FINISH,
 	WAIT_EVENT_PROCARRAY_GROUP_UPDATE,
 	WAIT_EVENT_PROC_SIGNAL_BARRIER,
 	WAIT_EVENT_PROMOTE,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fb9f936d43..df6f230715 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2021,7 +2021,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.38.1

#106Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#105)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Jan 5, 2023 at 4:24 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Thanks for the review!

Addressed the comments.

"Increment the indexes completed." (dot at the end) instead?

Used the commenting format being used in other places in this
file with an inclusion of a double-dash. i.,e.
/* Wraparound emergency -- end current index scan */

It seems to me that "#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (1024 * 1024 * 1024 / BLCKSZ))" would be fine too.

I kept this the same as it matches what we are doing in other places such
as FAILSAFE_EVERY_PAGES

v20 attached.

+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that will be vacuumed. This value will be
+       <literal>0</literal> if there are no indexes to vacuum,
<literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, or vacuum failsafe is triggered.

Similar to above three cases, vacuum can bypass index vacuuming if
there are almost zero TIDs. Should we set indexes_total to 0 in this
case too? If so, I think we can set both indexes_total and
indexes_completed at the beginning of the index vacuuming/cleanup and
reset them at the end. That is, these values are valid only in index
vacuum phase and index cleanup phase. Otherwise, 0.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#107Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#106)
Re: Add index scan progress to pg_stat_progress_vacuum

Similar to above three cases, vacuum can bypass index vacuuming if
there are almost zero TIDs. Should we set indexes_total to 0 in this
case too? If so, I think we can set both indexes_total and
indexes_completed at the beginning of the index vacuuming/cleanup and
reset them at the end.

Unlike the other 3 cases, in which the vacuum and cleanup are totally skipped,
a cleanup still occurs when the index vacuum is bypassed. From what I can tell,
this is to allow for things like a gin pending list cleanup even if the index
is not vacuumed. There could be other reasons as well.

if (bypass)
{
/*
* There are almost zero TIDs. Behave as if there were precisely
* zero: bypass index vacuuming, but do index cleanup.
*
* We expect that the ongoing VACUUM operation will finish very
* quickly, so there is no point in considering speeding up as a
* failsafe against wraparound failure. (Index cleanup is expected to
* finish very quickly in cases where there were no ambulkdelete()
* calls.)
*/
vacrel->do_index_vacuuming = false;
}

So it seems like we should still report the total number of indexes as we are currently
doing in the patch.

With that said, the documentation should make this be more clear.

For indexes_total, the description should be:

Number of indexes that will be vacuumed or cleaned up. This value will be
<literal>0</literal> if there are no indexes to vacuum, <literal>INDEX_CLEANUP</literal>
is set to <literal>OFF</literal>, or vacuum failsafe is triggered.
See <xref linkend="guc-vacuum-failsafe-age"/>

For indexes_completed, it should be:

Number of indexes vacuumed in the current vacuum cycle when the
phase is <literal>vacuuming indexes</liternal>, or the number
of indexes cleaned up in the <literal>cleaning up indexes<literal>
phase.

Regards,

--
Sami Imseih
Amazon Web Services: https://aws.amazon.com

#108Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#107)
Re: Add index scan progress to pg_stat_progress_vacuum

On Fri, Jan 6, 2023 at 12:07 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Similar to above three cases, vacuum can bypass index vacuuming if
there are almost zero TIDs. Should we set indexes_total to 0 in this
case too? If so, I think we can set both indexes_total and
indexes_completed at the beginning of the index vacuuming/cleanup and
reset them at the end.

Unlike the other 3 cases, in which the vacuum and cleanup are totally skipped,
a cleanup still occurs when the index vacuum is bypassed. From what I can tell,
this is to allow for things like a gin pending list cleanup even if the index
is not vacuumed.

Right. But since we set indexes_total also at the beginning of index
cleanup (i.e. lazy_cleanup_all_indexes()), can't we still show the
valid value in this case? My point is whether we should show
indexes_total throughout the vacuum execution (even also in not
relevant phases such as heap scanning/vacuum/truncation). For example,
in the analyze progress report, we have child_tables_total and
child_tables_done. child_tables_total is set before the actual work of
sampling rows from child tables, but not the beginning of the analyze.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#109Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#108)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

My point is whether we should show
indexes_total throughout the vacuum execution (even also in not
relevant phases such as heap scanning/vacuum/truncation).

That is a good point. We should only show indexes_total
and indexes_completed only during the relevant phases.

V21 addresses this along with a documentation fix.

indexes_total and indexes_completed can only be a value > 0 while in the
"vacuuming indexes" or "cleaning up indexes" phases of vacuum.

Indexes_total is set to 0 at the start of the index vacuum cycle or index cleanups
and set back to 0, along with indexes_completed, at the end of the index vacuum
cycle and index cleanups

Also, for the progress updates in vacuumlazy.c that should be done atomically,
I made a change to use pgstat_progress_update_multi_param.

Regards,

--
Sami Imseih
Amazon Web Services: https://aws.amazon.com

Attachments:

v21-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v21-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From d1fee558a90c7984d64c2c21fb9a03235a623934 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Fri, 6 Jan 2023 19:23:34 -0600
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 contrib/bloom/blvacuum.c                |  4 ++
 doc/src/sgml/monitoring.sgml            | 31 +++++++++
 src/backend/access/gin/ginvacuum.c      |  6 ++
 src/backend/access/gist/gistvacuum.c    |  4 ++
 src/backend/access/hash/hash.c          |  6 ++
 src/backend/access/heap/vacuumlazy.c    | 84 ++++++++++++++++++++++---
 src/backend/access/nbtree/nbtree.c      |  2 +
 src/backend/access/spgist/spgvacuum.c   |  3 +
 src/backend/catalog/index.c             |  2 +
 src/backend/catalog/system_views.sql    |  3 +-
 src/backend/commands/analyze.c          |  1 +
 src/backend/commands/vacuumparallel.c   | 74 +++++++++++++++++++++-
 src/backend/utils/activity/wait_event.c |  3 +
 src/include/access/genam.h              |  5 +-
 src/include/commands/progress.h         |  2 +
 src/include/commands/vacuum.h           |  7 +++
 src/include/utils/wait_event.h          |  1 +
 src/test/regress/expected/rules.out     |  4 +-
 18 files changed, 227 insertions(+), 15 deletions(-)

diff --git a/contrib/bloom/blvacuum.c b/contrib/bloom/blvacuum.c
index 2340d49e00..731c16782d 100644
--- a/contrib/bloom/blvacuum.c
+++ b/contrib/bloom/blvacuum.c
@@ -62,6 +62,8 @@ blbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 				   *itupEnd;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
@@ -192,6 +194,8 @@ blvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 		Page		page;
 
 		vacuum_delay_point();
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5bcba0fdec..b0583d6846 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1767,6 +1767,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index vacuum.</entry>
+     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -6842,6 +6846,33 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that will be vacuumed or cleaned up. This value will be
+       <literal>0</literal> if the phase is not <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal>, <literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, index vacuum is skipped due to very
+       few dead tuples in the table, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle when the
+       phase is <literal>vacuuming indexes</literal>, or the number
+       of indexes cleaned up during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/gin/ginvacuum.c b/src/backend/access/gin/ginvacuum.c
index e5d310d836..195065290a 100644
--- a/src/backend/access/gin/ginvacuum.c
+++ b/src/backend/access/gin/ginvacuum.c
@@ -665,6 +665,9 @@ ginbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		for (i = 0; i < nRoot; i++)
 		{
 			ginVacuumPostingTree(&gvs, rootOfPostingTree[i]);
@@ -751,6 +754,9 @@ ginvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
 
 		vacuum_delay_point();
 
+		if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
+
 		buffer = ReadBufferExtended(index, MAIN_FORKNUM, blkno,
 									RBM_NORMAL, info->strategy);
 		LockBuffer(buffer, GIN_SHARE);
diff --git a/src/backend/access/gist/gistvacuum.c b/src/backend/access/gist/gistvacuum.c
index 3f60d3274d..8783c766ce 100644
--- a/src/backend/access/gist/gistvacuum.c
+++ b/src/backend/access/gist/gistvacuum.c
@@ -223,7 +223,11 @@ gistvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			break;
 		/* Iterate over pages, then loop back to recheck length */
 		for (; blkno < num_pages; blkno++)
+		{
 			gistvacuumpage(&vstate, blkno, blkno);
+			if (info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
+		}
 	}
 
 	/*
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index eb258337d6..2426dcd968 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -505,6 +505,12 @@ loop_top:
 
 		blkno = bucket_blkno;
 
+		/*
+		 * For hash indexes, we report parallel vacuum progress
+		 * for every bucket.
+		 */
+		if (info->report_parallel_progress)
+			parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		/*
 		 * We need to acquire a cleanup lock on the primary bucket page to out
 		 * wait concurrent scans before deleting the dead tuples.
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index a42e881da3..c8785afd2f 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2294,6 +2294,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEX_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEX_TOTAL,
+		PROGRESS_VACUUM_INDEX_COMPLETED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2306,9 +2317,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes
+	 * and the number of indexes to vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2321,6 +2336,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Done vacuuming an index -- increment the indexes completed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2355,14 +2374,18 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Reset and report the total indexes to vacuum and the number of
+	 * indexes vacuumed.
+	 * Also, increase and report the number of index scans completed.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2607,12 +2630,23 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int   progress_index[] = {
+			PROGRESS_VACUUM_INDEX_TOTAL,
+			PROGRESS_VACUUM_INDEX_COMPLETED
+		};
+		int64       progress_val[2] = {0, 0};
+
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation.
+		 *
+		 * Also, report that we are no longer tracking index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2640,13 +2674,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEX_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEX_TOTAL,
+		PROGRESS_VACUUM_INDEX_COMPLETED
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[2];
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes
+	 * and the number of indexes to cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2658,6 +2706,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Done cleaning an index -- increment the indexes completed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2667,6 +2719,14 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/*
+	 * Reset and report the total number of indexes to cleanup
+	 * and the number of indexes cleaned.
+	 */
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
@@ -2692,10 +2752,12 @@ lazy_vacuum_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
@@ -2740,11 +2802,13 @@ lazy_cleanup_one_index(Relation indrel, IndexBulkDeleteResult *istat,
 	ivinfo.index = indrel;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = false;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = estimated_count;
 	ivinfo.message_level = DEBUG2;
 
 	ivinfo.num_heap_tuples = reltuples;
 	ivinfo.strategy = vacrel->bstrategy;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Update error traceback information.
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 1cc88da032..eb69409c6a 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,8 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
 			if (info->report_progress)
 				pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
 											 scanblkno);
+			if (info->report_parallel_progress && (scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/access/spgist/spgvacuum.c b/src/backend/access/spgist/spgvacuum.c
index 3adb18f2d8..b28036a830 100644
--- a/src/backend/access/spgist/spgvacuum.c
+++ b/src/backend/access/spgist/spgvacuum.c
@@ -843,6 +843,9 @@ spgvacuumscan(spgBulkDeleteState *bds)
 			/* empty the pending-list after each page */
 			if (bds->pendingList != NULL)
 				spgprocesspending(bds);
+			/* report parallel vacuum progress */
+			if (bds->info->report_parallel_progress && (blkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(bds->info->parallel_vacuum_state);
 		}
 	}
 
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index e6579f2979..c734b2d4ce 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -3348,10 +3348,12 @@ validate_index(Oid heapId, Oid indexId, Snapshot snapshot)
 	ivinfo.index = indexRelation;
 	ivinfo.analyze_only = false;
 	ivinfo.report_progress = true;
+	ivinfo.report_parallel_progress = false;
 	ivinfo.estimated_count = true;
 	ivinfo.message_level = DEBUG2;
 	ivinfo.num_heap_tuples = heapRelation->rd_rel->reltuples;
 	ivinfo.strategy = NULL;
+	ivinfo.parallel_vacuum_state = NULL;
 
 	/*
 	 * Encode TIDs as int8 values for the sort, rather than directly sorting
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 447c9b970f..0ab864185f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/analyze.c b/src/backend/commands/analyze.c
index c86e690980..84c1d2f3c7 100644
--- a/src/backend/commands/analyze.c
+++ b/src/backend/commands/analyze.c
@@ -712,6 +712,7 @@ do_analyze_rel(Relation onerel, VacuumParams *params,
 			ivinfo.message_level = elevel;
 			ivinfo.num_heap_tuples = onerel->rd_rel->reltuples;
 			ivinfo.strategy = vac_strategy;
+			ivinfo.parallel_vacuum_state = NULL;
 
 			stats = index_vacuum_cleanup(&ivinfo, NULL);
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index bcd40c80a1..8e2be7d15b 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -103,6 +104,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 nindexes_completed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -213,6 +225,7 @@ static void parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation
 static bool parallel_vacuum_index_is_parallel_safe(Relation indrel, int num_index_scans,
 												   bool vacuum);
 static void parallel_vacuum_error_callback(void *arg);
+static void parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs);
 
 /*
  * Try to enter parallel mode and create a parallel context.  Then initialize
@@ -364,6 +377,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_completed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +632,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_completed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -688,7 +703,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 	 */
 	if (nworkers > 0)
 	{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * Wait for all indexes to be vacuumed while
+		 * updating the parallel vacuum index progress,
+		 * and then wait for all workers to finish.
+		 */
+		parallel_vacuum_wait_to_finish(pvs);
+
 		WaitForParallelWorkersToFinish(pvs->pcxt);
 
 		for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)
@@ -839,9 +860,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	ivinfo.num_heap_tuples = pvs->shared->reltuples;
 	ivinfo.strategy = pvs->bstrategy;
 
+	/* Only the leader should report parallel vacuum progress */
+	ivinfo.report_parallel_progress = !IsParallelWorker();
 	/* Update error traceback information */
 	pvs->indname = pstrdup(RelationGetRelationName(indrel));
 	pvs->status = indstats->status;
+	/* Pass ParallelVacuumState to IndexVacuumInfo for progress reporting */
+	ivinfo.parallel_vacuum_state = pvs;
 
 	switch (indstats->status)
 	{
@@ -888,6 +913,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/* Update the number of indexes completed */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1);
+
+	if (ivinfo.report_parallel_progress)
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1103,42 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read pvs->shared->nindexes_completed and report the number of indexes
+ * vacuumed so far. This function is called periodically
+ * by index AMs as well as parallel_vacuum_process_one_index.
+ *
+ * Note: This function should be called by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker());
+
+	if (pvs)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_read_u32(&pvs->shared->nindexes_completed));
+}
+
+/*
+ * Waiting for all indexes to be vacuumed while updating the
+ * parallel index vacuum progress.
+ */
+void
+parallel_vacuum_wait_to_finish(ParallelVacuumState *pvs)
+{
+	Assert(!IsParallelWorker());
+
+	while (pg_atomic_read_u32(&pvs->shared->nindexes_completed) < pvs->nindexes)
+	{
+		CHECK_FOR_INTERRUPTS();
+
+		parallel_vacuum_update_progress(pvs);
+
+		(void) WaitLatch(MyLatch, WL_TIMEOUT | WL_LATCH_SET | WL_EXIT_ON_PM_DEATH, 1000L,
+						 WAIT_EVENT_PARALLEL_VACUUM_FINISH);
+		ResetLatch(MyLatch);
+	}
+}
diff --git a/src/backend/utils/activity/wait_event.c b/src/backend/utils/activity/wait_event.c
index f9574e800f..1a06428f3c 100644
--- a/src/backend/utils/activity/wait_event.c
+++ b/src/backend/utils/activity/wait_event.c
@@ -415,6 +415,9 @@ pgstat_get_wait_ipc(WaitEventIPC w)
 		case WAIT_EVENT_PARALLEL_FINISH:
 			event_name = "ParallelFinish";
 			break;
+		case WAIT_EVENT_PARALLEL_VACUUM_FINISH:
+			event_name = "ParallelVacuumFinish";
+			break;
 		case WAIT_EVENT_PROCARRAY_GROUP_UPDATE:
 			event_name = "ProcArrayGroupUpdate";
 			break;
diff --git a/src/include/access/genam.h b/src/include/access/genam.h
index 83dbee0fe6..6e0f12cef0 100644
--- a/src/include/access/genam.h
+++ b/src/include/access/genam.h
@@ -21,8 +21,9 @@
 #include "utils/relcache.h"
 #include "utils/snapshot.h"
 
-/* We don't want this file to depend on execnodes.h. */
+/* We don't want this file to depend on execnodes.h or vacuum.h. */
 struct IndexInfo;
+struct ParallelVacuumState;
 
 /*
  * Struct for statistics returned by ambuild
@@ -46,10 +47,12 @@ typedef struct IndexVacuumInfo
 	Relation	index;			/* the index being vacuumed */
 	bool		analyze_only;	/* ANALYZE (without any actual vacuum) */
 	bool		report_progress;	/* emit progress.h status reports */
+	bool		report_parallel_progress;	/* emit progress.h status reports for parallel vacuum */
 	bool		estimated_count;	/* num_heap_tuples is an estimate */
 	int			message_level;	/* ereport level for progress messages */
 	double		num_heap_tuples;	/* tuples remaining in heap */
 	BufferAccessStrategy strategy;	/* access strategy for reads */
+	struct ParallelVacuumState *parallel_vacuum_state; /* access parallel vacuum state for progress reporting */
 } IndexVacuumInfo;
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..6b8b609a4f 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 689dbb7702..a937f9af0e 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -64,6 +64,12 @@
 /* value for checking vacuum flags */
 #define VACUUM_OPTION_MAX_VALID_VALUE		((1 << 3) - 1)
 
+/*
+ * Parallel Index vacuum progress is reported every 1GB of blocks
+ * scanned.
+ */
+#define REPORT_PARALLEL_VACUUM_EVERY_PAGES ((BlockNumber) (((uint64) 1024 * 1024 * 1024) / BLCKSZ))
+
 /* Abstract type for parallel vacuum state */
 typedef struct ParallelVacuumState ParallelVacuumState;
 
@@ -370,5 +376,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(ParallelVacuumState *pvs);
 
 #endif							/* VACUUM_H */
diff --git a/src/include/utils/wait_event.h b/src/include/utils/wait_event.h
index f53254ad1f..46930fe7f5 100644
--- a/src/include/utils/wait_event.h
+++ b/src/include/utils/wait_event.h
@@ -114,6 +114,7 @@ typedef enum
 	WAIT_EVENT_PARALLEL_BITMAP_SCAN,
 	WAIT_EVENT_PARALLEL_CREATE_INDEX_SCAN,
 	WAIT_EVENT_PARALLEL_FINISH,
+	WAIT_EVENT_PARALLEL_VACUUM_FINISH,
 	WAIT_EVENT_PROCARRAY_GROUP_UPDATE,
 	WAIT_EVENT_PROC_SIGNAL_BARRIER,
 	WAIT_EVENT_PROMOTE,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fb9f936d43..df6f230715 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2021,7 +2021,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

#110Andres Freund
andres@anarazel.de
In reply to: Imseih (AWS), Sami (#109)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

On 2023-01-07 01:59:40 +0000, Imseih (AWS), Sami wrote:

--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -998,6 +998,8 @@ btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
if (info->report_progress)
pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
scanblkno);
+			if (info->report_parallel_progress && (scanblkno % REPORT_PARALLEL_VACUUM_EVERY_PAGES) == 0)
+				parallel_vacuum_update_progress(info->parallel_vacuum_state);
}
}

I still think it's wrong to need multiple pgstat_progress_*() calls within one
scan. If we really need it, it should be abstracted into a helper function
that wrapps all the vacuum progress stuff needed for an index.

@@ -688,7 +703,13 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
*/
if (nworkers > 0)
{
-		/* Wait for all vacuum workers to finish */
+		/*
+		 * Wait for all indexes to be vacuumed while
+		 * updating the parallel vacuum index progress,
+		 * and then wait for all workers to finish.
+		 */
+		parallel_vacuum_wait_to_finish(pvs);
+
WaitForParallelWorkersToFinish(pvs->pcxt);

for (int i = 0; i < pvs->pcxt->nworkers_launched; i++)

I don't think it's a good idea to have two difference routines that wait for
workers to exit. And certainly not when one of them basically just polls in a
regular interval as parallel_vacuum_wait_to_finish().

I think the problem here is that you're basically trying to work around the
lack of an asynchronous state update mechanism between leader and workers. The
workaround is to add a lot of different places that poll whether there has
been any progress. And you're not doing that by integrating with the existing
machinery for interrupt processing (i.e. CHECK_FOR_INTERRUPTS()), but by
developing a new mechanism.

I think your best bet would be to integrate with HandleParallelMessages().

Greetings,

Andres Freund

#111Imseih (AWS), Sami
simseih@amazon.com
In reply to: Andres Freund (#110)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Thanks for the feedback and I apologize for the delay in response.

I think the problem here is that you're basically trying to work around the
lack of an asynchronous state update mechanism between leader and workers. The
workaround is to add a lot of different places that poll whether there has
been any progress. And you're not doing that by integrating with the existing
machinery for interrupt processing (i.e. CHECK_FOR_INTERRUPTS()), but by
developing a new mechanism.

I think your best bet would be to integrate with HandleParallelMessages().

You are correct. I have been trying to work around the async nature
of parallel workers performing the index vacuum. As you have pointed out,
integrating with HandleParallelMessages does appear to be the proper way.
Doing so will also avoid having to do any progress updates in the index AMs.

In the attached patch, the parallel workers send a new type of protocol message
type to the leader called 'P' which signals the leader that it should handle a
progress update. The leader then performs the progress update by
invoking a callback set in the ParallelContext. This is done inside HandleParallelMessages.
In the index vacuum case, the callback is parallel_vacuum_update_progress.

The new message does not contain a payload, and it's merely used to
signal the leader that it can invoke a progress update.

Also, If the leader performs the index vacuum, it can call parallel_vacuum_update_progress
directly inside vacuumparallel.c.

Regards,

Sami Imseih
Amazon Web Services (AWS)

Attachments:

v22-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v22-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From 57f37c1fdab14557abd5a028625118e3faeebfd4 Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Thu, 12 Jan 2023 07:33:38 -0600
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 31 +++++++++++
 src/backend/access/heap/vacuumlazy.c  | 80 +++++++++++++++++++++++----
 src/backend/access/transam/parallel.c | 17 ++++++
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 58 ++++++++++++++++++-
 src/include/access/parallel.h         |  5 ++
 src/include/commands/progress.h       |  2 +
 src/include/commands/vacuum.h         |  1 +
 src/test/regress/expected/rules.out   |  4 +-
 9 files changed, 188 insertions(+), 13 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 358d2ff90f..b46c6a95aa 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1777,6 +1777,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index vacuum.</entry>
+     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -6857,6 +6861,33 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that will be vacuumed or cleaned up. This value will be
+       <literal>0</literal> if the phase is not <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal>, <literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, index vacuum is skipped due to very
+       few dead tuples in the table, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle when the
+       phase is <literal>vacuuming indexes</literal>, or the number
+       of indexes cleaned up during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 3694515167..6892f5a226 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2294,6 +2294,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEX_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEX_TOTAL,
+		PROGRESS_VACUUM_INDEX_COMPLETED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2306,9 +2317,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes
+	 * and the number of indexes to vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2321,6 +2336,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Done vacuuming an index -- increment the indexes completed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2355,14 +2374,18 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Reset and report the total indexes to vacuum and the number of
+	 * indexes vacuumed.
+	 * Also, increase and report the number of index scans completed.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2599,12 +2622,23 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int   progress_index[] = {
+			PROGRESS_VACUUM_INDEX_TOTAL,
+			PROGRESS_VACUUM_INDEX_COMPLETED
+		};
+		int64       progress_val[2] = {0, 0};
+
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation.
+		 *
+		 * Also, report that we are no longer tracking index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2632,13 +2666,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEX_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEX_TOTAL,
+		PROGRESS_VACUUM_INDEX_COMPLETED
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[2];
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes
+	 * and the number of indexes to cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2650,6 +2698,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Done cleaning an index -- increment the indexes completed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2659,6 +2711,14 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/*
+	 * Reset and report the total number of indexes to cleanup
+	 * and the number of indexes cleaned.
+	 */
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 9e3ec0d5d8..58e47d773b 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -185,6 +185,8 @@ CreateParallelContext(const char *library_name, const char *function_name,
 	pcxt->library_name = pstrdup(library_name);
 	pcxt->function_name = pstrdup(function_name);
 	pcxt->error_context_stack = error_context_stack;
+	pcxt->parallel_progress_callback = NULL;
+	pcxt->parallel_progress_callback_arg = NULL;
 	shm_toc_initialize_estimator(&pcxt->estimator);
 	dlist_push_head(&pcxt_list, &pcxt->node);
 
@@ -1199,6 +1201,21 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/*
+				 * A Leader process that receives this message
+				 * must be ready to update progress.
+				 */
+				Assert(pcxt->parallel_progress_callback);
+				Assert(pcxt->parallel_progress_callback_arg);
+
+				/* Report progress */
+				pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 447c9b970f..0ab864185f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index bcd40c80a1..3791ad3a2d 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,7 +30,9 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
+#include "libpq/pqformat.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
 #include "storage/bufmgr.h"
@@ -103,6 +105,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 nindexes_completed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -272,6 +285,8 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 								 parallel_workers);
 	Assert(pcxt->nworkers > 0);
 	pvs->pcxt = pcxt;
+	pcxt->parallel_progress_callback = parallel_vacuum_update_progress;
+	pcxt->parallel_progress_callback_arg = pvs;
 
 	/* Estimate size for index vacuum stats -- PARALLEL_VACUUM_KEY_INDEX_STATS */
 	est_indstats_len = mul_size(sizeof(PVIndStats), nindexes);
@@ -364,6 +379,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_completed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +634,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_completed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -888,6 +905,26 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Update index vacuum progress.
+	 *
+	 * When a parallel worker completes an
+	 * index vacuum, it sends a protocol message
+	 * to notify the leader. The leader then
+	 * updates the progress. See HandleParallelMessage().
+	 *
+	 * When a leader performs the index vacuum,
+	 * it can update the progress directly.
+	 */
+	if (IsParallelWorker())
+	{
+		StringInfoData msgbuf;
+
+		pq_beginmessage(&msgbuf, 'P');
+		pq_endmessage(&msgbuf);
+	} else
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1109,22 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read pvs->shared->nindexes_completed and report the number of indexes
+ * vacuumed so far.
+ *
+ * Note: This function should be called by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+
+	Assert(!IsParallelWorker());
+
+	if (pvs)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1));
+}
\ No newline at end of file
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 061f8a4c4c..7ddc71dae2 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -20,6 +20,9 @@
 #include "storage/shm_mq.h"
 #include "storage/shm_toc.h"
 
+/* progress callback definition */
+typedef void (*ParallelProgressCallback) (void *parallel_progress_callback_state);
+
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);
 
 typedef struct ParallelWorkerInfo
@@ -46,6 +49,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	ParallelProgressCallback parallel_progress_callback;
+	void		*parallel_progress_callback_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..6b8b609a4f 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 689dbb7702..7b13069d33 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -370,5 +370,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(void *arg);
 
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index fb9f936d43..df6f230715 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2021,7 +2021,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT s.stats_reset,
-- 
2.32.1 (Apple Git-133)

#112Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#111)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Jan 12, 2023 at 11:02 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Thanks for the feedback and I apologize for the delay in response.

I think the problem here is that you're basically trying to work around the
lack of an asynchronous state update mechanism between leader and workers. The
workaround is to add a lot of different places that poll whether there has
been any progress. And you're not doing that by integrating with the existing
machinery for interrupt processing (i.e. CHECK_FOR_INTERRUPTS()), but by
developing a new mechanism.

I think your best bet would be to integrate with HandleParallelMessages().

You are correct. I have been trying to work around the async nature
of parallel workers performing the index vacuum. As you have pointed out,
integrating with HandleParallelMessages does appear to be the proper way.
Doing so will also avoid having to do any progress updates in the index AMs.

Very interesting idea. I need to study the parallel query
infrastructure more to consider potential downside of this idea but it
seems okay as far as I researched so far.

In the attached patch, the parallel workers send a new type of protocol message
type to the leader called 'P' which signals the leader that it should handle a
progress update. The leader then performs the progress update by
invoking a callback set in the ParallelContext. This is done inside HandleParallelMessages.
In the index vacuum case, the callback is parallel_vacuum_update_progress.

The new message does not contain a payload, and it's merely used to
signal the leader that it can invoke a progress update.

Thank you for updating the patch. Here are some comments for v22 patch:

---
+      <para>
+       Number of indexes that will be vacuumed or cleaned up. This
value will be
+       <literal>0</literal> if the phase is not <literal>vacuuming
indexes</literal>
+       or <literal>cleaning up indexes</literal>,
<literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, index vacuum is skipped due to very
+       few dead tuples in the table, or vacuum failsafe is triggered.

I think that if INDEX_CLEANUP is set to OFF or index vacuum is skipped
due to failsafe mode, we enter neither vacuum indexes phase nor
cleanup indexes phase. So probably we can say something like:

Number of indexes that will be vacuumed or cleaned up. This counter only
advances when the phase is vacuuming indexes or cleaning up indexes.

---
-        /* Report that we are now vacuuming indexes */
-        pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-
PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+        /*
+         * Report that we are now vacuuming indexes
+         * and the number of indexes to vacuum.
+         */
+        progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+        progress_start_val[1] = vacrel->nindexes;
+        pgstat_progress_update_multi_param(2, progress_start_index,
progress_start_val);

According to our code style guideline[1]https://www.postgresql.org/docs/devel/source-format.html, we limit line lengths so
that the code is readable in an 80-column window. Some comments
updated in this patch seem too short.

---
+                StringInfoData msgbuf;
+
+                pq_beginmessage(&msgbuf, 'P');
+                pq_endmessage(&msgbuf);

I think we can use pq_putmessage() instead.

---
+/* progress callback definition */
+typedef void (*ParallelProgressCallback) (void
*parallel_progress_callback_state);

I think it's better to define "void *arg".

---
+                                /*
+                                 * A Leader process that receives this message
+                                 * must be ready to update progress.
+                                 */
+                                Assert(pcxt->parallel_progress_callback);
+                                Assert(pcxt->parallel_progress_callback_arg);
+
+                                /* Report progress */
+
pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);

I think the parallel query infra should not require
parallel_progress_callback_arg to always be set. I think it can be
NULL.

---
+void
+parallel_vacuum_update_progress(void *arg)
+{
+        ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+
+        Assert(!IsParallelWorker());
+
+        if (pvs)
+                pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+
   pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1));
+}

Since parallel vacuum always sets the arg, I think we don't need to check it.

Regards,

[1]: https://www.postgresql.org/docs/devel/source-format.html

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#113Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#112)
Re: Add index scan progress to pg_stat_progress_vacuum

Number of indexes that will be vacuumed or cleaned up. This counter only
advances when the phase is vacuuming indexes or cleaning up indexes.

I agree, this reads better.

    ---
    -        /* Report that we are now vacuuming indexes */
    -        pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
    -
    PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
    +        /*
    +         * Report that we are now vacuuming indexes
    +         * and the number of indexes to vacuum.
    +         */
    +        progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
    +        progress_start_val[1] = vacrel->nindexes;
    +        pgstat_progress_update_multi_param(2, progress_start_index,
    progress_start_val);

According to our code style guideline[1], we limit line lengths so
that the code is readable in an 80-column window. Some comments
updated in this patch seem too short.

I will correct this.

I think it's better to define "void *arg".

Agree

---
+                                /*
+                                 * A Leader process that receives this message
+                                 * must be ready to update progress.
+                                 */
+                                Assert(pcxt->parallel_progress_callback);
+                                Assert(pcxt->parallel_progress_callback_arg);
+
+                                /* Report progress */
+
pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);

I think the parallel query infra should not require
parallel_progress_callback_arg to always be set. I think it can be
NULL.

This assertion is inside the new 'P' message type handling.
If a leader is consuming this message, they must have a
progress callback set. Right now we only set the callback
in the parallel vacuum case only, so not all leaders will be prepared
to handle this case.

Would you agree this is needed for safety?

case 'P': /* Parallel progress reporting */
{
/*
* A Leader process that receives this message
* must be ready to update progress.
*/
Assert(pcxt->parallel_progress_callback);
Assert(pcxt->parallel_progress_callback_arg);

---

+void
+parallel_vacuum_update_progress(void *arg)
+{
+        ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+
+        Assert(!IsParallelWorker());
+
+        if (pvs)
+                pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+
pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1));
+}

Since parallel vacuum always sets the arg, I think we don't need to check it.

The arg is only set during parallel vacuum. During non-parallel vacuum,
It's NULL. This check can be removed, but I realize now that we do need
an Assert(pvs). Do you agree?

--

Regards,

Sami Imseih
Amazon Web Services (AWS)

#114Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#113)
Re: Add index scan progress to pg_stat_progress_vacuum

On Fri, Jan 20, 2023 at 11:38 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Number of indexes that will be vacuumed or cleaned up. This counter only
advances when the phase is vacuuming indexes or cleaning up indexes.

I agree, this reads better.

---
-        /* Report that we are now vacuuming indexes */
-        pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-
PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+        /*
+         * Report that we are now vacuuming indexes
+         * and the number of indexes to vacuum.
+         */
+        progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+        progress_start_val[1] = vacrel->nindexes;
+        pgstat_progress_update_multi_param(2, progress_start_index,
progress_start_val);

According to our code style guideline[1], we limit line lengths so
that the code is readable in an 80-column window. Some comments
updated in this patch seem too short.

I will correct this.

I think it's better to define "void *arg".

Agree

---
+                                /*
+                                 * A Leader process that receives this message
+                                 * must be ready to update progress.
+                                 */
+                                Assert(pcxt->parallel_progress_callback);
+                                Assert(pcxt->parallel_progress_callback_arg);
+
+                                /* Report progress */
+
pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);

I think the parallel query infra should not require
parallel_progress_callback_arg to always be set. I think it can be
NULL.

This assertion is inside the new 'P' message type handling.
If a leader is consuming this message, they must have a
progress callback set. Right now we only set the callback
in the parallel vacuum case only, so not all leaders will be prepared
to handle this case.

Would you agree this is needed for safety?

I think it makes sense that we assume pcxt->parallel_progress_callback
is always not NULL but my point is that in the future one might want
to use this callback without the argument and we should allow it. If
parallel vacuum assumes pcxt->parallel_progress_callback_arg is not
NULL, I think we should add an assertion in
parallel_vacuum_update_progress(), but not in HandleParallelMessage().

case 'P': /* Parallel progress reporting */
{
/*
* A Leader process that receives this message
* must be ready to update progress.
*/
Assert(pcxt->parallel_progress_callback);
Assert(pcxt->parallel_progress_callback_arg);

---

+void
+parallel_vacuum_update_progress(void *arg)
+{
+        ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+
+        Assert(!IsParallelWorker());
+
+        if (pvs)
+                pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+
pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1));
+}

Since parallel vacuum always sets the arg, I think we don't need to check it.

The arg is only set during parallel vacuum. During non-parallel vacuum,
It's NULL. This check can be removed, but I realize now that we do need
an Assert(pvs). Do you agree?

Agreed to add the assertion in this function.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#115Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#114)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

Thanks for your reply!

I addressed the latest comments in v23.

1/ cleaned up the asserts as discussed.
2/ used pq_putmessage to send the message on index scan completion.

Thanks

--
Sami Imseih
Amazon Web Services (AWS)

Attachments:

v23-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v23-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From 64258696f1e648cf5f7e05803c849c3f85be5c3a Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Tue, 7 Feb 2023 19:39:42 -0600
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Author: Sami Imseih, based on suggestions by Nathan Bossart, Peter Geoghegan and Masahiko Sawada
Reviewed by: Nathan Bossart, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 31 +++++++++++
 src/backend/access/heap/vacuumlazy.c  | 80 +++++++++++++++++++++++----
 src/backend/access/transam/parallel.c | 16 ++++++
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 55 +++++++++++++++++-
 src/include/access/parallel.h         |  5 ++
 src/include/commands/progress.h       |  2 +
 src/include/commands/vacuum.h         |  1 +
 src/test/regress/expected/rules.out   |  4 +-
 9 files changed, 184 insertions(+), 13 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 1756f1a4b6..2796d92f99 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1779,6 +1779,10 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index vacuum.</entry>
+     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -6883,6 +6887,33 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that will be vacuumed or cleaned up. This value will be
+       <literal>0</literal> if the phase is not <literal>vacuuming indexes</literal>
+       or <literal>cleaning up indexes</literal>, <literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, index vacuum is skipped due to very
+       few dead tuples in the table, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle when the
+       phase is <literal>vacuuming indexes</literal>, or the number
+       of indexes cleaned up during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 8f14cf85f3..28bac92591 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2316,6 +2316,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEX_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEX_TOTAL,
+		PROGRESS_VACUUM_INDEX_COMPLETED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2328,9 +2339,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes
+	 * and the number of indexes to vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2343,6 +2358,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Done vacuuming an index -- increment the indexes completed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2377,14 +2396,18 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Reset and report the total indexes to vacuum and the number of
+	 * indexes vacuumed.
+	 * Also, increase and report the number of index scans completed.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2621,12 +2644,23 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int   progress_index[] = {
+			PROGRESS_VACUUM_INDEX_TOTAL,
+			PROGRESS_VACUUM_INDEX_COMPLETED
+		};
+		int64       progress_val[2] = {0, 0};
+
 		vacrel->failsafe_active = true;
 
-		/* Disable index vacuuming, index cleanup, and heap rel truncation */
+		/*
+		 * Disable index vacuuming, index cleanup, and heap rel truncation.
+		 *
+		 * Also, report that we are no longer tracking index vacuum/cleanup.
+		 */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
 
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
@@ -2654,13 +2688,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEX_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEX_TOTAL,
+		PROGRESS_VACUUM_INDEX_COMPLETED
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[2];
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes
+	 * and the number of indexes to cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2672,6 +2720,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Done cleaning an index -- increment the indexes completed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2681,6 +2733,14 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/*
+	 * Reset and report the total number of indexes to cleanup
+	 * and the number of indexes cleaned.
+	 */
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 9e3ec0d5d8..a321f76999 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -185,6 +185,8 @@ CreateParallelContext(const char *library_name, const char *function_name,
 	pcxt->library_name = pstrdup(library_name);
 	pcxt->function_name = pstrdup(function_name);
 	pcxt->error_context_stack = error_context_stack;
+	pcxt->parallel_progress_callback = NULL;
+	pcxt->parallel_progress_callback_arg = NULL;
 	shm_toc_initialize_estimator(&pcxt->estimator);
 	dlist_push_head(&pcxt_list, &pcxt->node);
 
@@ -1199,6 +1201,20 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/*
+				 * A Leader process that receives this message
+				 * must be ready to update progress.
+				 */
+				Assert(pcxt->parallel_progress_callback);
+
+				/* Report progress */
+				pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 8608e3fa5b..799587dcd7 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1165,7 +1165,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_completed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index bcd40c80a1..9e5a300ba4 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,7 +30,9 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
+#include "libpq/libpq.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
 #include "storage/bufmgr.h"
@@ -103,6 +105,17 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Counter for vacuuming and cleanup progress reporting.
+	 * This value is used to report index vacuum/cleanup progress
+	 * in parallel_vacuum_progress_report. We keep this
+	 * counter to avoid having to loop through
+	 * ParallelVacuumState->indstats to determine the number
+	 * of indexes completed.
+	 */
+	pg_atomic_uint32 nindexes_completed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -272,6 +285,8 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 								 parallel_workers);
 	Assert(pcxt->nworkers > 0);
 	pvs->pcxt = pcxt;
+	pcxt->parallel_progress_callback = parallel_vacuum_update_progress;
+	pcxt->parallel_progress_callback_arg = pvs;
 
 	/* Estimate size for index vacuum stats -- PARALLEL_VACUUM_KEY_INDEX_STATS */
 	est_indstats_len = mul_size(sizeof(PVIndStats), nindexes);
@@ -364,6 +379,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_completed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +634,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_completed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -888,6 +905,22 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Update index vacuum progress.
+	 *
+	 * When a parallel worker completes an
+	 * index vacuum, it sends a protocol message
+	 * to notify the leader. The leader then
+	 * updates the progress. See HandleParallelMessage().
+	 *
+	 * When a leader performs the index vacuum,
+	 * it can update the progress directly.
+	 */
+	if (IsParallelWorker())
+		pq_putmessage('P', NULL, 0);
+	else
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1105,23 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Read pvs->shared->nindexes_completed and report the number of indexes
+ * vacuumed so far.
+ *
+ * Note: This function should be called by the leader process only,
+ * and it's up to the caller to ensure this.
+ */
+void
+parallel_vacuum_update_progress(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+
+	Assert(!IsParallelWorker());
+	Assert(pvs->pcxt->parallel_progress_callback_arg);
+
+	if (pvs)
+		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+									 pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1));
+}
\ No newline at end of file
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 061f8a4c4c..7ddc71dae2 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -20,6 +20,9 @@
 #include "storage/shm_mq.h"
 #include "storage/shm_toc.h"
 
+/* progress callback definition */
+typedef void (*ParallelProgressCallback) (void *parallel_progress_callback_state);
+
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);
 
 typedef struct ParallelWorkerInfo
@@ -46,6 +49,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	ParallelProgressCallback parallel_progress_callback;
+	void		*parallel_progress_callback_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..6b8b609a4f 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_COMPLETED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 689dbb7702..7b13069d33 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -370,5 +370,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(void *arg);
 
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index e7a2f5856a..a20978d335 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2021,7 +2021,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_completed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
-- 
2.37.1 (Apple Git-137.1)

#116Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#115)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Feb 8, 2023 at 11:03 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Hi,

Thanks for your reply!

I addressed the latest comments in v23.

1/ cleaned up the asserts as discussed.
2/ used pq_putmessage to send the message on index scan completion.

Thank you for updating the patch! Here are some comments for v23 patch:

+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index
vacuum.</entry>
+     </row>

This change is out-of-date.

---
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes that will be vacuumed or cleaned up. This
value will be
+       <literal>0</literal> if the phase is not <literal>vacuuming
indexes</literal>
+       or <literal>cleaning up indexes</literal>,
<literal>INDEX_CLEANUP</literal>
+       is set to <literal>OFF</literal>, index vacuum is skipped due to very
+       few dead tuples in the table, or vacuum failsafe is triggered.
+       See <xref linkend="guc-vacuum-failsafe-age"/>
+       for more on vacuum failsafe.
+      </para></entry>
+     </row>

This explanation looks redundant: setting INDEX_CLEANUP to OFF
essentially means the vacuum doesn't enter the vacuuming indexes
phase. The same is true for the case of skipping index vacuum and
failsafe mode. How about the following?

Total number of indexes that will be vacuumed or cleaned up. This
number is reported as of the beginning of the vacuuming indexes phase
or the cleaning up indexes phase.

---
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_completed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes vacuumed in the current vacuum cycle when the
+       phase is <literal>vacuuming indexes</literal>, or the number
+       of indexes cleaned up during the <literal>cleaning up indexes</literal>
+       phase.
+      </para></entry>
+     </row>

How about simplfyy the description to something like:

Number of indexes processed. This counter only advances when the phase
is vacuuming indexes or cleaning up indexes.

Also, index_processed sounds accurate to me. What do you think?

---
+        pcxt->parallel_progress_callback = NULL;
+        pcxt->parallel_progress_callback_arg = NULL;

I think these settings are not necessary since the pcxt is palloc0'ed.

---
+void
+parallel_vacuum_update_progress(void *arg)
+{
+        ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+
+        Assert(!IsParallelWorker());
+        Assert(pvs->pcxt->parallel_progress_callback_arg);
+
+        if (pvs)
+                pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+
   pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1));
+}

Assert(pvs->pcxt->parallel_progress_callback_arg) looks wrong to me.
If 'arg' is NULL, a SEGV happens.

I think it's better to update pvs->shared->nindexes_completed by both
leader and worker processes who processed the index.

---
+/* progress callback definition */
+typedef void (*ParallelProgressCallback) (void
*parallel_progress_callback_state);
+
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);

I think it's better to make the function type consistent with the
existing parallel_worker_main_type. How about
parallel_progress_callback_type?

I've attached a patch that incorporates the above comments and has
some suggestions of updating comments etc.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

Attachments:

fix_v23_masahiko.patchapplication/octet-stream; name=fix_v23_masahiko.patchDownload
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 136b441e0c..02ebb718d7 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -1790,10 +1790,6 @@ postgres   27093  0.0  0.0  30096  2752 ?        Ss   11:34   0:00 postgres: ser
       <entry><literal>ParallelFinish</literal></entry>
       <entry>Waiting for parallel workers to finish computing.</entry>
      </row>
-     <row>
-      <entry><literal>ParallelVacuumFinish</literal></entry>
-      <entry>Waiting for parallel vacuum workers to finish index vacuum.</entry>
-     </row>
      <row>
       <entry><literal>ProcArrayGroupUpdate</literal></entry>
       <entry>Waiting for the group leader to clear the transaction ID at
@@ -7193,25 +7189,19 @@ FROM pg_stat_get_backend_idset() AS backendid;
        <structfield>indexes_total</structfield> <type>bigint</type>
       </para>
       <para>
-       Number of indexes that will be vacuumed or cleaned up. This value will be
-       <literal>0</literal> if the phase is not <literal>vacuuming indexes</literal>
-       or <literal>cleaning up indexes</literal>, <literal>INDEX_CLEANUP</literal>
-       is set to <literal>OFF</literal>, index vacuum is skipped due to very
-       few dead tuples in the table, or vacuum failsafe is triggered.
-       See <xref linkend="guc-vacuum-failsafe-age"/>
-       for more on vacuum failsafe.
+       Total number of indexes that will be vacuumed or cleaned up. This number is
+       reported as of the beginning of the <literal>vacuuming indexes</literal> phase
+       or the <literal>cleaning up indexes</literal> phase.
       </para></entry>
      </row>
 
      <row>
       <entry role="catalog_table_entry"><para role="column_definition">
-       <structfield>indexes_completed</structfield> <type>bigint</type>
+       <structfield>indexes_processed</structfield> <type>bigint</type>
       </para>
       <para>
-       Number of indexes vacuumed in the current vacuum cycle when the
-       phase is <literal>vacuuming indexes</literal>, or the number
-       of indexes cleaned up during the <literal>cleaning up indexes</literal>
-       phase.
+       Number of indexes processed. This counter only advances when the phase is
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>.
       </para></entry>
      </row>
     </tbody>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 28bac92591..8ede3c1956 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2322,7 +2322,7 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	};
 	const int progress_end_index[] = {
 		PROGRESS_VACUUM_INDEX_TOTAL,
-		PROGRESS_VACUUM_INDEX_COMPLETED,
+		PROGRESS_VACUUM_INDEX_PROCESSED,
 		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
 	};
 	int64       progress_start_val[2];
@@ -2340,8 +2340,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	}
 
 	/*
-	 * Report that we are now vacuuming indexes
-	 * and the number of indexes to vacuum.
+	 * Report that we are now vacuuming indexes and the number of indexes
+	 * to vacuum.
 	 */
 	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
 	progress_start_val[1] = vacrel->nindexes;
@@ -2358,8 +2358,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
-			/* Done vacuuming an index -- increment the indexes completed */
-			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_PROCESSED,
 										 idx + 1);
 
 			if (lazy_check_wraparound_failsafe(vacrel))
@@ -2396,9 +2396,8 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Reset and report the total indexes to vacuum and the number of
-	 * indexes vacuumed.
-	 * Also, increase and report the number of index scans completed.
+	 * Increase and report the number of index. Also, we reset the progress
+	 * counters.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
@@ -2646,20 +2645,18 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 	{
 		const int   progress_index[] = {
 			PROGRESS_VACUUM_INDEX_TOTAL,
-			PROGRESS_VACUUM_INDEX_COMPLETED
+			PROGRESS_VACUUM_INDEX_PROCESSED
 		};
 		int64       progress_val[2] = {0, 0};
 
 		vacrel->failsafe_active = true;
 
-		/*
-		 * Disable index vacuuming, index cleanup, and heap rel truncation.
-		 *
-		 * Also, report that we are no longer tracking index vacuum/cleanup.
-		 */
+		/* Disable index vacuuming, index cleanup, and heap rel truncation */
 		vacrel->do_index_vacuuming = false;
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
+
+		/* Reset the progress counters */
 		pgstat_progress_update_multi_param(2, progress_index, progress_val);
 
 		ereport(WARNING,
@@ -2694,17 +2691,17 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 	};
 	const int progress_end_index[] = {
 		PROGRESS_VACUUM_INDEX_TOTAL,
-		PROGRESS_VACUUM_INDEX_COMPLETED
+		PROGRESS_VACUUM_INDEX_PROCESSED
 	};
 	int64       progress_start_val[2];
-	int64       progress_end_val[2];
+	int64       progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
 	/*
-	 * Report that we are now cleaning up indexes
-	 * and the number of indexes to cleanup.
+	 * Report that we are now cleaning up indexes and the number of indexes
+	 * to cleanup.
 	 */
 	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
 	progress_start_val[1] = vacrel->nindexes;
@@ -2721,8 +2718,8 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
 
-			/* Done cleaning an index -- increment the indexes completed */
-			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_PROCESSED,
 										 idx + 1);
 		}
 	}
@@ -2734,12 +2731,7 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											estimated_count);
 	}
 
-	/*
-	 * Reset and report the total number of indexes to cleanup
-	 * and the number of indexes cleaned.
-	 */
-	progress_end_val[0] = 0;
-	progress_end_val[1] = 0;
+	/* Reset the progress counters */
 	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 66fd3a0730..b0c406fe7a 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -185,8 +185,6 @@ CreateParallelContext(const char *library_name, const char *function_name,
 	pcxt->library_name = pstrdup(library_name);
 	pcxt->function_name = pstrdup(function_name);
 	pcxt->error_context_stack = error_context_stack;
-	pcxt->parallel_progress_callback = NULL;
-	pcxt->parallel_progress_callback_arg = NULL;
 	shm_toc_initialize_estimator(&pcxt->estimator);
 	dlist_push_head(&pcxt_list, &pcxt->node);
 
@@ -1203,13 +1201,8 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 
 		case 'P':				/* Parallel progress reporting */
 			{
-				/*
-				 * A Leader process that receives this message
-				 * must be ready to update progress.
-				 */
+				/* Call the progress reporting callback */
 				Assert(pcxt->parallel_progress_callback);
-
-				/* Report progress */
 				pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
 
 				break;
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index fc642bc25e..5886ee8b7c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1181,7 +1181,7 @@ CREATE VIEW pg_stat_progress_vacuum AS
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
         S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
-        S.param8 AS indexes_total, S.param9 AS indexes_completed
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 9e5a300ba4..57dc5fd8f0 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -107,14 +107,11 @@ typedef struct PVShared
 	pg_atomic_uint32 idx;
 
 	/*
-	 * Counter for vacuuming and cleanup progress reporting.
-	 * This value is used to report index vacuum/cleanup progress
-	 * in parallel_vacuum_progress_report. We keep this
-	 * counter to avoid having to loop through
-	 * ParallelVacuumState->indstats to determine the number
-	 * of indexes completed.
+	 * Number of indexes processed in a parallel index bulk-deletion or a
+	 * parallel index cleanup. This counter is used to report the progress
+	 * information.
 	 */
-	pg_atomic_uint32 nindexes_completed;
+	pg_atomic_uint32 nindexes_processed;
 
 } PVShared;
 
@@ -284,10 +281,13 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pcxt = CreateParallelContext("postgres", "parallel_vacuum_main",
 								 parallel_workers);
 	Assert(pcxt->nworkers > 0);
-	pvs->pcxt = pcxt;
+
+	/* Setup callback for updating the progress information */
 	pcxt->parallel_progress_callback = parallel_vacuum_update_progress;
 	pcxt->parallel_progress_callback_arg = pvs;
 
+	pvs->pcxt = pcxt;
+
 	/* Estimate size for index vacuum stats -- PARALLEL_VACUUM_KEY_INDEX_STATS */
 	est_indstats_len = mul_size(sizeof(PVIndStats), nindexes);
 	shm_toc_estimate_chunk(&pcxt->estimator, est_indstats_len);
@@ -379,7 +379,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
-	pg_atomic_init_u32(&(shared->nindexes_completed), 0);
+	pg_atomic_init_u32(&(shared->nindexes_processed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -636,7 +636,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 
 	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
-	pg_atomic_write_u32(&(pvs->shared->nindexes_completed), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_processed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -907,16 +907,10 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->indname = NULL;
 
 	/*
-	 * Update index vacuum progress.
-	 *
-	 * When a parallel worker completes an
-	 * index vacuum, it sends a protocol message
-	 * to notify the leader. The leader then
-	 * updates the progress. See HandleParallelMessage().
-	 *
-	 * When a leader performs the index vacuum,
-	 * it can update the progress directly.
+	 * Update the index vacuum progress information. Since the progress is
+	 * updated only by the leader, the worker notifies the leader of it.
 	 */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_processed), 1);
 	if (IsParallelWorker())
 		pq_putmessage('P', NULL, 0);
 	else
@@ -1107,21 +1101,17 @@ parallel_vacuum_error_callback(void *arg)
 }
 
 /*
- * Read pvs->shared->nindexes_completed and report the number of indexes
- * vacuumed so far.
- *
- * Note: This function should be called by the leader process only,
- * and it's up to the caller to ensure this.
+ * Update the number of indexes processed so far in the current index bulk-deletion
+ * or index cleanup.
  */
 void
 parallel_vacuum_update_progress(void *arg)
 {
-	ParallelVacuumState *pvs = (ParallelVacuumState *)arg;
+	ParallelVacuumState *pvs = (ParallelVacuumState *) arg;
 
 	Assert(!IsParallelWorker());
-	Assert(pvs->pcxt->parallel_progress_callback_arg);
+	Assert(pvs);
 
-	if (pvs)
-		pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_COMPLETED,
-									 pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_completed), 1));
-}
\ No newline at end of file
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_PROCESSED,
+								 pg_atomic_read_u32(&(pvs->shared->nindexes_processed)));
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 7ddc71dae2..f621d51c0d 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -20,10 +20,8 @@
 #include "storage/shm_mq.h"
 #include "storage/shm_toc.h"
 
-/* progress callback definition */
-typedef void (*ParallelProgressCallback) (void *parallel_progress_callback_state);
-
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);
+typedef void (*parallel_progress_callback_type) (void *arg);
 
 typedef struct ParallelWorkerInfo
 {
@@ -49,7 +47,7 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
-	ParallelProgressCallback parallel_progress_callback;
+	parallel_progress_callback_type parallel_progress_callback;
 	void		*parallel_progress_callback_arg;
 } ParallelContext;
 
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index 6b8b609a4f..23c38f2d0e 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -26,7 +26,7 @@
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
 #define PROGRESS_VACUUM_INDEX_TOTAL             7
-#define PROGRESS_VACUUM_INDEX_COMPLETED         8
+#define PROGRESS_VACUUM_INDEX_PROCESSED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index c1b32dfa20..bca4da7036 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2035,7 +2035,7 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param6 AS max_dead_tuples,
     s.param7 AS num_dead_tuples,
     s.param8 AS indexes_total,
-    s.param9 AS indexes_completed
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
#117Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#116)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Thanks for the review!

+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index
vacuum.</entry>
+     </row>

This change is out-of-date.

That was an oversight. Thanks for catching.

Total number of indexes that will be vacuumed or cleaned up. This
number is reported as of the beginning of the vacuuming indexes phase
or the cleaning up indexes phase.

This is cleaner. I was being unnecessarily verbose in the original description.

Number of indexes processed. This counter only advances when the phase
is vacuuming indexes or cleaning up indexes.

I agree.

Also, index_processed sounds accurate to me. What do you think?

At one point, II used index_processed, but decided to change it.
"processed" makes sense also. I will use this.

I think these settings are not necessary since the pcxt is palloc0'ed.

Good point.

Assert(pvs->pcxt->parallel_progress_callback_arg) looks wrong to me.
If 'arg' is NULL, a SEGV happens.

Correct, Assert(pvs) is all that is needed.

I think it's better to update pvs->shared->nindexes_completed by both
leader and worker processes who processed the index.

No reason for that, since only the leader process can report process to
backend_progress.

I think it's better to make the function type consistent with the
existing parallel_worker_main_type. How about
parallel_progress_callback_type?

Yes, that makes sense.

I've attached a patch that incorporates the above comments and has
some suggestions of updating comments etc.

I reviewed and incorporated these changes, with a slight change. See v24.

- * Increase and report the number of index. Also, we reset the progress
- * counters.

+        * Increase and report the number of index scans. Also, we reset the progress
+        * counters.

Thanks

--
Sami Imseih
Amazon Web Services (AWS)

Attachments:

v24-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v24-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From 3e146938fca29e2ace9bb86f60ac346f4312534e Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Fri, 17 Feb 2023 19:34:02 -0600
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Authors: Sami Imseih
Reviewed by: Masahiko Sawada, Nathan Bossart, Andres Freund
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 21 ++++++++
 src/backend/access/heap/vacuumlazy.c  | 70 +++++++++++++++++++++++----
 src/backend/access/transam/parallel.c |  9 ++++
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 45 ++++++++++++++++-
 src/include/access/parallel.h         |  3 ++
 src/include/commands/progress.h       |  2 +
 src/include/commands/vacuum.h         |  1 +
 src/test/regress/expected/rules.out   |  4 +-
 9 files changed, 146 insertions(+), 12 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index b0b997f092..02ebb718d7 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -7183,6 +7183,27 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Total number of indexes that will be vacuumed or cleaned up. This number is
+       reported as of the beginning of the <literal>vacuuming indexes</literal> phase
+       or the <literal>cleaning up indexes</literal> phase.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes processed. This counter only advances when the phase is
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 8f14cf85f3..443a44d6c9 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2316,6 +2316,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEX_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEX_TOTAL,
+		PROGRESS_VACUUM_INDEX_PROCESSED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2328,9 +2339,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes and the number of indexes
+	 * to vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2343,6 +2358,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_PROCESSED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2377,14 +2396,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Increase and report the number of index scans. Also, we reset the progress
+	 * counters.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2621,6 +2643,12 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int   progress_index[] = {
+			PROGRESS_VACUUM_INDEX_TOTAL,
+			PROGRESS_VACUUM_INDEX_PROCESSED
+		};
+		int64       progress_val[2] = {0, 0};
+
 		vacrel->failsafe_active = true;
 
 		/* Disable index vacuuming, index cleanup, and heap rel truncation */
@@ -2628,6 +2656,9 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Reset the progress counters */
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						vacrel->dbname, vacrel->relnamespace, vacrel->relname,
@@ -2654,13 +2685,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEX_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEX_TOTAL,
+		PROGRESS_VACUUM_INDEX_PROCESSED
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes and the number of indexes
+	 * to cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2672,6 +2717,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_PROCESSED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2681,6 +2730,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Reset the progress counters */
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index b26f2a64fb..b0c406fe7a 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -1199,6 +1199,15 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/* Call the progress reporting callback */
+				Assert(pcxt->parallel_progress_callback);
+				pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 34ca0e739f..5886ee8b7c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1180,7 +1180,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index bcd40c80a1..57dc5fd8f0 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,7 +30,9 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
+#include "libpq/libpq.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
 #include "storage/bufmgr.h"
@@ -103,6 +105,14 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Number of indexes processed in a parallel index bulk-deletion or a
+	 * parallel index cleanup. This counter is used to report the progress
+	 * information.
+	 */
+	pg_atomic_uint32 nindexes_processed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -271,6 +281,11 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pcxt = CreateParallelContext("postgres", "parallel_vacuum_main",
 								 parallel_workers);
 	Assert(pcxt->nworkers > 0);
+
+	/* Setup callback for updating the progress information */
+	pcxt->parallel_progress_callback = parallel_vacuum_update_progress;
+	pcxt->parallel_progress_callback_arg = pvs;
+
 	pvs->pcxt = pcxt;
 
 	/* Estimate size for index vacuum stats -- PARALLEL_VACUUM_KEY_INDEX_STATS */
@@ -364,6 +379,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_processed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +634,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_processed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -888,6 +905,16 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Update the index vacuum progress information. Since the progress is
+	 * updated only by the leader, the worker notifies the leader of it.
+	 */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_processed), 1);
+	if (IsParallelWorker())
+		pq_putmessage('P', NULL, 0);
+	else
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1099,19 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Update the number of indexes processed so far in the current index bulk-deletion
+ * or index cleanup.
+ */
+void
+parallel_vacuum_update_progress(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *) arg;
+
+	Assert(!IsParallelWorker());
+	Assert(pvs);
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEX_PROCESSED,
+								 pg_atomic_read_u32(&(pvs->shared->nindexes_processed)));
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 061f8a4c4c..f621d51c0d 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -21,6 +21,7 @@
 #include "storage/shm_toc.h"
 
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);
+typedef void (*parallel_progress_callback_type) (void *arg);
 
 typedef struct ParallelWorkerInfo
 {
@@ -46,6 +47,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	parallel_progress_callback_type parallel_progress_callback;
+	void		*parallel_progress_callback_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..23c38f2d0e 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_PROCESSED         8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 689dbb7702..7b13069d33 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -370,5 +370,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(void *arg);
 
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index a3a5a62329..b1badc485d 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2033,7 +2033,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
-- 
2.37.1 (Apple Git-137.1)

#118Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#117)
Re: Add index scan progress to pg_stat_progress_vacuum

On Sat, Feb 18, 2023 at 11:46 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Thanks for the review!

+     <row>
+      <entry><literal>ParallelVacuumFinish</literal></entry>
+      <entry>Waiting for parallel vacuum workers to finish index
vacuum.</entry>
+     </row>

This change is out-of-date.

That was an oversight. Thanks for catching.

Total number of indexes that will be vacuumed or cleaned up. This
number is reported as of the beginning of the vacuuming indexes phase
or the cleaning up indexes phase.

This is cleaner. I was being unnecessarily verbose in the original description.

Number of indexes processed. This counter only advances when the phase
is vacuuming indexes or cleaning up indexes.

I agree.

Also, index_processed sounds accurate to me. What do you think?

At one point, II used index_processed, but decided to change it.
"processed" makes sense also. I will use this.

I think these settings are not necessary since the pcxt is palloc0'ed.

Good point.

Assert(pvs->pcxt->parallel_progress_callback_arg) looks wrong to me.
If 'arg' is NULL, a SEGV happens.

Correct, Assert(pvs) is all that is needed.

I think it's better to update pvs->shared->nindexes_completed by both
leader and worker processes who processed the index.

No reason for that, since only the leader process can report process to
backend_progress.

I think it's better to make the function type consistent with the
existing parallel_worker_main_type. How about
parallel_progress_callback_type?

Yes, that makes sense.

I've attached a patch that incorporates the above comments and has
some suggestions of updating comments etc.

I reviewed and incorporated these changes, with a slight change. See v24.

- * Increase and report the number of index. Also, we reset the progress
- * counters.

+        * Increase and report the number of index scans. Also, we reset the progress
+        * counters.

Thanks

Thanks for updating the patch!

 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS              4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES                        5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES                        6
+#define PROGRESS_VACUUM_INDEX_TOTAL             7
+#define PROGRESS_VACUUM_INDEX_PROCESSED         8
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed

I think PROGRESS_VACUUM_INDEXES_TOTAL and
PROGRESS_VACUUM_INDEXES_PROCESSED are better for consistency. The rest
looks good to me.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#119Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#118)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

Thanks!

I think PROGRESS_VACUUM_INDEXES_TOTAL and
PROGRESS_VACUUM_INDEXES_PROCESSED are better for consistency. The rest
looks good to me.

Took care of that in v25.

Regards

--
Sami Imseih
Amazon Web Services

Attachments:

v25-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v25-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From 629f12b3e959e34a634f45e6f2dfe5e5fba7343a Mon Sep 17 00:00:00 2001
From: "Imseih (AWS)" <simseih@88665a22795f.ant.amazon.com>
Date: Mon, 20 Feb 2023 09:56:37 -0600
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Authors: Sami Imseih
Reviewed by: Masahiko Sawada, Nathan Bossart, Andres Freund
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 21 ++++++++
 src/backend/access/heap/vacuumlazy.c  | 70 +++++++++++++++++++++++----
 src/backend/access/transam/parallel.c |  9 ++++
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 45 ++++++++++++++++-
 src/include/access/parallel.h         |  3 ++
 src/include/commands/progress.h       |  2 +
 src/include/commands/vacuum.h         |  1 +
 src/test/regress/expected/rules.out   |  4 +-
 9 files changed, 146 insertions(+), 12 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index b0b997f092..02ebb718d7 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -7183,6 +7183,27 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Total number of indexes that will be vacuumed or cleaned up. This number is
+       reported as of the beginning of the <literal>vacuuming indexes</literal> phase
+       or the <literal>cleaning up indexes</literal> phase.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes processed. This counter only advances when the phase is
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 8f14cf85f3..2518dcba1e 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2316,6 +2316,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2328,9 +2339,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes and the number of indexes
+	 * to vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2343,6 +2358,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2377,14 +2396,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Increase and report the number of index scans. Also, we reset the progress
+	 * counters.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2621,6 +2643,12 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int   progress_index[] = {
+			PROGRESS_VACUUM_INDEXES_TOTAL,
+			PROGRESS_VACUUM_INDEXES_PROCESSED
+		};
+		int64       progress_val[2] = {0, 0};
+
 		vacrel->failsafe_active = true;
 
 		/* Disable index vacuuming, index cleanup, and heap rel truncation */
@@ -2628,6 +2656,9 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Reset the progress counters */
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						vacrel->dbname, vacrel->relnamespace, vacrel->relname,
@@ -2654,13 +2685,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes and the number of indexes
+	 * to cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2672,6 +2717,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2681,6 +2730,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Reset the progress counters */
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index b26f2a64fb..b0c406fe7a 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -1199,6 +1199,15 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/* Call the progress reporting callback */
+				Assert(pcxt->parallel_progress_callback);
+				pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 34ca0e739f..5886ee8b7c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1180,7 +1180,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index bcd40c80a1..8944f065bc 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,7 +30,9 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
+#include "libpq/libpq.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
 #include "storage/bufmgr.h"
@@ -103,6 +105,14 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Number of indexes processed in a parallel index bulk-deletion or a
+	 * parallel index cleanup. This counter is used to report the progress
+	 * information.
+	 */
+	pg_atomic_uint32 nindexes_processed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -271,6 +281,11 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pcxt = CreateParallelContext("postgres", "parallel_vacuum_main",
 								 parallel_workers);
 	Assert(pcxt->nworkers > 0);
+
+	/* Setup callback for updating the progress information */
+	pcxt->parallel_progress_callback = parallel_vacuum_update_progress;
+	pcxt->parallel_progress_callback_arg = pvs;
+
 	pvs->pcxt = pcxt;
 
 	/* Estimate size for index vacuum stats -- PARALLEL_VACUUM_KEY_INDEX_STATS */
@@ -364,6 +379,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_processed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -618,8 +634,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_processed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -888,6 +905,16 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Update the index vacuum progress information. Since the progress is
+	 * updated only by the leader, the worker notifies the leader of it.
+	 */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_processed), 1);
+	if (IsParallelWorker())
+		pq_putmessage('P', NULL, 0);
+	else
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1072,3 +1099,19 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Update the number of indexes processed so far in the current index bulk-deletion
+ * or index cleanup.
+ */
+void
+parallel_vacuum_update_progress(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *) arg;
+
+	Assert(!IsParallelWorker());
+	Assert(pvs);
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+								 pg_atomic_read_u32(&(pvs->shared->nindexes_processed)));
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 061f8a4c4c..f621d51c0d 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -21,6 +21,7 @@
 #include "storage/shm_toc.h"
 
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);
+typedef void (*parallel_progress_callback_type) (void *arg);
 
 typedef struct ParallelWorkerInfo
 {
@@ -46,6 +47,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	parallel_progress_callback_type parallel_progress_callback;
+	void		*parallel_progress_callback_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..2478e87425 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 689dbb7702..7b13069d33 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -370,5 +370,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(void *arg);
 
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index a3a5a62329..b1badc485d 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2033,7 +2033,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
-- 
2.37.1 (Apple Git-137.1)

#120Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#119)
Re: Add index scan progress to pg_stat_progress_vacuum

On Tue, Feb 21, 2023 at 1:48 AM Imseih (AWS), Sami <simseih@amazon.com> wrote:

Thanks!

I think PROGRESS_VACUUM_INDEXES_TOTAL and
PROGRESS_VACUUM_INDEXES_PROCESSED are better for consistency. The rest
looks good to me.

Took care of that in v25.

Thanks! It looks good to me so I've marked it as Ready for Committer.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#121Michael Paquier
michael@paquier.xyz
In reply to: Masahiko Sawada (#120)
Re: Add index scan progress to pg_stat_progress_vacuum

On Fri, Feb 24, 2023 at 03:16:10PM +0900, Masahiko Sawada wrote:

Thanks! It looks good to me so I've marked it as Ready for Committer.

+       case 'P':               /* Parallel progress reporting */
+           {
+               /* Call the progress reporting callback */
+               Assert(pcxt->parallel_progress_callback);
+               pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+
+               break;
+           }

The key point of the patch is here. From what I understand based on
the information of the thread, this is used as a way to make the
progress reporting done by the leader more responsive so as we'd
update the index counters each time the leader is poked at with a 'P'
message by one of its workers, once a worker is done with the parallel
cleanup of one of the indexes. That's appealing, because this design
is responsive and cheap, while we'd rely on CFIs to make sure that the
leader triggers its callback on a timely basis. Unfortunately,
sticking a concept of "Parallel progress reporting" is rather
confusing here? This stuff can be used for much more purposes than
just progress reporting: the leader would execute the callback it has
registered based on the timing given by one or more of its workers,
these willing to push an event on the leader. Progress reporting is
one application of that to force a refresh and make the information of
the leader accurate. What about things like a chain of callbacks, for
example? Could the leader want to register more than one callback and
act on all of them with one single P message?

Another question I have: could the reporting of each individual worker
make sense on its own? The cleanup of the indexes depends on the
order they are processed, their number, size and AM with their cleanup
strategy, still there may be a point in getting information about how
much work a single worker is doing rather than just have the
aggregated information given to the leader?

Btw, Is an assertion really helpful here? If
parallel_progress_callback is not set, we'd just crash one line
after. It seems to me that it could be cleaner to do nothing if a
leader gets a poke message from a worker if there are no callbacks
registered.
--
Michael

#122Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Michael Paquier (#121)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Apr 5, 2023 at 4:47 PM Michael Paquier <michael@paquier.xyz> wrote:

On Fri, Feb 24, 2023 at 03:16:10PM +0900, Masahiko Sawada wrote:

Thanks! It looks good to me so I've marked it as Ready for Committer.

+       case 'P':               /* Parallel progress reporting */
+           {
+               /* Call the progress reporting callback */
+               Assert(pcxt->parallel_progress_callback);
+               pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+
+               break;
+           }

The key point of the patch is here. From what I understand based on
the information of the thread, this is used as a way to make the
progress reporting done by the leader more responsive so as we'd
update the index counters each time the leader is poked at with a 'P'
message by one of its workers, once a worker is done with the parallel
cleanup of one of the indexes. That's appealing, because this design
is responsive and cheap, while we'd rely on CFIs to make sure that the
leader triggers its callback on a timely basis. Unfortunately,
sticking a concept of "Parallel progress reporting" is rather
confusing here? This stuff can be used for much more purposes than
just progress reporting: the leader would execute the callback it has
registered based on the timing given by one or more of its workers,
these willing to push an event on the leader. Progress reporting is
one application of that to force a refresh and make the information of
the leader accurate. What about things like a chain of callbacks, for
example? Could the leader want to register more than one callback and
act on all of them with one single P message?

That seems a valid argument. I was thinking that such an asynchronous
state update mechanism would be a good infrastructure for progress
reporting of parallel operations. It might be worth considering to use
it in more general usage but since the current implementation is
minimal can we extend it in the future when we need it for other use
cases?

Another question I have: could the reporting of each individual worker
make sense on its own? The cleanup of the indexes depends on the
order they are processed, their number, size and AM with their cleanup
strategy, still there may be a point in getting information about how
much work a single worker is doing rather than just have the
aggregated information given to the leader?

It would also be useful information for users but I don't think it can
alternate the aggregated information. The aggregated information can
answer the question from the user like "how many indexes to vacuum are
remaining?", which helps estimate the remaining time to complete.

Btw, Is an assertion really helpful here? If
parallel_progress_callback is not set, we'd just crash one line
after. It seems to me that it could be cleaner to do nothing if a
leader gets a poke message from a worker if there are no callbacks
registered.

Agreed.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

#123Imseih (AWS), Sami
simseih@amazon.com
In reply to: Masahiko Sawada (#122)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

The key point of the patch is here. From what I understand based on
the information of the thread, this is used as a way to make the

progress reporting done by the leader more responsive so as we'd

update the index counters each time the leader is poked at with a 'P'
message by one of its workers, once a worker is done with the parallel
cleanup of one of the indexes. That's appealing, because this design
is responsive and cheap, while we'd rely on CFIs to make sure that the
leader triggers its callback on a timely basis. Unfortunately,
sticking a concept of "Parallel progress reporting" is rather
confusing here? This stuff can be used for much more purposes than
just progress reporting: the leader would execute the callback it has
registered based on the timing given by one or more of its workers,
these willing to push an event on the leader. Progress reporting is
one application of that to force a refresh and make the information of
the leader accurate. What about things like a chain of callbacks, for
example? Could the leader want to register more than one callback and
act on all of them with one single P message?

That seems a valid argument. I was thinking that such an asynchronous
state update mechanism would be a good infrastructure for progress
reporting of parallel operations. It might be worth considering to use
it in more general usage but since the current implementation is
minimal can we extend it in the future when we need it for other use
cases?

I don't think we should delay this patch to design a more general
infrastructure. I agree this can be handled by a future requirement.

Another question I have: could the reporting of each individual worker
make sense on its own? The cleanup of the indexes depends on the
order they are processed, their number, size and AM with their cleanup
strategy, still there may be a point in getting information about how
much work a single worker is doing rather than just have the
aggregated information given to the leader?

It would also be useful information for users but I don't think it can
alternate the aggregated information. The aggregated information can
answer the question from the user like "how many indexes to vacuum are
remaining?", which helps estimate the remaining time to complete.

The original intention of the thread was to expose stats for both
aggregate (leader level) and individual index progress. Both the aggregate
and individual indexes information have benefit as mentioned by Swada-San.

For the individual index progress, a suggested patch was suggested earlier in
the thread, v1-0001-Function-to-return-currently-vacuumed-or-cleaned-ind.patch.

However, since this particular thread has focused mainly on the aggregated stats work,
my thoughts have been to start a new thread for the individual index progress
once this gets committed.

Btw, Is an assertion really helpful here? If
parallel_progress_callback is not set, we'd just crash one line
after. It seems to me that it could be cleaner to do nothing if a
leader gets a poke message from a worker if there are no callbacks
registered.

Agreed.

I removed the assert and added an if condition instead.

See the attached v26 please.

Regards,

--
Sami Imseih
Amazon Web Services (AWS)

Attachments:

v26-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchapplication/octet-stream; name=v26-0001-Add-2-new-columns-to-pg_stat_progress_vacuum.-Th.patchDownload
From d837126ed19fd21fda47af7fed5425fdc3d4bf3b Mon Sep 17 00:00:00 2001
From: EC2 Default User <ec2-user@ip-172-31-26-221.ec2.internal>
Date: Wed, 5 Apr 2023 14:14:26 +0000
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Authors: Sami Imseih
Reviewed by: Masahiko Sawada, Nathan Bossart, Andres Freund
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 21 ++++++++
 src/backend/access/heap/vacuumlazy.c  | 70 +++++++++++++++++++++++----
 src/backend/access/transam/parallel.c |  9 ++++
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c | 45 ++++++++++++++++-
 src/include/access/parallel.h         |  3 ++
 src/include/commands/progress.h       |  2 +
 src/include/commands/vacuum.h         |  1 +
 src/test/regress/expected/rules.out   |  4 +-
 9 files changed, 146 insertions(+), 12 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index fd0ffbb1e0..10c1149bfa 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -7246,6 +7246,27 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Total number of indexes that will be vacuumed or cleaned up. This number is
+       reported as of the beginning of the <literal>vacuuming indexes</literal> phase
+       or the <literal>cleaning up indexes</literal> phase.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes processed. This counter only advances when the phase is
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 639179aa46..70191e487a 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2316,6 +2316,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2328,9 +2339,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes and the number of indexes
+	 * to vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2343,6 +2358,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2377,14 +2396,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || vacrel->failsafe_active);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Increase and report the number of index scans. Also, we reset the progress
+	 * counters.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2621,6 +2643,12 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int   progress_index[] = {
+			PROGRESS_VACUUM_INDEXES_TOTAL,
+			PROGRESS_VACUUM_INDEXES_PROCESSED
+		};
+		int64       progress_val[2] = {0, 0};
+
 		vacrel->failsafe_active = true;
 
 		/*
@@ -2635,6 +2663,9 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Reset the progress counters */
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						vacrel->dbname, vacrel->relnamespace, vacrel->relname,
@@ -2661,13 +2692,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes and the number of indexes
+	 * to cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2679,6 +2724,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2688,6 +2737,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Reset the progress counters */
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index b26f2a64fb..0af4b7b0e0 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -1199,6 +1199,15 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/* Call the progress reporting callback, if set */
+				if (pcxt->parallel_progress_callback)
+					pcxt->parallel_progress_callback(pcxt->parallel_progress_callback_arg);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 6b098234f8..d5a72519b9 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1183,7 +1183,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 563117a8f6..e668f3e914 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,7 +30,9 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
+#include "libpq/libpq.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
 #include "storage/bufmgr.h"
@@ -103,6 +105,14 @@ typedef struct PVShared
 
 	/* Counter for vacuuming and cleanup */
 	pg_atomic_uint32 idx;
+
+	/*
+	 * Number of indexes processed in a parallel index bulk-deletion or a
+	 * parallel index cleanup. This counter is used to report the progress
+	 * information.
+	 */
+	pg_atomic_uint32 nindexes_processed;
+
 } PVShared;
 
 /* Status used during parallel index vacuum or cleanup */
@@ -275,6 +285,11 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pcxt = CreateParallelContext("postgres", "parallel_vacuum_main",
 								 parallel_workers);
 	Assert(pcxt->nworkers > 0);
+
+	/* Setup callback for updating the progress information */
+	pcxt->parallel_progress_callback = parallel_vacuum_update_progress;
+	pcxt->parallel_progress_callback_arg = pvs;
+
 	pvs->pcxt = pcxt;
 
 	/* Estimate size for index vacuum stats -- PARALLEL_VACUUM_KEY_INDEX_STATS */
@@ -368,6 +383,7 @@ parallel_vacuum_init(Relation rel, Relation *indrels, int nindexes,
 	pg_atomic_init_u32(&(shared->cost_balance), 0);
 	pg_atomic_init_u32(&(shared->active_nworkers), 0);
 	pg_atomic_init_u32(&(shared->idx), 0);
+	pg_atomic_init_u32(&(shared->nindexes_processed), 0);
 
 	shm_toc_insert(pcxt->toc, PARALLEL_VACUUM_KEY_SHARED, shared);
 	pvs->shared = shared;
@@ -622,8 +638,9 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
+	pg_atomic_write_u32(&(pvs->shared->nindexes_processed), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
 	if (nworkers > 0)
@@ -893,6 +910,16 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Update the index vacuum progress information. Since the progress is
+	 * updated only by the leader, the worker notifies the leader of it.
+	 */
+	pg_atomic_add_fetch_u32(&(pvs->shared->nindexes_processed), 1);
+	if (IsParallelWorker())
+		pq_putmessage('P', NULL, 0);
+	else
+		parallel_vacuum_update_progress(pvs);
 }
 
 /*
@@ -1078,3 +1105,19 @@ parallel_vacuum_error_callback(void *arg)
 			return;
 	}
 }
+
+/*
+ * Update the number of indexes processed so far in the current index bulk-deletion
+ * or index cleanup.
+ */
+void
+parallel_vacuum_update_progress(void *arg)
+{
+	ParallelVacuumState *pvs = (ParallelVacuumState *) arg;
+
+	Assert(!IsParallelWorker());
+	Assert(pvs);
+
+	pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+								 pg_atomic_read_u32(&(pvs->shared->nindexes_processed)));
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 061f8a4c4c..f621d51c0d 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -21,6 +21,7 @@
 #include "storage/shm_toc.h"
 
 typedef void (*parallel_worker_main_type) (dsm_segment *seg, shm_toc *toc);
+typedef void (*parallel_progress_callback_type) (void *arg);
 
 typedef struct ParallelWorkerInfo
 {
@@ -46,6 +47,8 @@ typedef struct ParallelContext
 	ParallelWorkerInfo *worker;
 	int			nknown_attached_workers;
 	bool	   *known_attached_workers;
+	parallel_progress_callback_type parallel_progress_callback;
+	void		*parallel_progress_callback_arg;
 } ParallelContext;
 
 typedef struct ParallelWorkerContext
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..2478e87425 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index bdfd96cfec..96f052eec0 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -371,5 +371,6 @@ extern bool std_typanalyze(VacAttrStats *stats);
 extern double anl_random_fract(void);
 extern double anl_init_selection_state(int n);
 extern double anl_get_next_S(double t, int n, double *stateptr);
+extern void parallel_vacuum_update_progress(void *arg);
 
 #endif							/* VACUUM_H */
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ab1aebfde4..9ff5dca49a 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2035,7 +2035,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
-- 
2.39.2

#124Michael Paquier
michael@paquier.xyz
In reply to: Imseih (AWS), Sami (#123)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Apr 05, 2023 at 02:31:54PM +0000, Imseih (AWS), Sami wrote:

That seems a valid argument. I was thinking that such an asynchronous
state update mechanism would be a good infrastructure for progress
reporting of parallel operations. It might be worth considering to use
it in more general usage but since the current implementation is
minimal can we extend it in the future when we need it for other use
cases?

I don't think we should delay this patch to design a more general
infrastructure. I agree this can be handled by a future requirement.

Not so sure to agree on that. As the patch stands, we have a rather
generally-purposed new message type and facility (callback trigger
poke from workers to their leader) used for a not-so-general purpose,
while being documented under this not-so-general purpose, which is
progress reporting. I agree that relying on pqmq.c to force the
leader to be more sensible to refreshes is sensible, because it is
cheap, but the interface seems kind of misplaced to me. As one thing,
for example, it introduces a dependency to parallel.h to do progress
reporting without touching at backend_progress.h. Is a callback
approach combined with a counter in shared memory the best thing there
could be? Could it be worth thinking about a different design where
the value incremented and the parameters of
pgstat_progress_update_param() are passed through the 'P' message
instead? It strikes me that gathering data in the leader from a poke
of the workers is something that could be useful in so much more areas
than just the parallel index operations done in a vacuum because we do
more and more things in parallel these days, so the API interface
ought to have some attention.

As some say, the introduction of a new message type in pqmq.c would be
basically a one-way door, because we'd have to maintain it in a stable
branch. I would not take that lightly. One idea of interface that
could be used is an extra set of APIs for workers to do progress
reporting, part of backend_progress.h, where we use a pqmq message
type in a centralized location, say something like a
pgstat_progress_parallel_incr_param().

About the callback interface, we may also want to be more careful
about more things, like the handling of callback chains, or even
unregistrations of callbacks? There could be much more uses to that
than just progress reporting, though this comes to a balance of what
the leader needs to do before the workers are able to poke at it on a
periodic basis to make the refresh of the aggregated progress
reporting data more verbose. There is also an argument where we could
have each worker report their progress independently of the leader?
--
Michael

#125Imseih (AWS), Sami
simseih@amazon.com
In reply to: Michael Paquier (#124)
Re: Add index scan progress to pg_stat_progress_vacuum

As one thing,
for example, it introduces a dependency to parallel.h to do progress
reporting without touching at backend_progress.h.

Containing the logic in backend_progress.h is a reasonable point
from a maintenance standpoint.

We can create a new function in backend_progress.h called
pgstat_progress_update_leader which is called from
vacuumparallel.c.

pgstat_progress_update_leader can then call pq_putmessage('P', NULL, 0)

Is a callback
approach combined with a counter in shared memory the best thing there
could be?

It seems to be the best way.

The shared memory, ParallelVacuumState, is already tracking the
counters for the Parallel Vacuum.

Also, the callback in ParallelContext is the only way I can see
to let the 'P' message know what to do for updating progress
to the leader.

Could it be worth thinking about a different design where
the value incremented and the parameters of
pgstat_progress_update_param() are passed through the 'P' message
instead?

I am not sure how this is different than the approach suggested.
In the current design, the 'P' message is used to pass the
ParallelvacuumState to parallel_vacuum_update_progress which then
calls pgstat_progress_update_param.

It strikes me that gathering data in the leader from a poke
of the workers is something that could be useful in so much more areas
than just the parallel index operations done in a vacuum because we do
more and more things in parallel these days, so the API interface
ought to have some attention.

We may need an interface that does more than progress
reporting, but I am not sure what those use cases are at
this point, besides progress reporting.

There is also an argument where we could
have each worker report their progress independently of the leader?

In this case, we don't need ParallelContext at all or to go through the
'P' message.

--
Regards,

Sami Imseih
Amazon Web Services (AWS)

#126Michael Paquier
michael@paquier.xyz
In reply to: Imseih (AWS), Sami (#125)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Apr 06, 2023 at 03:14:20PM +0000, Imseih (AWS), Sami wrote:

Could it be worth thinking about a different design where
the value incremented and the parameters of
pgstat_progress_update_param() are passed through the 'P' message
instead?

I am not sure how this is different than the approach suggested.
In the current design, the 'P' message is used to pass the
ParallelvacuumState to parallel_vacuum_update_progress which then
calls pgstat_progress_update_param.

The arguments of pgstat_progress_update_param() would be given by the
worker directly as components of the 'P' message. It seems to me that
this approach would have the simplicity to not require the setup of a
shmem area for the extra counters, and there would be no need for a
callback. Hence, the only thing the code paths of workers would need
to do is to call this routine, then the leaders would increment their
progress when they see a CFI to process the 'P' message. Also, I
guess that we would only need an interface in backend_progress.c to
increment counters, like pgstat_progress_incr_param(), but usable by
workers. Like a pgstat_progress_worker_incr_param()?
--
Michael

#127Andres Freund
andres@anarazel.de
In reply to: Michael Paquier (#124)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

On 2023-04-06 12:28:04 +0900, Michael Paquier wrote:

As some say, the introduction of a new message type in pqmq.c would be
basically a one-way door, because we'd have to maintain it in a stable
branch.

Why would it mean that? Parallel workers are updated together with the leader,
so there's no compatibility issue?

Greetings,

Andres Freund

#128Imseih (AWS), Sami
simseih@amazon.com
In reply to: Michael Paquier (#126)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

The arguments of pgstat_progress_update_param() would be given by the
worker directly as components of the 'P' message. It seems to me that
this approach would have the simplicity to not require the setup of a
shmem area for the extra counters, and there would be no need for a
callback. Hence, the only thing the code paths of workers would need
to do is to call this routine, then the leaders would increment their
progress when they see a CFI to process the 'P' message. Also, I
guess that we would only need an interface in backend_progress.c to
increment counters, like pgstat_progress_incr_param(), but usable by
workers. Like a pgstat_progress_worker_incr_param()?

So, here is what I think should be workable to give a generic
progress interface.

pgstat_progress_parallel_incr_param will be a new API that
can be called by either worker of leader from any parallel
code path that chooses to increment a progress index.

If called by a worker, it will send a 'P' message to the front end
passing both the progress index, i.e. PROGRESS_VACUUM_INDEXES_PROCESSED
And the value to increment by, i.e. 1 for index vacuum progress.

With that, the additional shared memory counters in ParallelVacuumState
are not needed, and the poke of the worker to the leader goes directly
through a generic backend_progress API.

Let me know your thoughts.

Thanks!

--
Sami Imseih
Amazon Web Services (AWS)

Attachments:

v27-0001-Report-index-vacuum-progress.patchapplication/octet-stream; name=v27-0001-Report-index-vacuum-progress.patchDownload
From d4933a6383578d7255acb1f31912b8c88dd34d52 Mon Sep 17 00:00:00 2001
From: EC2 Default User <ec2-user@ip-172-31-26-221.ec2.internal>
Date: Fri, 7 Apr 2023 18:46:33 +0000
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Authors: Sami Imseih
Reviewed by: Masahiko Sawada, Michael Paquier, Nathan Bossart, Andres Freund
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml                  | 21 ++++++
 src/backend/access/heap/vacuumlazy.c          | 70 ++++++++++++++++---
 src/backend/access/transam/parallel.c         | 12 ++++
 src/backend/catalog/system_views.sql          |  3 +-
 src/backend/commands/vacuumparallel.c         | 10 ++-
 src/backend/utils/activity/backend_progress.c | 33 +++++++++
 src/include/commands/progress.h               |  2 +
 src/include/utils/backend_progress.h          |  1 +
 src/test/regress/expected/rules.out           |  4 +-
 9 files changed, 144 insertions(+), 12 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index bce9ae4661..41296db236 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -7246,6 +7246,27 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Total number of indexes that will be vacuumed or cleaned up. This number is
+       reported as of the beginning of the <literal>vacuuming indexes</literal> phase
+       or the <literal>cleaning up indexes</literal> phase.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes processed. This counter only advances when the phase is
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 0a9ebd22bd..93d83a77c9 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2314,6 +2314,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2326,9 +2337,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes and the number of indexes
+	 * to vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2341,6 +2356,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2375,14 +2394,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || VacuumFailsafeActive);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Increase and report the number of index scans. Also, we reset the progress
+	 * counters.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2619,6 +2641,12 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int   progress_index[] = {
+			PROGRESS_VACUUM_INDEXES_TOTAL,
+			PROGRESS_VACUUM_INDEXES_PROCESSED
+		};
+		int64       progress_val[2] = {0, 0};
+
 		VacuumFailsafeActive = true;
 
 		/*
@@ -2633,6 +2661,9 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Reset the progress counters */
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						vacrel->dbname, vacrel->relnamespace, vacrel->relname,
@@ -2659,13 +2690,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes and the number of indexes
+	 * to cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2677,6 +2722,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2686,6 +2735,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Reset the progress counters */
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index b26f2a64fb..8e83b3de5e 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -24,6 +24,7 @@
 #include "catalog/pg_enum.h"
 #include "catalog/storage.h"
 #include "commands/async.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "executor/execParallel.h"
 #include "libpq/libpq.h"
@@ -1199,6 +1200,17 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/* update progress */
+				int index = pq_getmsgint(msg, 4);
+				int incr = pq_getmsgint(msg, 1);
+
+				pgstat_progress_incr_param(index, incr);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 6b098234f8..d5a72519b9 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1183,7 +1183,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 87ea5c5242..aa9aae042a 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -631,7 +632,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
@@ -902,6 +903,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Call the parallel variant of pgstat_progress_incr_param
+	 * so workers can report progress of index vacuum to the
+	 * leader.
+	 */
+	pgstat_progress_parallel_incr_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 }
 
 /*
diff --git a/src/backend/utils/activity/backend_progress.c b/src/backend/utils/activity/backend_progress.c
index fb48eafef9..249bfce143 100644
--- a/src/backend/utils/activity/backend_progress.c
+++ b/src/backend/utils/activity/backend_progress.c
@@ -10,6 +10,8 @@
  */
 #include "postgres.h"
 
+#include "access/parallel.h"
+#include "libpq/pqformat.h"
 #include "port/atomics.h"		/* for memory barriers */
 #include "utils/backend_progress.h"
 #include "utils/backend_status.h"
@@ -79,6 +81,37 @@ pgstat_progress_incr_param(int index, int64 incr)
 	PGSTAT_END_WRITE_ACTIVITY(beentry);
 }
 
+/*-----------
+ * pgstat_progress_parallel_incr_param() -
+ *
+ * A variant of pgstat_progress_incr_param to allow a worker
+ * to poke a leader to update progress.
+ *-----------
+ */
+void
+pgstat_progress_parallel_incr_param(int index, int64 incr)
+{
+	/*
+	 * Parallel workers notify a leader through a 'P'
+	 * protocol message to update progress, passing the
+	 * progress index and increment value. Leaders can
+	 * just call pgstat_progress_incr_param directly.
+	 */
+	if (IsParallelWorker())
+	{
+		static StringInfoData progress_message;
+
+		initStringInfo(&progress_message);
+
+		pq_beginmessage(&progress_message, 'P');
+		pq_sendint32(&progress_message, index);
+		pq_sendint64(&progress_message, incr);
+		pq_endmessage(&progress_message);
+	}
+	else
+		pgstat_progress_incr_param(index, incr);
+}
+
 /*-----------
  * pgstat_progress_update_multi_param() -
  *
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..2478e87425 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index a84752ade9..70dea55fc0 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -37,6 +37,7 @@ extern void pgstat_progress_start_command(ProgressCommandType cmdtype,
 										  Oid relid);
 extern void pgstat_progress_update_param(int index, int64 val);
 extern void pgstat_progress_incr_param(int index, int64 incr);
+extern void pgstat_progress_parallel_incr_param(int index, int64 incr);
 extern void pgstat_progress_update_multi_param(int nparam, const int *index,
 											   const int64 *val);
 extern void pgstat_progress_end_command(void);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ab1aebfde4..9ff5dca49a 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2035,7 +2035,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
-- 
2.39.2

#129Michael Paquier
michael@paquier.xyz
In reply to: Imseih (AWS), Sami (#128)
Re: Add index scan progress to pg_stat_progress_vacuum

On Fri, Apr 07, 2023 at 07:27:17PM +0000, Imseih (AWS), Sami wrote:

If called by a worker, it will send a 'P' message to the front end
passing both the progress index, i.e. PROGRESS_VACUUM_INDEXES_PROCESSED
And the value to increment by, i.e. 1 for index vacuum progress.

With that, the additional shared memory counters in ParallelVacuumState
are not needed, and the poke of the worker to the leader goes directly
through a generic backend_progress API.

Thanks for the new version. This has unfortunately not been able to
make the cut for v16, but let's see it done at the beginning of the
v17 cycle.

+void
+pgstat_progress_parallel_incr_param(int index, int64 incr)
+{
+   /*
+    * Parallel workers notify a leader through a 'P'
+    * protocol message to update progress, passing the
+    * progress index and increment value. Leaders can
+    * just call pgstat_progress_incr_param directly.
+    */
+   if (IsParallelWorker())
+   {
+       static StringInfoData progress_message;
+
+       initStringInfo(&progress_message);
+
+       pq_beginmessage(&progress_message, 'P');
+       pq_sendint32(&progress_message, index);
+       pq_sendint64(&progress_message, incr);
+       pq_endmessage(&progress_message);
+   }
+   else
+       pgstat_progress_incr_param(index, incr);
+}

I see. You need to handle both the leader and worker case because
parallel_vacuum_process_one_index() can be called by either of them.

+ case 'P': /* Parallel progress reporting */

Perhaps this comment should say that this is only about incremental
progress reporting, for the moment.

+    * Increase and report the number of index scans. Also, we reset the progress
+    * counters.

The counters reset are the two index counts, perhaps this comment
should mention this fact.

+               /* update progress */
+               int index = pq_getmsgint(msg, 4);
+               int incr = pq_getmsgint(msg, 1);
[...]
+       pq_beginmessage(&progress_message, 'P');
+       pq_sendint32(&progress_message, index);
+       pq_sendint64(&progress_message, incr);
+       pq_endmessage(&progress_message);

It seems to me that the receiver side is missing one pq_getmsgend()?
incr is defined and sent as an int64 on the sender side, hence the
receiver should use pq_getmsgint64(), no? pq_getmsgint(msg, 1) means
to receive only one byte, see pqformat.c. And the order is reversed?

There may be a case in the future about making 'P' more complicated
with more arguments, but what you have here should be sufficient for
your use-case. Were there plans to increment more data for some
different and/or new progress indexes in the VACUUM path, by the way?
Most of that looked a bit tricky to me as this was AM-dependent, but I
may have missed something.
--
Michael

#130Michael Paquier
michael@paquier.xyz
In reply to: Andres Freund (#127)
Re: Add index scan progress to pg_stat_progress_vacuum

On Fri, Apr 07, 2023 at 12:01:17PM -0700, Andres Freund wrote:

Why would it mean that? Parallel workers are updated together with the leader,
so there's no compatibility issue?

My point is that the callback system would still need to be maintained
in a stable branch, and, while useful, it could be used for much more
than it is originally written. I guess that this could be used in
custom nodes with their own custom parallel nodes.
--
Michael

#131Imseih (AWS), Sami
simseih@amazon.com
In reply to: Michael Paquier (#129)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

+ case 'P': /* Parallel progress reporting */

I kept this comment as-is but inside case code block I added
more comments. This is to avoid cluttering up the one-liner comment.

+ * Increase and report the number of index scans. Also, we reset the progress
+ * counters.

The counters reset are the two index counts, perhaps this comment
should mention this fact.

Yes, since we are using the multi_param API here, it makes sense to
mention the progress fields being reset in the comments.

+ /* update progress */
+ int index = pq_getmsgint(msg, 4);
+ int incr = pq_getmsgint(msg, 1);
[...]
+ pq_beginmessage(&progress_message, 'P');
+ pq_sendint32(&progress_message, index);
+ pq_sendint64(&progress_message, incr);
+ pq_endmessage(&progress_message);

It seems to me that the receiver side is missing one pq_getmsgend()?

Yes. I added this.

incr is defined and sent as an int64 on the sender side, hence the
receiver should use pq_getmsgint64(), no? pq_getmsgint(msg, 1) means
to receive only one byte, see pqformat.c.

Ah correct, incr is an int64 so what we need is.

int64 incr = pq_getmsgint64(msg);

I also added the pq_getmsgend call.

And the order is reversed?

I don't think so. The index then incr are sent and they are
back in the same order. Testing the patch shows the value
increments correctly.

See v28 addressing the comments.

Regards,

Sami Imseih
AWS (Amazon Web Services)

Attachments:

v28-0001-Report-index-vacuum-progress.patchapplication/octet-stream; name=v28-0001-Report-index-vacuum-progress.patchDownload
From 65c589c6af3c91954a66d02f538faf741cf548a8 Mon Sep 17 00:00:00 2001
From: EC2 Default User <ec2-user@ip-172-31-26-221.ec2.internal>
Date: Fri, 7 Apr 2023 18:46:33 +0000
Subject: [PATCH 1/1] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Authors: Sami Imseih
Reviewed by: Masahiko Sawada, Michael Paquier, Nathan Bossart, Andres Freund
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml                  | 21 ++++++
 src/backend/access/heap/vacuumlazy.c          | 70 ++++++++++++++++---
 src/backend/access/transam/parallel.c         | 19 +++++
 src/backend/catalog/system_views.sql          |  3 +-
 src/backend/commands/vacuumparallel.c         | 10 ++-
 src/backend/utils/activity/backend_progress.c | 33 +++++++++
 src/include/commands/progress.h               |  2 +
 src/include/utils/backend_progress.h          |  1 +
 src/test/regress/expected/rules.out           |  4 +-
 9 files changed, 151 insertions(+), 12 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index bce9ae4661..41296db236 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -7246,6 +7246,27 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Total number of indexes that will be vacuumed or cleaned up. This number is
+       reported as of the beginning of the <literal>vacuuming indexes</literal> phase
+       or the <literal>cleaning up indexes</literal> phase.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes processed. This counter only advances when the phase is
+       <literal>vacuuming indexes</literal> or <literal>cleaning up indexes</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 0a9ebd22bd..685bff3855 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2314,6 +2314,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2326,9 +2337,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes and the number of indexes
+	 * to vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2341,6 +2356,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2375,14 +2394,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || VacuumFailsafeActive);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Increase and report the number of index scans. Also, we reset
+	 * PROGRESS_VACUUM_INDEXES_TOTAL and PROGRESS_VACUUM_INDEXES_PROCESSED.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2619,6 +2641,12 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int   progress_index[] = {
+			PROGRESS_VACUUM_INDEXES_TOTAL,
+			PROGRESS_VACUUM_INDEXES_PROCESSED
+		};
+		int64       progress_val[2] = {0, 0};
+
 		VacuumFailsafeActive = true;
 
 		/*
@@ -2633,6 +2661,9 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Reset the progress counters */
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						vacrel->dbname, vacrel->relnamespace, vacrel->relname,
@@ -2659,13 +2690,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED
+	};
+	int64       progress_start_val[2];
+	int64       progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes and the number of indexes
+	 * to cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2677,6 +2722,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2686,6 +2735,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Reset the progress counters */
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index b26f2a64fb..f401cb07a4 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -24,6 +24,7 @@
 #include "catalog/pg_enum.h"
 #include "catalog/storage.h"
 #include "commands/async.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "executor/execParallel.h"
 #include "libpq/libpq.h"
@@ -1199,6 +1200,24 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/*
+				 * Only incremental progress reporting is currently
+				 * supported. However, it's possible to add more
+				 * fields to the message to allow for handling of
+				 * other backend progress APIs.
+				 */
+				int index = pq_getmsgint(msg, 4);
+				int64 incr = pq_getmsgint64(msg);
+
+				pq_getmsgend(msg);
+
+				pgstat_progress_incr_param(index, incr);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 6b098234f8..d5a72519b9 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1183,7 +1183,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 87ea5c5242..aa9aae042a 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -631,7 +632,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
@@ -902,6 +903,13 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Call the parallel variant of pgstat_progress_incr_param
+	 * so workers can report progress of index vacuum to the
+	 * leader.
+	 */
+	pgstat_progress_parallel_incr_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 }
 
 /*
diff --git a/src/backend/utils/activity/backend_progress.c b/src/backend/utils/activity/backend_progress.c
index fb48eafef9..249bfce143 100644
--- a/src/backend/utils/activity/backend_progress.c
+++ b/src/backend/utils/activity/backend_progress.c
@@ -10,6 +10,8 @@
  */
 #include "postgres.h"
 
+#include "access/parallel.h"
+#include "libpq/pqformat.h"
 #include "port/atomics.h"		/* for memory barriers */
 #include "utils/backend_progress.h"
 #include "utils/backend_status.h"
@@ -79,6 +81,37 @@ pgstat_progress_incr_param(int index, int64 incr)
 	PGSTAT_END_WRITE_ACTIVITY(beentry);
 }
 
+/*-----------
+ * pgstat_progress_parallel_incr_param() -
+ *
+ * A variant of pgstat_progress_incr_param to allow a worker
+ * to poke a leader to update progress.
+ *-----------
+ */
+void
+pgstat_progress_parallel_incr_param(int index, int64 incr)
+{
+	/*
+	 * Parallel workers notify a leader through a 'P'
+	 * protocol message to update progress, passing the
+	 * progress index and increment value. Leaders can
+	 * just call pgstat_progress_incr_param directly.
+	 */
+	if (IsParallelWorker())
+	{
+		static StringInfoData progress_message;
+
+		initStringInfo(&progress_message);
+
+		pq_beginmessage(&progress_message, 'P');
+		pq_sendint32(&progress_message, index);
+		pq_sendint64(&progress_message, incr);
+		pq_endmessage(&progress_message);
+	}
+	else
+		pgstat_progress_incr_param(index, incr);
+}
+
 /*-----------
  * pgstat_progress_update_multi_param() -
  *
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..2478e87425 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index a84752ade9..70dea55fc0 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -37,6 +37,7 @@ extern void pgstat_progress_start_command(ProgressCommandType cmdtype,
 										  Oid relid);
 extern void pgstat_progress_update_param(int index, int64 val);
 extern void pgstat_progress_incr_param(int index, int64 incr);
+extern void pgstat_progress_parallel_incr_param(int index, int64 incr);
 extern void pgstat_progress_update_multi_param(int nparam, const int *index,
 											   const int64 *val);
 extern void pgstat_progress_end_command(void);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index ab1aebfde4..9ff5dca49a 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2035,7 +2035,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
-- 
2.39.2

#132Andres Freund
andres@anarazel.de
In reply to: Michael Paquier (#130)
Re: Add index scan progress to pg_stat_progress_vacuum

Hi,

On 2023-04-10 08:14:18 +0900, Michael Paquier wrote:

On Fri, Apr 07, 2023 at 12:01:17PM -0700, Andres Freund wrote:

Why would it mean that? Parallel workers are updated together with the leader,
so there's no compatibility issue?

My point is that the callback system would still need to be maintained
in a stable branch, and, while useful, it could be used for much more
than it is originally written. I guess that this could be used in
custom nodes with their own custom parallel nodes.

Hm, I'm somewhat doubtful that that's something we should encourage. And
doubtful we'd get it right without a concrete use case at hand to verify the
design.

Greetings,

Andres Freund

#133Michael Paquier
michael@paquier.xyz
In reply to: Imseih (AWS), Sami (#131)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

On Mon, Apr 10, 2023 at 07:20:42PM +0000, Imseih (AWS), Sami wrote:

See v28 addressing the comments.

This should be OK (also checked the code paths where the reports are
added). Note that the patch needed a few adjustments for its
indentation.
--
Michael

Attachments:

v29-0001-Report-index-vacuum-progress.patchtext/x-diff; charset=us-asciiDownload
From 9267342ba2a1b8b120efaa98871b736a5bbde9a9 Mon Sep 17 00:00:00 2001
From: Michael Paquier <michael@paquier.xyz>
Date: Wed, 12 Apr 2023 13:45:59 +0900
Subject: [PATCH v29] Report index vacuum progress.

Add 2 new columns to pg_stat_progress_vacuum.
The columns are ndexes_total as the total indexes to be vacuumed or cleaned and
indexes_processed as the number of indexes vacuumed or cleaned up so far.

Authors: Sami Imseih
Reviewed by: Masahiko Sawada, Michael Paquier, Nathan Bossart, Andres Freund
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 src/include/commands/progress.h               |  2 +
 src/include/utils/backend_progress.h          |  1 +
 src/backend/access/heap/vacuumlazy.c          | 70 ++++++++++++++++---
 src/backend/access/transam/parallel.c         | 18 +++++
 src/backend/catalog/system_views.sql          |  3 +-
 src/backend/commands/vacuumparallel.c         |  9 ++-
 src/backend/utils/activity/backend_progress.c | 32 +++++++++
 src/test/regress/expected/rules.out           |  4 +-
 doc/src/sgml/monitoring.sgml                  | 23 ++++++
 9 files changed, 150 insertions(+), 12 deletions(-)

diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..2478e87425 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index a84752ade9..70dea55fc0 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -37,6 +37,7 @@ extern void pgstat_progress_start_command(ProgressCommandType cmdtype,
 										  Oid relid);
 extern void pgstat_progress_update_param(int index, int64 val);
 extern void pgstat_progress_incr_param(int index, int64 incr);
+extern void pgstat_progress_parallel_incr_param(int index, int64 incr);
 extern void pgstat_progress_update_multi_param(int nparam, const int *index,
 											   const int64 *val);
 extern void pgstat_progress_end_command(void);
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 0a9ebd22bd..e5707098bd 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2314,6 +2314,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int	progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int	progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64		progress_start_val[2];
+	int64		progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2326,9 +2337,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes and the number of indexes to
+	 * vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2341,6 +2356,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2375,14 +2394,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || VacuumFailsafeActive);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Increase and report the number of index scans.  Also, we reset
+	 * PROGRESS_VACUUM_INDEXES_TOTAL and PROGRESS_VACUUM_INDEXES_PROCESSED.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2619,6 +2641,12 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int	progress_index[] = {
+			PROGRESS_VACUUM_INDEXES_TOTAL,
+			PROGRESS_VACUUM_INDEXES_PROCESSED
+		};
+		int64		progress_val[2] = {0, 0};
+
 		VacuumFailsafeActive = true;
 
 		/*
@@ -2633,6 +2661,9 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Reset the progress counters */
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						vacrel->dbname, vacrel->relnamespace, vacrel->relname,
@@ -2659,13 +2690,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int	progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int	progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED
+	};
+	int64		progress_start_val[2];
+	int64		progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes and the number of indexes to
+	 * cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2677,6 +2722,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2686,6 +2735,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Reset the progress counters */
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index b26f2a64fb..88dcb43daa 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -24,6 +24,7 @@
 #include "catalog/pg_enum.h"
 #include "catalog/storage.h"
 #include "commands/async.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "executor/execParallel.h"
 #include "libpq/libpq.h"
@@ -1199,6 +1200,23 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/*
+				 * Only incremental progress reporting is currently supported.
+				 * However, it's possible to add more fields to the message to
+				 * allow for handling of other backend progress APIs.
+				 */
+				int			index = pq_getmsgint(msg, 4);
+				int64		incr = pq_getmsgint64(msg);
+
+				pq_getmsgend(msg);
+
+				pgstat_progress_incr_param(index, incr);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 701c340fc4..1e44f94d3e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1189,7 +1189,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index 87ea5c5242..e8f1291955 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -631,7 +632,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
@@ -902,6 +903,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Call the parallel variant of pgstat_progress_incr_param so workers can
+	 * report progress of index vacuum to the leader.
+	 */
+	pgstat_progress_parallel_incr_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 }
 
 /*
diff --git a/src/backend/utils/activity/backend_progress.c b/src/backend/utils/activity/backend_progress.c
index fb48eafef9..67447ef03a 100644
--- a/src/backend/utils/activity/backend_progress.c
+++ b/src/backend/utils/activity/backend_progress.c
@@ -10,6 +10,8 @@
  */
 #include "postgres.h"
 
+#include "access/parallel.h"
+#include "libpq/pqformat.h"
 #include "port/atomics.h"		/* for memory barriers */
 #include "utils/backend_progress.h"
 #include "utils/backend_status.h"
@@ -79,6 +81,36 @@ pgstat_progress_incr_param(int index, int64 incr)
 	PGSTAT_END_WRITE_ACTIVITY(beentry);
 }
 
+/*-----------
+ * pgstat_progress_parallel_incr_param() -
+ *
+ * A variant of pgstat_progress_incr_param to allow a worker to poke at
+ * a leader to do an incremental progress update.
+ *-----------
+ */
+void
+pgstat_progress_parallel_incr_param(int index, int64 incr)
+{
+	/*
+	 * Parallel workers notify a leader through a 'P' protocol message to
+	 * update progress, passing the progress index and incremented value.
+	 * Leaders can just call pgstat_progress_incr_param directly.
+	 */
+	if (IsParallelWorker())
+	{
+		static StringInfoData progress_message;
+
+		initStringInfo(&progress_message);
+
+		pq_beginmessage(&progress_message, 'P');
+		pq_sendint32(&progress_message, index);
+		pq_sendint64(&progress_message, incr);
+		pq_endmessage(&progress_message);
+	}
+	else
+		pgstat_progress_incr_param(index, incr);
+}
+
 /*-----------
  * pgstat_progress_update_multi_param() -
  *
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 2d75dd6656..5e5f9b5a9a 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2041,7 +2041,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f33a1c56c..b6bbb85be9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -7316,6 +7316,29 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Total number of indexes that will be vacuumed or cleaned up. This
+       number is reported at the beginning of the
+       <literal>vacuuming indexes</literal> phase or the
+       <literal>cleaning up indexes</literal> phase.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes processed. This counter only advances when the
+       phase is <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
-- 
2.40.0

#134Imseih (AWS), Sami
simseih@amazon.com
In reply to: Michael Paquier (#133)
Re: Add index scan progress to pg_stat_progress_vacuum

This should be OK (also checked the code paths where the reports are
added). Note that the patch needed a few adjustments for its
indentation.

Thanks for the formatting corrections! This looks good to me.

--
Sami

#135Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Imseih (AWS), Sami (#134)
1 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

On Wed, Apr 12, 2023 at 9:22 PM Imseih (AWS), Sami <simseih@amazon.com> wrote:

This should be OK (also checked the code paths where the reports are
added). Note that the patch needed a few adjustments for its
indentation.

Thanks for the formatting corrections! This looks good to me.

Thank you for updating the patch. It looks good to me too. I've
updated the commit message.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

Attachments:

v30-0001-Report-index-vacuum-progress.patchapplication/x-patch; name=v30-0001-Report-index-vacuum-progress.patchDownload
From f92929f21d4052cfbc3b068ca5b8be954741da3d Mon Sep 17 00:00:00 2001
From: Michael Paquier <michael@paquier.xyz>
Date: Wed, 12 Apr 2023 13:45:59 +0900
Subject: [PATCH v30] Report index vacuum progress.

This commit adds two columns: indexes_total and indexes_processed, to
pg_stat_progress_vacuum system view to show the index vacuum
progress. These numbers are reported in the "vacuuming indexes" and
"cleaning up indexes" phases.

It also adds a new type of parallel message, 'P'. Which is used to
convey the progress updates made by parallel workers to the leader
process. Therefore, the parallel workers' progress updates are
reflected in an asynchronous manner. Currently it supports only
incremental progress reporting but it's possible to allow for
supporting of other backend process APIs in the future.

Authors: Sami Imseih
Reviewed by: Masahiko Sawada, Michael Paquier, Nathan Bossart, Andres Freund
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml                  | 23 ++++++
 src/backend/access/heap/vacuumlazy.c          | 70 ++++++++++++++++---
 src/backend/access/transam/parallel.c         | 18 +++++
 src/backend/catalog/system_views.sql          |  3 +-
 src/backend/commands/vacuumparallel.c         |  9 ++-
 src/backend/utils/activity/backend_progress.c | 32 +++++++++
 src/include/commands/progress.h               |  2 +
 src/include/utils/backend_progress.h          |  1 +
 src/test/regress/expected/rules.out           |  4 +-
 9 files changed, 150 insertions(+), 12 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 506aeaa879..588b720f57 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6110,6 +6110,29 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Total number of indexes that will be vacuumed or cleaned up. This
+       number is reported at the beginning of the
+       <literal>vacuuming indexes</literal> phase or the
+       <literal>cleaning up indexes</literal> phase.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes processed. This counter only advances when the
+       phase is <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 4eb953f904..6a41ee635d 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2319,6 +2319,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int	progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int	progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64		progress_start_val[2];
+	int64		progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2331,9 +2342,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes and the number of indexes to
+	 * vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2346,6 +2361,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2380,14 +2399,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || VacuumFailsafeActive);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Increase and report the number of index scans.  Also, we reset
+	 * PROGRESS_VACUUM_INDEXES_TOTAL and PROGRESS_VACUUM_INDEXES_PROCESSED.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2624,6 +2646,12 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int	progress_index[] = {
+			PROGRESS_VACUUM_INDEXES_TOTAL,
+			PROGRESS_VACUUM_INDEXES_PROCESSED
+		};
+		int64		progress_val[2] = {0, 0};
+
 		VacuumFailsafeActive = true;
 
 		/*
@@ -2638,6 +2666,9 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Reset the progress counters */
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						vacrel->dbname, vacrel->relnamespace, vacrel->relname,
@@ -2664,13 +2695,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int	progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int	progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED
+	};
+	int64		progress_start_val[2];
+	int64		progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes and the number of indexes to
+	 * cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2682,6 +2727,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2691,6 +2740,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Reset the progress counters */
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 2b8bc2f58d..2bd04bd177 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -24,6 +24,7 @@
 #include "catalog/pg_enum.h"
 #include "catalog/storage.h"
 #include "commands/async.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "executor/execParallel.h"
 #include "libpq/libpq.h"
@@ -1199,6 +1200,23 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/*
+				 * Only incremental progress reporting is currently supported.
+				 * However, it's possible to add more fields to the message to
+				 * allow for handling of other backend progress APIs.
+				 */
+				int			index = pq_getmsgint(msg, 4);
+				int64		incr = pq_getmsgint64(msg);
+
+				pq_getmsgend(msg);
+
+				pgstat_progress_incr_param(index, incr);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c18fea8362..af65af6bdd 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1192,7 +1192,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index a79067fd46..351ab4957a 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -631,7 +632,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
@@ -902,6 +903,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Call the parallel variant of pgstat_progress_incr_param so workers can
+	 * report progress of index vacuum to the leader.
+	 */
+	pgstat_progress_parallel_incr_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 }
 
 /*
diff --git a/src/backend/utils/activity/backend_progress.c b/src/backend/utils/activity/backend_progress.c
index fb48eafef9..67447ef03a 100644
--- a/src/backend/utils/activity/backend_progress.c
+++ b/src/backend/utils/activity/backend_progress.c
@@ -10,6 +10,8 @@
  */
 #include "postgres.h"
 
+#include "access/parallel.h"
+#include "libpq/pqformat.h"
 #include "port/atomics.h"		/* for memory barriers */
 #include "utils/backend_progress.h"
 #include "utils/backend_status.h"
@@ -79,6 +81,36 @@ pgstat_progress_incr_param(int index, int64 incr)
 	PGSTAT_END_WRITE_ACTIVITY(beentry);
 }
 
+/*-----------
+ * pgstat_progress_parallel_incr_param() -
+ *
+ * A variant of pgstat_progress_incr_param to allow a worker to poke at
+ * a leader to do an incremental progress update.
+ *-----------
+ */
+void
+pgstat_progress_parallel_incr_param(int index, int64 incr)
+{
+	/*
+	 * Parallel workers notify a leader through a 'P' protocol message to
+	 * update progress, passing the progress index and incremented value.
+	 * Leaders can just call pgstat_progress_incr_param directly.
+	 */
+	if (IsParallelWorker())
+	{
+		static StringInfoData progress_message;
+
+		initStringInfo(&progress_message);
+
+		pq_beginmessage(&progress_message, 'P');
+		pq_sendint32(&progress_message, index);
+		pq_sendint64(&progress_message, incr);
+		pq_endmessage(&progress_message);
+	}
+	else
+		pgstat_progress_incr_param(index, incr);
+}
+
 /*-----------
  * pgstat_progress_update_multi_param() -
  *
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..2478e87425 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index a84752ade9..70dea55fc0 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -37,6 +37,7 @@ extern void pgstat_progress_start_command(ProgressCommandType cmdtype,
 										  Oid relid);
 extern void pgstat_progress_update_param(int index, int64 val);
 extern void pgstat_progress_incr_param(int index, int64 incr);
+extern void pgstat_progress_parallel_incr_param(int index, int64 incr);
 extern void pgstat_progress_update_multi_param(int nparam, const int *index,
 											   const int64 *val);
 extern void pgstat_progress_end_command(void);
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 7fd81e6a7d..e07afcd4aa 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2044,7 +2044,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
-- 
2.31.1

#136Michael Paquier
michael@paquier.xyz
In reply to: Masahiko Sawada (#135)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Jul 06, 2023 at 11:07:14AM +0900, Masahiko Sawada wrote:

Thank you for updating the patch. It looks good to me too. I've
updated the commit message.

Thanks. I was planning to review this patch today and/or tomorrow now
that my stack of things to do is getting slightly lower (registered my
name as committer as well a few weeks ago to not format).

One thing I was planning to do is to move the new message processing
API for the incrementational updates in its own commit for clarity, as
that's a separate concept than the actual feature, useful on its own.

Anyway, would you prefer taking care of it?
--
Michael

#137Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Michael Paquier (#136)
2 attachment(s)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Jul 6, 2023 at 11:15 AM Michael Paquier <michael@paquier.xyz> wrote:

On Thu, Jul 06, 2023 at 11:07:14AM +0900, Masahiko Sawada wrote:

Thank you for updating the patch. It looks good to me too. I've
updated the commit message.

Thanks. I was planning to review this patch today and/or tomorrow now
that my stack of things to do is getting slightly lower (registered my
name as committer as well a few weeks ago to not format).

One thing I was planning to do is to move the new message processing
API for the incrementational updates in its own commit for clarity, as
that's a separate concept than the actual feature, useful on its own.

+1. I had the same idea. Please find the attached patches.

Anyway, would you prefer taking care of it?

I can take care of it if you're okay.

Regards,

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com

Attachments:

v31-0002-Report-index-vacuum-progress.patchapplication/octet-stream; name=v31-0002-Report-index-vacuum-progress.patchDownload
From 31776cfb8c2d3b0282f099bdaed8a70848bb1900 Mon Sep 17 00:00:00 2001
From: Masahiko Sawada <sawada.mshk@gmail.com>
Date: Thu, 6 Jul 2023 14:14:53 +0900
Subject: [PATCH v31 2/2] Report index vacuum progress.

This commit adds two columns: indexes_total and indexes_processed, to
pg_stat_progress_vacuum system view to show the index vacuum
progress. These numbers are reported in the "vacuuming indexes" and
"cleaning up indexes" phases.

This uses the new parallel message type for progress reporting added
by XXXXXXXXX.

Authors: Sami Imseih
Reviewed by: Masahiko Sawada, Michael Paquier, Nathan Bossart, Andres Freund
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 doc/src/sgml/monitoring.sgml          | 23 +++++++++
 src/backend/access/heap/vacuumlazy.c  | 70 +++++++++++++++++++++++----
 src/backend/catalog/system_views.sql  |  3 +-
 src/backend/commands/vacuumparallel.c |  9 +++-
 src/include/commands/progress.h       |  2 +
 src/test/regress/expected/rules.out   |  4 +-
 6 files changed, 99 insertions(+), 12 deletions(-)

diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 506aeaa879..588b720f57 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -6110,6 +6110,29 @@ FROM pg_stat_get_backend_idset() AS backendid;
        Number of dead tuples collected since the last index vacuum cycle.
       </para></entry>
      </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_total</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Total number of indexes that will be vacuumed or cleaned up. This
+       number is reported at the beginning of the
+       <literal>vacuuming indexes</literal> phase or the
+       <literal>cleaning up indexes</literal> phase.
+      </para></entry>
+     </row>
+
+     <row>
+      <entry role="catalog_table_entry"><para role="column_definition">
+       <structfield>indexes_processed</structfield> <type>bigint</type>
+      </para>
+      <para>
+       Number of indexes processed. This counter only advances when the
+       phase is <literal>vacuuming indexes</literal> or
+       <literal>cleaning up indexes</literal>.
+      </para></entry>
+     </row>
     </tbody>
    </tgroup>
   </table>
diff --git a/src/backend/access/heap/vacuumlazy.c b/src/backend/access/heap/vacuumlazy.c
index 4eb953f904..6a41ee635d 100644
--- a/src/backend/access/heap/vacuumlazy.c
+++ b/src/backend/access/heap/vacuumlazy.c
@@ -2319,6 +2319,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 {
 	bool		allindexes = true;
 	double		old_live_tuples = vacrel->rel->rd_rel->reltuples;
+	const int	progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int	progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED,
+		PROGRESS_VACUUM_NUM_INDEX_VACUUMS
+	};
+	int64		progress_start_val[2];
+	int64		progress_end_val[3];
 
 	Assert(vacrel->nindexes > 0);
 	Assert(vacrel->do_index_vacuuming);
@@ -2331,9 +2342,13 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 		return false;
 	}
 
-	/* Report that we are now vacuuming indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_VACUUM_INDEX);
+	/*
+	 * Report that we are now vacuuming indexes and the number of indexes to
+	 * vacuum.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_VACUUM_INDEX;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2346,6 +2361,10 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 														  old_live_tuples,
 														  vacrel);
 
+			/* Report the number of indexes vacuumed */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
+
 			if (lazy_check_wraparound_failsafe(vacrel))
 			{
 				/* Wraparound emergency -- end current index scan */
@@ -2380,14 +2399,17 @@ lazy_vacuum_all_indexes(LVRelState *vacrel)
 	Assert(allindexes || VacuumFailsafeActive);
 
 	/*
-	 * Increase and report the number of index scans.
+	 * Increase and report the number of index scans.  Also, we reset
+	 * PROGRESS_VACUUM_INDEXES_TOTAL and PROGRESS_VACUUM_INDEXES_PROCESSED.
 	 *
 	 * We deliberately include the case where we started a round of bulk
 	 * deletes that we weren't able to finish due to the failsafe triggering.
 	 */
 	vacrel->num_index_scans++;
-	pgstat_progress_update_param(PROGRESS_VACUUM_NUM_INDEX_VACUUMS,
-								 vacrel->num_index_scans);
+	progress_end_val[0] = 0;
+	progress_end_val[1] = 0;
+	progress_end_val[2] = vacrel->num_index_scans;
+	pgstat_progress_update_multi_param(3, progress_end_index, progress_end_val);
 
 	return allindexes;
 }
@@ -2624,6 +2646,12 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 
 	if (unlikely(vacuum_xid_failsafe_check(&vacrel->cutoffs)))
 	{
+		const int	progress_index[] = {
+			PROGRESS_VACUUM_INDEXES_TOTAL,
+			PROGRESS_VACUUM_INDEXES_PROCESSED
+		};
+		int64		progress_val[2] = {0, 0};
+
 		VacuumFailsafeActive = true;
 
 		/*
@@ -2638,6 +2666,9 @@ lazy_check_wraparound_failsafe(LVRelState *vacrel)
 		vacrel->do_index_cleanup = false;
 		vacrel->do_rel_truncate = false;
 
+		/* Reset the progress counters */
+		pgstat_progress_update_multi_param(2, progress_index, progress_val);
+
 		ereport(WARNING,
 				(errmsg("bypassing nonessential maintenance of table \"%s.%s.%s\" as a failsafe after %d index scans",
 						vacrel->dbname, vacrel->relnamespace, vacrel->relname,
@@ -2664,13 +2695,27 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 {
 	double		reltuples = vacrel->new_rel_tuples;
 	bool		estimated_count = vacrel->scanned_pages < vacrel->rel_pages;
+	const int	progress_start_index[] = {
+		PROGRESS_VACUUM_PHASE,
+		PROGRESS_VACUUM_INDEXES_TOTAL
+	};
+	const int	progress_end_index[] = {
+		PROGRESS_VACUUM_INDEXES_TOTAL,
+		PROGRESS_VACUUM_INDEXES_PROCESSED
+	};
+	int64		progress_start_val[2];
+	int64		progress_end_val[2] = {0, 0};
 
 	Assert(vacrel->do_index_cleanup);
 	Assert(vacrel->nindexes > 0);
 
-	/* Report that we are now cleaning up indexes */
-	pgstat_progress_update_param(PROGRESS_VACUUM_PHASE,
-								 PROGRESS_VACUUM_PHASE_INDEX_CLEANUP);
+	/*
+	 * Report that we are now cleaning up indexes and the number of indexes to
+	 * cleanup.
+	 */
+	progress_start_val[0] = PROGRESS_VACUUM_PHASE_INDEX_CLEANUP;
+	progress_start_val[1] = vacrel->nindexes;
+	pgstat_progress_update_multi_param(2, progress_start_index, progress_start_val);
 
 	if (!ParallelVacuumIsActive(vacrel))
 	{
@@ -2682,6 +2727,10 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 			vacrel->indstats[idx] =
 				lazy_cleanup_one_index(indrel, istat, reltuples,
 									   estimated_count, vacrel);
+
+			/* Report the number of indexes cleaned up */
+			pgstat_progress_update_param(PROGRESS_VACUUM_INDEXES_PROCESSED,
+										 idx + 1);
 		}
 	}
 	else
@@ -2691,6 +2740,9 @@ lazy_cleanup_all_indexes(LVRelState *vacrel)
 											vacrel->num_index_scans,
 											estimated_count);
 	}
+
+	/* Reset the progress counters */
+	pgstat_progress_update_multi_param(2, progress_end_index, progress_end_val);
 }
 
 /*
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c18fea8362..af65af6bdd 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1192,7 +1192,8 @@ CREATE VIEW pg_stat_progress_vacuum AS
                       END AS phase,
         S.param2 AS heap_blks_total, S.param3 AS heap_blks_scanned,
         S.param4 AS heap_blks_vacuumed, S.param5 AS index_vacuum_count,
-        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples
+        S.param6 AS max_dead_tuples, S.param7 AS num_dead_tuples,
+        S.param8 AS indexes_total, S.param9 AS indexes_processed
     FROM pg_stat_get_progress_info('VACUUM') AS S
         LEFT JOIN pg_database D ON S.datid = D.oid;
 
diff --git a/src/backend/commands/vacuumparallel.c b/src/backend/commands/vacuumparallel.c
index a79067fd46..351ab4957a 100644
--- a/src/backend/commands/vacuumparallel.c
+++ b/src/backend/commands/vacuumparallel.c
@@ -30,6 +30,7 @@
 #include "access/table.h"
 #include "access/xact.h"
 #include "catalog/index.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "optimizer/paths.h"
 #include "pgstat.h"
@@ -631,7 +632,7 @@ parallel_vacuum_process_all_indexes(ParallelVacuumState *pvs, int num_index_scan
 													vacuum));
 	}
 
-	/* Reset the parallel index processing counter */
+	/* Reset the parallel index processing and progress counters */
 	pg_atomic_write_u32(&(pvs->shared->idx), 0);
 
 	/* Setup the shared cost-based vacuum delay and launch workers */
@@ -902,6 +903,12 @@ parallel_vacuum_process_one_index(ParallelVacuumState *pvs, Relation indrel,
 	pvs->status = PARALLEL_INDVAC_STATUS_COMPLETED;
 	pfree(pvs->indname);
 	pvs->indname = NULL;
+
+	/*
+	 * Call the parallel variant of pgstat_progress_incr_param so workers can
+	 * report progress of index vacuum to the leader.
+	 */
+	pgstat_progress_parallel_incr_param(PROGRESS_VACUUM_INDEXES_PROCESSED, 1);
 }
 
 /*
diff --git a/src/include/commands/progress.h b/src/include/commands/progress.h
index e5add41352..2478e87425 100644
--- a/src/include/commands/progress.h
+++ b/src/include/commands/progress.h
@@ -25,6 +25,8 @@
 #define PROGRESS_VACUUM_NUM_INDEX_VACUUMS		4
 #define PROGRESS_VACUUM_MAX_DEAD_TUPLES			5
 #define PROGRESS_VACUUM_NUM_DEAD_TUPLES			6
+#define PROGRESS_VACUUM_INDEXES_TOTAL			7
+#define PROGRESS_VACUUM_INDEXES_PROCESSED		8
 
 /* Phases of vacuum (as advertised via PROGRESS_VACUUM_PHASE) */
 #define PROGRESS_VACUUM_PHASE_SCAN_HEAP			1
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 7fd81e6a7d..e07afcd4aa 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2044,7 +2044,9 @@ pg_stat_progress_vacuum| SELECT s.pid,
     s.param4 AS heap_blks_vacuumed,
     s.param5 AS index_vacuum_count,
     s.param6 AS max_dead_tuples,
-    s.param7 AS num_dead_tuples
+    s.param7 AS num_dead_tuples,
+    s.param8 AS indexes_total,
+    s.param9 AS indexes_processed
    FROM (pg_stat_get_progress_info('VACUUM'::text) s(pid, datid, relid, param1, param2, param3, param4, param5, param6, param7, param8, param9, param10, param11, param12, param13, param14, param15, param16, param17, param18, param19, param20)
      LEFT JOIN pg_database d ON ((s.datid = d.oid)));
 pg_stat_recovery_prefetch| SELECT stats_reset,
-- 
2.31.1

v31-0001-Add-new-parallel-message-type-to-progress-report.patchapplication/octet-stream; name=v31-0001-Add-new-parallel-message-type-to-progress-report.patchDownload
From f9e1cc968f4e3f1067df0159d194c6462c0fd53f Mon Sep 17 00:00:00 2001
From: Masahiko Sawada <sawada.mshk@gmail.com>
Date: Thu, 6 Jul 2023 14:06:47 +0900
Subject: [PATCH v31 1/2] Add new parallel message type to progress reporting.

This commit adds a new type of parallel message 'P' to allow a
parallel worker to poke at a leader to update the progress.

Currently it supports only incremental progress reporting but it's
possible to allow for supporting of other backend progress APIs in the
future.

There are no users of this new message type as of this commit. That
will follow in future commits.

Idea from Andres Freund.

Authors: Sami Imseih
Reviewed by: Michael Paquier, Masahiko Sawada
Discussion: https://www.postgresql.org/message-id/flat/5478DFCD-2333-401A-B2F0-0D186AB09228@amazon.com
---
 src/backend/access/transam/parallel.c         | 18 +++++++++++
 src/backend/utils/activity/backend_progress.c | 32 +++++++++++++++++++
 src/include/utils/backend_progress.h          |  1 +
 3 files changed, 51 insertions(+)

diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 2b8bc2f58d..2bd04bd177 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -24,6 +24,7 @@
 #include "catalog/pg_enum.h"
 #include "catalog/storage.h"
 #include "commands/async.h"
+#include "commands/progress.h"
 #include "commands/vacuum.h"
 #include "executor/execParallel.h"
 #include "libpq/libpq.h"
@@ -1199,6 +1200,23 @@ HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
 				break;
 			}
 
+		case 'P':				/* Parallel progress reporting */
+			{
+				/*
+				 * Only incremental progress reporting is currently supported.
+				 * However, it's possible to add more fields to the message to
+				 * allow for handling of other backend progress APIs.
+				 */
+				int			index = pq_getmsgint(msg, 4);
+				int64		incr = pq_getmsgint64(msg);
+
+				pq_getmsgend(msg);
+
+				pgstat_progress_incr_param(index, incr);
+
+				break;
+			}
+
 		case 'X':				/* Terminate, indicating clean exit */
 			{
 				shm_mq_detach(pcxt->worker[i].error_mqh);
diff --git a/src/backend/utils/activity/backend_progress.c b/src/backend/utils/activity/backend_progress.c
index fb48eafef9..67447ef03a 100644
--- a/src/backend/utils/activity/backend_progress.c
+++ b/src/backend/utils/activity/backend_progress.c
@@ -10,6 +10,8 @@
  */
 #include "postgres.h"
 
+#include "access/parallel.h"
+#include "libpq/pqformat.h"
 #include "port/atomics.h"		/* for memory barriers */
 #include "utils/backend_progress.h"
 #include "utils/backend_status.h"
@@ -79,6 +81,36 @@ pgstat_progress_incr_param(int index, int64 incr)
 	PGSTAT_END_WRITE_ACTIVITY(beentry);
 }
 
+/*-----------
+ * pgstat_progress_parallel_incr_param() -
+ *
+ * A variant of pgstat_progress_incr_param to allow a worker to poke at
+ * a leader to do an incremental progress update.
+ *-----------
+ */
+void
+pgstat_progress_parallel_incr_param(int index, int64 incr)
+{
+	/*
+	 * Parallel workers notify a leader through a 'P' protocol message to
+	 * update progress, passing the progress index and incremented value.
+	 * Leaders can just call pgstat_progress_incr_param directly.
+	 */
+	if (IsParallelWorker())
+	{
+		static StringInfoData progress_message;
+
+		initStringInfo(&progress_message);
+
+		pq_beginmessage(&progress_message, 'P');
+		pq_sendint32(&progress_message, index);
+		pq_sendint64(&progress_message, incr);
+		pq_endmessage(&progress_message);
+	}
+	else
+		pgstat_progress_incr_param(index, incr);
+}
+
 /*-----------
  * pgstat_progress_update_multi_param() -
  *
diff --git a/src/include/utils/backend_progress.h b/src/include/utils/backend_progress.h
index a84752ade9..70dea55fc0 100644
--- a/src/include/utils/backend_progress.h
+++ b/src/include/utils/backend_progress.h
@@ -37,6 +37,7 @@ extern void pgstat_progress_start_command(ProgressCommandType cmdtype,
 										  Oid relid);
 extern void pgstat_progress_update_param(int index, int64 val);
 extern void pgstat_progress_incr_param(int index, int64 incr);
+extern void pgstat_progress_parallel_incr_param(int index, int64 incr);
 extern void pgstat_progress_update_multi_param(int nparam, const int *index,
 											   const int64 *val);
 extern void pgstat_progress_end_command(void);
-- 
2.31.1

#138Masahiko Sawada
sawada.mshk@gmail.com
In reply to: Masahiko Sawada (#137)
Re: Add index scan progress to pg_stat_progress_vacuum

On Thu, Jul 6, 2023 at 2:28 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:

On Thu, Jul 6, 2023 at 11:15 AM Michael Paquier <michael@paquier.xyz> wrote:

On Thu, Jul 06, 2023 at 11:07:14AM +0900, Masahiko Sawada wrote:

Thank you for updating the patch. It looks good to me too. I've
updated the commit message.

Thanks. I was planning to review this patch today and/or tomorrow now
that my stack of things to do is getting slightly lower (registered my
name as committer as well a few weeks ago to not format).

One thing I was planning to do is to move the new message processing
API for the incrementational updates in its own commit for clarity, as
that's a separate concept than the actual feature, useful on its own.

+1. I had the same idea. Please find the attached patches.

Anyway, would you prefer taking care of it?

I can take care of it if you're okay.

Pushed.

--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com