1
0
mirror of https://github.com/postgres/postgres.git synced 2025-11-12 05:01:15 +03:00

Support parallel btree index builds.

To make this work, tuplesort.c and logtape.c must also support
parallelism, so this patch adds that infrastructure and then applies
it to the particular case of parallel btree index builds.  Testing
to date shows that this can often be 2-3x faster than a serial
index build.

The model for deciding how many workers to use is fairly primitive
at present, but it's better than not having the feature.  We can
refine it as we get more experience.

Peter Geoghegan with some help from Rushabh Lathia.  While Heikki
Linnakangas is not an author of this patch, he wrote other patches
without which this feature would not have been possible, and
therefore the release notes should possibly credit him as an author
of this feature.  Reviewed by Claudio Freire, Heikki Linnakangas,
Thomas Munro, Tels, Amit Kapila, me.

Discussion: http://postgr.es/m/CAM3SWZQKM=Pzc=CAHzRixKjp2eO5Q0Jg1SoFQqeXFQ647JiwqQ@mail.gmail.com
Discussion: http://postgr.es/m/CAH2-Wz=AxWqDoVvGU7dq856S4r6sJAj6DBn7VMtigkB33N5eyg@mail.gmail.com
This commit is contained in:
Robert Haas
2018-02-02 13:25:55 -05:00
parent 9aef173163
commit 9da0cc3528
51 changed files with 2238 additions and 362 deletions

View File

@@ -706,7 +706,7 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
* heap blocks in physical order.
*/
reltuples = IndexBuildHeapScan(heap, index, indexInfo, false,
brinbuildCallback, (void *) state);
brinbuildCallback, (void *) state, NULL);
/* process the final batch */
form_and_insert_tuple(state);
@@ -1205,7 +1205,7 @@ summarize_range(IndexInfo *indexInfo, BrinBuildState *state, Relation heapRel,
state->bs_currRangeStart = heapBlk;
IndexBuildHeapRangeScan(heapRel, state->bs_irel, indexInfo, false, true,
heapBlk, scanNumBlks,
brinbuildCallback, (void *) state);
brinbuildCallback, (void *) state, NULL);
/*
* Now we update the values obtained by the scan with the placeholder

View File

@@ -391,7 +391,7 @@ ginbuild(Relation heap, Relation index, IndexInfo *indexInfo)
* prefers to receive tuples in TID order.
*/
reltuples = IndexBuildHeapScan(heap, index, indexInfo, false,
ginBuildCallback, (void *) &buildstate);
ginBuildCallback, (void *) &buildstate, NULL);
/* dump remaining entries to the index */
oldCtx = MemoryContextSwitchTo(buildstate.tmpCtx);

View File

@@ -203,7 +203,7 @@ gistbuild(Relation heap, Relation index, IndexInfo *indexInfo)
* Do the heap scan.
*/
reltuples = IndexBuildHeapScan(heap, index, indexInfo, true,
gistBuildCallback, (void *) &buildstate);
gistBuildCallback, (void *) &buildstate, NULL);
/*
* If buffering was used, flush out all the tuples that are still in the

View File

@@ -159,7 +159,7 @@ hashbuild(Relation heap, Relation index, IndexInfo *indexInfo)
/* do the heap scan */
reltuples = IndexBuildHeapScan(heap, index, indexInfo, true,
hashbuildCallback, (void *) &buildstate);
hashbuildCallback, (void *) &buildstate, NULL);
if (buildstate.spool)
{

View File

@@ -82,6 +82,7 @@ _h_spoolinit(Relation heap, Relation index, uint32 num_buckets)
hspool->low_mask,
hspool->max_buckets,
maintenance_work_mem,
NULL,
false);
return hspool;

View File

@@ -1627,7 +1627,16 @@ heap_parallelscan_initialize(ParallelHeapScanDesc target, Relation relation,
SpinLockInit(&target->phs_mutex);
target->phs_startblock = InvalidBlockNumber;
pg_atomic_init_u64(&target->phs_nallocated, 0);
SerializeSnapshot(snapshot, target->phs_snapshot_data);
if (IsMVCCSnapshot(snapshot))
{
SerializeSnapshot(snapshot, target->phs_snapshot_data);
target->phs_snapshot_any = false;
}
else
{
Assert(snapshot == SnapshotAny);
target->phs_snapshot_any = true;
}
}
/* ----------------
@@ -1655,11 +1664,22 @@ heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
Snapshot snapshot;
Assert(RelationGetRelid(relation) == parallel_scan->phs_relid);
snapshot = RestoreSnapshot(parallel_scan->phs_snapshot_data);
RegisterSnapshot(snapshot);
if (!parallel_scan->phs_snapshot_any)
{
/* Snapshot was serialized -- restore it */
snapshot = RestoreSnapshot(parallel_scan->phs_snapshot_data);
RegisterSnapshot(snapshot);
}
else
{
/* SnapshotAny passed by caller (not serialized) */
snapshot = SnapshotAny;
}
return heap_beginscan_internal(relation, snapshot, 0, NULL, parallel_scan,
true, true, true, false, false, true);
true, true, true, false, false,
!parallel_scan->phs_snapshot_any);
}
/* ----------------

View File

@@ -21,36 +21,19 @@
#include "access/nbtree.h"
#include "access/relscan.h"
#include "access/xlog.h"
#include "catalog/index.h"
#include "commands/vacuum.h"
#include "nodes/execnodes.h"
#include "pgstat.h"
#include "storage/condition_variable.h"
#include "storage/indexfsm.h"
#include "storage/ipc.h"
#include "storage/lmgr.h"
#include "storage/smgr.h"
#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/builtins.h"
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
/* Working state for btbuild and its callback */
typedef struct
{
bool isUnique;
bool haveDead;
Relation heapRel;
BTSpool *spool;
/*
* spool2 is needed only when the index is a unique index. Dead tuples are
* put into spool2 instead of spool in order to avoid uniqueness check.
*/
BTSpool *spool2;
double indtuples;
} BTBuildState;
/* Working state needed by btvacuumpage */
typedef struct
{
@@ -104,12 +87,6 @@ typedef struct BTParallelScanDescData
typedef struct BTParallelScanDescData *BTParallelScanDesc;
static void btbuildCallback(Relation index,
HeapTuple htup,
Datum *values,
bool *isnull,
bool tupleIsAlive,
void *state);
static void btvacuumscan(IndexVacuumInfo *info, IndexBulkDeleteResult *stats,
IndexBulkDeleteCallback callback, void *callback_state,
BTCycleId cycleid);
@@ -166,115 +143,6 @@ bthandler(PG_FUNCTION_ARGS)
PG_RETURN_POINTER(amroutine);
}
/*
* btbuild() -- build a new btree index.
*/
IndexBuildResult *
btbuild(Relation heap, Relation index, IndexInfo *indexInfo)
{
IndexBuildResult *result;
double reltuples;
BTBuildState buildstate;
buildstate.isUnique = indexInfo->ii_Unique;
buildstate.haveDead = false;
buildstate.heapRel = heap;
buildstate.spool = NULL;
buildstate.spool2 = NULL;
buildstate.indtuples = 0;
#ifdef BTREE_BUILD_STATS
if (log_btree_build_stats)
ResetUsage();
#endif /* BTREE_BUILD_STATS */
/*
* We expect to be called exactly once for any index relation. If that's
* not the case, big trouble's what we have.
*/
if (RelationGetNumberOfBlocks(index) != 0)
elog(ERROR, "index \"%s\" already contains data",
RelationGetRelationName(index));
buildstate.spool = _bt_spoolinit(heap, index, indexInfo->ii_Unique, false);
/*
* If building a unique index, put dead tuples in a second spool to keep
* them out of the uniqueness check.
*/
if (indexInfo->ii_Unique)
buildstate.spool2 = _bt_spoolinit(heap, index, false, true);
/* do the heap scan */
reltuples = IndexBuildHeapScan(heap, index, indexInfo, true,
btbuildCallback, (void *) &buildstate);
/* okay, all heap tuples are indexed */
if (buildstate.spool2 && !buildstate.haveDead)
{
/* spool2 turns out to be unnecessary */
_bt_spooldestroy(buildstate.spool2);
buildstate.spool2 = NULL;
}
/*
* Finish the build by (1) completing the sort of the spool file, (2)
* inserting the sorted tuples into btree pages and (3) building the upper
* levels.
*/
_bt_leafbuild(buildstate.spool, buildstate.spool2);
_bt_spooldestroy(buildstate.spool);
if (buildstate.spool2)
_bt_spooldestroy(buildstate.spool2);
#ifdef BTREE_BUILD_STATS
if (log_btree_build_stats)
{
ShowUsage("BTREE BUILD STATS");
ResetUsage();
}
#endif /* BTREE_BUILD_STATS */
/*
* Return statistics
*/
result = (IndexBuildResult *) palloc(sizeof(IndexBuildResult));
result->heap_tuples = reltuples;
result->index_tuples = buildstate.indtuples;
return result;
}
/*
* Per-tuple callback from IndexBuildHeapScan
*/
static void
btbuildCallback(Relation index,
HeapTuple htup,
Datum *values,
bool *isnull,
bool tupleIsAlive,
void *state)
{
BTBuildState *buildstate = (BTBuildState *) state;
/*
* insert the index tuple into the appropriate spool file for subsequent
* processing
*/
if (tupleIsAlive || buildstate->spool2 == NULL)
_bt_spool(buildstate->spool, &htup->t_self, values, isnull);
else
{
/* dead tuples are put into spool2 */
buildstate->haveDead = true;
_bt_spool(buildstate->spool2, &htup->t_self, values, isnull);
}
buildstate->indtuples += 1;
}
/*
* btbuildempty() -- build an empty btree index in the initialization fork
*/

View File

@@ -67,28 +67,168 @@
#include "postgres.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/relscan.h"
#include "access/xact.h"
#include "access/xlog.h"
#include "access/xloginsert.h"
#include "catalog/index.h"
#include "miscadmin.h"
#include "pgstat.h"
#include "storage/smgr.h"
#include "tcop/tcopprot.h"
#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/rel.h"
#include "utils/sortsupport.h"
#include "utils/tuplesort.h"
/* Magic numbers for parallel state sharing */
#define PARALLEL_KEY_BTREE_SHARED UINT64CONST(0xA000000000000001)
#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)
#define PARALLEL_KEY_TUPLESORT_SPOOL2 UINT64CONST(0xA000000000000003)
/*
* DISABLE_LEADER_PARTICIPATION disables the leader's participation in
* parallel index builds. This may be useful as a debugging aid.
#undef DISABLE_LEADER_PARTICIPATION
*/
/*
* Status record for spooling/sorting phase. (Note we may have two of
* these due to the special requirements for uniqueness-checking with
* dead tuples.)
*/
struct BTSpool
typedef struct BTSpool
{
Tuplesortstate *sortstate; /* state data for tuplesort.c */
Relation heap;
Relation index;
bool isunique;
};
} BTSpool;
/*
* Status for index builds performed in parallel. This is allocated in a
* dynamic shared memory segment. Note that there is a separate tuplesort TOC
* entry, private to tuplesort.c but allocated by this module on its behalf.
*/
typedef struct BTShared
{
/*
* These fields are not modified during the sort. They primarily exist
* for the benefit of worker processes that need to create BTSpool state
* corresponding to that used by the leader.
*/
Oid heaprelid;
Oid indexrelid;
bool isunique;
bool isconcurrent;
int scantuplesortstates;
/*
* workersdonecv is used to monitor the progress of workers. All parallel
* participants must indicate that they are done before leader can use
* mutable state that workers maintain during scan (and before leader can
* proceed to tuplesort_performsort()).
*/
ConditionVariable workersdonecv;
/*
* mutex protects all fields before heapdesc.
*
* These fields contain status information of interest to B-Tree index
* builds that must work just the same when an index is built in parallel.
*/
slock_t mutex;
/*
* Mutable state that is maintained by workers, and reported back to
* leader at end of parallel scan.
*
* nparticipantsdone is number of worker processes finished.
*
* reltuples is the total number of input heap tuples.
*
* havedead indicates if RECENTLY_DEAD tuples were encountered during
* build.
*
* indtuples is the total number of tuples that made it into the index.
*
* brokenhotchain indicates if any worker detected a broken HOT chain
* during build.
*/
int nparticipantsdone;
double reltuples;
bool havedead;
double indtuples;
bool brokenhotchain;
/*
* This variable-sized field must come last.
*
* See _bt_parallel_estimate_shared().
*/
ParallelHeapScanDescData heapdesc;
} BTShared;
/*
* Status for leader in parallel index build.
*/
typedef struct BTLeader
{
/* parallel context itself */
ParallelContext *pcxt;
/*
* nparticipanttuplesorts is the exact number of worker processes
* successfully launched, plus one leader process if it participates as a
* worker (only DISABLE_LEADER_PARTICIPATION builds avoid leader
* participating as a worker).
*/
int nparticipanttuplesorts;
/*
* Leader process convenience pointers to shared state (leader avoids TOC
* lookups).
*
* btshared is the shared state for entire build. sharedsort is the
* shared, tuplesort-managed state passed to each process tuplesort.
* sharedsort2 is the corresponding btspool2 shared state, used only when
* building unique indexes. snapshot is the snapshot used by the scan iff
* an MVCC snapshot is required.
*/
BTShared *btshared;
Sharedsort *sharedsort;
Sharedsort *sharedsort2;
Snapshot snapshot;
} BTLeader;
/*
* Working state for btbuild and its callback.
*
* When parallel CREATE INDEX is used, there is a BTBuildState for each
* participant.
*/
typedef struct BTBuildState
{
bool isunique;
bool havedead;
Relation heap;
BTSpool *spool;
/*
* spool2 is needed only when the index is a unique index. Dead tuples are
* put into spool2 instead of spool in order to avoid uniqueness check.
*/
BTSpool *spool2;
double indtuples;
/*
* btleader is only present when a parallel index build is performed, and
* only in the leader process. (Actually, only the leader has a
* BTBuildState. Workers have their own spool and spool2, though.)
*/
BTLeader *btleader;
} BTBuildState;
/*
* Status record for a btree page being built. We have one of these
@@ -128,6 +268,14 @@ typedef struct BTWriteState
} BTWriteState;
static double _bt_spools_heapscan(Relation heap, Relation index,
BTBuildState *buildstate, IndexInfo *indexInfo);
static void _bt_spooldestroy(BTSpool *btspool);
static void _bt_spool(BTSpool *btspool, ItemPointer self,
Datum *values, bool *isnull);
static void _bt_leafbuild(BTSpool *btspool, BTSpool *btspool2);
static void _bt_build_callback(Relation index, HeapTuple htup, Datum *values,
bool *isnull, bool tupleIsAlive, void *state);
static Page _bt_blnewpage(uint32 level);
static BTPageState *_bt_pagestate(BTWriteState *wstate, uint32 level);
static void _bt_slideleft(Page page);
@@ -138,45 +286,219 @@ static void _bt_buildadd(BTWriteState *wstate, BTPageState *state,
static void _bt_uppershutdown(BTWriteState *wstate, BTPageState *state);
static void _bt_load(BTWriteState *wstate,
BTSpool *btspool, BTSpool *btspool2);
static void _bt_begin_parallel(BTBuildState *buildstate, bool isconcurrent,
int request);
static void _bt_end_parallel(BTLeader *btleader);
static Size _bt_parallel_estimate_shared(Snapshot snapshot);
static double _bt_parallel_heapscan(BTBuildState *buildstate,
bool *brokenhotchain);
static void _bt_leader_participate_as_worker(BTBuildState *buildstate);
static void _bt_parallel_scan_and_sort(BTSpool *btspool, BTSpool *btspool2,
BTShared *btshared, Sharedsort *sharedsort,
Sharedsort *sharedsort2, int sortmem);
/*
* Interface routines
* btbuild() -- build a new btree index.
*/
IndexBuildResult *
btbuild(Relation heap, Relation index, IndexInfo *indexInfo)
{
IndexBuildResult *result;
BTBuildState buildstate;
double reltuples;
#ifdef BTREE_BUILD_STATS
if (log_btree_build_stats)
ResetUsage();
#endif /* BTREE_BUILD_STATS */
buildstate.isunique = indexInfo->ii_Unique;
buildstate.havedead = false;
buildstate.heap = heap;
buildstate.spool = NULL;
buildstate.spool2 = NULL;
buildstate.indtuples = 0;
buildstate.btleader = NULL;
/*
* We expect to be called exactly once for any index relation. If that's
* not the case, big trouble's what we have.
*/
if (RelationGetNumberOfBlocks(index) != 0)
elog(ERROR, "index \"%s\" already contains data",
RelationGetRelationName(index));
reltuples = _bt_spools_heapscan(heap, index, &buildstate, indexInfo);
/*
* Finish the build by (1) completing the sort of the spool file, (2)
* inserting the sorted tuples into btree pages and (3) building the upper
* levels. Finally, it may also be necessary to end use of parallelism.
*/
_bt_leafbuild(buildstate.spool, buildstate.spool2);
_bt_spooldestroy(buildstate.spool);
if (buildstate.spool2)
_bt_spooldestroy(buildstate.spool2);
if (buildstate.btleader)
_bt_end_parallel(buildstate.btleader);
result = (IndexBuildResult *) palloc(sizeof(IndexBuildResult));
result->heap_tuples = reltuples;
result->index_tuples = buildstate.indtuples;
#ifdef BTREE_BUILD_STATS
if (log_btree_build_stats)
{
ShowUsage("BTREE BUILD STATS");
ResetUsage();
}
#endif /* BTREE_BUILD_STATS */
return result;
}
/*
* create and initialize a spool structure
* Create and initialize one or two spool structures, and save them in caller's
* buildstate argument. May also fill-in fields within indexInfo used by index
* builds.
*
* Scans the heap, possibly in parallel, filling spools with IndexTuples. This
* routine encapsulates all aspects of managing parallelism. Caller need only
* call _bt_end_parallel() in parallel case after it is done with spool/spool2.
*
* Returns the total number of heap tuples scanned.
*/
BTSpool *
_bt_spoolinit(Relation heap, Relation index, bool isunique, bool isdead)
static double
_bt_spools_heapscan(Relation heap, Relation index, BTBuildState *buildstate,
IndexInfo *indexInfo)
{
BTSpool *btspool = (BTSpool *) palloc0(sizeof(BTSpool));
int btKbytes;
btspool->heap = heap;
btspool->index = index;
btspool->isunique = isunique;
SortCoordinate coordinate = NULL;
double reltuples = 0;
/*
* We size the sort area as maintenance_work_mem rather than work_mem to
* speed index creation. This should be OK since a single backend can't
* run multiple index creations in parallel. Note that creation of a
* unique index actually requires two BTSpool objects. We expect that the
* second one (for dead tuples) won't get very full, so we give it only
* work_mem.
* run multiple index creations in parallel (see also: notes on
* parallelism and maintenance_work_mem below).
*/
btKbytes = isdead ? work_mem : maintenance_work_mem;
btspool->sortstate = tuplesort_begin_index_btree(heap, index, isunique,
btKbytes, false);
btspool->heap = heap;
btspool->index = index;
btspool->isunique = indexInfo->ii_Unique;
return btspool;
/* Save as primary spool */
buildstate->spool = btspool;
/* Attempt to launch parallel worker scan when required */
if (indexInfo->ii_ParallelWorkers > 0)
_bt_begin_parallel(buildstate, indexInfo->ii_Concurrent,
indexInfo->ii_ParallelWorkers);
/*
* If parallel build requested and at least one worker process was
* successfully launched, set up coordination state
*/
if (buildstate->btleader)
{
coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
coordinate->isWorker = false;
coordinate->nParticipants =
buildstate->btleader->nparticipanttuplesorts;
coordinate->sharedsort = buildstate->btleader->sharedsort;
}
/*
* Begin serial/leader tuplesort.
*
* In cases where parallelism is involved, the leader receives the same
* share of maintenance_work_mem as a serial sort (it is generally treated
* in the same way as a serial sort once we return). Parallel worker
* Tuplesortstates will have received only a fraction of
* maintenance_work_mem, though.
*
* We rely on the lifetime of the Leader Tuplesortstate almost not
* overlapping with any worker Tuplesortstate's lifetime. There may be
* some small overlap, but that's okay because we rely on leader
* Tuplesortstate only allocating a small, fixed amount of memory here.
* When its tuplesort_performsort() is called (by our caller), and
* significant amounts of memory are likely to be used, all workers must
* have already freed almost all memory held by their Tuplesortstates
* (they are about to go away completely, too). The overall effect is
* that maintenance_work_mem always represents an absolute high watermark
* on the amount of memory used by a CREATE INDEX operation, regardless of
* the use of parallelism or any other factor.
*/
buildstate->spool->sortstate =
tuplesort_begin_index_btree(heap, index, buildstate->isunique,
maintenance_work_mem, coordinate,
false);
/*
* If building a unique index, put dead tuples in a second spool to keep
* them out of the uniqueness check. We expect that the second spool (for
* dead tuples) won't get very full, so we give it only work_mem.
*/
if (indexInfo->ii_Unique)
{
BTSpool *btspool2 = (BTSpool *) palloc0(sizeof(BTSpool));
SortCoordinate coordinate2 = NULL;
/* Initialize secondary spool */
btspool2->heap = heap;
btspool2->index = index;
btspool2->isunique = false;
/* Save as secondary spool */
buildstate->spool2 = btspool2;
if (buildstate->btleader)
{
/*
* Set up non-private state that is passed to
* tuplesort_begin_index_btree() about the basic high level
* coordination of a parallel sort.
*/
coordinate2 = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
coordinate2->isWorker = false;
coordinate2->nParticipants =
buildstate->btleader->nparticipanttuplesorts;
coordinate2->sharedsort = buildstate->btleader->sharedsort2;
}
/*
* We expect that the second one (for dead tuples) won't get very
* full, so we give it only work_mem
*/
buildstate->spool2->sortstate =
tuplesort_begin_index_btree(heap, index, false, work_mem,
coordinate2, false);
}
/* Fill spool using either serial or parallel heap scan */
if (!buildstate->btleader)
reltuples = IndexBuildHeapScan(heap, index, indexInfo, true,
_bt_build_callback, (void *) buildstate,
NULL);
else
reltuples = _bt_parallel_heapscan(buildstate,
&indexInfo->ii_BrokenHotChain);
/* okay, all heap tuples are spooled */
if (buildstate->spool2 && !buildstate->havedead)
{
/* spool2 turns out to be unnecessary */
_bt_spooldestroy(buildstate->spool2);
buildstate->spool2 = NULL;
}
return reltuples;
}
/*
* clean up a spool structure and its substructures.
*/
void
static void
_bt_spooldestroy(BTSpool *btspool)
{
tuplesort_end(btspool->sortstate);
@@ -186,7 +508,7 @@ _bt_spooldestroy(BTSpool *btspool)
/*
* spool an index entry into the sort file.
*/
void
static void
_bt_spool(BTSpool *btspool, ItemPointer self, Datum *values, bool *isnull)
{
tuplesort_putindextuplevalues(btspool->sortstate, btspool->index,
@@ -197,7 +519,7 @@ _bt_spool(BTSpool *btspool, ItemPointer self, Datum *values, bool *isnull)
* given a spool loaded by successive calls to _bt_spool,
* create an entire btree.
*/
void
static void
_bt_leafbuild(BTSpool *btspool, BTSpool *btspool2)
{
BTWriteState wstate;
@@ -231,11 +553,34 @@ _bt_leafbuild(BTSpool *btspool, BTSpool *btspool2)
_bt_load(&wstate, btspool, btspool2);
}
/*
* Internal routines.
* Per-tuple callback from IndexBuildHeapScan
*/
static void
_bt_build_callback(Relation index,
HeapTuple htup,
Datum *values,
bool *isnull,
bool tupleIsAlive,
void *state)
{
BTBuildState *buildstate = (BTBuildState *) state;
/*
* insert the index tuple into the appropriate spool file for subsequent
* processing
*/
if (tupleIsAlive || buildstate->spool2 == NULL)
_bt_spool(buildstate->spool, &htup->t_self, values, isnull);
else
{
/* dead tuples are put into spool2 */
buildstate->havedead = true;
_bt_spool(buildstate->spool2, &htup->t_self, values, isnull);
}
buildstate->indtuples += 1;
}
/*
* allocate workspace for a new, clean btree page, not linked to any siblings.
@@ -819,3 +1164,488 @@ _bt_load(BTWriteState *wstate, BTSpool *btspool, BTSpool *btspool2)
smgrimmedsync(wstate->index->rd_smgr, MAIN_FORKNUM);
}
}
/*
* Create parallel context, and launch workers for leader.
*
* buildstate argument should be initialized (with the exception of the
* tuplesort state in spools, which may later be created based on shared
* state initially set up here).
*
* isconcurrent indicates if operation is CREATE INDEX CONCURRENTLY.
*
* request is the target number of parallel worker processes to launch.
*
* Sets buildstate's BTLeader, which caller must use to shut down parallel
* mode by passing it to _bt_end_parallel() at the very end of its index
* build. If not even a single worker process can be launched, this is
* never set, and caller should proceed with a serial index build.
*/
static void
_bt_begin_parallel(BTBuildState *buildstate, bool isconcurrent, int request)
{
ParallelContext *pcxt;
int scantuplesortstates;
Snapshot snapshot;
Size estbtshared;
Size estsort;
BTShared *btshared;
Sharedsort *sharedsort;
Sharedsort *sharedsort2;
BTSpool *btspool = buildstate->spool;
BTLeader *btleader = (BTLeader *) palloc0(sizeof(BTLeader));
bool leaderparticipates = true;
#ifdef DISABLE_LEADER_PARTICIPATION
leaderparticipates = false;
#endif
/*
* Enter parallel mode, and create context for parallel build of btree
* index
*/
EnterParallelMode();
Assert(request > 0);
pcxt = CreateParallelContext("postgres", "_bt_parallel_build_main",
request, true);
scantuplesortstates = leaderparticipates ? request + 1 : request;
/*
* Prepare for scan of the base relation. In a normal index build, we use
* SnapshotAny because we must retrieve all tuples and do our own time
* qual checks (because we have to index RECENTLY_DEAD tuples). In a
* concurrent build, we take a regular MVCC snapshot and index whatever's
* live according to that.
*/
if (!isconcurrent)
snapshot = SnapshotAny;
else
snapshot = RegisterSnapshot(GetTransactionSnapshot());
/*
* Estimate size for at least two keys -- our own
* PARALLEL_KEY_BTREE_SHARED workspace, and PARALLEL_KEY_TUPLESORT
* tuplesort workspace
*/
estbtshared = _bt_parallel_estimate_shared(snapshot);
shm_toc_estimate_chunk(&pcxt->estimator, estbtshared);
estsort = tuplesort_estimate_shared(scantuplesortstates);
shm_toc_estimate_chunk(&pcxt->estimator, estsort);
/*
* Unique case requires a second spool, and so we may have to account for
* a third shared workspace -- PARALLEL_KEY_TUPLESORT_SPOOL2
*/
if (!btspool->isunique)
shm_toc_estimate_keys(&pcxt->estimator, 2);
else
{
shm_toc_estimate_chunk(&pcxt->estimator, estsort);
shm_toc_estimate_keys(&pcxt->estimator, 3);
}
/* Everyone's had a chance to ask for space, so now create the DSM */
InitializeParallelDSM(pcxt);
/* Store shared build state, for which we reserved space */
btshared = (BTShared *) shm_toc_allocate(pcxt->toc, estbtshared);
/* Initialize immutable state */
btshared->heaprelid = RelationGetRelid(btspool->heap);
btshared->indexrelid = RelationGetRelid(btspool->index);
btshared->isunique = btspool->isunique;
btshared->isconcurrent = isconcurrent;
btshared->scantuplesortstates = scantuplesortstates;
ConditionVariableInit(&btshared->workersdonecv);
SpinLockInit(&btshared->mutex);
/* Initialize mutable state */
btshared->nparticipantsdone = 0;
btshared->reltuples = 0.0;
btshared->havedead = false;
btshared->indtuples = 0.0;
btshared->brokenhotchain = false;
heap_parallelscan_initialize(&btshared->heapdesc, btspool->heap, snapshot);
/*
* Store shared tuplesort-private state, for which we reserved space.
* Then, initialize opaque state using tuplesort routine.
*/
sharedsort = (Sharedsort *) shm_toc_allocate(pcxt->toc, estsort);
tuplesort_initialize_shared(sharedsort, scantuplesortstates,
pcxt->seg);
shm_toc_insert(pcxt->toc, PARALLEL_KEY_BTREE_SHARED, btshared);
shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
/* Unique case requires a second spool, and associated shared state */
if (!btspool->isunique)
sharedsort2 = NULL;
else
{
/*
* Store additional shared tuplesort-private state, for which we
* reserved space. Then, initialize opaque state using tuplesort
* routine.
*/
sharedsort2 = (Sharedsort *) shm_toc_allocate(pcxt->toc, estsort);
tuplesort_initialize_shared(sharedsort2, scantuplesortstates,
pcxt->seg);
shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT_SPOOL2, sharedsort2);
}
/* Launch workers, saving status for leader/caller */
LaunchParallelWorkers(pcxt);
btleader->pcxt = pcxt;
btleader->nparticipanttuplesorts = pcxt->nworkers_launched;
if (leaderparticipates)
btleader->nparticipanttuplesorts++;
btleader->btshared = btshared;
btleader->sharedsort = sharedsort;
btleader->sharedsort2 = sharedsort2;
btleader->snapshot = snapshot;
/* If no workers were successfully launched, back out (do serial build) */
if (pcxt->nworkers_launched == 0)
{
_bt_end_parallel(btleader);
return;
}
/* Save leader state now that it's clear build will be parallel */
buildstate->btleader = btleader;
/* Join heap scan ourselves */
if (leaderparticipates)
_bt_leader_participate_as_worker(buildstate);
/*
* Caller needs to wait for all launched workers when we return. Make
* sure that the failure-to-start case will not hang forever.
*/
WaitForParallelWorkersToAttach(pcxt);
}
/*
* Shut down workers, destroy parallel context, and end parallel mode.
*/
static void
_bt_end_parallel(BTLeader *btleader)
{
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(btleader->pcxt);
/* Free last reference to MVCC snapshot, if one was used */
if (IsMVCCSnapshot(btleader->snapshot))
UnregisterSnapshot(btleader->snapshot);
DestroyParallelContext(btleader->pcxt);
ExitParallelMode();
}
/*
* Returns size of shared memory required to store state for a parallel
* btree index build based on the snapshot its parallel scan will use.
*/
static Size
_bt_parallel_estimate_shared(Snapshot snapshot)
{
if (!IsMVCCSnapshot(snapshot))
{
Assert(snapshot == SnapshotAny);
return sizeof(BTShared);
}
return add_size(offsetof(BTShared, heapdesc) +
offsetof(ParallelHeapScanDescData, phs_snapshot_data),
EstimateSnapshotSpace(snapshot));
}
/*
* Within leader, wait for end of heap scan.
*
* When called, parallel heap scan started by _bt_begin_parallel() will
* already be underway within worker processes (when leader participates
* as a worker, we should end up here just as workers are finishing).
*
* Fills in fields needed for ambuild statistics, and lets caller set
* field indicating that some worker encountered a broken HOT chain.
*
* Returns the total number of heap tuples scanned.
*/
static double
_bt_parallel_heapscan(BTBuildState *buildstate, bool *brokenhotchain)
{
BTShared *btshared = buildstate->btleader->btshared;
int nparticipanttuplesorts;
double reltuples;
nparticipanttuplesorts = buildstate->btleader->nparticipanttuplesorts;
for (;;)
{
SpinLockAcquire(&btshared->mutex);
if (btshared->nparticipantsdone == nparticipanttuplesorts)
{
buildstate->havedead = btshared->havedead;
buildstate->indtuples = btshared->indtuples;
*brokenhotchain = btshared->brokenhotchain;
reltuples = btshared->reltuples;
SpinLockRelease(&btshared->mutex);
break;
}
SpinLockRelease(&btshared->mutex);
ConditionVariableSleep(&btshared->workersdonecv,
WAIT_EVENT_PARALLEL_CREATE_INDEX_SCAN);
}
ConditionVariableCancelSleep();
return reltuples;
}
/*
* Within leader, participate as a parallel worker.
*/
static void
_bt_leader_participate_as_worker(BTBuildState *buildstate)
{
BTLeader *btleader = buildstate->btleader;
BTSpool *leaderworker;
BTSpool *leaderworker2;
int sortmem;
/* Allocate memory and initialize private spool */
leaderworker = (BTSpool *) palloc0(sizeof(BTSpool));
leaderworker->heap = buildstate->spool->heap;
leaderworker->index = buildstate->spool->index;
leaderworker->isunique = buildstate->spool->isunique;
/* Initialize second spool, if required */
if (!btleader->btshared->isunique)
leaderworker2 = NULL;
else
{
/* Allocate memory for worker's own private secondary spool */
leaderworker2 = (BTSpool *) palloc0(sizeof(BTSpool));
/* Initialize worker's own secondary spool */
leaderworker2->heap = leaderworker->heap;
leaderworker2->index = leaderworker->index;
leaderworker2->isunique = false;
}
/*
* Might as well use reliable figure when doling out maintenance_work_mem
* (when requested number of workers were not launched, this will be
* somewhat higher than it is for other workers).
*/
sortmem = maintenance_work_mem / btleader->nparticipanttuplesorts;
/* Perform work common to all participants */
_bt_parallel_scan_and_sort(leaderworker, leaderworker2, btleader->btshared,
btleader->sharedsort, btleader->sharedsort2,
sortmem);
#ifdef BTREE_BUILD_STATS
if (log_btree_build_stats)
{
ShowUsage("BTREE BUILD (Leader Partial Spool) STATISTICS");
ResetUsage();
}
#endif /* BTREE_BUILD_STATS */
}
/*
* Perform work within a launched parallel process.
*/
void
_bt_parallel_build_main(dsm_segment *seg, shm_toc *toc)
{
BTSpool *btspool;
BTSpool *btspool2;
BTShared *btshared;
Sharedsort *sharedsort;
Sharedsort *sharedsort2;
Relation heapRel;
Relation indexRel;
LOCKMODE heapLockmode;
LOCKMODE indexLockmode;
int sortmem;
#ifdef BTREE_BUILD_STATS
if (log_btree_build_stats)
ResetUsage();
#endif /* BTREE_BUILD_STATS */
/* Look up shared state */
btshared = shm_toc_lookup(toc, PARALLEL_KEY_BTREE_SHARED, false);
/* Open relations using lock modes known to be obtained by index.c */
if (!btshared->isconcurrent)
{
heapLockmode = ShareLock;
indexLockmode = AccessExclusiveLock;
}
else
{
heapLockmode = ShareUpdateExclusiveLock;
indexLockmode = RowExclusiveLock;
}
/* Open relations within worker */
heapRel = heap_open(btshared->heaprelid, heapLockmode);
indexRel = index_open(btshared->indexrelid, indexLockmode);
/* Initialize worker's own spool */
btspool = (BTSpool *) palloc0(sizeof(BTSpool));
btspool->heap = heapRel;
btspool->index = indexRel;
btspool->isunique = btshared->isunique;
/* Look up shared state private to tuplesort.c */
sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
tuplesort_attach_shared(sharedsort, seg);
if (!btshared->isunique)
{
btspool2 = NULL;
sharedsort2 = NULL;
}
else
{
/* Allocate memory for worker's own private secondary spool */
btspool2 = (BTSpool *) palloc0(sizeof(BTSpool));
/* Initialize worker's own secondary spool */
btspool2->heap = btspool->heap;
btspool2->index = btspool->index;
btspool2->isunique = false;
/* Look up shared state private to tuplesort.c */
sharedsort2 = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT_SPOOL2, false);
tuplesort_attach_shared(sharedsort2, seg);
}
/* Perform sorting of spool, and possibly a spool2 */
sortmem = maintenance_work_mem / btshared->scantuplesortstates;
_bt_parallel_scan_and_sort(btspool, btspool2, btshared, sharedsort,
sharedsort2, sortmem);
#ifdef BTREE_BUILD_STATS
if (log_btree_build_stats)
{
ShowUsage("BTREE BUILD (Worker Partial Spool) STATISTICS");
ResetUsage();
}
#endif /* BTREE_BUILD_STATS */
index_close(indexRel, indexLockmode);
heap_close(heapRel, heapLockmode);
}
/*
* Perform a worker's portion of a parallel sort.
*
* This generates a tuplesort for passed btspool, and a second tuplesort
* state if a second btspool is need (i.e. for unique index builds). All
* other spool fields should already be set when this is called.
*
* sortmem is the amount of working memory to use within each worker,
* expressed in KBs.
*
* When this returns, workers are done, and need only release resources.
*/
static void
_bt_parallel_scan_and_sort(BTSpool *btspool, BTSpool *btspool2,
BTShared *btshared, Sharedsort *sharedsort,
Sharedsort *sharedsort2, int sortmem)
{
SortCoordinate coordinate;
BTBuildState buildstate;
HeapScanDesc scan;
double reltuples;
IndexInfo *indexInfo;
/* Initialize local tuplesort coordination state */
coordinate = palloc0(sizeof(SortCoordinateData));
coordinate->isWorker = true;
coordinate->nParticipants = -1;
coordinate->sharedsort = sharedsort;
/* Begin "partial" tuplesort */
btspool->sortstate = tuplesort_begin_index_btree(btspool->heap,
btspool->index,
btspool->isunique,
sortmem, coordinate,
false);
/*
* Just as with serial case, there may be a second spool. If so, a
* second, dedicated spool2 partial tuplesort is required.
*/
if (btspool2)
{
SortCoordinate coordinate2;
/*
* We expect that the second one (for dead tuples) won't get very
* full, so we give it only work_mem (unless sortmem is less for
* worker). Worker processes are generally permitted to allocate
* work_mem independently.
*/
coordinate2 = palloc0(sizeof(SortCoordinateData));
coordinate2->isWorker = true;
coordinate2->nParticipants = -1;
coordinate2->sharedsort = sharedsort2;
btspool2->sortstate =
tuplesort_begin_index_btree(btspool->heap, btspool->index, false,
Min(sortmem, work_mem), coordinate2,
false);
}
/* Fill in buildstate for _bt_build_callback() */
buildstate.isunique = btshared->isunique;
buildstate.havedead = false;
buildstate.heap = btspool->heap;
buildstate.spool = btspool;
buildstate.spool2 = btspool2;
buildstate.indtuples = 0;
buildstate.btleader = NULL;
/* Join parallel scan */
indexInfo = BuildIndexInfo(btspool->index);
indexInfo->ii_Concurrent = btshared->isconcurrent;
scan = heap_beginscan_parallel(btspool->heap, &btshared->heapdesc);
reltuples = IndexBuildHeapScan(btspool->heap, btspool->index, indexInfo,
true, _bt_build_callback,
(void *) &buildstate, scan);
/*
* Execute this worker's part of the sort.
*
* Unlike leader and serial cases, we cannot avoid calling
* tuplesort_performsort() for spool2 if it ends up containing no dead
* tuples (this is disallowed for workers by tuplesort).
*/
tuplesort_performsort(btspool->sortstate);
if (btspool2)
tuplesort_performsort(btspool2->sortstate);
/*
* Done. Record ambuild statistics, and whether we encountered a broken
* HOT chain.
*/
SpinLockAcquire(&btshared->mutex);
btshared->nparticipantsdone++;
btshared->reltuples += reltuples;
if (buildstate.havedead)
btshared->havedead = true;
btshared->indtuples += buildstate.indtuples;
if (indexInfo->ii_BrokenHotChain)
btshared->brokenhotchain = true;
SpinLockRelease(&btshared->mutex);
/* Notify leader */
ConditionVariableSignal(&btshared->workersdonecv);
/* We can end tuplesorts immediately */
tuplesort_end(btspool->sortstate);
if (btspool2)
tuplesort_end(btspool2->sortstate);
}

View File

@@ -138,7 +138,8 @@ spgbuild(Relation heap, Relation index, IndexInfo *indexInfo)
ALLOCSET_DEFAULT_SIZES);
reltuples = IndexBuildHeapScan(heap, index, indexInfo, true,
spgistBuildCallback, (void *) &buildstate);
spgistBuildCallback, (void *) &buildstate,
NULL);
MemoryContextDelete(buildstate.tmpCtx);

View File

@@ -14,6 +14,7 @@
#include "postgres.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
#include "access/xact.h"
@@ -129,6 +130,9 @@ static const struct
{
{
"ParallelQueryMain", ParallelQueryMain
},
{
"_bt_parallel_build_main", _bt_parallel_build_main
}
};
@@ -146,7 +150,7 @@ static void ParallelWorkerShutdown(int code, Datum arg);
*/
ParallelContext *
CreateParallelContext(const char *library_name, const char *function_name,
int nworkers)
int nworkers, bool serializable_okay)
{
MemoryContext oldcontext;
ParallelContext *pcxt;
@@ -167,9 +171,11 @@ CreateParallelContext(const char *library_name, const char *function_name,
/*
* If we are running under serializable isolation, we can't use parallel
* workers, at least not until somebody enhances that mechanism to be
* parallel-aware.
* parallel-aware. Utility statement callers may ask us to ignore this
* restriction because they're always able to safely ignore the fact that
* SIREAD locks do not work with parallelism.
*/
if (IsolationIsSerializable())
if (IsolationIsSerializable() && !serializable_okay)
nworkers = 0;
/* We might be running in a short-lived memory context. */