1
0
mirror of https://github.com/postgres/postgres.git synced 2025-11-19 13:42:17 +03:00

Show index search count in EXPLAIN ANALYZE, take 2.

Expose the count of index searches/index descents in EXPLAIN ANALYZE's
output for index scan/index-only scan/bitmap index scan nodes.  This
information is particularly useful with scans that use ScalarArrayOp
quals, where the number of index searches can be unpredictable due to
implementation details that interact with physical index characteristics
(at least with nbtree SAOP scans, since Postgres 17 commit 5bf748b8).
The information shown also provides useful context when EXPLAIN ANALYZE
runs a plan with an index scan node that successfully applied the skip
scan optimization (set to be added to nbtree by an upcoming patch).

The instrumentation works by teaching all index AMs to increment a new
nsearches counter whenever a new index search begins.  The counter is
incremented at exactly the same point that index AMs already increment
the pg_stat_*_indexes.idx_scan counter (we're counting the same event,
but at the scan level rather than the relation level).  Parallel queries
have workers copy their local counter struct into shared memory when an
index scan node ends -- even when it isn't a parallel aware scan node.
An earlier version of this patch that only worked with parallel aware
scans became commit 5ead85fb (though that was quickly reverted by commit
d00107cd following "debug_parallel_query=regress" buildfarm failures).

Our approach doesn't match the approach used when tracking other index
scan related costs (e.g., "Rows Removed by Filter:").  It is comparable
to the approach used in similar cases involving costs that are only
readily accessible inside an access method, not from the executor proper
(e.g., "Heap Blocks:" output for a Bitmap Heap Scan, which was recently
enhanced to show per-worker costs by commit 5a1e6df3, using essentially
the same scheme as the one used here).  It is necessary for index AMs to
have direct responsibility for maintaining the new counter, since the
counter might need to be incremented multiple times per amgettuple call
(or per amgetbitmap call).  But it is also necessary for the executor
proper to manage the shared memory now used to transfer each worker's
counter struct to the leader.

Author: Peter Geoghegan <pg@bowt.ie>
Reviewed-By: Robert Haas <robertmhaas@gmail.com>
Reviewed-By: Tomas Vondra <tomas@vondra.me>
Reviewed-By: Masahiro Ikeda <ikedamsh@oss.nttdata.com>
Reviewed-By: Matthias van de Meent <boekewurm+postgres@gmail.com>
Discussion: https://postgr.es/m/CAH2-WzkRqvaqR2CTNqTZP0z6FuL4-3ED6eQB0yx38XBNj1v-4Q@mail.gmail.com
Discussion: https://postgr.es/m/CAH2-Wz=PKR6rB7qbx+Vnd7eqeB5VTcrW=iJvAsTsKbdG+kW_UA@mail.gmail.com
This commit is contained in:
Peter Geoghegan
2025-03-11 09:20:50 -04:00
parent 12c5f797ea
commit 0fbceae841
37 changed files with 797 additions and 98 deletions

View File

@@ -592,6 +592,8 @@ bringetbitmap(IndexScanDesc scan, TIDBitmap *tbm)
opaque = (BrinOpaque *) scan->opaque;
bdesc = opaque->bo_bdesc;
pgstat_count_index_scan(idxRel);
if (scan->instrument)
scan->instrument->nsearches++;
/*
* We need to know the size of the table so that we know how long to

View File

@@ -442,6 +442,8 @@ ginNewScanKey(IndexScanDesc scan)
MemoryContextSwitchTo(oldCtx);
pgstat_count_index_scan(scan->indexRelation);
if (scan->instrument)
scan->instrument->nsearches++;
}
void

View File

@@ -625,6 +625,8 @@ gistgettuple(IndexScanDesc scan, ScanDirection dir)
GISTSearchItem fakeItem;
pgstat_count_index_scan(scan->indexRelation);
if (scan->instrument)
scan->instrument->nsearches++;
so->firstCall = false;
so->curPageData = so->nPageData = 0;
@@ -750,6 +752,8 @@ gistgetbitmap(IndexScanDesc scan, TIDBitmap *tbm)
return 0;
pgstat_count_index_scan(scan->indexRelation);
if (scan->instrument)
scan->instrument->nsearches++;
/* Begin the scan by processing the root page */
so->curPageData = so->nPageData = 0;

View File

@@ -298,6 +298,8 @@ _hash_first(IndexScanDesc scan, ScanDirection dir)
HashScanPosItem *currItem;
pgstat_count_index_scan(rel);
if (scan->instrument)
scan->instrument->nsearches++;
/*
* We do not support hash scans with no index qualification, because we

View File

@@ -749,7 +749,7 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
tableScan = NULL;
heapScan = NULL;
indexScan = index_beginscan(OldHeap, OldIndex, SnapshotAny, 0, 0);
indexScan = index_beginscan(OldHeap, OldIndex, SnapshotAny, NULL, 0, 0);
index_rescan(indexScan, NULL, 0, NULL, 0);
}
else

View File

@@ -119,6 +119,7 @@ RelationGetIndexScan(Relation indexRelation, int nkeys, int norderbys)
scan->ignore_killed_tuples = !scan->xactStartedInRecovery;
scan->opaque = NULL;
scan->instrument = NULL;
scan->xs_itup = NULL;
scan->xs_itupdesc = NULL;
@@ -446,7 +447,7 @@ systable_beginscan(Relation heapRelation,
}
sysscan->iscan = index_beginscan(heapRelation, irel,
snapshot, nkeys, 0);
snapshot, NULL, nkeys, 0);
index_rescan(sysscan->iscan, idxkey, nkeys, NULL, 0);
sysscan->scan = NULL;
@@ -711,7 +712,7 @@ systable_beginscan_ordered(Relation heapRelation,
}
sysscan->iscan = index_beginscan(heapRelation, indexRelation,
snapshot, nkeys, 0);
snapshot, NULL, nkeys, 0);
index_rescan(sysscan->iscan, idxkey, nkeys, NULL, 0);
sysscan->scan = NULL;

View File

@@ -256,6 +256,7 @@ IndexScanDesc
index_beginscan(Relation heapRelation,
Relation indexRelation,
Snapshot snapshot,
IndexScanInstrumentation *instrument,
int nkeys, int norderbys)
{
IndexScanDesc scan;
@@ -270,6 +271,7 @@ index_beginscan(Relation heapRelation,
*/
scan->heapRelation = heapRelation;
scan->xs_snapshot = snapshot;
scan->instrument = instrument;
/* prepare to fetch index matches from table */
scan->xs_heapfetch = table_index_fetch_begin(heapRelation);
@@ -286,6 +288,7 @@ index_beginscan(Relation heapRelation,
IndexScanDesc
index_beginscan_bitmap(Relation indexRelation,
Snapshot snapshot,
IndexScanInstrumentation *instrument,
int nkeys)
{
IndexScanDesc scan;
@@ -299,6 +302,7 @@ index_beginscan_bitmap(Relation indexRelation,
* up by RelationGetIndexScan.
*/
scan->xs_snapshot = snapshot;
scan->instrument = instrument;
return scan;
}
@@ -448,14 +452,19 @@ index_restrpos(IndexScanDesc scan)
/*
* index_parallelscan_estimate - estimate shared memory for parallel scan
*
* When instrument=true, estimate includes SharedIndexScanInstrumentation
* space. When parallel_aware=true, estimate includes whatever space the
* index AM's amestimateparallelscan routine requested when called.
*/
Size
index_parallelscan_estimate(Relation indexRelation, int nkeys, int norderbys,
Snapshot snapshot)
Snapshot snapshot, bool instrument,
bool parallel_aware, int nworkers)
{
Size nbytes;
Assert(snapshot != InvalidSnapshot);
Assert(instrument || parallel_aware);
RELATION_CHECKS;
@@ -463,12 +472,22 @@ index_parallelscan_estimate(Relation indexRelation, int nkeys, int norderbys,
nbytes = add_size(nbytes, EstimateSnapshotSpace(snapshot));
nbytes = MAXALIGN(nbytes);
if (instrument)
{
Size sharedinfosz;
sharedinfosz = offsetof(SharedIndexScanInstrumentation, winstrument) +
nworkers * sizeof(IndexScanInstrumentation);
nbytes = add_size(nbytes, sharedinfosz);
nbytes = MAXALIGN(nbytes);
}
/*
* If amestimateparallelscan is not provided, assume there is no
* AM-specific data needed. (It's hard to believe that could work, but
* it's easy enough to cater to it here.)
* If parallel scan index AM interface can't be used (or index AM provides
* no such interface), assume there is no AM-specific data needed
*/
if (indexRelation->rd_indam->amestimateparallelscan != NULL)
if (parallel_aware &&
indexRelation->rd_indam->amestimateparallelscan != NULL)
nbytes = add_size(nbytes,
indexRelation->rd_indam->amestimateparallelscan(nkeys,
norderbys));
@@ -488,11 +507,14 @@ index_parallelscan_estimate(Relation indexRelation, int nkeys, int norderbys,
*/
void
index_parallelscan_initialize(Relation heapRelation, Relation indexRelation,
Snapshot snapshot, ParallelIndexScanDesc target)
Snapshot snapshot, bool instrument,
bool parallel_aware, int nworkers,
SharedIndexScanInstrumentation **sharedinfo,
ParallelIndexScanDesc target)
{
Size offset;
Assert(snapshot != InvalidSnapshot);
Assert(instrument || parallel_aware);
RELATION_CHECKS;
@@ -502,15 +524,34 @@ index_parallelscan_initialize(Relation heapRelation, Relation indexRelation,
target->ps_locator = heapRelation->rd_locator;
target->ps_indexlocator = indexRelation->rd_locator;
target->ps_offset = offset;
target->ps_offset_ins = 0;
target->ps_offset_am = 0;
SerializeSnapshot(snapshot, target->ps_snapshot_data);
if (instrument)
{
Size sharedinfosz;
target->ps_offset_ins = offset;
sharedinfosz = offsetof(SharedIndexScanInstrumentation, winstrument) +
nworkers * sizeof(IndexScanInstrumentation);
offset = add_size(offset, sharedinfosz);
offset = MAXALIGN(offset);
/* Set leader's *sharedinfo pointer, and initialize stats */
*sharedinfo = (SharedIndexScanInstrumentation *)
OffsetToPointer(target, target->ps_offset_ins);
memset(*sharedinfo, 0, sharedinfosz);
(*sharedinfo)->num_workers = nworkers;
}
/* aminitparallelscan is optional; assume no-op if not provided by AM */
if (indexRelation->rd_indam->aminitparallelscan != NULL)
if (parallel_aware && indexRelation->rd_indam->aminitparallelscan != NULL)
{
void *amtarget;
amtarget = OffsetToPointer(target, offset);
target->ps_offset_am = offset;
amtarget = OffsetToPointer(target, target->ps_offset_am);
indexRelation->rd_indam->aminitparallelscan(amtarget);
}
}
@@ -538,8 +579,10 @@ index_parallelrescan(IndexScanDesc scan)
* Caller must be holding suitable locks on the heap and the index.
*/
IndexScanDesc
index_beginscan_parallel(Relation heaprel, Relation indexrel, int nkeys,
int norderbys, ParallelIndexScanDesc pscan)
index_beginscan_parallel(Relation heaprel, Relation indexrel,
IndexScanInstrumentation *instrument,
int nkeys, int norderbys,
ParallelIndexScanDesc pscan)
{
Snapshot snapshot;
IndexScanDesc scan;
@@ -558,6 +601,7 @@ index_beginscan_parallel(Relation heaprel, Relation indexrel, int nkeys,
*/
scan->heapRelation = heaprel;
scan->xs_snapshot = snapshot;
scan->instrument = instrument;
/* prepare to fetch index matches from table */
scan->xs_heapfetch = table_index_fetch_begin(heaprel);

View File

@@ -574,7 +574,7 @@ btparallelrescan(IndexScanDesc scan)
Assert(parallel_scan);
btscan = (BTParallelScanDesc) OffsetToPointer(parallel_scan,
parallel_scan->ps_offset);
parallel_scan->ps_offset_am);
/*
* In theory, we don't need to acquire the LWLock here, because there
@@ -652,7 +652,7 @@ _bt_parallel_seize(IndexScanDesc scan, BlockNumber *next_scan_page,
}
btscan = (BTParallelScanDesc) OffsetToPointer(parallel_scan,
parallel_scan->ps_offset);
parallel_scan->ps_offset_am);
while (1)
{
@@ -760,7 +760,7 @@ _bt_parallel_release(IndexScanDesc scan, BlockNumber next_scan_page,
Assert(BlockNumberIsValid(next_scan_page));
btscan = (BTParallelScanDesc) OffsetToPointer(parallel_scan,
parallel_scan->ps_offset);
parallel_scan->ps_offset_am);
LWLockAcquire(&btscan->btps_lock, LW_EXCLUSIVE);
btscan->btps_nextScanPage = next_scan_page;
@@ -799,7 +799,7 @@ _bt_parallel_done(IndexScanDesc scan)
return;
btscan = (BTParallelScanDesc) OffsetToPointer(parallel_scan,
parallel_scan->ps_offset);
parallel_scan->ps_offset_am);
/*
* Mark the parallel scan as done, unless some other process did so
@@ -837,7 +837,7 @@ _bt_parallel_primscan_schedule(IndexScanDesc scan, BlockNumber curr_page)
Assert(so->numArrayKeys);
btscan = (BTParallelScanDesc) OffsetToPointer(parallel_scan,
parallel_scan->ps_offset);
parallel_scan->ps_offset_am);
LWLockAcquire(&btscan->btps_lock, LW_EXCLUSIVE);
if (btscan->btps_lastCurrPage == curr_page &&

View File

@@ -950,6 +950,8 @@ _bt_first(IndexScanDesc scan, ScanDirection dir)
* _bt_search/_bt_endpoint below
*/
pgstat_count_index_scan(rel);
if (scan->instrument)
scan->instrument->nsearches++;
/*----------
* Examine the scan keys to discover where we need to start the scan.

View File

@@ -421,6 +421,8 @@ spgrescan(IndexScanDesc scan, ScanKey scankey, int nscankeys,
/* count an indexscan for stats */
pgstat_count_index_scan(scan->indexRelation);
if (scan->instrument)
scan->instrument->nsearches++;
}
void

View File

@@ -125,6 +125,7 @@ static void show_recursive_union_info(RecursiveUnionState *rstate,
static void show_memoize_info(MemoizeState *mstate, List *ancestors,
ExplainState *es);
static void show_hashagg_info(AggState *aggstate, ExplainState *es);
static void show_indexsearches_info(PlanState *planstate, ExplainState *es);
static void show_tidbitmap_info(BitmapHeapScanState *planstate,
ExplainState *es);
static void show_instrumentation_count(const char *qlabel, int which,
@@ -2096,6 +2097,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
if (plan->qual)
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
show_indexsearches_info(planstate, es);
break;
case T_IndexOnlyScan:
show_scan_qual(((IndexOnlyScan *) plan)->indexqual,
@@ -2112,10 +2114,12 @@ ExplainNode(PlanState *planstate, List *ancestors,
if (es->analyze)
ExplainPropertyFloat("Heap Fetches", NULL,
planstate->instrument->ntuples2, 0, es);
show_indexsearches_info(planstate, es);
break;
case T_BitmapIndexScan:
show_scan_qual(((BitmapIndexScan *) plan)->indexqualorig,
"Index Cond", planstate, ancestors, es);
show_indexsearches_info(planstate, es);
break;
case T_BitmapHeapScan:
show_scan_qual(((BitmapHeapScan *) plan)->bitmapqualorig,
@@ -3855,6 +3859,65 @@ show_hashagg_info(AggState *aggstate, ExplainState *es)
}
}
/*
* Show the total number of index searches for a
* IndexScan/IndexOnlyScan/BitmapIndexScan node
*/
static void
show_indexsearches_info(PlanState *planstate, ExplainState *es)
{
Plan *plan = planstate->plan;
SharedIndexScanInstrumentation *SharedInfo = NULL;
uint64 nsearches = 0;
if (!es->analyze)
return;
/* Initialize counters with stats from the local process first */
switch (nodeTag(plan))
{
case T_IndexScan:
{
IndexScanState *indexstate = ((IndexScanState *) planstate);
nsearches = indexstate->iss_Instrument.nsearches;
SharedInfo = indexstate->iss_SharedInfo;
break;
}
case T_IndexOnlyScan:
{
IndexOnlyScanState *indexstate = ((IndexOnlyScanState *) planstate);
nsearches = indexstate->ioss_Instrument.nsearches;
SharedInfo = indexstate->ioss_SharedInfo;
break;
}
case T_BitmapIndexScan:
{
BitmapIndexScanState *indexstate = ((BitmapIndexScanState *) planstate);
nsearches = indexstate->biss_Instrument.nsearches;
SharedInfo = indexstate->biss_SharedInfo;
break;
}
default:
break;
}
/* Next get the sum of the counters set within each and every process */
if (SharedInfo)
{
for (int i = 0; i < SharedInfo->num_workers; ++i)
{
IndexScanInstrumentation *winstrument = &SharedInfo->winstrument[i];
nsearches += winstrument->nsearches;
}
}
ExplainPropertyUInteger("Index Searches", NULL, nsearches, es);
}
/*
* Show exact/lossy pages for a BitmapHeapScan node
*/

View File

@@ -816,7 +816,7 @@ check_exclusion_or_unique_constraint(Relation heap, Relation index,
retry:
conflict = false;
found_self = false;
index_scan = index_beginscan(heap, index, &DirtySnapshot, indnkeyatts, 0);
index_scan = index_beginscan(heap, index, &DirtySnapshot, NULL, indnkeyatts, 0);
index_rescan(index_scan, scankeys, indnkeyatts, NULL, 0);
while (index_getnext_slot(index_scan, ForwardScanDirection, existing_slot))

View File

@@ -28,6 +28,7 @@
#include "executor/nodeAgg.h"
#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeBitmapIndexscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeHash.h"
@@ -244,14 +245,19 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
e->pcxt);
break;
case T_IndexScanState:
if (planstate->plan->parallel_aware)
ExecIndexScanEstimate((IndexScanState *) planstate,
e->pcxt);
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecIndexScanEstimate((IndexScanState *) planstate,
e->pcxt);
break;
case T_IndexOnlyScanState:
if (planstate->plan->parallel_aware)
ExecIndexOnlyScanEstimate((IndexOnlyScanState *) planstate,
e->pcxt);
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecIndexOnlyScanEstimate((IndexOnlyScanState *) planstate,
e->pcxt);
break;
case T_BitmapIndexScanState:
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecBitmapIndexScanEstimate((BitmapIndexScanState *) planstate,
e->pcxt);
break;
case T_ForeignScanState:
if (planstate->plan->parallel_aware)
@@ -468,14 +474,17 @@ ExecParallelInitializeDSM(PlanState *planstate,
d->pcxt);
break;
case T_IndexScanState:
if (planstate->plan->parallel_aware)
ExecIndexScanInitializeDSM((IndexScanState *) planstate,
d->pcxt);
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecIndexScanInitializeDSM((IndexScanState *) planstate, d->pcxt);
break;
case T_IndexOnlyScanState:
if (planstate->plan->parallel_aware)
ExecIndexOnlyScanInitializeDSM((IndexOnlyScanState *) planstate,
d->pcxt);
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecIndexOnlyScanInitializeDSM((IndexOnlyScanState *) planstate,
d->pcxt);
break;
case T_BitmapIndexScanState:
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecBitmapIndexScanInitializeDSM((BitmapIndexScanState *) planstate, d->pcxt);
break;
case T_ForeignScanState:
if (planstate->plan->parallel_aware)
@@ -1002,6 +1011,7 @@ ExecParallelReInitializeDSM(PlanState *planstate,
ExecHashJoinReInitializeDSM((HashJoinState *) planstate,
pcxt);
break;
case T_BitmapIndexScanState:
case T_HashState:
case T_SortState:
case T_IncrementalSortState:
@@ -1063,6 +1073,15 @@ ExecParallelRetrieveInstrumentation(PlanState *planstate,
/* Perform any node-type-specific work that needs to be done. */
switch (nodeTag(planstate))
{
case T_IndexScanState:
ExecIndexScanRetrieveInstrumentation((IndexScanState *) planstate);
break;
case T_IndexOnlyScanState:
ExecIndexOnlyScanRetrieveInstrumentation((IndexOnlyScanState *) planstate);
break;
case T_BitmapIndexScanState:
ExecBitmapIndexScanRetrieveInstrumentation((BitmapIndexScanState *) planstate);
break;
case T_SortState:
ExecSortRetrieveInstrumentation((SortState *) planstate);
break;
@@ -1330,14 +1349,18 @@ ExecParallelInitializeWorker(PlanState *planstate, ParallelWorkerContext *pwcxt)
ExecSeqScanInitializeWorker((SeqScanState *) planstate, pwcxt);
break;
case T_IndexScanState:
if (planstate->plan->parallel_aware)
ExecIndexScanInitializeWorker((IndexScanState *) planstate,
pwcxt);
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecIndexScanInitializeWorker((IndexScanState *) planstate, pwcxt);
break;
case T_IndexOnlyScanState:
if (planstate->plan->parallel_aware)
ExecIndexOnlyScanInitializeWorker((IndexOnlyScanState *) planstate,
pwcxt);
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecIndexOnlyScanInitializeWorker((IndexOnlyScanState *) planstate,
pwcxt);
break;
case T_BitmapIndexScanState:
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecBitmapIndexScanInitializeWorker((BitmapIndexScanState *) planstate,
pwcxt);
break;
case T_ForeignScanState:
if (planstate->plan->parallel_aware)

View File

@@ -202,7 +202,7 @@ RelationFindReplTupleByIndex(Relation rel, Oid idxoid,
skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot);
/* Start an index scan. */
scan = index_beginscan(rel, idxrel, &snap, skey_attoff, 0);
scan = index_beginscan(rel, idxrel, &snap, NULL, skey_attoff, 0);
retry:
found = false;

View File

@@ -183,6 +183,27 @@ ExecEndBitmapIndexScan(BitmapIndexScanState *node)
indexRelationDesc = node->biss_RelationDesc;
indexScanDesc = node->biss_ScanDesc;
/*
* When ending a parallel worker, copy the statistics gathered by the
* worker back into shared memory so that it can be picked up by the main
* process to report in EXPLAIN ANALYZE
*/
if (node->biss_SharedInfo != NULL && IsParallelWorker())
{
IndexScanInstrumentation *winstrument;
Assert(ParallelWorkerNumber <= node->biss_SharedInfo->num_workers);
winstrument = &node->biss_SharedInfo->winstrument[ParallelWorkerNumber];
/*
* We have to accumulate the stats rather than performing a memcpy.
* When a Gather/GatherMerge node finishes it will perform planner
* shutdown on the workers. On rescan it will spin up new workers
* which will have a new BitmapIndexScanState and zeroed stats.
*/
winstrument->nsearches += node->biss_Instrument.nsearches;
}
/*
* close the index relation (no-op if we didn't open it)
*/
@@ -302,6 +323,7 @@ ExecInitBitmapIndexScan(BitmapIndexScan *node, EState *estate, int eflags)
indexstate->biss_ScanDesc =
index_beginscan_bitmap(indexstate->biss_RelationDesc,
estate->es_snapshot,
&indexstate->biss_Instrument,
indexstate->biss_NumScanKeys);
/*
@@ -319,3 +341,97 @@ ExecInitBitmapIndexScan(BitmapIndexScan *node, EState *estate, int eflags)
*/
return indexstate;
}
/* ----------------------------------------------------------------
* ExecBitmapIndexScanEstimate
*
* Compute the amount of space we'll need in the parallel
* query DSM, and inform pcxt->estimator about our needs.
* ----------------------------------------------------------------
*/
void
ExecBitmapIndexScanEstimate(BitmapIndexScanState *node, ParallelContext *pcxt)
{
Size size;
/*
* Parallel bitmap index scans are not supported, but we still need to
* store the scan's instrumentation in DSM during parallel query
*/
if (!node->ss.ps.instrument || pcxt->nworkers == 0)
return;
size = offsetof(SharedIndexScanInstrumentation, winstrument) +
pcxt->nworkers * sizeof(IndexScanInstrumentation);
shm_toc_estimate_chunk(&pcxt->estimator, size);
shm_toc_estimate_keys(&pcxt->estimator, 1);
}
/* ----------------------------------------------------------------
* ExecBitmapIndexScanInitializeDSM
*
* Set up bitmap index scan shared instrumentation.
* ----------------------------------------------------------------
*/
void
ExecBitmapIndexScanInitializeDSM(BitmapIndexScanState *node,
ParallelContext *pcxt)
{
Size size;
/* don't need this if not instrumenting or no workers */
if (!node->ss.ps.instrument || pcxt->nworkers == 0)
return;
size = offsetof(SharedIndexScanInstrumentation, winstrument) +
pcxt->nworkers * sizeof(IndexScanInstrumentation);
node->biss_SharedInfo =
(SharedIndexScanInstrumentation *) shm_toc_allocate(pcxt->toc,
size);
shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id,
node->biss_SharedInfo);
/* Each per-worker area must start out as zeroes */
memset(node->biss_SharedInfo, 0, size);
node->biss_SharedInfo->num_workers = pcxt->nworkers;
}
/* ----------------------------------------------------------------
* ExecBitmapIndexScanInitializeWorker
*
* Copy relevant information from TOC into planstate.
* ----------------------------------------------------------------
*/
void
ExecBitmapIndexScanInitializeWorker(BitmapIndexScanState *node,
ParallelWorkerContext *pwcxt)
{
/* don't need this if not instrumenting */
if (!node->ss.ps.instrument)
return;
node->biss_SharedInfo = (SharedIndexScanInstrumentation *)
shm_toc_lookup(pwcxt->toc, node->ss.ps.plan->plan_node_id, false);
}
/* ----------------------------------------------------------------
* ExecBitmapIndexScanRetrieveInstrumentation
*
* Transfer bitmap index scan statistics from DSM to private memory.
* ----------------------------------------------------------------
*/
void
ExecBitmapIndexScanRetrieveInstrumentation(BitmapIndexScanState *node)
{
SharedIndexScanInstrumentation *SharedInfo = node->biss_SharedInfo;
size_t size;
if (SharedInfo == NULL)
return;
/* Create a copy of SharedInfo in backend-local memory */
size = offsetof(SharedIndexScanInstrumentation, winstrument) +
SharedInfo->num_workers * sizeof(IndexScanInstrumentation);
node->biss_SharedInfo = palloc(size);
memcpy(node->biss_SharedInfo, SharedInfo, size);
}

View File

@@ -92,6 +92,7 @@ IndexOnlyNext(IndexOnlyScanState *node)
scandesc = index_beginscan(node->ss.ss_currentRelation,
node->ioss_RelationDesc,
estate->es_snapshot,
&node->ioss_Instrument,
node->ioss_NumScanKeys,
node->ioss_NumOrderByKeys);
@@ -413,6 +414,27 @@ ExecEndIndexOnlyScan(IndexOnlyScanState *node)
node->ioss_VMBuffer = InvalidBuffer;
}
/*
* When ending a parallel worker, copy the statistics gathered by the
* worker back into shared memory so that it can be picked up by the main
* process to report in EXPLAIN ANALYZE
*/
if (node->ioss_SharedInfo != NULL && IsParallelWorker())
{
IndexScanInstrumentation *winstrument;
Assert(ParallelWorkerNumber <= node->ioss_SharedInfo->num_workers);
winstrument = &node->ioss_SharedInfo->winstrument[ParallelWorkerNumber];
/*
* We have to accumulate the stats rather than performing a memcpy.
* When a Gather/GatherMerge node finishes it will perform planner
* shutdown on the workers. On rescan it will spin up new workers
* which will have a new IndexOnlyScanState and zeroed stats.
*/
winstrument->nsearches += node->ioss_Instrument.nsearches;
}
/*
* close the index relation (no-op if we didn't open it)
*/
@@ -707,11 +729,21 @@ ExecIndexOnlyScanEstimate(IndexOnlyScanState *node,
ParallelContext *pcxt)
{
EState *estate = node->ss.ps.state;
bool instrument = (node->ss.ps.instrument != NULL);
bool parallel_aware = node->ss.ps.plan->parallel_aware;
if (!instrument && !parallel_aware)
{
/* No DSM required by the scan */
return;
}
node->ioss_PscanLen = index_parallelscan_estimate(node->ioss_RelationDesc,
node->ioss_NumScanKeys,
node->ioss_NumOrderByKeys,
estate->es_snapshot);
estate->es_snapshot,
instrument, parallel_aware,
pcxt->nworkers);
shm_toc_estimate_chunk(&pcxt->estimator, node->ioss_PscanLen);
shm_toc_estimate_keys(&pcxt->estimator, 1);
}
@@ -728,16 +760,33 @@ ExecIndexOnlyScanInitializeDSM(IndexOnlyScanState *node,
{
EState *estate = node->ss.ps.state;
ParallelIndexScanDesc piscan;
bool instrument = node->ss.ps.instrument != NULL;
bool parallel_aware = node->ss.ps.plan->parallel_aware;
if (!instrument && !parallel_aware)
{
/* No DSM required by the scan */
return;
}
piscan = shm_toc_allocate(pcxt->toc, node->ioss_PscanLen);
index_parallelscan_initialize(node->ss.ss_currentRelation,
node->ioss_RelationDesc,
estate->es_snapshot,
piscan);
instrument, parallel_aware, pcxt->nworkers,
&node->ioss_SharedInfo, piscan);
shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, piscan);
if (!parallel_aware)
{
/* Only here to initialize SharedInfo in DSM */
return;
}
node->ioss_ScanDesc =
index_beginscan_parallel(node->ss.ss_currentRelation,
node->ioss_RelationDesc,
&node->ioss_Instrument,
node->ioss_NumScanKeys,
node->ioss_NumOrderByKeys,
piscan);
@@ -764,6 +813,7 @@ void
ExecIndexOnlyScanReInitializeDSM(IndexOnlyScanState *node,
ParallelContext *pcxt)
{
Assert(node->ss.ps.plan->parallel_aware);
index_parallelrescan(node->ioss_ScanDesc);
}
@@ -778,11 +828,31 @@ ExecIndexOnlyScanInitializeWorker(IndexOnlyScanState *node,
ParallelWorkerContext *pwcxt)
{
ParallelIndexScanDesc piscan;
bool instrument = node->ss.ps.instrument != NULL;
bool parallel_aware = node->ss.ps.plan->parallel_aware;
if (!instrument && !parallel_aware)
{
/* No DSM required by the scan */
return;
}
piscan = shm_toc_lookup(pwcxt->toc, node->ss.ps.plan->plan_node_id, false);
if (instrument)
node->ioss_SharedInfo = (SharedIndexScanInstrumentation *)
OffsetToPointer(piscan, piscan->ps_offset_ins);
if (!parallel_aware)
{
/* Only here to set up worker node's SharedInfo */
return;
}
node->ioss_ScanDesc =
index_beginscan_parallel(node->ss.ss_currentRelation,
node->ioss_RelationDesc,
&node->ioss_Instrument,
node->ioss_NumScanKeys,
node->ioss_NumOrderByKeys,
piscan);
@@ -797,3 +867,25 @@ ExecIndexOnlyScanInitializeWorker(IndexOnlyScanState *node,
node->ioss_ScanKeys, node->ioss_NumScanKeys,
node->ioss_OrderByKeys, node->ioss_NumOrderByKeys);
}
/* ----------------------------------------------------------------
* ExecIndexOnlyScanRetrieveInstrumentation
*
* Transfer index-only scan statistics from DSM to private memory.
* ----------------------------------------------------------------
*/
void
ExecIndexOnlyScanRetrieveInstrumentation(IndexOnlyScanState *node)
{
SharedIndexScanInstrumentation *SharedInfo = node->ioss_SharedInfo;
size_t size;
if (SharedInfo == NULL)
return;
/* Create a copy of SharedInfo in backend-local memory */
size = offsetof(SharedIndexScanInstrumentation, winstrument) +
SharedInfo->num_workers * sizeof(IndexScanInstrumentation);
node->ioss_SharedInfo = palloc(size);
memcpy(node->ioss_SharedInfo, SharedInfo, size);
}

View File

@@ -109,6 +109,7 @@ IndexNext(IndexScanState *node)
scandesc = index_beginscan(node->ss.ss_currentRelation,
node->iss_RelationDesc,
estate->es_snapshot,
&node->iss_Instrument,
node->iss_NumScanKeys,
node->iss_NumOrderByKeys);
@@ -204,6 +205,7 @@ IndexNextWithReorder(IndexScanState *node)
scandesc = index_beginscan(node->ss.ss_currentRelation,
node->iss_RelationDesc,
estate->es_snapshot,
&node->iss_Instrument,
node->iss_NumScanKeys,
node->iss_NumOrderByKeys);
@@ -793,6 +795,27 @@ ExecEndIndexScan(IndexScanState *node)
indexRelationDesc = node->iss_RelationDesc;
indexScanDesc = node->iss_ScanDesc;
/*
* When ending a parallel worker, copy the statistics gathered by the
* worker back into shared memory so that it can be picked up by the main
* process to report in EXPLAIN ANALYZE
*/
if (node->iss_SharedInfo != NULL && IsParallelWorker())
{
IndexScanInstrumentation *winstrument;
Assert(ParallelWorkerNumber <= node->iss_SharedInfo->num_workers);
winstrument = &node->iss_SharedInfo->winstrument[ParallelWorkerNumber];
/*
* We have to accumulate the stats rather than performing a memcpy.
* When a Gather/GatherMerge node finishes it will perform planner
* shutdown on the workers. On rescan it will spin up new workers
* which will have a new IndexOnlyScanState and zeroed stats.
*/
winstrument->nsearches += node->iss_Instrument.nsearches;
}
/*
* close the index relation (no-op if we didn't open it)
*/
@@ -1642,11 +1665,21 @@ ExecIndexScanEstimate(IndexScanState *node,
ParallelContext *pcxt)
{
EState *estate = node->ss.ps.state;
bool instrument = node->ss.ps.instrument != NULL;
bool parallel_aware = node->ss.ps.plan->parallel_aware;
if (!instrument && !parallel_aware)
{
/* No DSM required by the scan */
return;
}
node->iss_PscanLen = index_parallelscan_estimate(node->iss_RelationDesc,
node->iss_NumScanKeys,
node->iss_NumOrderByKeys,
estate->es_snapshot);
estate->es_snapshot,
instrument, parallel_aware,
pcxt->nworkers);
shm_toc_estimate_chunk(&pcxt->estimator, node->iss_PscanLen);
shm_toc_estimate_keys(&pcxt->estimator, 1);
}
@@ -1663,16 +1696,33 @@ ExecIndexScanInitializeDSM(IndexScanState *node,
{
EState *estate = node->ss.ps.state;
ParallelIndexScanDesc piscan;
bool instrument = node->ss.ps.instrument != NULL;
bool parallel_aware = node->ss.ps.plan->parallel_aware;
if (!instrument && !parallel_aware)
{
/* No DSM required by the scan */
return;
}
piscan = shm_toc_allocate(pcxt->toc, node->iss_PscanLen);
index_parallelscan_initialize(node->ss.ss_currentRelation,
node->iss_RelationDesc,
estate->es_snapshot,
piscan);
instrument, parallel_aware, pcxt->nworkers,
&node->iss_SharedInfo, piscan);
shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, piscan);
if (!parallel_aware)
{
/* Only here to initialize SharedInfo in DSM */
return;
}
node->iss_ScanDesc =
index_beginscan_parallel(node->ss.ss_currentRelation,
node->iss_RelationDesc,
&node->iss_Instrument,
node->iss_NumScanKeys,
node->iss_NumOrderByKeys,
piscan);
@@ -1697,6 +1747,7 @@ void
ExecIndexScanReInitializeDSM(IndexScanState *node,
ParallelContext *pcxt)
{
Assert(node->ss.ps.plan->parallel_aware);
index_parallelrescan(node->iss_ScanDesc);
}
@@ -1711,11 +1762,31 @@ ExecIndexScanInitializeWorker(IndexScanState *node,
ParallelWorkerContext *pwcxt)
{
ParallelIndexScanDesc piscan;
bool instrument = node->ss.ps.instrument != NULL;
bool parallel_aware = node->ss.ps.plan->parallel_aware;
if (!instrument && !parallel_aware)
{
/* No DSM required by the scan */
return;
}
piscan = shm_toc_lookup(pwcxt->toc, node->ss.ps.plan->plan_node_id, false);
if (instrument)
node->iss_SharedInfo = (SharedIndexScanInstrumentation *)
OffsetToPointer(piscan, piscan->ps_offset_ins);
if (!parallel_aware)
{
/* Only here to set up worker node's SharedInfo */
return;
}
node->iss_ScanDesc =
index_beginscan_parallel(node->ss.ss_currentRelation,
node->iss_RelationDesc,
&node->iss_Instrument,
node->iss_NumScanKeys,
node->iss_NumOrderByKeys,
piscan);
@@ -1729,3 +1800,25 @@ ExecIndexScanInitializeWorker(IndexScanState *node,
node->iss_ScanKeys, node->iss_NumScanKeys,
node->iss_OrderByKeys, node->iss_NumOrderByKeys);
}
/* ----------------------------------------------------------------
* ExecIndexScanRetrieveInstrumentation
*
* Transfer index scan statistics from DSM to private memory.
* ----------------------------------------------------------------
*/
void
ExecIndexScanRetrieveInstrumentation(IndexScanState *node)
{
SharedIndexScanInstrumentation *SharedInfo = node->iss_SharedInfo;
size_t size;
if (SharedInfo == NULL)
return;
/* Create a copy of SharedInfo in backend-local memory */
size = offsetof(SharedIndexScanInstrumentation, winstrument) +
SharedInfo->num_workers * sizeof(IndexScanInstrumentation);
node->iss_SharedInfo = palloc(size);
memcpy(node->iss_SharedInfo, SharedInfo, size);
}

View File

@@ -6551,7 +6551,7 @@ get_actual_variable_endpoint(Relation heapRel,
GlobalVisTestFor(heapRel));
index_scan = index_beginscan(heapRel, indexRel,
&SnapshotNonVacuumable,
&SnapshotNonVacuumable, NULL,
1, 0);
/* Set it up for index-only scan */
index_scan->xs_want_itup = true;