diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml index 1d70fae..6a3481f 100644 --- a/doc/src/sgml/monitoring.sgml +++ b/doc/src/sgml/monitoring.sgml @@ -1248,6 +1248,10 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser Waiting for parallel workers to finish computing. + ParallelBitmapPopulate + Waiting for the leader to populate the TidBitmap. + + SafeSnapshot Waiting for a snapshot for a READ ONLY DEFERRABLE transaction. diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c index af25836..bffc971 100644 --- a/src/backend/access/heap/heapam.c +++ b/src/backend/access/heap/heapam.c @@ -1754,6 +1754,22 @@ retry: } /* ---------------- + * heap_update_snapshot + * + * Update snapshot info in heap scan descriptor. + * ---------------- + */ +void +heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot) +{ + Assert(IsMVCCSnapshot(snapshot)); + + RegisterSnapshot(snapshot); + scan->rs_snapshot = snapshot; + scan->rs_temp_snap = true; +} + +/* ---------------- * heap_getnext - retrieve next tuple in scan * * Fix to work with index relations. diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c index 646df08..e5263f2 100644 --- a/src/backend/executor/execParallel.c +++ b/src/backend/executor/execParallel.c @@ -25,6 +25,7 @@ #include "executor/execParallel.h" #include "executor/executor.h" +#include "executor/nodeBitmapHeapscan.h" #include "executor/nodeCustom.h" #include "executor/nodeForeignscan.h" #include "executor/nodeSeqscan.h" @@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e) ExecCustomScanEstimate((CustomScanState *) planstate, e->pcxt); break; + case T_BitmapHeapScanState: + ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate, + e->pcxt); + break; default: break; } @@ -275,6 +280,11 @@ ExecParallelInitializeDSM(PlanState *planstate, ExecCustomScanInitializeDSM((CustomScanState *) planstate, d->pcxt); break; + case T_BitmapHeapScanState: + ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate, + d->pcxt); + break; + default: break; } @@ -757,6 +767,10 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc) ExecCustomScanInitializeWorker((CustomScanState *) planstate, toc); break; + case T_BitmapHeapScanState: + ExecBitmapHeapInitializeWorker( + (BitmapHeapScanState *) planstate, toc); + break; default: break; } diff --git a/src/backend/executor/nodeBitmapHeapscan.c b/src/backend/executor/nodeBitmapHeapscan.c index f18827d..1d92a25 100644 --- a/src/backend/executor/nodeBitmapHeapscan.c +++ b/src/backend/executor/nodeBitmapHeapscan.c @@ -48,10 +48,11 @@ #include "utils/snapmgr.h" #include "utils/tqual.h" - static TupleTableSlot *BitmapHeapNext(BitmapHeapScanState *node); static void bitgetpage(HeapScanDesc scan, TBMIterateResult *tbmres); +static bool pbms_is_leader(ParallelBitmapState *pbminfo); +static void pbms_set_parallel(PlanState *node); /* ---------------------------------------------------------------- * BitmapHeapNext @@ -66,13 +67,17 @@ BitmapHeapNext(BitmapHeapScanState *node) HeapScanDesc scan; TIDBitmap *tbm; TBMIterator *tbmiterator; + TBMSharedIterator *stbmiterator; TBMIterateResult *tbmres; #ifdef USE_PREFETCH TBMIterator *prefetch_iterator; + TBMSharedIterator *sprefetch_iterator; #endif OffsetNumber targoffset; TupleTableSlot *slot; + ParallelBitmapState *pstate = node->pstate; + dsa_area *dsa = node->ss.ps.state->es_query_dsa; /* * extract necessary information from index scan node @@ -82,11 +87,16 @@ BitmapHeapNext(BitmapHeapScanState *node) scan = node->ss.ss_currentScanDesc; tbm = node->tbm; tbmiterator = node->tbmiterator; + stbmiterator = node->stbmiterator; tbmres = node->tbmres; #ifdef USE_PREFETCH prefetch_iterator = node->prefetch_iterator; + sprefetch_iterator = node->sprefetch_iterator; #endif + if (node->inited) + goto start_iterate; + /* * If we haven't yet performed the underlying index scan, do it, and begin * the iteration over the bitmap. @@ -99,7 +109,7 @@ BitmapHeapNext(BitmapHeapScanState *node) * node->prefetch_maximum. This is to avoid doing a lot of prefetching in * a scan that stops after a few tuples because of a LIMIT. */ - if (tbm == NULL) + if (!pstate) { tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node)); @@ -119,7 +129,71 @@ BitmapHeapNext(BitmapHeapScanState *node) } #endif /* USE_PREFETCH */ } + else + { + /* + * The leader will immediately come out of the function, but others + * will be blocked until leader populates the TBM and wakes them up. + */ + if (pbms_is_leader(pstate)) + { + /* + * Set the flag in the lower nodes to indicate that we need a + * shared TBM. + */ + pbms_set_parallel(outerPlanState(node)); + tbm = (TIDBitmap *) MultiExecProcNode(outerPlanState(node)); + if (!tbm || !IsA(tbm, TIDBitmap)) + elog(ERROR, "unrecognized result from subplan"); + + node->tbm = tbm; + + /* + * Prepare to iterate over the TBM. This will return the + * dsa_pointer of the iterator state which will be used by + * multiple processes to iterate jointly. + */ + pstate->tbmiterator = tbm_prepare_shared_iterate(tbm); +#ifdef USE_PREFETCH + if (node->prefetch_maximum > 0) + { + pstate->prefetch_iterator = tbm_prepare_shared_iterate(tbm); + + /* We don't need mutex here as we haven't yet woke up others */ + pstate->prefetch_pages = 0; + pstate->prefetch_target = -1; + } +#endif + + /* + * By this time we have already populated the TBM and initialized + * the shared iterators so set the state to PBM_FINISHED and wake + * up others. + */ + SpinLockAcquire(&pstate->state_mutex); + pstate->state = PBM_FINISHED; + SpinLockRelease(&pstate->state_mutex); + ConditionVariableBroadcast(&pstate->cv); + } + + /* Allocate a private iterator and attach the shared state to it */ + stbmiterator = tbm_attach_shared_iterate(dsa, pstate->tbmiterator); + node->stbmiterator = stbmiterator; + node->tbmres = tbmres = NULL; + +#ifdef USE_PREFETCH + if (node->prefetch_maximum > 0) + { + sprefetch_iterator = + tbm_attach_shared_iterate(dsa, pstate->prefetch_iterator); + node->sprefetch_iterator = sprefetch_iterator; + } +#endif /* USE_PREFETCH */ + } + + node->inited = true; +start_iterate: for (;;) { Page dp; @@ -130,7 +204,14 @@ BitmapHeapNext(BitmapHeapScanState *node) */ if (tbmres == NULL) { - node->tbmres = tbmres = tbm_iterate(tbmiterator); + /* + * If we are in parallel mode perform shared iterate otherwise + * local iterate. + */ + if (!pstate) + node->tbmres = tbmres = tbm_iterate(tbmiterator); + else + node->tbmres = tbmres = tbm_shared_iterate(stbmiterator); if (tbmres == NULL) { /* no more entries in the bitmap */ @@ -138,19 +219,50 @@ BitmapHeapNext(BitmapHeapScanState *node) } #ifdef USE_PREFETCH - if (node->prefetch_pages > 0) + + if (!pstate) { - /* The main iterator has closed the distance by one page */ - node->prefetch_pages--; + if (node->prefetch_pages > 0) + /* The main iterator has closed the distance by one page */ + node->prefetch_pages--; + else if (prefetch_iterator) + { + /* + * Do not let the prefetch iterator get behind the main + * one. + */ + TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator); + + if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno) + elog(ERROR, "prefetch and main iterators are out of sync"); + } } - else if (prefetch_iterator) + else if (node->prefetch_maximum > 0) { - /* Do not let the prefetch iterator get behind the main one */ - TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator); - - if (tbmpre == NULL || tbmpre->blockno != tbmres->blockno) - elog(ERROR, "prefetch and main iterators are out of sync"); + SpinLockAcquire(&pstate->prefetch_mutex); + if (pstate->prefetch_pages > 0) + { + node->prefetch_pages--; + SpinLockRelease(&pstate->prefetch_mutex); + } + else + { + /* Release the mutex before iterating */ + SpinLockRelease(&pstate->prefetch_mutex); + + /* + * In case of shared mode, we can not ensure that the + * current blockno of the main iterator and that of the + * prefetch iterator are same. It's possible that + * whatever blockno we are prefetching will be processed + * by another process. Therefore we don't validate the + * blockno like we do in non-parallel case. + */ + if (sprefetch_iterator) + tbm_shared_iterate(sprefetch_iterator); + } } + #endif /* USE_PREFETCH */ /* @@ -188,14 +300,32 @@ BitmapHeapNext(BitmapHeapScanState *node) * page/tuple, then to one after the second tuple is fetched, then * it doubles as later pages are fetched. */ - if (node->prefetch_target >= node->prefetch_maximum) - /* don't increase any further */ ; - else if (node->prefetch_target >= node->prefetch_maximum / 2) - node->prefetch_target = node->prefetch_maximum; - else if (node->prefetch_target > 0) - node->prefetch_target *= 2; - else - node->prefetch_target++; + if (!pstate) + { + if (node->prefetch_target >= node->prefetch_maximum) + /* don't increase any further */ ; + else if (node->prefetch_target >= node->prefetch_maximum / 2) + node->prefetch_target = node->prefetch_maximum; + else if (node->prefetch_target > 0) + node->prefetch_target *= 2; + else + node->prefetch_target++; + } + else if (pstate->prefetch_target < node->prefetch_maximum) + { + SpinLockAcquire(&pstate->prefetch_mutex); + if (pstate->prefetch_target >= node->prefetch_maximum) + /* don't increase any further */ ; + else if (pstate->prefetch_target >= + node->prefetch_maximum / 2) + pstate->prefetch_target = node->prefetch_maximum; + else if (pstate->prefetch_target > 0) + pstate->prefetch_target *= 2; + else + pstate->prefetch_target++; + SpinLockRelease(&pstate->prefetch_mutex); + } + #endif /* USE_PREFETCH */ } else @@ -211,8 +341,22 @@ BitmapHeapNext(BitmapHeapScanState *node) * Try to prefetch at least a few pages even before we get to the * second page if we don't stop reading after the first tuple. */ - if (node->prefetch_target < node->prefetch_maximum) - node->prefetch_target++; + if (!pstate) + { + if (node->prefetch_target < node->prefetch_maximum) + node->prefetch_target++; + } + else if (pstate->prefetch_target < node->prefetch_maximum) + { + /* + * If we are in parallel mode then grab prefetch_mutex before + * updating prefetch target. + */ + SpinLockAcquire(&pstate->prefetch_mutex); + if (pstate->prefetch_target < node->prefetch_maximum) + pstate->prefetch_target++; + SpinLockRelease(&pstate->prefetch_mutex); + } #endif /* USE_PREFETCH */ } @@ -236,21 +380,71 @@ BitmapHeapNext(BitmapHeapScanState *node) */ if (prefetch_iterator) { - while (node->prefetch_pages < node->prefetch_target) + if (!pstate) { - TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator); + while (node->prefetch_pages < node->prefetch_target) + { + TBMIterateResult *tbmpre = tbm_iterate(prefetch_iterator); + + if (tbmpre == NULL) + { + /* No more pages to prefetch */ + tbm_end_iterate(prefetch_iterator); + node->prefetch_iterator = prefetch_iterator = NULL; + break; + } + node->prefetch_pages++; + PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno); + } + } + else if (pstate->prefetch_pages < pstate->prefetch_target) + { + SpinLockAcquire(&pstate->prefetch_mutex); - if (tbmpre == NULL) + /* Recheck whether we still need to prefetch under the mutex */ + if (!pstate->prefetching && + pstate->prefetch_pages < pstate->prefetch_target) { - /* No more pages to prefetch */ - tbm_end_iterate(prefetch_iterator); - node->prefetch_iterator = prefetch_iterator = NULL; - break; + /* Allow only one process to prefetch */ + pstate->prefetching = true; + SpinLockRelease(&pstate->prefetch_mutex); + + do + { + TBMIterateResult *tbmpre; + + tbmpre = tbm_shared_iterate(sprefetch_iterator); + if (tbmpre == NULL) + { + /* No more pages to prefetch */ + tbm_end_shared_iterate(sprefetch_iterator); + node->sprefetch_iterator = + sprefetch_iterator = NULL; + break; + } + + SpinLockAcquire(&pstate->prefetch_mutex); + pstate->prefetch_pages++; + if (pstate->prefetch_pages >= pstate->prefetch_target) + { + SpinLockRelease(&pstate->prefetch_mutex); + break; + } + SpinLockRelease(&pstate->prefetch_mutex); + + PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, + tbmpre->blockno); + } while (true); + + SpinLockAcquire(&pstate->prefetch_mutex); + pstate->prefetching = false; + SpinLockRelease(&pstate->prefetch_mutex); } - node->prefetch_pages++; - PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, tbmpre->blockno); + else + SpinLockRelease(&pstate->prefetch_mutex); } } + #endif /* USE_PREFETCH */ /* @@ -458,12 +652,40 @@ ExecReScanBitmapHeapScan(BitmapHeapScanState *node) tbm_end_iterate(node->tbmiterator); if (node->prefetch_iterator) tbm_end_iterate(node->prefetch_iterator); + if (node->stbmiterator) + tbm_end_shared_iterate(node->stbmiterator); + if (node->sprefetch_iterator) + tbm_end_shared_iterate(node->sprefetch_iterator); if (node->tbm) tbm_free(node->tbm); node->tbm = NULL; node->tbmiterator = NULL; node->tbmres = NULL; node->prefetch_iterator = NULL; + node->inited = false; + node->stbmiterator = NULL; + node->sprefetch_iterator = NULL; + + /* Reset parallel bitmap state, if present */ + if (node->pstate) + { + dsa_area *dsa = node->ss.ps.state->es_query_dsa; + + node->pstate->state = PBM_INITIAL; + + if (DsaPointerIsValid(node->pstate->tbmiterator)) + { + /* First we free the shared TBM members using the shared state */ + tbm_free_shared_area(dsa, node->pstate->tbmiterator); + dsa_free(dsa, node->pstate->tbmiterator); + } + + if (DsaPointerIsValid(node->pstate->prefetch_iterator)) + dsa_free(dsa, node->pstate->prefetch_iterator); + + node->pstate->tbmiterator = InvalidDsaPointer; + node->pstate->prefetch_iterator = InvalidDsaPointer; + } ExecScanReScan(&node->ss); @@ -516,6 +738,10 @@ ExecEndBitmapHeapScan(BitmapHeapScanState *node) tbm_end_iterate(node->prefetch_iterator); if (node->tbm) tbm_free(node->tbm); + if (node->stbmiterator) + tbm_end_shared_iterate(node->stbmiterator); + if (node->sprefetch_iterator) + tbm_end_shared_iterate(node->sprefetch_iterator); /* * close heap scan @@ -567,6 +793,11 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags) scanstate->prefetch_target = 0; /* may be updated below */ scanstate->prefetch_maximum = target_prefetch_pages; + scanstate->pscan_len = 0; + scanstate->inited = false; + scanstate->stbmiterator = NULL; + scanstate->sprefetch_iterator = NULL; + scanstate->pstate = NULL; /* * Miscellaneous initialization @@ -651,3 +882,161 @@ ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags) */ return scanstate; } + +/*---------------- + * pbms_is_leader + * + * The first process to come here and see the state to the PBM_INITIAL + * will become the leader for the parallel bitmap scan and will be + * responsible for populating the TIDBitmap. The other processes will + * be blocked by the condition variable until the leader wakes them up. + * --------------- + */ +static bool +pbms_is_leader(ParallelBitmapState *pstate) +{ + bool needWait = false; + bool leader = false; + + while (1) + { + /*--------------- + * Check the current state + * If state is + * PBM_INITIAL : We become the leader and set it to PBM_INPROGRESS + * PBM_INPROGRESS : We need to wait till leader creates bitmap + * PBM_FINISHED : bitmap is ready so no need to wait + *--------------- + */ + SpinLockAcquire(&pstate->state_mutex); + + if (pstate->state == PBM_INITIAL) + { + pstate->state = PBM_INPROGRESS; + leader = true; + } + else if (pstate->state == PBM_INPROGRESS) + needWait = true; + else + needWait = false; + + SpinLockRelease(&pstate->state_mutex); + + /* If we are leader or leader has already created a TIDBITMAP */ + if (leader || !needWait) + break; + + /* Sleep until leader send wake up signal */ + ConditionVariableSleep(&pstate->cv, WAIT_EVENT_PARALLEL_BITMAP_SCAN); + } + + ConditionVariableCancelSleep(); + + return leader; +} + +/*------------------- + * pbms_set_parallel + * + * Recursively process the node and set the parallel flag. This flag + * will be used to indicate the underlying layer to allocate the + * pagetable elements from the DSA. + * ----------------- + */ +static void +pbms_set_parallel(PlanState *node) +{ + /* + * For BitmapAnd node, set parallel flag only for the first child because + * only that node will create the main TBM and other TBMs will be merged + * to the main one. Only BitmapOr and BitmapIndex nodes create TBM, + * therefore set the parallel flag only for these types. + */ + switch (node->type) + { + case T_BitmapIndexScanState: + ((BitmapIndexScanState *) node)->biss_Parallel = true; + break; + case T_BitmapOrState: + ((BitmapOrState *) node)->is_parallel = true;; + break; + case T_BitmapAndState: + pbms_set_parallel(((BitmapAndState *) node)->bitmapplans[0]); + break; + default: + break; + } +} + +/* ---------------------------------------------------------------- + * ExecBitmapHeapEstimate + * + * estimates the space required to serialize bitmap scan node. + * ---------------------------------------------------------------- + */ +void +ExecBitmapHeapEstimate(BitmapHeapScanState *node, + ParallelContext *pcxt) +{ + EState *estate = node->ss.ps.state; + + /* Estimate the size for sharing parallel Bitmap info. */ + node->pscan_len = add_size(offsetof(ParallelBitmapState, + phs_snapshot_data), + EstimateSnapshotSpace(estate->es_snapshot)); + + shm_toc_estimate_chunk(&pcxt->estimator, node->pscan_len); + shm_toc_estimate_keys(&pcxt->estimator, 1); +} + +/* ---------------------------------------------------------------- + * ExecBitmapHeapInitializeDSM + * + * Set up a parallel bitmap heap scan descriptor. + * ---------------------------------------------------------------- + */ +void +ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node, + ParallelContext *pcxt) +{ + ParallelBitmapState *pstate; + EState *estate = node->ss.ps.state; + + pstate = shm_toc_allocate(pcxt->toc, node->pscan_len); + + /* Initialize mutex to protect prefetch pages and target */ + SpinLockInit(&pstate->prefetch_mutex); + pstate->tbmiterator = 0; + pstate->prefetch_iterator = 0; + pstate->prefetch_pages = 0; + pstate->prefetch_target = 0; + pstate->prefetching = false; + + /* Initialize mutex to protect current state */ + SpinLockInit(&pstate->state_mutex); + pstate->state = PBM_INITIAL; + ConditionVariableInit(&pstate->cv); + SerializeSnapshot(estate->es_snapshot, pstate->phs_snapshot_data); + + shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pstate); + node->pstate = pstate; +} + +/* ---------------------------------------------------------------- + * ExecBitmapHeapInitializeWorker + * + * Copy relevant information from TOC into planstate. + * ---------------------------------------------------------------- + */ +void +ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, shm_toc *toc) +{ + ParallelBitmapState *pstate; + Snapshot snapshot; + + pstate = shm_toc_lookup(toc, node->ss.ps.plan->plan_node_id); + node->pstate = pstate; + + snapshot = RestoreSnapshot(pstate->phs_snapshot_data); + heap_update_snapshot(node->ss.ss_currentScanDesc, snapshot); +} diff --git a/src/backend/executor/nodeBitmapIndexscan.c b/src/backend/executor/nodeBitmapIndexscan.c index 94bb289..2f01cad 100644 --- a/src/backend/executor/nodeBitmapIndexscan.c +++ b/src/backend/executor/nodeBitmapIndexscan.c @@ -78,7 +78,8 @@ MultiExecBitmapIndexScan(BitmapIndexScanState *node) else { /* XXX should we use less than work_mem for this? */ - tbm = tbm_create(work_mem * 1024L, NULL); + tbm = tbm_create(work_mem * 1024L, + node->biss_Parallel ? node->ss.ps.state->es_query_dsa: NULL); } /* diff --git a/src/backend/executor/nodeBitmapOr.c b/src/backend/executor/nodeBitmapOr.c index 1d280be..e8f110b 100644 --- a/src/backend/executor/nodeBitmapOr.c +++ b/src/backend/executor/nodeBitmapOr.c @@ -129,7 +129,8 @@ MultiExecBitmapOr(BitmapOrState *node) if (result == NULL) /* first subplan */ { /* XXX should we use less than work_mem for this? */ - result = tbm_create(work_mem * 1024L, NULL); + result = tbm_create(work_mem * 1024L, node->is_parallel ? + node->ps.state->es_query_dsa: NULL); } ((BitmapIndexScanState *) subnode)->biss_result = result; diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c index eeacf81..f671f0a 100644 --- a/src/backend/optimizer/path/allpaths.c +++ b/src/backend/optimizer/path/allpaths.c @@ -2875,6 +2875,30 @@ remove_unused_subquery_outputs(Query *subquery, RelOptInfo *rel) } /* + * create_partial_bitmap_paths + * Build partial bitmap heap path for the relation + */ +void +create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel, + Path *bitmapqual) +{ + int parallel_workers; + double pages_fetched; + + /* Compute heap pages for bitmap heap scan */ + pages_fetched = compute_bitmap_pages(root, rel, bitmapqual, 1.0, + NULL, NULL); + + parallel_workers = compute_parallel_worker(rel, pages_fetched, 0); + + if (parallel_workers <= 0) + return; + + add_partial_path(rel, (Path *) create_bitmap_heap_path(root, rel, + bitmapqual, rel->lateral_relids, 1.0, parallel_workers)); +} + +/* * Compute the number of parallel workers that should be used to scan a * relation. We compute the parallel workers based on the size of the heap to * be scanned and the size of the index to be scanned, then choose a minimum diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c index d01630f..6240819 100644 --- a/src/backend/optimizer/path/costsize.c +++ b/src/backend/optimizer/path/costsize.c @@ -861,6 +861,7 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel, QualCost qpqual_cost; Cost cpu_per_tuple; Cost cost_per_page; + Cost cpu_run_cost; double tuples_fetched; double pages_fetched; double spc_seq_page_cost, @@ -922,8 +923,21 @@ cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel, startup_cost += qpqual_cost.startup; cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple; + cpu_run_cost = cpu_per_tuple * tuples_fetched; + + /* Adjust costing for parallelism, if used. */ + if (path->parallel_workers > 0) + { + double parallel_divisor = get_parallel_divisor(path); + + /* The CPU cost is divided among all the workers. */ + cpu_run_cost /= parallel_divisor; - run_cost += cpu_per_tuple * tuples_fetched; + path->rows = clamp_row_est(path->rows / parallel_divisor); + } + + + run_cost += cpu_run_cost; /* tlist eval costs are paid per output row, not per tuple scanned */ startup_cost += path->pathtarget->cost.startup; diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c index d92826b..c411962 100644 --- a/src/backend/optimizer/path/indxpath.c +++ b/src/backend/optimizer/path/indxpath.c @@ -337,8 +337,12 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel) bitmapqual = choose_bitmap_and(root, rel, bitindexpaths); bpath = create_bitmap_heap_path(root, rel, bitmapqual, - rel->lateral_relids, 1.0); + rel->lateral_relids, 1.0, 0); add_path(rel, (Path *) bpath); + + /* create a partial bitmap heap path */ + if (rel->consider_parallel && rel->lateral_relids == NULL) + create_partial_bitmap_paths(root, rel, bitmapqual); } /* @@ -410,7 +414,7 @@ create_index_paths(PlannerInfo *root, RelOptInfo *rel) required_outer = get_bitmap_tree_required_outer(bitmapqual); loop_count = get_loop_count(root, rel->relid, required_outer); bpath = create_bitmap_heap_path(root, rel, bitmapqual, - required_outer, loop_count); + required_outer, loop_count, 0); add_path(rel, (Path *) bpath); } } @@ -1618,6 +1622,11 @@ bitmap_scan_cost_est(PlannerInfo *root, RelOptInfo *rel, Path *ipath) bpath.path.pathkeys = NIL; bpath.bitmapqual = ipath; + /* + * Check the cost of temporary path without considering parallelism. + * Parallel bitmap heap path will be considered at later stage. + */ + bpath.path.parallel_workers = 0; cost_bitmap_heap_scan(&bpath.path, root, rel, bpath.path.param_info, ipath, @@ -1660,6 +1669,12 @@ bitmap_and_cost_est(PlannerInfo *root, RelOptInfo *rel, List *paths) bpath.path.pathkeys = NIL; bpath.bitmapqual = (Path *) &apath; + /* + * Check the cost of temporary path without considering parallelism. + * Parallel bitmap heap path will be considered at later stage. + */ + bpath.path.parallel_workers = 0; + /* Now we can do cost_bitmap_heap_scan */ cost_bitmap_heap_scan(&bpath.path, root, rel, bpath.path.param_info, diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c index 3248296..6e70808 100644 --- a/src/backend/optimizer/util/pathnode.c +++ b/src/backend/optimizer/util/pathnode.c @@ -1068,7 +1068,8 @@ create_bitmap_heap_path(PlannerInfo *root, RelOptInfo *rel, Path *bitmapqual, Relids required_outer, - double loop_count) + double loop_count, + int parallel_degree) { BitmapHeapPath *pathnode = makeNode(BitmapHeapPath); @@ -1077,9 +1078,9 @@ create_bitmap_heap_path(PlannerInfo *root, pathnode->path.pathtarget = rel->reltarget; pathnode->path.param_info = get_baserel_parampathinfo(root, rel, required_outer); - pathnode->path.parallel_aware = false; + pathnode->path.parallel_aware = parallel_degree > 0 ? true : false; pathnode->path.parallel_safe = rel->consider_parallel; - pathnode->path.parallel_workers = 0; + pathnode->path.parallel_workers = parallel_degree; pathnode->path.pathkeys = NIL; /* always unordered */ pathnode->bitmapqual = bitmapqual; @@ -3255,7 +3256,7 @@ reparameterize_path(PlannerInfo *root, Path *path, rel, bpath->bitmapqual, required_outer, - loop_count); + loop_count, 0); } case T_SubqueryScan: { diff --git a/src/backend/postmaster/pgstat.c b/src/backend/postmaster/pgstat.c index ada374c..9b191df 100644 --- a/src/backend/postmaster/pgstat.c +++ b/src/backend/postmaster/pgstat.c @@ -3395,6 +3395,9 @@ pgstat_get_wait_ipc(WaitEventIPC w) case WAIT_EVENT_PARALLEL_FINISH: event_name = "ParallelFinish"; break; + case WAIT_EVENT_PARALLEL_BITMAP_SCAN: + event_name = "ParallelBitmapScan"; + break; case WAIT_EVENT_SAFE_SNAPSHOT: event_name = "SafeSnapshot"; break; diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h index a864f78..7e85510 100644 --- a/src/include/access/heapam.h +++ b/src/include/access/heapam.h @@ -179,6 +179,7 @@ extern void simple_heap_update(Relation relation, ItemPointer otid, HeapTuple tup); extern void heap_sync(Relation relation); +extern void heap_update_snapshot(HeapScanDesc scan, Snapshot snapshot); /* in heap/pruneheap.c */ extern void heap_page_prune_opt(Relation relation, Buffer buffer); diff --git a/src/include/executor/nodeBitmapHeapscan.h b/src/include/executor/nodeBitmapHeapscan.h index d7659b9..465c58e 100644 --- a/src/include/executor/nodeBitmapHeapscan.h +++ b/src/include/executor/nodeBitmapHeapscan.h @@ -15,10 +15,17 @@ #define NODEBITMAPHEAPSCAN_H #include "nodes/execnodes.h" +#include "access/parallel.h" extern BitmapHeapScanState *ExecInitBitmapHeapScan(BitmapHeapScan *node, EState *estate, int eflags); extern TupleTableSlot *ExecBitmapHeapScan(BitmapHeapScanState *node); extern void ExecEndBitmapHeapScan(BitmapHeapScanState *node); extern void ExecReScanBitmapHeapScan(BitmapHeapScanState *node); +extern void ExecBitmapHeapEstimate(BitmapHeapScanState *node, + ParallelContext *pcxt); +extern void ExecBitmapHeapInitializeDSM(BitmapHeapScanState *node, + ParallelContext *pcxt); +extern void ExecBitmapHeapInitializeWorker(BitmapHeapScanState *node, + shm_toc *toc); #endif /* NODEBITMAPHEAPSCAN_H */ diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h index 1c1cb80..912a4cd 100644 --- a/src/include/nodes/execnodes.h +++ b/src/include/nodes/execnodes.h @@ -26,6 +26,8 @@ #include "utils/sortsupport.h" #include "utils/tuplestore.h" #include "utils/tuplesort.h" +#include "nodes/tidbitmap.h" +#include "storage/condition_variable.h" /* ---------------- @@ -1261,6 +1263,7 @@ typedef struct BitmapOrState PlanState ps; /* its first field is NodeTag */ PlanState **bitmapplans; /* array of PlanStates for my inputs */ int nplans; /* number of input plans */ + bool is_parallel; /* create shared TBM pagetable if it's set */ } BitmapOrState; /* ---------------------------------------------------------------- @@ -1445,6 +1448,7 @@ typedef struct IndexOnlyScanState * RuntimeContext expr context for evaling runtime Skeys * RelationDesc index relation descriptor * ScanDesc index scan descriptor + * Parallel Create shared TBM pagetable if it's set * ---------------- */ typedef struct BitmapIndexScanState @@ -1461,9 +1465,60 @@ typedef struct BitmapIndexScanState ExprContext *biss_RuntimeContext; Relation biss_RelationDesc; IndexScanDesc biss_ScanDesc; + bool biss_Parallel; } BitmapIndexScanState; /* ---------------- + * PBMState information : Current status of the TIDBitmap creation during + * parallel bitmap heap scan. + * + * PBM_INITIAL TIDBitmap creation is not yet started, so + * first worker to see this state will set the + * state to PBM_INPROGRESS and that process will + * be responsible for creating TIDBitmap. + * PBM_INPROGRESS TIDBitmap creation is already in progress, + * therefore workers need to sleep until leader set + * the state to PBM_FINISHED and wake us up. + * PBM_FINISHED TIDBitmap creation is done, so now all worker can + * proceed to iterate over TIDBitmap. + * ---------------- + */ +typedef enum +{ + PBM_INITIAL, + PBM_INPROGRESS, + PBM_FINISHED +} PBMState; + +/* ---------------- + * ParallelBitmapState : Shared state for BitmapHeapScanState + * tbmiterator iterator for scanning current pages + * prefetch_iterator iterator for prefetching ahead of current page + * prefetch_mutex mutual exclusion for the prefetching variables + * prefetch_pages # pages prefetch iterator is ahead of current + * prefetch_target current target prefetch distance + * prefetching set true if prefetching is in progress + * state_mutex mutual exclusion for state + * state current state of the TIDBitmap + * cv conditional wait variable + * phs_snapshot_data snapshot data shared to workers + * ---------------- + */ +typedef struct ParallelBitmapState +{ + dsa_pointer tbmiterator; + dsa_pointer prefetch_iterator; + slock_t prefetch_mutex; + int prefetch_pages; + int prefetch_target; + bool prefetching; + slock_t state_mutex; + PBMState state; + ConditionVariable cv; + char phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER]; +} ParallelBitmapState; + +/* ---------------- * BitmapHeapScanState information * * bitmapqualorig execution state for bitmapqualorig expressions @@ -1476,7 +1531,12 @@ typedef struct BitmapIndexScanState * prefetch_pages # pages prefetch iterator is ahead of current * prefetch_target current target prefetch distance * prefetch_maximum maximum value for prefetch_target - * ---------------- + * pscan_len size of the shared memory for parallel bitmap + * inited is node is ready to iterate + * stbmiterator shared iterator + * sprefetch_iterator shared iterator for prefetching + * pstate shared state for parallel bitmap scan + *---------------- */ typedef struct BitmapHeapScanState { @@ -1491,6 +1551,11 @@ typedef struct BitmapHeapScanState int prefetch_pages; int prefetch_target; int prefetch_maximum; + Size pscan_len; + bool inited; + TBMSharedIterator *stbmiterator; + TBMSharedIterator *sprefetch_iterator; + ParallelBitmapState *pstate; } BitmapHeapScanState; /* ---------------- diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h index 53cad24..d22b039 100644 --- a/src/include/optimizer/pathnode.h +++ b/src/include/optimizer/pathnode.h @@ -53,7 +53,8 @@ extern BitmapHeapPath *create_bitmap_heap_path(PlannerInfo *root, RelOptInfo *rel, Path *bitmapqual, Relids required_outer, - double loop_count); + double loop_count, + int parallel_degree); extern BitmapAndPath *create_bitmap_and_path(PlannerInfo *root, RelOptInfo *rel, List *bitmapquals); diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h index ebda308..3ec2d9d 100644 --- a/src/include/optimizer/paths.h +++ b/src/include/optimizer/paths.h @@ -56,6 +56,8 @@ extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed, extern void generate_gather_paths(PlannerInfo *root, RelOptInfo *rel); extern int compute_parallel_worker(RelOptInfo *rel, BlockNumber heap_pages, BlockNumber index_pages); +extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel, + Path *bitmapqual); #ifdef OPTIMIZER_DEBUG extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel); diff --git a/src/include/pgstat.h b/src/include/pgstat.h index 8b710ec..e31389f 100644 --- a/src/include/pgstat.h +++ b/src/include/pgstat.h @@ -787,6 +787,7 @@ typedef enum WAIT_EVENT_MQ_RECEIVE, WAIT_EVENT_MQ_SEND, WAIT_EVENT_PARALLEL_FINISH, + WAIT_EVENT_PARALLEL_BITMAP_SCAN, WAIT_EVENT_SAFE_SNAPSHOT, WAIT_EVENT_SYNC_REP } WaitEventIPC;