Parallel CREATE INDEX for BRIN indexes
Hi,
Here's a WIP patch allowing parallel CREATE INDEX for BRIN indexes. The
infrastructure (starting workers etc.) is "inspired" by the BTREE code
(i.e. copied from that and massaged a bit to call brin stuff).
_bt_begin_parallel -> _brin_begin_parallel
_bt_end_parallel -> _brin_end_parallel
_bt_parallel_estimate_shared -> _brin_parallel_estimate_shared
_bt_leader_participate_as_worker -> _brin_leader_participate_as_worker
_bt_parallel_scan_and_sort -> _brin_parallel_scan_and_build
This is mostly mechanical stuff - setting up the parallel workers,
starting the scan etc.
The tricky part is how to divide the work between workers and how we
combine the partial results. For BTREE we simply let each worker to read
a subset of the table (using a parallel scan), sort it and then do a
merge sort on the partial results.
For BRIN it's a bit different, because the indexes essentially splits
the table into smaller ranges and treat them independently. So the
easiest way is to organize the table scan so that each range gets
processed by exactly one worker. Each worker writes the index tuples
into a temporary file, and then when all workers are done we read and
write them into the index.
The problem is a parallel scan assigns mostly random subset of the table
to each worker - it's not guaranteed a BRIN page range to be processed
by a single worker.
0001 does that in a bit silly way - instead of doing single large scan,
each worker does a sequence of TID range scans for each worker (see
_brin_parallel_scan_and_build), and BrinShared has fields used to track
which ranges were already assigned to workers. A bit cumbersome, but it
works pretty well.
0002 replaces the TID range scan sequence with a single parallel scan,
modified to assign "chunks" in multiple of pagesPerRange.
In both cases _brin_end_parallel then reads the summaries from worker
files, and adds them into the index. In 0001 this is fairly simple,
although we could do one more improvement and sort the ranges by range
start to make the index nicer (and possibly a bit more efficient). This
should be simple, because the per-worker results are already sorted like
that (so a merge sort in _brin_end_parallel would be enough).
For 0002 it's a bit more complicated, because with a single parallel
scan brinbuildCallbackParallel can't decide if a range is assigned to a
different worker or empty. And we want to generate summaries for empty
ranges in the index. We could either skip such range during index build,
and then add empty summaries in _brin_end_parallel (if needed), or add
them and then merge them using "union".
I just realized there's a third option to do this - we could just do
regular parallel scan (with no particular regard to pagesPerRange), and
then do "union" when merging results from workers. It doesn't require
the sequence of TID scans, and the union would also handle the empty
ranges. The per-worker results might be much larger, though, because
each worker might produce up to the "full" BRIN index.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
0001-parallel-CREATE-INDEX-for-BRIN-20230608.patchtext/x-patch; charset=UTF-8; name=0001-parallel-CREATE-INDEX-for-BRIN-20230608.patchDownload
From 0d37a829e768772ef3e9c080f96333e24cdd43b7 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Sun, 26 Mar 2023 00:44:01 +0100
Subject: [PATCH 1/2] parallel CREATE INDEX for BRIN
---
src/backend/access/brin/brin.c | 714 +++++++++++++++++++++++++-
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 3 +-
src/include/access/brin.h | 3 +
4 files changed, 719 insertions(+), 5 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 3c6a956eaa3..13d94931efc 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -31,8 +31,10 @@
#include "miscadmin.h"
#include "pgstat.h"
#include "postmaster/autovacuum.h"
+#include "storage/buffile.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -41,6 +43,98 @@
#include "utils/memutils.h"
#include "utils/rel.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000002)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xA000000000000003)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xA000000000000004)
+
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist for
+ * the benefit of worker processes that need to create state corresponding
+ * to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /* Used to pass built BRIN tuples from workers to leader (for insert). */
+ SharedFileSet fileset;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN index
+ * builds that must work just the same when an index is built in parallel.
+ */
+ slock_t mutex;
+
+ /*
+ *
+ */
+ int last_worker_id;
+ BlockNumber next_range;
+ BlockNumber last_range;
+
+ /*
+ * Mutable state that is maintained by workers, and reported back to
+ * leader at end of the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+} BrinShared;
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* parallel context itself */
+ ParallelContext *pcxt;
+
+ /*
+ * nparticipantworkers 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 nparticipantworkers;
+
+ /*
+ * Leader process convenience pointers to shared state (leader avoids TOC
+ * lookups).
+ *
+ * brinshared is the shared state for entire build. snapshot is the snapshot
+ * used by the scan iff an MVCC snapshot is required.
+ */
+ BrinShared *brinshared;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -50,12 +144,22 @@ typedef struct BrinBuildState
{
Relation bs_irel;
int bs_numtuples;
+ int bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BufFile *bs_file;
} BrinBuildState;
/*
@@ -76,6 +180,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -83,6 +188,18 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, Datum *values, bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinShared *brinshared,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -822,6 +939,54 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a temporary file, and leave the insertion up to the leader (which may reorder
+ * them a bit etc.).
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; make sure to insert index
+ * tuples for those too.
+ */
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the temporary file */
+ form_and_spill_tuple(state);
+
+ /* set state to correspond to the next range */
+ state->bs_currRangeStart += state->bs_pagesPerRange;
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -883,18 +1048,46 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
revmap = brinRevmapInitialize(index, &pagesPerRange, NULL);
state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ /*
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
+ */
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
+
/*
* Now scan the relation. No syncscan allowed here because we want the
* heap blocks in physical order.
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
- form_and_insert_tuple(state);
+ /* no parallel index build, just do the usual thing */
+ if (state->bs_leader == NULL)
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
+ form_and_insert_tuple(state);
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
+ }
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read
+ * the tuples from the temporary files and insert them into the index.
+ */
+ if (state->bs_leader)
+ _brin_end_parallel(state->bs_leader, state);
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1299,12 +1492,15 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
return state;
}
@@ -1597,6 +1793,31 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a temporary file (leader will insert it into the
+ * index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ Assert(state->bs_file);
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ BufFileWrite(state->bs_file, &size, sizeof(Size));
+ BufFileWrite(state->bs_file, &state->bs_currRangeStart, sizeof(BlockNumber));
+ BufFileWrite(state->bs_file, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -1916,3 +2137,488 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ Snapshot snapshot;
+ Size estbrinshared;
+ BrinShared *brinshared;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* */
+ SharedFileSetInit(&brinshared->fileset, pcxt->seg);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ /* Track work assigned to workers etc. */
+ brinshared->last_worker_id = 0;
+ brinshared->next_range = 0;
+ brinshared->last_range = RelationGetNumberOfBlocks(heap);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipantworkers = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipantworkers++;
+ brinleader->brinshared = brinshared;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinShared *brinshared = brinleader->brinshared;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ /*
+ * XXX maybe we should sort the ranges by rangeStart? That'd give us index
+ * that is cheaper to walk sequentially, because we'd not have any page
+ * misses (mostly getting data from the same page as before). Although the
+ * index should be pretty small in general, and thus cached. OTOH each
+ * worker should produce tuples in the right order, so we could just merge
+ * sort them.
+ *
+ * XXX Alternatively, we could arrange the build so that the workers read
+ * a continuous chunk of the table. For example, with K workers we might
+ * leave the first 1/K to the first worker, then 1/K to the second etc.
+ * The we would not need to reorder anything, we would just read the
+ * results for all workers.
+ *
+ * XXX That's also mean we don't do many small TID scans, as now.
+ *
+ * The problem is we don't know how many workers will be started while
+ * determining this, but maybe we could postpone that decision somehow?
+ * We'd have to wait for all the launched workers to attach, I guess.
+ */
+ for (i = 1; i <= brinshared->last_worker_id; i++)
+ {
+ BufFile *f;
+ char name[MAXPGPATH];
+ int64 fsize;
+ int64 fpos = 0;
+
+ snprintf(name, MAXPGPATH, "tuples.%d", i);
+
+ f = BufFileOpenFileSet(&brinshared->fileset.fs, name, O_RDONLY, false);
+ fsize = BufFileSize(f);
+
+ while (fpos < fsize)
+ {
+ Size size;
+ BlockNumber rangeStart;
+ BrinTuple *tup;
+
+ if (BufFileRead(f, &size, sizeof(Size)) != sizeof(Size))
+ elog(ERROR, "failed read");
+
+ if (BufFileRead(f, &rangeStart, sizeof(BlockNumber)) != sizeof(BlockNumber))
+ elog(ERROR, "failed read");
+
+ tup = (BrinTuple *) palloc(size);
+
+ if (BufFileRead(f, tup, size) != size)
+ elog(ERROR, "failed read");
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, rangeStart, tup, size);
+
+ fpos += sizeof(Size) + sizeof(BlockNumber) + size;
+ }
+
+ BufFileClose(f);
+ }
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /*
+ * 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 / brinleader->nparticipantworkers;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, brinleader->brinshared,
+ heap, index, sortmem, true);
+
+ /* insert the last range */
+ form_and_spill_tuple(buildstate);
+
+ BufFileClose(buildstate->bs_file);
+ buildstate->bs_file = NULL;
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinShared *brinshared,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ double reltuples;
+ IndexInfo *indexInfo;
+ char name[MAXPGPATH];
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ SpinLockAcquire(&brinshared->mutex);
+ state->bs_worker_id = (++brinshared->last_worker_id);
+ SpinLockRelease(&brinshared->mutex);
+
+ snprintf(name, MAXPGPATH, "tuples.%d", state->bs_worker_id);
+
+ state->bs_file = BufFileCreateFileSet(&brinshared->fileset.fs, name);
+
+ /* Get chunks of the table, do TID Scans and build the ranges */
+ while (true)
+ {
+ TableScanDesc scan;
+ BlockNumber startBlock,
+ lastBlock,
+ chunkBlocks;
+ ItemPointerData mintid,
+ maxtid;
+
+ /*
+ * Acquire larger chunks of data - this matters especially for low
+ * pages_per_range settings (e.g. set to 1). Otherwise there would
+ * be a lot of trashing and overhead with multiple workers.
+ *
+ * Not sure where's the sweet spot. Maybe tie this to the prefetching
+ * too (maintenance_effective_io_concucrrency)?
+ *
+ * FIXME The chunkBlocks needs to be a multiple of bs_pagesPerRange.
+ */
+ chunkBlocks = Max(128, state->bs_pagesPerRange);
+
+ SpinLockAcquire(&brinshared->mutex);
+ startBlock = brinshared->next_range;
+ lastBlock = brinshared->last_range;
+ brinshared->next_range += chunkBlocks;
+ SpinLockRelease(&brinshared->mutex);
+
+ state->bs_currRangeStart = startBlock;
+
+ /* did we reach the end of the heap relation? */
+ if (startBlock > lastBlock)
+ break;
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+
+ ItemPointerSet(&mintid, startBlock, 0);
+ ItemPointerSet(&maxtid, startBlock + (chunkBlocks - 1),
+ MaxHeapTuplesPerPage);
+
+ /* start tidscan to read the relevant part of the table */
+ scan = table_beginscan_tidrange(heap, SnapshotAny, // FIXME which snapshot to use?
+ &mintid, &maxtid);
+
+#ifdef USE_PREFETCH
+ /* do prefetching (this prefetches the whole range. not sure that's good) */
+ for (BlockNumber blkno = startBlock; blkno < startBlock + chunkBlocks; blkno++)
+ PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, blkno);
+#endif
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* spill the last tuple */
+ form_and_spill_tuple(state);
+
+ state->bs_reltuples += reltuples;
+
+ /* set state to invalid range */
+ state->bs_currRangeStart = InvalidBlockNumber;
+ }
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL, brinshared->pagesPerRange);
+
+ /* Attach to the shared fileset. */
+ SharedFileSetAttach(&brinshared->fileset, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /* FIXME tie this to number of participants, somehow */
+ sortmem = maintenance_work_mem / 2;
+ _brin_parallel_scan_and_build(buildstate, brinshared,
+ heapRel, indexRel, sortmem, false);
+
+ /* insert the last range */
+ form_and_spill_tuple(buildstate);
+
+ BufFileClose(buildstate->bs_file);
+ buildstate->bs_file = NULL;
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 2b8bc2f58dd..72086212f47 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -144,6 +145,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 352e43d0e61..5bdd025bb25 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2980,7 +2980,8 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ (indexRelation->rd_rel->relam == BTREE_AM_OID ||
+ indexRelation->rd_rel->relam == BRIN_AM_OID))
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d51..3451ecb211f 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
--
2.40.1
0002-switch-CREATE-INDEX-for-BRIN-to-parallel-sc-20230608.patchtext/x-patch; charset=UTF-8; name=0002-switch-CREATE-INDEX-for-BRIN-to-parallel-sc-20230608.patchDownload
From 5e19887efa57f77b17df45a65a101b84721f665c Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Sat, 8 Apr 2023 16:39:51 +0200
Subject: [PATCH 2/2] switch CREATE INDEX for BRIN to parallel scan
---
src/backend/access/brin/brin.c | 110 +++++++++++-----------------
src/backend/access/nbtree/nbtsort.c | 2 +-
src/backend/access/table/tableam.c | 26 +++++--
src/backend/executor/nodeSeqscan.c | 3 +-
src/include/access/relscan.h | 1 +
src/include/access/tableam.h | 9 ++-
6 files changed, 73 insertions(+), 78 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 13d94931efc..796c0ecf06e 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -85,12 +85,8 @@ typedef struct BrinShared
*/
slock_t mutex;
- /*
- *
- */
+ /* XXX Probably not needed. Identifies the worker. */
int last_worker_id;
- BlockNumber next_range;
- BlockNumber last_range;
/*
* Mutable state that is maintained by workers, and reported back to
@@ -105,8 +101,23 @@ typedef struct BrinShared
int nparticipantsdone;
double reltuples;
double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
} BrinShared;
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
/*
* Status for leader in parallel index build.
*/
@@ -975,8 +986,20 @@ brinbuildCallbackParallel(Relation index,
/* create the index tuple and write it into the temporary file */
form_and_spill_tuple(state);
- /* set state to correspond to the next range */
- state->bs_currRangeStart += state->bs_pagesPerRange;
+ /*
+ * set state to correspond to the next range
+ *
+ * XXX This has the issue that it skips ranges summarized by other
+ * workers, but it also skips empty ranges that should have been
+ * summarized. We'd need to either make the workers aware which
+ * chunk they are actually processing (which is currently known
+ * only in the ParallelBlockTableScan bit). Or we could ignore it
+ * here, and then decide it while "merging" results from workers
+ * (if there's no entry for the range, it had to be empty so we
+ * just add an empty one).
+ */
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ state->bs_currRangeStart += state->bs_pagesPerRange;
/* re-initialize state for it */
brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
@@ -2243,8 +2266,10 @@ _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
/* Track work assigned to workers etc. */
brinshared->last_worker_id = 0;
- brinshared->next_range = 0;
- brinshared->last_range = RelationGetNumberOfBlocks(heap);
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ snapshot, brinshared->pagesPerRange);
/*
* Store shared tuplesort-private state, for which we reserved space.
@@ -2452,6 +2477,7 @@ _brin_parallel_scan_and_build(BrinBuildState *state, BrinShared *brinshared,
Relation heap, Relation index, int sortmem,
bool progress)
{
+ TableScanDesc scan;
double reltuples;
IndexInfo *indexInfo;
char name[MAXPGPATH];
@@ -2468,68 +2494,16 @@ _brin_parallel_scan_and_build(BrinBuildState *state, BrinShared *brinshared,
state->bs_file = BufFileCreateFileSet(&brinshared->fileset.fs, name);
- /* Get chunks of the table, do TID Scans and build the ranges */
- while (true)
- {
- TableScanDesc scan;
- BlockNumber startBlock,
- lastBlock,
- chunkBlocks;
- ItemPointerData mintid,
- maxtid;
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
- /*
- * Acquire larger chunks of data - this matters especially for low
- * pages_per_range settings (e.g. set to 1). Otherwise there would
- * be a lot of trashing and overhead with multiple workers.
- *
- * Not sure where's the sweet spot. Maybe tie this to the prefetching
- * too (maintenance_effective_io_concucrrency)?
- *
- * FIXME The chunkBlocks needs to be a multiple of bs_pagesPerRange.
- */
- chunkBlocks = Max(128, state->bs_pagesPerRange);
-
- SpinLockAcquire(&brinshared->mutex);
- startBlock = brinshared->next_range;
- lastBlock = brinshared->last_range;
- brinshared->next_range += chunkBlocks;
- SpinLockRelease(&brinshared->mutex);
-
- state->bs_currRangeStart = startBlock;
-
- /* did we reach the end of the heap relation? */
- if (startBlock > lastBlock)
- break;
-
- /* re-initialize state for it */
- brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
- ItemPointerSet(&mintid, startBlock, 0);
- ItemPointerSet(&maxtid, startBlock + (chunkBlocks - 1),
- MaxHeapTuplesPerPage);
+ /* spill the last tuple */
+ form_and_spill_tuple(state);
- /* start tidscan to read the relevant part of the table */
- scan = table_beginscan_tidrange(heap, SnapshotAny, // FIXME which snapshot to use?
- &mintid, &maxtid);
-
-#ifdef USE_PREFETCH
- /* do prefetching (this prefetches the whole range. not sure that's good) */
- for (BlockNumber blkno = startBlock; blkno < startBlock + chunkBlocks; blkno++)
- PrefetchBuffer(scan->rs_rd, MAIN_FORKNUM, blkno);
-#endif
-
- reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
- brinbuildCallbackParallel, state, scan);
-
- /* spill the last tuple */
- form_and_spill_tuple(state);
-
- state->bs_reltuples += reltuples;
-
- /* set state to invalid range */
- state->bs_currRangeStart = InvalidBlockNumber;
- }
+ state->bs_reltuples += reltuples;
/*
* Done. Record ambuild statistics.
diff --git a/src/backend/access/nbtree/nbtsort.c b/src/backend/access/nbtree/nbtsort.c
index 6ad3f3c54d5..69ec9d5fe9c 100644
--- a/src/backend/access/nbtree/nbtsort.c
+++ b/src/backend/access/nbtree/nbtsort.c
@@ -1575,7 +1575,7 @@ _bt_begin_parallel(BTBuildState *buildstate, bool isconcurrent, int request)
btshared->brokenhotchain = false;
table_parallelscan_initialize(btspool->heap,
ParallelTableScanFromBTShared(btshared),
- snapshot);
+ snapshot, InvalidBlockNumber);
/*
* Store shared tuplesort-private state, for which we reserved space.
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 771438c8cec..834cc9cd7c6 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -153,9 +153,9 @@ table_parallelscan_estimate(Relation rel, Snapshot snapshot)
void
table_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan,
- Snapshot snapshot)
+ Snapshot snapshot, BlockNumber chunk_factor)
{
- Size snapshot_off = rel->rd_tableam->parallelscan_initialize(rel, pscan);
+ Size snapshot_off = rel->rd_tableam->parallelscan_initialize(rel, pscan, chunk_factor);
pscan->phs_snapshot_off = snapshot_off;
@@ -395,16 +395,21 @@ table_block_parallelscan_estimate(Relation rel)
}
Size
-table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan)
+table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan, BlockNumber chunk_factor)
{
ParallelBlockTableScanDesc bpscan = (ParallelBlockTableScanDesc) pscan;
bpscan->base.phs_relid = RelationGetRelid(rel);
bpscan->phs_nblocks = RelationGetNumberOfBlocks(rel);
- /* compare phs_syncscan initialization to similar logic in initscan */
+ bpscan->phs_chunk_factor = chunk_factor;
+ /* compare phs_syncscan initialization to similar logic in initscan
+ *
+ * Disable sync scans if the chunk factor is set (valid block number).
+ */
bpscan->base.phs_syncscan = synchronize_seqscans &&
!RelationUsesLocalBuffers(rel) &&
- bpscan->phs_nblocks > NBuffers / 4;
+ (bpscan->phs_nblocks > NBuffers / 4) &&
+ !BlockNumberIsValid(bpscan->phs_chunk_factor);
SpinLockInit(&bpscan->phs_mutex);
bpscan->phs_startblock = InvalidBlockNumber;
pg_atomic_init_u64(&bpscan->phs_nallocated, 0);
@@ -459,6 +464,17 @@ table_block_parallelscan_startblock_init(Relation rel,
pbscanwork->phsw_chunk_size = Min(pbscanwork->phsw_chunk_size,
PARALLEL_SEQSCAN_MAX_CHUNK_SIZE);
+ /*
+ * If the chunk size factor is set, we need to make sure the chunk size is
+ * a multiple of that value.
+ */
+ if (pbscan->phs_chunk_factor != InvalidBlockNumber)
+ {
+ int nchunks = (pbscanwork->phsw_chunk_size / pbscan->phs_chunk_factor);
+
+ pbscanwork->phsw_chunk_size = Max(1, nchunks) * pbscan->phs_chunk_factor;
+ }
+
retry:
/* Grab the spinlock. */
SpinLockAcquire(&pbscan->phs_mutex);
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 4da0f28f7ba..e017f05f780 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -274,7 +274,8 @@ ExecSeqScanInitializeDSM(SeqScanState *node,
pscan = shm_toc_allocate(pcxt->toc, node->pscan_len);
table_parallelscan_initialize(node->ss.ss_currentRelation,
pscan,
- estate->es_snapshot);
+ estate->es_snapshot,
+ InvalidBlockNumber);
shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pscan);
node->ss.ss_currentScanDesc =
table_beginscan_parallel(node->ss.ss_currentRelation, pscan);
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index d03360eac04..72a20d882f5 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -79,6 +79,7 @@ typedef struct ParallelBlockTableScanDescData
BlockNumber phs_nblocks; /* # blocks in relation at start of scan */
slock_t phs_mutex; /* mutual exclusion for setting startblock */
BlockNumber phs_startblock; /* starting block number */
+ BlockNumber phs_chunk_factor; /* chunks need to be a multiple of this */
pg_atomic_uint64 phs_nallocated; /* number of blocks allocated to
* workers so far. */
} ParallelBlockTableScanDescData;
diff --git a/src/include/access/tableam.h b/src/include/access/tableam.h
index 230bc39cc0e..82297546e27 100644
--- a/src/include/access/tableam.h
+++ b/src/include/access/tableam.h
@@ -390,7 +390,8 @@ typedef struct TableAmRoutine
* relation.
*/
Size (*parallelscan_initialize) (Relation rel,
- ParallelTableScanDesc pscan);
+ ParallelTableScanDesc pscan,
+ BlockNumber chunk_factor);
/*
* Reinitialize `pscan` for a new scan. `rel` will be the same relation as
@@ -1148,7 +1149,8 @@ extern Size table_parallelscan_estimate(Relation rel, Snapshot snapshot);
*/
extern void table_parallelscan_initialize(Relation rel,
ParallelTableScanDesc pscan,
- Snapshot snapshot);
+ Snapshot snapshot,
+ BlockNumber chunk_factor);
/*
* Begin a parallel scan. `pscan` needs to have been initialized with
@@ -2064,7 +2066,8 @@ extern void simple_table_tuple_update(Relation rel, ItemPointer otid,
extern Size table_block_parallelscan_estimate(Relation rel);
extern Size table_block_parallelscan_initialize(Relation rel,
- ParallelTableScanDesc pscan);
+ ParallelTableScanDesc pscan,
+ BlockNumber chunk_factor);
extern void table_block_parallelscan_reinitialize(Relation rel,
ParallelTableScanDesc pscan);
extern BlockNumber table_block_parallelscan_nextpage(Relation rel,
--
2.40.1
On Thu, 8 Jun 2023 at 14:55, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
Here's a WIP patch allowing parallel CREATE INDEX for BRIN indexes. The
infrastructure (starting workers etc.) is "inspired" by the BTREE code
(i.e. copied from that and massaged a bit to call brin stuff).
Nice work.
In both cases _brin_end_parallel then reads the summaries from worker
files, and adds them into the index. In 0001 this is fairly simple,
although we could do one more improvement and sort the ranges by range
start to make the index nicer (and possibly a bit more efficient). This
should be simple, because the per-worker results are already sorted like
that (so a merge sort in _brin_end_parallel would be enough).
I see that you manually built the passing and sorting of tuples
between workers, but can't we use the parallel tuplesort
infrastructure for that? It already has similar features in place and
improves code commonality.
For 0002 it's a bit more complicated, because with a single parallel
scan brinbuildCallbackParallel can't decide if a range is assigned to a
different worker or empty. And we want to generate summaries for empty
ranges in the index. We could either skip such range during index build,
and then add empty summaries in _brin_end_parallel (if needed), or add
them and then merge them using "union".I just realized there's a third option to do this - we could just do
regular parallel scan (with no particular regard to pagesPerRange), and
then do "union" when merging results from workers. It doesn't require
the sequence of TID scans, and the union would also handle the empty
ranges. The per-worker results might be much larger, though, because
each worker might produce up to the "full" BRIN index.
Would it be too much effort to add a 'min_chunk_size' argument to
table_beginscan_parallel (or ParallelTableScanDesc) that defines the
minimum granularity of block ranges to be assigned to each process? I
think that would be the most elegant solution that would require
relatively little effort: table_block_parallelscan_nextpage already
does parallel management of multiple chunk sizes, and I think this
modification would fit quite well in that code.
Kind regards,
Matthias van de Meent
On 7/4/23 23:53, Matthias van de Meent wrote:
On Thu, 8 Jun 2023 at 14:55, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
Here's a WIP patch allowing parallel CREATE INDEX for BRIN indexes. The
infrastructure (starting workers etc.) is "inspired" by the BTREE code
(i.e. copied from that and massaged a bit to call brin stuff).Nice work.
In both cases _brin_end_parallel then reads the summaries from worker
files, and adds them into the index. In 0001 this is fairly simple,
although we could do one more improvement and sort the ranges by range
start to make the index nicer (and possibly a bit more efficient). This
should be simple, because the per-worker results are already sorted like
that (so a merge sort in _brin_end_parallel would be enough).I see that you manually built the passing and sorting of tuples
between workers, but can't we use the parallel tuplesort
infrastructure for that? It already has similar features in place and
improves code commonality.
Maybe. I wasn't that familiar with what parallel tuplesort can and can't
do, and the little I knew I managed to forget since I wrote this patch.
Which similar features do you have in mind?
The workers are producing the results in "start_block" order, so if they
pass that to the leader, it probably can do the usual merge sort.
For 0002 it's a bit more complicated, because with a single parallel
scan brinbuildCallbackParallel can't decide if a range is assigned to a
different worker or empty. And we want to generate summaries for empty
ranges in the index. We could either skip such range during index build,
and then add empty summaries in _brin_end_parallel (if needed), or add
them and then merge them using "union".I just realized there's a third option to do this - we could just do
regular parallel scan (with no particular regard to pagesPerRange), and
then do "union" when merging results from workers. It doesn't require
the sequence of TID scans, and the union would also handle the empty
ranges. The per-worker results might be much larger, though, because
each worker might produce up to the "full" BRIN index.Would it be too much effort to add a 'min_chunk_size' argument to
table_beginscan_parallel (or ParallelTableScanDesc) that defines the
minimum granularity of block ranges to be assigned to each process? I
think that would be the most elegant solution that would require
relatively little effort: table_block_parallelscan_nextpage already
does parallel management of multiple chunk sizes, and I think this
modification would fit quite well in that code.
I'm confused. Isn't that pretty much exactly what 0002 does? I mean,
that passes pagesPerRange to table_parallelscan_initialize(), so that
each pagesPerRange is assigned to a single worker.
The trouble I described above is that the scan returns tuples, and the
consumer has no idea what was the chunk size or how many other workers
are there. Imagine you get a tuple from block 1, and then a tuple from
block 1000. Does that mean that the blocks in between are empty or that
they were processed by some other worker?
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On Wed, 5 Jul 2023 at 00:08, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
On 7/4/23 23:53, Matthias van de Meent wrote:
On Thu, 8 Jun 2023 at 14:55, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
Here's a WIP patch allowing parallel CREATE INDEX for BRIN indexes. The
infrastructure (starting workers etc.) is "inspired" by the BTREE code
(i.e. copied from that and massaged a bit to call brin stuff).Nice work.
In both cases _brin_end_parallel then reads the summaries from worker
files, and adds them into the index. In 0001 this is fairly simple,
although we could do one more improvement and sort the ranges by range
start to make the index nicer (and possibly a bit more efficient). This
should be simple, because the per-worker results are already sorted like
that (so a merge sort in _brin_end_parallel would be enough).I see that you manually built the passing and sorting of tuples
between workers, but can't we use the parallel tuplesort
infrastructure for that? It already has similar features in place and
improves code commonality.Maybe. I wasn't that familiar with what parallel tuplesort can and can't
do, and the little I knew I managed to forget since I wrote this patch.
Which similar features do you have in mind?The workers are producing the results in "start_block" order, so if they
pass that to the leader, it probably can do the usual merge sort.For 0002 it's a bit more complicated, because with a single parallel
scan brinbuildCallbackParallel can't decide if a range is assigned to a
different worker or empty. And we want to generate summaries for empty
ranges in the index. We could either skip such range during index build,
and then add empty summaries in _brin_end_parallel (if needed), or add
them and then merge them using "union".I just realized there's a third option to do this - we could just do
regular parallel scan (with no particular regard to pagesPerRange), and
then do "union" when merging results from workers. It doesn't require
the sequence of TID scans, and the union would also handle the empty
ranges. The per-worker results might be much larger, though, because
each worker might produce up to the "full" BRIN index.Would it be too much effort to add a 'min_chunk_size' argument to
table_beginscan_parallel (or ParallelTableScanDesc) that defines the
minimum granularity of block ranges to be assigned to each process? I
think that would be the most elegant solution that would require
relatively little effort: table_block_parallelscan_nextpage already
does parallel management of multiple chunk sizes, and I think this
modification would fit quite well in that code.I'm confused. Isn't that pretty much exactly what 0002 does? I mean,
that passes pagesPerRange to table_parallelscan_initialize(), so that
each pagesPerRange is assigned to a single worker.
Huh, I overlooked that one... Sorry for that.
The trouble I described above is that the scan returns tuples, and the
consumer has no idea what was the chunk size or how many other workers
are there. Imagine you get a tuple from block 1, and then a tuple from
block 1000. Does that mean that the blocks in between are empty or that
they were processed by some other worker?
If the unit of work for parallel table scans is the index's
pages_per_range, then I think we can just fill in expected-but-missing
ranges as 'empty' in the parallel leader during index loading, like
the first of the two solutions you proposed.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech/)
On 7/5/23 10:44, Matthias van de Meent wrote:
On Wed, 5 Jul 2023 at 00:08, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
On 7/4/23 23:53, Matthias van de Meent wrote:
On Thu, 8 Jun 2023 at 14:55, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
Here's a WIP patch allowing parallel CREATE INDEX for BRIN indexes. The
infrastructure (starting workers etc.) is "inspired" by the BTREE code
(i.e. copied from that and massaged a bit to call brin stuff).Nice work.
In both cases _brin_end_parallel then reads the summaries from worker
files, and adds them into the index. In 0001 this is fairly simple,
although we could do one more improvement and sort the ranges by range
start to make the index nicer (and possibly a bit more efficient). This
should be simple, because the per-worker results are already sorted like
that (so a merge sort in _brin_end_parallel would be enough).I see that you manually built the passing and sorting of tuples
between workers, but can't we use the parallel tuplesort
infrastructure for that? It already has similar features in place and
improves code commonality.Maybe. I wasn't that familiar with what parallel tuplesort can and can't
do, and the little I knew I managed to forget since I wrote this patch.
Which similar features do you have in mind?The workers are producing the results in "start_block" order, so if they
pass that to the leader, it probably can do the usual merge sort.For 0002 it's a bit more complicated, because with a single parallel
scan brinbuildCallbackParallel can't decide if a range is assigned to a
different worker or empty. And we want to generate summaries for empty
ranges in the index. We could either skip such range during index build,
and then add empty summaries in _brin_end_parallel (if needed), or add
them and then merge them using "union".I just realized there's a third option to do this - we could just do
regular parallel scan (with no particular regard to pagesPerRange), and
then do "union" when merging results from workers. It doesn't require
the sequence of TID scans, and the union would also handle the empty
ranges. The per-worker results might be much larger, though, because
each worker might produce up to the "full" BRIN index.Would it be too much effort to add a 'min_chunk_size' argument to
table_beginscan_parallel (or ParallelTableScanDesc) that defines the
minimum granularity of block ranges to be assigned to each process? I
think that would be the most elegant solution that would require
relatively little effort: table_block_parallelscan_nextpage already
does parallel management of multiple chunk sizes, and I think this
modification would fit quite well in that code.I'm confused. Isn't that pretty much exactly what 0002 does? I mean,
that passes pagesPerRange to table_parallelscan_initialize(), so that
each pagesPerRange is assigned to a single worker.Huh, I overlooked that one... Sorry for that.
The trouble I described above is that the scan returns tuples, and the
consumer has no idea what was the chunk size or how many other workers
are there. Imagine you get a tuple from block 1, and then a tuple from
block 1000. Does that mean that the blocks in between are empty or that
they were processed by some other worker?If the unit of work for parallel table scans is the index's
pages_per_range, then I think we can just fill in expected-but-missing
ranges as 'empty' in the parallel leader during index loading, like
the first of the two solutions you proposed.
Right, I think that's the right solution.
Or rather the only solution, because the other idea (generating the
empty ranges in workers) relies on the workers knowing when to generate
that. But I don't think the workers have the necessary information.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On Wed, 5 Jul 2023 at 00:08, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
On 7/4/23 23:53, Matthias van de Meent wrote:
On Thu, 8 Jun 2023 at 14:55, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
Here's a WIP patch allowing parallel CREATE INDEX for BRIN indexes. The
infrastructure (starting workers etc.) is "inspired" by the BTREE code
(i.e. copied from that and massaged a bit to call brin stuff).Nice work.
In both cases _brin_end_parallel then reads the summaries from worker
files, and adds them into the index. In 0001 this is fairly simple,
although we could do one more improvement and sort the ranges by range
start to make the index nicer (and possibly a bit more efficient). This
should be simple, because the per-worker results are already sorted like
that (so a merge sort in _brin_end_parallel would be enough).I see that you manually built the passing and sorting of tuples
between workers, but can't we use the parallel tuplesort
infrastructure for that? It already has similar features in place and
improves code commonality.Maybe. I wasn't that familiar with what parallel tuplesort can and can't
do, and the little I knew I managed to forget since I wrote this patch.
Which similar features do you have in mind?
I was referring to the feature that is "emitting a single sorted run
of tuples at the leader backend based on data gathered in parallel
worker backends". It manages the sort state, on-disk runs etc. so that
you don't have to manage that yourself.
Adding a new storage format for what is effectively a logical tape
(logtape.{c,h}) and manually merging it seems like a lot of changes if
that functionality is readily available, standardized and optimized in
sortsupport; and adds an additional place to manually go through for
disk-related changes like TDE.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech/)
On 7/5/23 16:33, Matthias van de Meent wrote:
...
Maybe. I wasn't that familiar with what parallel tuplesort can and can't
do, and the little I knew I managed to forget since I wrote this patch.
Which similar features do you have in mind?I was referring to the feature that is "emitting a single sorted run
of tuples at the leader backend based on data gathered in parallel
worker backends". It manages the sort state, on-disk runs etc. so that
you don't have to manage that yourself.Adding a new storage format for what is effectively a logical tape
(logtape.{c,h}) and manually merging it seems like a lot of changes if
that functionality is readily available, standardized and optimized in
sortsupport; and adds an additional place to manually go through for
disk-related changes like TDE.
Here's a new version of the patch, with three main changes:
1) Adoption of the parallel scan approach, instead of the homegrown
solution with a sequence of TID scans. This is mostly what the 0002
patch did, except for fixing a bug - parallel scan has a "rampdown"
close to the end, and this needs to consider the chunk size too.
2) Switches to the parallel tuplesort, as proposed. This turned out to
be easier than I expected - most of the work was in adding methods to
tuplesortvariants.c to allow reading/writing BrinTuple items. The main
limitation is that we need to pass around the length of the tuple
(AFAICS it's not in the BrinTuple itself). I'm not entirely sure about
the memory management aspect of this, and maybe there's a more elegant
solution.
Overall it seems to work - the brin.c code is heavily based on how
nbtsearch.c does parallel builds for btree, so hopefully it's fine. At
some point I got a bit confused about which spool to create/use, but it
seems to work.
3) Handling of empty ranges - I ended up ignoring empty ranges in
workers (i.e. those are not written to the tuplesort), and instead the
leader fills them in when reading data from the shared tuplesort.
One thing I was wondering about is whether it might be better to allow
the workers to process overlapping ranges, and then let the leader to
merge the summaries. That would mean we might not need the tableam.c
changes at all, but the leader would need to do more work (although the
BRIN indexes tend to be fairly small). The main reason that got me
thinking about this is that we have pretty much no tests for the union
procedures, because triggering that is really difficult. But for
parallel index builds that'd be much more common.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
0001-parallel-CREATE-INDEX-for-BRIN-20230706.patchtext/x-patch; charset=UTF-8; name=0001-parallel-CREATE-INDEX-for-BRIN-20230706.patchDownload
From 6485c26d0546ec35d81bad33f3893ec5ee421f87 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Sun, 26 Mar 2023 00:44:01 +0100
Subject: [PATCH] parallel CREATE INDEX for BRIN
---
src/backend/access/brin/brin.c | 781 ++++++++++++++++++++-
src/backend/access/nbtree/nbtsort.c | 2 +-
src/backend/access/table/tableam.c | 49 +-
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 3 +-
src/backend/executor/nodeSeqscan.c | 3 +-
src/backend/utils/sort/tuplesort.c | 3 +
src/backend/utils/sort/tuplesortvariants.c | 211 ++++++
src/include/access/brin.h | 3 +
src/include/access/relscan.h | 1 +
src/include/access/tableam.h | 9 +-
src/include/utils/tuplesort.h | 11 +
12 files changed, 1065 insertions(+), 15 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 3c6a956eaa..9e72c54572 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -33,6 +33,7 @@
#include "postmaster/autovacuum.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -40,7 +41,118 @@
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+#include "utils/tuplesort.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xA000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xA000000000000005)
+
+/*
+ * Status record for spooling/sorting phase.
+ */
+typedef struct BrinSpool
+{
+ Tuplesortstate *sortstate; /* state data for tuplesort.c */
+ Relation heap;
+ Relation index;
+} BrinSpool;
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist for
+ * the benefit of worker processes that need to create state corresponding
+ * to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+ int scantuplesortstates;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN 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 the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
+} BrinShared;
+
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* 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).
+ *
+ * brinshared is the shared state for entire build. sharedsort is the
+ * shared, tuplesort-managed state passed to each process tuplesort.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is required.
+ */
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -50,12 +162,23 @@ typedef struct BrinBuildState
{
Relation bs_irel;
int bs_numtuples;
+ int bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BrinSpool *bs_spool;
+ BrinSpool *bs_spool_out;
} BrinBuildState;
/*
@@ -76,6 +199,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -83,6 +207,20 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, Datum *values, bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinSpool *brinspool,
+ BrinShared *brinshared,
+ Sharedsort *sharedsort,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -822,6 +960,67 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a shared tuplestore file, and leave the insertion up to the leader (which may
+ * reorder them a bit etc.). The callback also does not generate empty ranges,
+ * those may be added by the leader when merging results from workers.
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; make sure to insert index
+ * tuples for those too.
+ */
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the tuplesort */
+ form_and_spill_tuple(state);
+
+ /*
+ * set state to correspond to the next range
+ *
+ * XXX This has the issue that it skips ranges summarized by other
+ * workers, but it also skips empty ranges that should have been
+ * summarized. We'd need to either make the workers aware which
+ * chunk they are actually processing (which is currently known
+ * only in the ParallelBlockTableScan bit). Or we could ignore it
+ * here, and then decide it while "merging" results from workers
+ * (if there's no entry for the range, it had to be empty so we
+ * just add an empty one).
+ */
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ state->bs_currRangeStart += state->bs_pagesPerRange;
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -883,18 +1082,93 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
revmap = brinRevmapInitialize(index, &pagesPerRange, NULL);
state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool->heap = heap;
+ state->bs_spool->index = index;
+
+ /*
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
+ */
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
+
/*
* Now scan the relation. No syncscan allowed here because we want the
* heap blocks in physical order.
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
- form_and_insert_tuple(state);
+ /*
+ * If parallel build requested and at least one worker process was
+ * successfully launched, set up coordination state
+ */
+ if (state->bs_leader)
+ {
+ SortCoordinate coordinate;
+
+ coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
+ coordinate->isWorker = false;
+ coordinate->nParticipants =
+ state->bs_leader->nparticipanttuplesorts;
+ coordinate->sharedsort = state->bs_leader->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.
+ */
+ state->bs_spool_out = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool_out->heap = heap;
+ state->bs_spool_out->index = index;
+
+ state->bs_spool_out->sortstate =
+ tuplesort_begin_index_brin(heap, index,
+ maintenance_work_mem, coordinate,
+ TUPLESORT_NONE);
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read all
+ * tuples from the shared tuplesort and insert them into the index.
+ */
+ _brin_end_parallel(state->bs_leader, state);
+ }
+ else /* no parallel index build, just do the usual thing */
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
+ form_and_insert_tuple(state);
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
+ }
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1299,12 +1573,16 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
+ state->bs_spool = NULL;
return state;
}
@@ -1597,6 +1875,32 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a (shared) tuplestore (the leader will insert it
+ * into the index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ /* don't insert empty tuples in parallel build */
+ if (state->bs_dtuple->bt_empty_range)
+ return;
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ /* write the BRIN tuple to the tuplesort */
+ tuplesort_putbrintuple(state->bs_spool->sortstate, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -1916,3 +2220,472 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ int scantuplesortstates;
+ Snapshot snapshot;
+ Size estbrinshared;
+ Size estsort;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ request);
+
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+ estsort = tuplesort_estimate_shared(scantuplesortstates);
+ shm_toc_estimate_chunk(&pcxt->estimator, estsort);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->scantuplesortstates = scantuplesortstates;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ snapshot, brinshared->pagesPerRange);
+
+ /*
+ * 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);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipanttuplesorts = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipanttuplesorts++;
+ brinleader->brinshared = brinshared;
+ brinleader->sharedsort = sharedsort;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinTuple *btup;
+ Size tuplen;
+ BrinShared *brinshared = brinleader->brinshared;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ tuplesort_performsort(state->bs_spool_out->sortstate);
+
+ /*
+ * Read the BRIN tuples from the shared tuplesort, sorted by block number.
+ * That probably gives us an index that is cheaper to scan, thanks to mostly
+ * getting data from the same index page as before.
+ *
+ * FIXME This probably needs some memory management fixes - we're reading
+ * tuples from the tuplesort, we're allocating an emty tuple, and so on.
+ * Probably better to release this memory.
+ *
+ * XXX We can't quite free the BrinTuple, though, because that's a field
+ * in BrinSortTuple.
+ */
+ while ((btup = tuplesort_getbrintuple(state->bs_spool_out->sortstate, &tuplen, true)) != NULL)
+ {
+ /*
+ * We should not get two summaries for the same range. The workers
+ * are producing ranges for non-overlapping sections of the table.
+ */
+ Assert(btup->bt_blkno != prevblkno);
+
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, btup->bt_blkno, btup, tuplen);
+
+ prevblkno = btup->bt_blkno;
+ }
+
+ tuplesort_end(state->bs_spool_out->sortstate);
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /* Allocate memory and initialize private spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = buildstate->bs_spool->heap;
+ buildstate->bs_spool->index = buildstate->bs_spool->index;
+
+ /*
+ * 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 / brinleader->nparticipanttuplesorts;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool, brinleader->brinshared,
+ brinleader->sharedsort, heap, index, sortmem, true);
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
+ BrinShared *brinshared, Sharedsort *sharedsort,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ SortCoordinate coordinate;
+ TableScanDesc 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 */
+ brinspool->sortstate = tuplesort_begin_index_brin(brinspool->heap,
+ brinspool->index,
+ sortmem, coordinate,
+ TUPLESORT_NONE);
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* insert the last item */
+ form_and_spill_tuple(state);
+
+ /* sort the BRIN ranges built by this worker */
+ tuplesort_performsort(brinspool->sortstate);
+
+ state->bs_reltuples += reltuples;
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+
+ tuplesort_end(brinspool->sortstate);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL, brinshared->pagesPerRange);
+
+ /* Initialize worker's own spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = heapRel;
+ buildstate->bs_spool->index = indexRel;
+
+ /* Look up shared state private to tuplesort.c */
+ sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
+ tuplesort_attach_shared(sharedsort, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /*
+ * 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 / brinshared->scantuplesortstates;
+
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool,
+ brinshared, sharedsort,
+ heapRel, indexRel, sortmem, false);
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/nbtree/nbtsort.c b/src/backend/access/nbtree/nbtsort.c
index c2665fce41..6241baeea8 100644
--- a/src/backend/access/nbtree/nbtsort.c
+++ b/src/backend/access/nbtree/nbtsort.c
@@ -1575,7 +1575,7 @@ _bt_begin_parallel(BTBuildState *buildstate, bool isconcurrent, int request)
btshared->brokenhotchain = false;
table_parallelscan_initialize(btspool->heap,
ParallelTableScanFromBTShared(btshared),
- snapshot);
+ snapshot, InvalidBlockNumber);
/*
* Store shared tuplesort-private state, for which we reserved space.
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index c6bdb7e1c6..4af0d433e9 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -153,9 +153,9 @@ table_parallelscan_estimate(Relation rel, Snapshot snapshot)
void
table_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan,
- Snapshot snapshot)
+ Snapshot snapshot, BlockNumber chunk_factor)
{
- Size snapshot_off = rel->rd_tableam->parallelscan_initialize(rel, pscan);
+ Size snapshot_off = rel->rd_tableam->parallelscan_initialize(rel, pscan, chunk_factor);
pscan->phs_snapshot_off = snapshot_off;
@@ -395,16 +395,21 @@ table_block_parallelscan_estimate(Relation rel)
}
Size
-table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan)
+table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan, BlockNumber chunk_factor)
{
ParallelBlockTableScanDesc bpscan = (ParallelBlockTableScanDesc) pscan;
bpscan->base.phs_relid = RelationGetRelid(rel);
bpscan->phs_nblocks = RelationGetNumberOfBlocks(rel);
- /* compare phs_syncscan initialization to similar logic in initscan */
+ bpscan->phs_chunk_factor = chunk_factor;
+ /* compare phs_syncscan initialization to similar logic in initscan
+ *
+ * Disable sync scans if the chunk factor is set (valid block number).
+ */
bpscan->base.phs_syncscan = synchronize_seqscans &&
!RelationUsesLocalBuffers(rel) &&
- bpscan->phs_nblocks > NBuffers / 4;
+ (bpscan->phs_nblocks > NBuffers / 4) &&
+ !BlockNumberIsValid(bpscan->phs_chunk_factor);
SpinLockInit(&bpscan->phs_mutex);
bpscan->phs_startblock = InvalidBlockNumber;
pg_atomic_init_u64(&bpscan->phs_nallocated, 0);
@@ -459,6 +464,25 @@ table_block_parallelscan_startblock_init(Relation rel,
pbscanwork->phsw_chunk_size = Min(pbscanwork->phsw_chunk_size,
PARALLEL_SEQSCAN_MAX_CHUNK_SIZE);
+ /*
+ * If the chunk size factor is set, we need to make sure the chunk size is
+ * a multiple of that value. We round the chunk size to the nearest chunk
+ * factor multiple, at least one chunk_factor.
+ *
+ * XXX Note this may override PARALLEL_SEQSCAN_MAX_CHUNK_SIZE, in case the
+ * chunk factor (e.g. BRIN pages_per_range) is larger.
+ */
+ if (pbscan->phs_chunk_factor != InvalidBlockNumber)
+ {
+ /* nearest (smaller) multiple of chunk_factor */
+ pbscanwork->phsw_chunk_size
+ = pbscan->phs_chunk_factor * (pbscanwork->phsw_chunk_size / pbscan->phs_chunk_factor);
+
+ /* but at least one chunk_factor */
+ pbscanwork->phsw_chunk_size = Max(pbscanwork->phsw_chunk_size,
+ pbscan->phs_chunk_factor);
+ }
+
retry:
/* Grab the spinlock. */
SpinLockAcquire(&pbscan->phs_mutex);
@@ -575,6 +599,21 @@ table_block_parallelscan_nextpage(Relation rel,
(pbscanwork->phsw_chunk_size * PARALLEL_SEQSCAN_RAMPDOWN_CHUNKS))
pbscanwork->phsw_chunk_size >>= 1;
+ /*
+ * We need to make sure the new chunk_size is still a suitable multiple
+ * of chunk_factor.
+ */
+ if (pbscan->phs_chunk_factor != InvalidBlockNumber)
+ {
+ /* nearest (smaller) multiple of chunk_factor */
+ pbscanwork->phsw_chunk_size
+ = pbscan->phs_chunk_factor * (pbscanwork->phsw_chunk_size / pbscan->phs_chunk_factor);
+
+ /* but at least one chunk_factor */
+ pbscanwork->phsw_chunk_size = Max(pbscanwork->phsw_chunk_size,
+ pbscan->phs_chunk_factor);
+ }
+
nallocated = pbscanwork->phsw_nallocated =
pg_atomic_fetch_add_u64(&pbscan->phs_nallocated,
pbscanwork->phsw_chunk_size);
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 2b8bc2f58d..72086212f4 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -144,6 +145,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 352e43d0e6..5bdd025bb2 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2980,7 +2980,8 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ (indexRelation->rd_rel->relam == BTREE_AM_OID ||
+ indexRelation->rd_rel->relam == BRIN_AM_OID))
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 4da0f28f7b..e017f05f78 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -274,7 +274,8 @@ ExecSeqScanInitializeDSM(SeqScanState *node,
pscan = shm_toc_allocate(pcxt->toc, node->pscan_len);
table_parallelscan_initialize(node->ss.ss_currentRelation,
pscan,
- estate->es_snapshot);
+ estate->es_snapshot,
+ InvalidBlockNumber);
shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pscan);
node->ss.ss_currentScanDesc =
table_beginscan_parallel(node->ss.ss_currentRelation, pscan);
diff --git a/src/backend/utils/sort/tuplesort.c b/src/backend/utils/sort/tuplesort.c
index e5a4e5b371..7b348195be 100644
--- a/src/backend/utils/sort/tuplesort.c
+++ b/src/backend/utils/sort/tuplesort.c
@@ -1331,6 +1331,7 @@ tuplesort_puttuple_common(Tuplesortstate *state, SortTuple *tuple, bool useAbbre
break;
default:
+ Assert(false);
elog(ERROR, "invalid tuplesort state");
break;
}
@@ -1465,6 +1466,7 @@ tuplesort_performsort(Tuplesortstate *state)
break;
default:
+ Assert(false);
elog(ERROR, "invalid tuplesort state");
break;
}
@@ -1721,6 +1723,7 @@ tuplesort_gettuple_common(Tuplesortstate *state, bool forward,
return false;
default:
+ Assert(false);
elog(ERROR, "invalid tuplesort state");
return false; /* keep compiler quiet */
}
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index eb6cfcfd00..719ec57123 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -19,6 +19,7 @@
#include "postgres.h"
+#include "access/brin_tuple.h"
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -43,6 +44,8 @@ static void removeabbrev_cluster(Tuplesortstate *state, SortTuple *stups,
int count);
static void removeabbrev_index(Tuplesortstate *state, SortTuple *stups,
int count);
+static void removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups,
+ int count);
static void removeabbrev_datum(Tuplesortstate *state, SortTuple *stups,
int count);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
@@ -61,10 +64,16 @@ static int comparetup_index_btree(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
+static int comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state);
static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len);
+static void writetup_index_brin(Tuplesortstate *state, LogicalTape *tape,
+ SortTuple *stup);
+static void readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static void writetup_datum(Tuplesortstate *state, LogicalTape *tape,
@@ -118,6 +127,16 @@ typedef struct
uint32 max_buckets;
} TuplesortIndexHashArg;
+/*
+ * Data struture pointed by "TuplesortPublic.arg" for the index_brin subcase.
+ */
+typedef struct
+{
+ TuplesortIndexArg index;
+
+ /* XXX do we need something here? */
+} TuplesortIndexBrinArg;
+
/*
* Data struture pointed by "TuplesortPublic.arg" for the Datum case.
* Set by tuplesort_begin_datum and used only by the DatumTuple routines.
@@ -130,6 +149,22 @@ typedef struct
int datumTypeLen;
} TuplesortDatumArg;
+/*
+ * Computing BrinTuple size with only the tuple is difficult, so we want to track
+ * the length for r referenced by SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by length. We only write the
+ * BrinTuple to the logtapes, though.
+ */
+typedef struct BrinSortTuple
+{
+ Size tuplen;
+ BrinTuple tuple;
+} BrinSortTuple;
+
+/* Size of the BrinSortTuple, given length of the BrinTuple. */
+#define BRINSORTTUPLE_SIZE(len) (offsetof(BrinSortTuple, tuple) + (len))
+
+
Tuplesortstate *
tuplesort_begin_heap(TupleDesc tupDesc,
int nkeys, AttrNumber *attNums,
@@ -512,6 +547,47 @@ tuplesort_begin_index_gist(Relation heapRel,
return state;
}
+Tuplesortstate *
+tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem,
+ SortCoordinate coordinate,
+ int sortopt)
+{
+ Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
+ sortopt);
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext;
+ TuplesortIndexBrinArg *arg;
+
+ oldcontext = MemoryContextSwitchTo(base->maincontext);
+ arg = (TuplesortIndexBrinArg *) palloc(sizeof(TuplesortIndexBrinArg));
+
+#ifdef TRACE_SORT
+ if (trace_sort)
+ elog(LOG,
+ "begin index sort: workMem = %d, randomAccess = %c",
+ workMem,
+ sortopt & TUPLESORT_RANDOMACCESS ? 't' : 'f');
+#endif
+
+ base->nKeys = 1; /* Only one sort column, the block number */
+
+ base->removeabbrev = removeabbrev_index_brin;
+ base->comparetup = comparetup_index_brin;
+ base->writetup = writetup_index_brin;
+ base->readtup = readtup_index_brin;
+ base->haveDatum1 = true;
+ base->arg = arg;
+
+ arg->index.heapRel = heapRel;
+ arg->index.indexRel = indexRel;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return state;
+}
+
Tuplesortstate *
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag, int workMem,
@@ -691,6 +767,35 @@ tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
!stup.isnull1);
}
+/*
+ * Collect one BRIN tuple while collecting input data for sort.
+ */
+void
+tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
+{
+ SortTuple stup;
+ BrinSortTuple *bstup;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->tuplecontext);
+
+ /* allocate space for the whole BRIN sort tuple */
+ bstup = palloc(BRINSORTTUPLE_SIZE(size));
+ stup.tuple = bstup;
+
+ bstup->tuplen = size;
+ memcpy(&bstup->tuple, tuple, size);
+
+ stup.datum1 = tuple->bt_blkno;
+ stup.isnull1 = false;
+
+ tuplesort_puttuple_common(state, &stup,
+ base->sortKeys &&
+ base->sortKeys->abbrev_converter &&
+ !stup.isnull1);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
/*
* Accept one Datum while collecting input data for sort.
*
@@ -834,6 +939,35 @@ tuplesort_getindextuple(Tuplesortstate *state, bool forward)
return (IndexTuple) stup.tuple;
}
+/*
+ * Fetch the next BRIN tuple in either forward or back direction.
+ * Returns NULL if no more tuples. Returned tuple belongs to tuplesort memory
+ * context, and must not be freed by caller. Caller may not rely on tuple
+ * remaining valid after any further manipulation of tuplesort.
+ */
+BrinTuple *
+tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
+ SortTuple stup;
+ BrinSortTuple *btup;
+
+ if (!tuplesort_gettuple_common(state, forward, &stup))
+ stup.tuple = NULL;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!stup.tuple)
+ return NULL;
+
+ btup = (BrinSortTuple *) stup.tuple;
+
+ *len = btup->tuplen;
+
+ return &btup->tuple;
+}
+
/*
* Fetch the next Datum in either forward or back direction.
* Returns false if no more datums.
@@ -1501,6 +1635,83 @@ readtup_index(Tuplesortstate *state, SortTuple *stup,
&stup->isnull1);
}
+/*
+ * Routines specialized for BrinTuple case
+ */
+
+static void
+removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
+{
+ int i;
+
+ for (i = 0; i < count; i++)
+ {
+ BrinSortTuple *tuple;
+
+ tuple = stups[i].tuple;
+ stups[i].datum1 = tuple->tuple.bt_blkno;
+ }
+}
+
+static int
+comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+{
+ BrinTuple *tuple1;
+ BrinTuple *tuple2;
+
+ tuple1 = &((BrinSortTuple *) a)->tuple;
+ tuple2 = &((BrinSortTuple *) b)->tuple;
+
+ if (tuple1->bt_blkno > tuple2->bt_blkno)
+ return 1;
+ else if (tuple1->bt_blkno < tuple2->bt_blkno)
+ return -1;
+
+ /* silence compilers */
+ return 0;
+}
+
+static void
+writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
+
+ tuplen = tuplen + sizeof(tuplen);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->tuple, tuple->tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+}
+
+static void
+readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ unsigned int tuplen = len - sizeof(unsigned int);
+
+ /*
+ * Allocate space for the BRIN sort tuple, which is BrinTuple with an
+ * extra length field.
+ */
+ BrinSortTuple *tuple
+ = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
+
+ tuple->tuplen = tuplen;
+
+ LogicalTapeReadExact(tape, &tuple->tuple, tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
+ stup->tuple = (void *) tuple;
+
+ /* set up first-column key value, which is block number */
+ stup->datum1 = tuple->tuple.bt_blkno;
+}
+
/*
* Routines specialized for DatumTuple case
*/
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d5..3451ecb211 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index d03360eac0..72a20d882f 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -79,6 +79,7 @@ typedef struct ParallelBlockTableScanDescData
BlockNumber phs_nblocks; /* # blocks in relation at start of scan */
slock_t phs_mutex; /* mutual exclusion for setting startblock */
BlockNumber phs_startblock; /* starting block number */
+ BlockNumber phs_chunk_factor; /* chunks need to be a multiple of this */
pg_atomic_uint64 phs_nallocated; /* number of blocks allocated to
* workers so far. */
} ParallelBlockTableScanDescData;
diff --git a/src/include/access/tableam.h b/src/include/access/tableam.h
index 230bc39cc0..82297546e2 100644
--- a/src/include/access/tableam.h
+++ b/src/include/access/tableam.h
@@ -390,7 +390,8 @@ typedef struct TableAmRoutine
* relation.
*/
Size (*parallelscan_initialize) (Relation rel,
- ParallelTableScanDesc pscan);
+ ParallelTableScanDesc pscan,
+ BlockNumber chunk_factor);
/*
* Reinitialize `pscan` for a new scan. `rel` will be the same relation as
@@ -1148,7 +1149,8 @@ extern Size table_parallelscan_estimate(Relation rel, Snapshot snapshot);
*/
extern void table_parallelscan_initialize(Relation rel,
ParallelTableScanDesc pscan,
- Snapshot snapshot);
+ Snapshot snapshot,
+ BlockNumber chunk_factor);
/*
* Begin a parallel scan. `pscan` needs to have been initialized with
@@ -2064,7 +2066,8 @@ extern void simple_table_tuple_update(Relation rel, ItemPointer otid,
extern Size table_block_parallelscan_estimate(Relation rel);
extern Size table_block_parallelscan_initialize(Relation rel,
- ParallelTableScanDesc pscan);
+ ParallelTableScanDesc pscan,
+ BlockNumber chunk_factor);
extern void table_block_parallelscan_reinitialize(Relation rel,
ParallelTableScanDesc pscan);
extern BlockNumber table_block_parallelscan_nextpage(Relation rel,
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index af057b6358..d6aa629233 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -21,6 +21,7 @@
#ifndef TUPLESORT_H
#define TUPLESORT_H
+#include "access/brin_tuple.h"
#include "access/itup.h"
#include "executor/tuptable.h"
#include "storage/dsm.h"
@@ -275,6 +276,9 @@ typedef struct
* The "index_hash" API is similar to index_btree, but the tuples are
* actually sorted by their hash codes not the raw data.
*
+ * The "index_brin" API is similar to index_btree, but the tuples are
+ * BrinTuple and are sorted by their block number not the raw data.
+ *
* Parallel sort callers are required to coordinate multiple tuplesort states
* in a leader process and one or more worker processes. The leader process
* must launch workers, and have each perform an independent "partial"
@@ -419,6 +423,10 @@ extern Tuplesortstate *tuplesort_begin_index_gist(Relation heapRel,
Relation indexRel,
int workMem, SortCoordinate coordinate,
int sortopt);
+extern Tuplesortstate *tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem, SortCoordinate coordinate,
+ int sortopt);
extern Tuplesortstate *tuplesort_begin_datum(Oid datumType,
Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag,
@@ -431,6 +439,7 @@ extern void tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup);
extern void tuplesort_putindextuplevalues(Tuplesortstate *state,
Relation rel, ItemPointer self,
Datum *values, bool *isnull);
+extern void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tup, Size len);
extern void tuplesort_putdatum(Tuplesortstate *state, Datum val,
bool isNull);
@@ -438,6 +447,8 @@ extern bool tuplesort_gettupleslot(Tuplesortstate *state, bool forward,
bool copy, TupleTableSlot *slot, Datum *abbrev);
extern HeapTuple tuplesort_getheaptuple(Tuplesortstate *state, bool forward);
extern IndexTuple tuplesort_getindextuple(Tuplesortstate *state, bool forward);
+extern BrinTuple *tuplesort_getbrintuple(Tuplesortstate *state, Size *len,
+ bool forward);
extern bool tuplesort_getdatum(Tuplesortstate *state, bool forward, bool copy,
Datum *val, bool *isNull, Datum *abbrev);
--
2.41.0
On Thu, 6 Jul 2023 at 16:13, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
On 7/5/23 16:33, Matthias van de Meent wrote:
...
Maybe. I wasn't that familiar with what parallel tuplesort can and can't
do, and the little I knew I managed to forget since I wrote this patch.
Which similar features do you have in mind?I was referring to the feature that is "emitting a single sorted run
of tuples at the leader backend based on data gathered in parallel
worker backends". It manages the sort state, on-disk runs etc. so that
you don't have to manage that yourself.Adding a new storage format for what is effectively a logical tape
(logtape.{c,h}) and manually merging it seems like a lot of changes if
that functionality is readily available, standardized and optimized in
sortsupport; and adds an additional place to manually go through for
disk-related changes like TDE.Here's a new version of the patch, with three main changes:
Thanks! I've done a review on the patch, and most looks good. Some
places need cleanup and polish, some others more documentations, and
there are some other issues, but so far it's looking OK.
One thing I was wondering about is whether it might be better to allow
the workers to process overlapping ranges, and then let the leader to
merge the summaries. That would mean we might not need the tableam.c
changes at all, but the leader would need to do more work (although the
BRIN indexes tend to be fairly small). The main reason that got me
thinking about this is that we have pretty much no tests for the union
procedures, because triggering that is really difficult. But for
parallel index builds that'd be much more common.
Hmm, that's a good point. I don't mind either way, but it would add
overhead in the leader to do all of that merging - especially when you
configure pages_per_range > PARALLEL_SEQSCAN_MAX_CHUNK_SIZE as we'd
need to merge up to parallel_workers tuples. That could be a
significant overhead.
... thinks a bit.
Hmm, but with the current P_S_M_C_S of 8192 blocks that's quite
unlikely to be a serious problem - the per-backend IO saved of such
large ranges on a single backend has presumably much more impact than
the merging of n_parallel_tasks max-sized brin tuples. So, seems fine
with me.
Review follows below.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech/)
-----------
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
+ BrinShared *brinshared;
Needs some indentation fixes.
+ int bs_reltuples; [...] + state->bs_reltuples += reltuples;
My IDE warns me that reltuples is a double. Looking deeper into the
value, it contains the number of live tuples in the table, so this
conversion may not result in a meaningful value for tables with >=2^31
live tuples. Tables > 56GB could begin to get affected by this.
+ int bs_worker_id;
This variable seems to be unused.
+ BrinSpool *bs_spool;
+ BrinSpool *bs_spool_out;
Are both used? If so, could you add comments why we have two spools
here, instead of only one?
+/* + * A version of the callback, used by parallel index builds. The main difference + * is that instead of writing the BRIN tuples into the index, we write them into + * a shared tuplestore file, and leave the insertion up to the leader (which may
+ ... shared tuplesort, and ...
brinbuildCallbackParallel(...)
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
shouldn't this be an 'if' now?
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1) + state->bs_currRangeStart += state->bs_pagesPerRange;
Is there a reason why you went with iterative addition instead of a
single divide-and-multiply like the following?:
+ state->bs_currRangeStart += state->bs_pagesPerRange *
((state->bs_currRangeStart - thisblock) / state->bs_pagesPerRange);
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c [...] -table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan) +table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan, BlockNumber chunk_factor) [...] - /* compare phs_syncscan initialization to similar logic in initscan */ + bpscan->phs_chunk_factor = chunk_factor; + /* compare phs_syncscan initialization to similar logic in initscan + * + * Disable sync scans if the chunk factor is set (valid block number). + */
I think this needs some pgindent or other style work, both on comment
style and line lengths
diff --git a/src/backend/utils/sort/tuplesort.c b/src/backend/utils/sort/tuplesort.c [...] + Assert(false); (x3)
I think these can be cleaned up, right?
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c [...] + * Computing BrinTuple size with only the tuple is difficult, so we want to track + * the length for r referenced by SortTuple. That's what BrinSortTuple is meant + * to do - it's essentially a BrinTuple prefixed by length. We only write the + * BrinTuple to the logtapes, though.
Why don't we write the full BrinSortTuple to disk? Doesn't that make more sense?
+ tuplesort_puttuple_common(state, &stup, + base->sortKeys && + base->sortKeys->abbrev_converter && + !stup.isnull1);
Can't this last argument just be inlined, based on knowledge that we
don't use sortKeys in brin?
+comparetup_index_brin(const SortTuple *a, const SortTuple *b, + Tuplesortstate *state) +{ + BrinTuple *tuple1; [...] + tuple1 = &((BrinSortTuple *) a)->tuple; [...]
I'm fairly sure that this cast (and it's neighbour) is incorrect and
should be the following instead:
+ tuple1 = &((BrinSortTuple *) (a->tuple))->tuple;
Additionally, I think the following would be a better approach here,
as we wouldn't need to do pointer-chasing:
+ static int
+ comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+ {
+ Assert(TuplesortstateGetPublic(state)->haveDatum1);
+
+ if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1))
+ return 1;
+ if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1))
+ return -1;
+ /* silence compilers */
+ return 0;
+ }
---
Thanks for working on this!
On 7/11/23 23:11, Matthias van de Meent wrote:
On Thu, 6 Jul 2023 at 16:13, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
On 7/5/23 16:33, Matthias van de Meent wrote:
...
Maybe. I wasn't that familiar with what parallel tuplesort can and can't
do, and the little I knew I managed to forget since I wrote this patch.
Which similar features do you have in mind?I was referring to the feature that is "emitting a single sorted run
of tuples at the leader backend based on data gathered in parallel
worker backends". It manages the sort state, on-disk runs etc. so that
you don't have to manage that yourself.Adding a new storage format for what is effectively a logical tape
(logtape.{c,h}) and manually merging it seems like a lot of changes if
that functionality is readily available, standardized and optimized in
sortsupport; and adds an additional place to manually go through for
disk-related changes like TDE.Here's a new version of the patch, with three main changes:
Thanks! I've done a review on the patch, and most looks good. Some
places need cleanup and polish, some others more documentations, and
there are some other issues, but so far it's looking OK.One thing I was wondering about is whether it might be better to allow
the workers to process overlapping ranges, and then let the leader to
merge the summaries. That would mean we might not need the tableam.c
changes at all, but the leader would need to do more work (although the
BRIN indexes tend to be fairly small). The main reason that got me
thinking about this is that we have pretty much no tests for the union
procedures, because triggering that is really difficult. But for
parallel index builds that'd be much more common.Hmm, that's a good point. I don't mind either way, but it would add
overhead in the leader to do all of that merging - especially when you
configure pages_per_range > PARALLEL_SEQSCAN_MAX_CHUNK_SIZE as we'd
need to merge up to parallel_workers tuples. That could be a
significant overhead.... thinks a bit.
Hmm, but with the current P_S_M_C_S of 8192 blocks that's quite
unlikely to be a serious problem - the per-backend IO saved of such
large ranges on a single backend has presumably much more impact than
the merging of n_parallel_tasks max-sized brin tuples. So, seems fine
with me.
As for PARALLEL_SEQSCAN_MAX_CHUNK_SIZE, the last patch actually
considers the chunk_factor (i.e. pages_per_range) *after* doing
pbscanwork->phsw_chunk_size = Min(pbscanwork->phsw_chunk_size,
PARALLEL_SEQSCAN_MAX_CHUNK_SIZE);
so even with (pages_per_range > PARALLEL_SEQSCAN_MAX_CHUNK_SIZE) it
would not need to merge anything.
Now, that might have been a bad idea and PARALLEL_SEQSCAN_MAX_CHUNK_SIZE
should be considered. In which case this *has* to do the union, even if
only for the rare corner case.
But I don't think that's a major issue - it's pretty sure summarizing
the tuples is way more expensive than merging the summaries. Which is
what matters for Amdahl's law ...
Review follows below.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech/)-----------
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c+ BrinShared *brinshared;
Needs some indentation fixes.
+ int bs_reltuples; [...] + state->bs_reltuples += reltuples;My IDE warns me that reltuples is a double. Looking deeper into the
value, it contains the number of live tuples in the table, so this
conversion may not result in a meaningful value for tables with >=2^31
live tuples. Tables > 56GB could begin to get affected by this.+ int bs_worker_id;
This variable seems to be unused.
+ BrinSpool *bs_spool;
+ BrinSpool *bs_spool_out;Are both used? If so, could you add comments why we have two spools
here, instead of only one?
OK, I admit I'm not sure both are actually necessary. I was struggling
getting it working with just one spool, because when the leader
participates as a worker, it needs to both summarize some of the chunks
(and put the tuples somewhere). And then it also needs to consume the
final output.
Maybe it's just a case of cargo cult programming - I was mostly copying
stuff from the btree index build, trying to make it work, and then with
two spools it started working.
+/* + * A version of the callback, used by parallel index builds. The main difference + * is that instead of writing the BRIN tuples into the index, we write them into + * a shared tuplestore file, and leave the insertion up to the leader (which may+ ... shared tuplesort, and ...
brinbuildCallbackParallel(...)
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)shouldn't this be an 'if' now?
Hmmm, probably ... that way we'd skip the empty ranges.
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1) + state->bs_currRangeStart += state->bs_pagesPerRange;Is there a reason why you went with iterative addition instead of a
single divide-and-multiply like the following?:+ state->bs_currRangeStart += state->bs_pagesPerRange *
((state->bs_currRangeStart - thisblock) / state->bs_pagesPerRange);
Probably laziness ... You're right the divide-multiply seems simpler.
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c [...] -table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan) +table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan, BlockNumber chunk_factor) [...] - /* compare phs_syncscan initialization to similar logic in initscan */ + bpscan->phs_chunk_factor = chunk_factor; + /* compare phs_syncscan initialization to similar logic in initscan + * + * Disable sync scans if the chunk factor is set (valid block number). + */I think this needs some pgindent or other style work, both on comment
style and line lengths
Right.
diff --git a/src/backend/utils/sort/tuplesort.c b/src/backend/utils/sort/tuplesort.c [...] + Assert(false); (x3)I think these can be cleaned up, right?
Duh! Absolutely, this shouldn't have been in the patch at all. I only
added those to quickly identify places that got the tuplesort into
unexpected state (much easier with a coredump and a backtrace).
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c [...] + * Computing BrinTuple size with only the tuple is difficult, so we want to track + * the length for r referenced by SortTuple. That's what BrinSortTuple is meant + * to do - it's essentially a BrinTuple prefixed by length. We only write the + * BrinTuple to the logtapes, though.Why don't we write the full BrinSortTuple to disk? Doesn't that make more sense?
Not sure I understand. We ultimately do, because we write
(length + BrinTuple)
and BrinSortTuple is exactly that. But if we write BrinSortTuple, we
would end up writing length for that too, no?
Or maybe I just don't understand how would that make things simpler.
+ tuplesort_puttuple_common(state, &stup, + base->sortKeys && + base->sortKeys->abbrev_converter && + !stup.isnull1);Can't this last argument just be inlined, based on knowledge that we
don't use sortKeys in brin?
What does "inlined" mean for an argument? But yeah, I guess it might be
just set to false. And we should probably have an assert that there are
no sortKeys.
+comparetup_index_brin(const SortTuple *a, const SortTuple *b, + Tuplesortstate *state) +{ + BrinTuple *tuple1; [...] + tuple1 = &((BrinSortTuple *) a)->tuple; [...]I'm fairly sure that this cast (and it's neighbour) is incorrect and
should be the following instead:+ tuple1 = &((BrinSortTuple *) (a->tuple))->tuple;
Additionally, I think the following would be a better approach here,
as we wouldn't need to do pointer-chasing:
Uh, right. This only works because 'tuple' happens to be the first field
in SortTuple.
+ static int + comparetup_index_brin(const SortTuple *a, const SortTuple *b, + Tuplesortstate *state) + { + Assert(TuplesortstateGetPublic(state)->haveDatum1); + + if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1)) + return 1; + if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1)) + return -1; + /* silence compilers */ + return 0; + }
Good idea! I forgot we're guaranteed to have datum1.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On Fri, 14 Jul 2023 at 15:57, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:
On 7/11/23 23:11, Matthias van de Meent wrote:
On Thu, 6 Jul 2023 at 16:13, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
One thing I was wondering about is whether it might be better to allow
the workers to process overlapping ranges, and then let the leader to
merge the summaries. That would mean we might not need the tableam.c
changes at all, but the leader would need to do more work (although the
BRIN indexes tend to be fairly small). The main reason that got me
thinking about this is that we have pretty much no tests for the union
procedures, because triggering that is really difficult. But for
parallel index builds that'd be much more common.Hmm, that's a good point. I don't mind either way, but it would add
overhead in the leader to do all of that merging - especially when you
configure pages_per_range > PARALLEL_SEQSCAN_MAX_CHUNK_SIZE as we'd
need to merge up to parallel_workers tuples. That could be a
significant overhead.... thinks a bit.
Hmm, but with the current P_S_M_C_S of 8192 blocks that's quite
unlikely to be a serious problem - the per-backend IO saved of such
large ranges on a single backend has presumably much more impact than
the merging of n_parallel_tasks max-sized brin tuples. So, seems fine
with me.As for PARALLEL_SEQSCAN_MAX_CHUNK_SIZE, the last patch actually
considers the chunk_factor (i.e. pages_per_range) *after* doingpbscanwork->phsw_chunk_size = Min(pbscanwork->phsw_chunk_size,
PARALLEL_SEQSCAN_MAX_CHUNK_SIZE);so even with (pages_per_range > PARALLEL_SEQSCAN_MAX_CHUNK_SIZE) it
would not need to merge anything.Now, that might have been a bad idea and PARALLEL_SEQSCAN_MAX_CHUNK_SIZE
should be considered. In which case this *has* to do the union, even if
only for the rare corner case.But I don't think that's a major issue - it's pretty sure summarizing
the tuples is way more expensive than merging the summaries. Which is
what matters for Amdahl's law ...
Agreed.
+ BrinSpool *bs_spool;
+ BrinSpool *bs_spool_out;Are both used? If so, could you add comments why we have two spools
here, instead of only one?OK, I admit I'm not sure both are actually necessary. I was struggling
getting it working with just one spool, because when the leader
participates as a worker, it needs to both summarize some of the chunks
(and put the tuples somewhere). And then it also needs to consume the
final output.Maybe it's just a case of cargo cult programming - I was mostly copying
stuff from the btree index build, trying to make it work, and then with
two spools it started working.
Two spools seem to be necessary in a participating leader, but both
spools have non-overlapping lifetimes. In the btree code actually two
pairs of spools are actually used (in unique indexes): you can see the
pairs being allocated in both _bt_leader_participate_as_worker (called
from _bt_begin_parallel, from _bt_spools_heapscan) and in
_bt_spools_heapscan.
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c [...] + * Computing BrinTuple size with only the tuple is difficult, so we want to track + * the length for r referenced by SortTuple. That's what BrinSortTuple is meant + * to do - it's essentially a BrinTuple prefixed by length. We only write the + * BrinTuple to the logtapes, though.Why don't we write the full BrinSortTuple to disk? Doesn't that make more sense?
Not sure I understand. We ultimately do, because we write
(length + BrinTuple)
and BrinSortTuple is exactly that. But if we write BrinSortTuple, we
would end up writing length for that too, no?Or maybe I just don't understand how would that make things simpler.
I don't quite understand the intricacies of the tape storage format
quite yet (specifically, I'm continuously getting confused by the len
-= sizeof(int)), so you might well be correct.
My comment was written based on just the comment's contents, which
claims "we can't easily recompute the length, so we store it with the
tuple in memory. However, we don't store the length when we write it
to the tape", which seems self-contradictory.
+ tuplesort_puttuple_common(state, &stup, + base->sortKeys && + base->sortKeys->abbrev_converter && + !stup.isnull1);Can't this last argument just be inlined, based on knowledge that we
don't use sortKeys in brin?What does "inlined" mean for an argument? But yeah, I guess it might be
just set to false. And we should probably have an assert that there are
no sortKeys.
"inlined", "precomputed", "const-ified"? I'm not super good at
vocabulary. But, indeed, thanks.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech)
Hi,
here's an updated patch, addressing the review comments, and reworking
how the work is divided between the workers & leader etc.
0001 is just v2, rebased to current master
0002 and 0003 address most of the issues, in particular it
- removes the unnecessary spool
- fixes bs_reltuples type to double
- a couple comments are reworded to be clearer
- changes loop/condition in brinbuildCallbackParallel
- removes asserts added for debugging
- fixes cast in comparetup_index_brin
- 0003 then simplifies comparetup_index_brin
- I haven't inlined the tuplesort_puttuple_common parameter
(didn't seem worth it)
0004 Reworks how the work is divided between workers and combined by the
leader. It undoes the tableam.c changes that attempted to divide the
relation into chunks matching the BRIN ranges, and instead merges the
results in the leader (using the BRIN "union" function).
I haven't done any indentation fixes yet.
I did fairly extensive testing, using pageinspect to compare indexes
built with/without parallelism. More testing is needed, but it seems to
work fine (with other opclasses and so on).
In general I'm quite happy with the current state, and I believe it's
fairly close to be committable.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
v3-0001-parallel-CREATE-INDEX-for-BRIN-v2.patchtext/x-patch; charset=UTF-8; name=v3-0001-parallel-CREATE-INDEX-for-BRIN-v2.patchDownload
From 3e5fabc6e5a30bb16ec11fdc7fcf65880e172d0c Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 7 Nov 2023 17:04:28 +0100
Subject: [PATCH v3 1/4] parallel CREATE INDEX for BRIN v2
---
src/backend/access/brin/brin.c | 781 ++++++++++++++++++++-
src/backend/access/nbtree/nbtsort.c | 2 +-
src/backend/access/table/tableam.c | 49 +-
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 3 +-
src/backend/executor/nodeSeqscan.c | 3 +-
src/backend/utils/sort/tuplesort.c | 3 +
src/backend/utils/sort/tuplesortvariants.c | 211 ++++++
src/include/access/brin.h | 3 +
src/include/access/relscan.h | 1 +
src/include/access/tableam.h | 9 +-
src/include/utils/tuplesort.h | 11 +
12 files changed, 1065 insertions(+), 15 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 25338a90e29..b7cd29c5968 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -33,6 +33,7 @@
#include "postmaster/autovacuum.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -40,7 +41,118 @@
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+#include "utils/tuplesort.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xA000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xA000000000000005)
+
+/*
+ * Status record for spooling/sorting phase.
+ */
+typedef struct BrinSpool
+{
+ Tuplesortstate *sortstate; /* state data for tuplesort.c */
+ Relation heap;
+ Relation index;
+} BrinSpool;
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist for
+ * the benefit of worker processes that need to create state corresponding
+ * to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+ int scantuplesortstates;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN 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 the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
+} BrinShared;
+
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* 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).
+ *
+ * brinshared is the shared state for entire build. sharedsort is the
+ * shared, tuplesort-managed state passed to each process tuplesort.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is required.
+ */
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -50,12 +162,23 @@ typedef struct BrinBuildState
{
Relation bs_irel;
int bs_numtuples;
+ int bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BrinSpool *bs_spool;
+ BrinSpool *bs_spool_out;
} BrinBuildState;
/*
@@ -76,6 +199,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -83,6 +207,20 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinSpool *brinspool,
+ BrinShared *brinshared,
+ Sharedsort *sharedsort,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -820,6 +958,67 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a shared tuplestore file, and leave the insertion up to the leader (which may
+ * reorder them a bit etc.). The callback also does not generate empty ranges,
+ * those may be added by the leader when merging results from workers.
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; make sure to insert index
+ * tuples for those too.
+ */
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the tuplesort */
+ form_and_spill_tuple(state);
+
+ /*
+ * set state to correspond to the next range
+ *
+ * XXX This has the issue that it skips ranges summarized by other
+ * workers, but it also skips empty ranges that should have been
+ * summarized. We'd need to either make the workers aware which
+ * chunk they are actually processing (which is currently known
+ * only in the ParallelBlockTableScan bit). Or we could ignore it
+ * here, and then decide it while "merging" results from workers
+ * (if there's no entry for the range, it had to be empty so we
+ * just add an empty one).
+ */
+ while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ state->bs_currRangeStart += state->bs_pagesPerRange;
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -881,18 +1080,93 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
revmap = brinRevmapInitialize(index, &pagesPerRange);
state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool->heap = heap;
+ state->bs_spool->index = index;
+
+ /*
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
+ */
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
+
/*
* Now scan the relation. No syncscan allowed here because we want the
* heap blocks in physical order.
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
- form_and_insert_tuple(state);
+ /*
+ * If parallel build requested and at least one worker process was
+ * successfully launched, set up coordination state
+ */
+ if (state->bs_leader)
+ {
+ SortCoordinate coordinate;
+
+ coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
+ coordinate->isWorker = false;
+ coordinate->nParticipants =
+ state->bs_leader->nparticipanttuplesorts;
+ coordinate->sharedsort = state->bs_leader->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.
+ */
+ state->bs_spool_out = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool_out->heap = heap;
+ state->bs_spool_out->index = index;
+
+ state->bs_spool_out->sortstate =
+ tuplesort_begin_index_brin(heap, index,
+ maintenance_work_mem, coordinate,
+ TUPLESORT_NONE);
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read all
+ * tuples from the shared tuplesort and insert them into the index.
+ */
+ _brin_end_parallel(state->bs_leader, state);
+ }
+ else /* no parallel index build, just do the usual thing */
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
+ form_and_insert_tuple(state);
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
+ }
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1312,12 +1586,16 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
+ state->bs_spool = NULL;
return state;
}
@@ -1609,6 +1887,32 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a (shared) tuplestore (the leader will insert it
+ * into the index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ /* don't insert empty tuples in parallel build */
+ if (state->bs_dtuple->bt_empty_range)
+ return;
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ /* write the BRIN tuple to the tuplesort */
+ tuplesort_putbrintuple(state->bs_spool->sortstate, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -1928,3 +2232,472 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ int scantuplesortstates;
+ Snapshot snapshot;
+ Size estbrinshared;
+ Size estsort;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ request);
+
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+ estsort = tuplesort_estimate_shared(scantuplesortstates);
+ shm_toc_estimate_chunk(&pcxt->estimator, estsort);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->scantuplesortstates = scantuplesortstates;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ snapshot, brinshared->pagesPerRange);
+
+ /*
+ * 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);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipanttuplesorts = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipanttuplesorts++;
+ brinleader->brinshared = brinshared;
+ brinleader->sharedsort = sharedsort;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinTuple *btup;
+ Size tuplen;
+ BrinShared *brinshared = brinleader->brinshared;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ tuplesort_performsort(state->bs_spool_out->sortstate);
+
+ /*
+ * Read the BRIN tuples from the shared tuplesort, sorted by block number.
+ * That probably gives us an index that is cheaper to scan, thanks to mostly
+ * getting data from the same index page as before.
+ *
+ * FIXME This probably needs some memory management fixes - we're reading
+ * tuples from the tuplesort, we're allocating an emty tuple, and so on.
+ * Probably better to release this memory.
+ *
+ * XXX We can't quite free the BrinTuple, though, because that's a field
+ * in BrinSortTuple.
+ */
+ while ((btup = tuplesort_getbrintuple(state->bs_spool_out->sortstate, &tuplen, true)) != NULL)
+ {
+ /*
+ * We should not get two summaries for the same range. The workers
+ * are producing ranges for non-overlapping sections of the table.
+ */
+ Assert(btup->bt_blkno != prevblkno);
+
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, btup->bt_blkno, btup, tuplen);
+
+ prevblkno = btup->bt_blkno;
+ }
+
+ tuplesort_end(state->bs_spool_out->sortstate);
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /* Allocate memory and initialize private spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = buildstate->bs_spool->heap;
+ buildstate->bs_spool->index = buildstate->bs_spool->index;
+
+ /*
+ * 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 / brinleader->nparticipanttuplesorts;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool, brinleader->brinshared,
+ brinleader->sharedsort, heap, index, sortmem, true);
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
+ BrinShared *brinshared, Sharedsort *sharedsort,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ SortCoordinate coordinate;
+ TableScanDesc 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 */
+ brinspool->sortstate = tuplesort_begin_index_brin(brinspool->heap,
+ brinspool->index,
+ sortmem, coordinate,
+ TUPLESORT_NONE);
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* insert the last item */
+ form_and_spill_tuple(state);
+
+ /* sort the BRIN ranges built by this worker */
+ tuplesort_performsort(brinspool->sortstate);
+
+ state->bs_reltuples += reltuples;
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+
+ tuplesort_end(brinspool->sortstate);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL, brinshared->pagesPerRange);
+
+ /* Initialize worker's own spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = heapRel;
+ buildstate->bs_spool->index = indexRel;
+
+ /* Look up shared state private to tuplesort.c */
+ sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
+ tuplesort_attach_shared(sharedsort, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /*
+ * 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 / brinshared->scantuplesortstates;
+
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool,
+ brinshared, sharedsort,
+ heapRel, indexRel, sortmem, false);
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/nbtree/nbtsort.c b/src/backend/access/nbtree/nbtsort.c
index c2665fce411..6241baeea86 100644
--- a/src/backend/access/nbtree/nbtsort.c
+++ b/src/backend/access/nbtree/nbtsort.c
@@ -1575,7 +1575,7 @@ _bt_begin_parallel(BTBuildState *buildstate, bool isconcurrent, int request)
btshared->brokenhotchain = false;
table_parallelscan_initialize(btspool->heap,
ParallelTableScanFromBTShared(btshared),
- snapshot);
+ snapshot, InvalidBlockNumber);
/*
* Store shared tuplesort-private state, for which we reserved space.
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index c6bdb7e1c68..4af0d433e9d 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -153,9 +153,9 @@ table_parallelscan_estimate(Relation rel, Snapshot snapshot)
void
table_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan,
- Snapshot snapshot)
+ Snapshot snapshot, BlockNumber chunk_factor)
{
- Size snapshot_off = rel->rd_tableam->parallelscan_initialize(rel, pscan);
+ Size snapshot_off = rel->rd_tableam->parallelscan_initialize(rel, pscan, chunk_factor);
pscan->phs_snapshot_off = snapshot_off;
@@ -395,16 +395,21 @@ table_block_parallelscan_estimate(Relation rel)
}
Size
-table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan)
+table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan, BlockNumber chunk_factor)
{
ParallelBlockTableScanDesc bpscan = (ParallelBlockTableScanDesc) pscan;
bpscan->base.phs_relid = RelationGetRelid(rel);
bpscan->phs_nblocks = RelationGetNumberOfBlocks(rel);
- /* compare phs_syncscan initialization to similar logic in initscan */
+ bpscan->phs_chunk_factor = chunk_factor;
+ /* compare phs_syncscan initialization to similar logic in initscan
+ *
+ * Disable sync scans if the chunk factor is set (valid block number).
+ */
bpscan->base.phs_syncscan = synchronize_seqscans &&
!RelationUsesLocalBuffers(rel) &&
- bpscan->phs_nblocks > NBuffers / 4;
+ (bpscan->phs_nblocks > NBuffers / 4) &&
+ !BlockNumberIsValid(bpscan->phs_chunk_factor);
SpinLockInit(&bpscan->phs_mutex);
bpscan->phs_startblock = InvalidBlockNumber;
pg_atomic_init_u64(&bpscan->phs_nallocated, 0);
@@ -459,6 +464,25 @@ table_block_parallelscan_startblock_init(Relation rel,
pbscanwork->phsw_chunk_size = Min(pbscanwork->phsw_chunk_size,
PARALLEL_SEQSCAN_MAX_CHUNK_SIZE);
+ /*
+ * If the chunk size factor is set, we need to make sure the chunk size is
+ * a multiple of that value. We round the chunk size to the nearest chunk
+ * factor multiple, at least one chunk_factor.
+ *
+ * XXX Note this may override PARALLEL_SEQSCAN_MAX_CHUNK_SIZE, in case the
+ * chunk factor (e.g. BRIN pages_per_range) is larger.
+ */
+ if (pbscan->phs_chunk_factor != InvalidBlockNumber)
+ {
+ /* nearest (smaller) multiple of chunk_factor */
+ pbscanwork->phsw_chunk_size
+ = pbscan->phs_chunk_factor * (pbscanwork->phsw_chunk_size / pbscan->phs_chunk_factor);
+
+ /* but at least one chunk_factor */
+ pbscanwork->phsw_chunk_size = Max(pbscanwork->phsw_chunk_size,
+ pbscan->phs_chunk_factor);
+ }
+
retry:
/* Grab the spinlock. */
SpinLockAcquire(&pbscan->phs_mutex);
@@ -575,6 +599,21 @@ table_block_parallelscan_nextpage(Relation rel,
(pbscanwork->phsw_chunk_size * PARALLEL_SEQSCAN_RAMPDOWN_CHUNKS))
pbscanwork->phsw_chunk_size >>= 1;
+ /*
+ * We need to make sure the new chunk_size is still a suitable multiple
+ * of chunk_factor.
+ */
+ if (pbscan->phs_chunk_factor != InvalidBlockNumber)
+ {
+ /* nearest (smaller) multiple of chunk_factor */
+ pbscanwork->phsw_chunk_size
+ = pbscan->phs_chunk_factor * (pbscanwork->phsw_chunk_size / pbscan->phs_chunk_factor);
+
+ /* but at least one chunk_factor */
+ pbscanwork->phsw_chunk_size = Max(pbscanwork->phsw_chunk_size,
+ pbscan->phs_chunk_factor);
+ }
+
nallocated = pbscanwork->phsw_nallocated =
pg_atomic_fetch_add_u64(&pbscan->phs_nallocated,
pbscanwork->phsw_chunk_size);
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 194a1207be6..d78314062e0 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -145,6 +146,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 143fae01ebd..37e4305d50a 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2982,7 +2982,8 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ (indexRelation->rd_rel->relam == BTREE_AM_OID ||
+ indexRelation->rd_rel->relam == BRIN_AM_OID))
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 49a5933aff6..529a7ed3284 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -262,7 +262,8 @@ ExecSeqScanInitializeDSM(SeqScanState *node,
pscan = shm_toc_allocate(pcxt->toc, node->pscan_len);
table_parallelscan_initialize(node->ss.ss_currentRelation,
pscan,
- estate->es_snapshot);
+ estate->es_snapshot,
+ InvalidBlockNumber);
shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pscan);
node->ss.ss_currentScanDesc =
table_beginscan_parallel(node->ss.ss_currentRelation, pscan);
diff --git a/src/backend/utils/sort/tuplesort.c b/src/backend/utils/sort/tuplesort.c
index ab6353bdcd1..4c6b396a8a8 100644
--- a/src/backend/utils/sort/tuplesort.c
+++ b/src/backend/utils/sort/tuplesort.c
@@ -1328,6 +1328,7 @@ tuplesort_puttuple_common(Tuplesortstate *state, SortTuple *tuple, bool useAbbre
break;
default:
+ Assert(false);
elog(ERROR, "invalid tuplesort state");
break;
}
@@ -1462,6 +1463,7 @@ tuplesort_performsort(Tuplesortstate *state)
break;
default:
+ Assert(false);
elog(ERROR, "invalid tuplesort state");
break;
}
@@ -1718,6 +1720,7 @@ tuplesort_gettuple_common(Tuplesortstate *state, bool forward,
return false;
default:
+ Assert(false);
elog(ERROR, "invalid tuplesort state");
return false; /* keep compiler quiet */
}
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 2cd508e5130..343ed4bbc54 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -19,6 +19,7 @@
#include "postgres.h"
+#include "access/brin_tuple.h"
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -43,6 +44,8 @@ static void removeabbrev_cluster(Tuplesortstate *state, SortTuple *stups,
int count);
static void removeabbrev_index(Tuplesortstate *state, SortTuple *stups,
int count);
+static void removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups,
+ int count);
static void removeabbrev_datum(Tuplesortstate *state, SortTuple *stups,
int count);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
@@ -69,10 +72,16 @@ static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash_tiebreak(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
+static int comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state);
static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len);
+static void writetup_index_brin(Tuplesortstate *state, LogicalTape *tape,
+ SortTuple *stup);
+static void readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_datum_tiebreak(const SortTuple *a, const SortTuple *b,
@@ -128,6 +137,16 @@ typedef struct
uint32 max_buckets;
} TuplesortIndexHashArg;
+/*
+ * Data struture pointed by "TuplesortPublic.arg" for the index_brin subcase.
+ */
+typedef struct
+{
+ TuplesortIndexArg index;
+
+ /* XXX do we need something here? */
+} TuplesortIndexBrinArg;
+
/*
* Data struture pointed by "TuplesortPublic.arg" for the Datum case.
* Set by tuplesort_begin_datum and used only by the DatumTuple routines.
@@ -140,6 +159,22 @@ typedef struct
int datumTypeLen;
} TuplesortDatumArg;
+/*
+ * Computing BrinTuple size with only the tuple is difficult, so we want to track
+ * the length for r referenced by SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by length. We only write the
+ * BrinTuple to the logtapes, though.
+ */
+typedef struct BrinSortTuple
+{
+ Size tuplen;
+ BrinTuple tuple;
+} BrinSortTuple;
+
+/* Size of the BrinSortTuple, given length of the BrinTuple. */
+#define BRINSORTTUPLE_SIZE(len) (offsetof(BrinSortTuple, tuple) + (len))
+
+
Tuplesortstate *
tuplesort_begin_heap(TupleDesc tupDesc,
int nkeys, AttrNumber *attNums,
@@ -527,6 +562,47 @@ tuplesort_begin_index_gist(Relation heapRel,
return state;
}
+Tuplesortstate *
+tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem,
+ SortCoordinate coordinate,
+ int sortopt)
+{
+ Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
+ sortopt);
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext;
+ TuplesortIndexBrinArg *arg;
+
+ oldcontext = MemoryContextSwitchTo(base->maincontext);
+ arg = (TuplesortIndexBrinArg *) palloc(sizeof(TuplesortIndexBrinArg));
+
+#ifdef TRACE_SORT
+ if (trace_sort)
+ elog(LOG,
+ "begin index sort: workMem = %d, randomAccess = %c",
+ workMem,
+ sortopt & TUPLESORT_RANDOMACCESS ? 't' : 'f');
+#endif
+
+ base->nKeys = 1; /* Only one sort column, the block number */
+
+ base->removeabbrev = removeabbrev_index_brin;
+ base->comparetup = comparetup_index_brin;
+ base->writetup = writetup_index_brin;
+ base->readtup = readtup_index_brin;
+ base->haveDatum1 = true;
+ base->arg = arg;
+
+ arg->index.heapRel = heapRel;
+ arg->index.indexRel = indexRel;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return state;
+}
+
Tuplesortstate *
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag, int workMem,
@@ -707,6 +783,35 @@ tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
!stup.isnull1);
}
+/*
+ * Collect one BRIN tuple while collecting input data for sort.
+ */
+void
+tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
+{
+ SortTuple stup;
+ BrinSortTuple *bstup;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->tuplecontext);
+
+ /* allocate space for the whole BRIN sort tuple */
+ bstup = palloc(BRINSORTTUPLE_SIZE(size));
+ stup.tuple = bstup;
+
+ bstup->tuplen = size;
+ memcpy(&bstup->tuple, tuple, size);
+
+ stup.datum1 = tuple->bt_blkno;
+ stup.isnull1 = false;
+
+ tuplesort_puttuple_common(state, &stup,
+ base->sortKeys &&
+ base->sortKeys->abbrev_converter &&
+ !stup.isnull1);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
/*
* Accept one Datum while collecting input data for sort.
*
@@ -850,6 +955,35 @@ tuplesort_getindextuple(Tuplesortstate *state, bool forward)
return (IndexTuple) stup.tuple;
}
+/*
+ * Fetch the next BRIN tuple in either forward or back direction.
+ * Returns NULL if no more tuples. Returned tuple belongs to tuplesort memory
+ * context, and must not be freed by caller. Caller may not rely on tuple
+ * remaining valid after any further manipulation of tuplesort.
+ */
+BrinTuple *
+tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
+ SortTuple stup;
+ BrinSortTuple *btup;
+
+ if (!tuplesort_gettuple_common(state, forward, &stup))
+ stup.tuple = NULL;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!stup.tuple)
+ return NULL;
+
+ btup = (BrinSortTuple *) stup.tuple;
+
+ *len = btup->tuplen;
+
+ return &btup->tuple;
+}
+
/*
* Fetch the next Datum in either forward or back direction.
* Returns false if no more datums.
@@ -1564,6 +1698,83 @@ readtup_index(Tuplesortstate *state, SortTuple *stup,
&stup->isnull1);
}
+/*
+ * Routines specialized for BrinTuple case
+ */
+
+static void
+removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
+{
+ int i;
+
+ for (i = 0; i < count; i++)
+ {
+ BrinSortTuple *tuple;
+
+ tuple = stups[i].tuple;
+ stups[i].datum1 = tuple->tuple.bt_blkno;
+ }
+}
+
+static int
+comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+{
+ BrinTuple *tuple1;
+ BrinTuple *tuple2;
+
+ tuple1 = &((BrinSortTuple *) a)->tuple;
+ tuple2 = &((BrinSortTuple *) b)->tuple;
+
+ if (tuple1->bt_blkno > tuple2->bt_blkno)
+ return 1;
+ else if (tuple1->bt_blkno < tuple2->bt_blkno)
+ return -1;
+
+ /* silence compilers */
+ return 0;
+}
+
+static void
+writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
+
+ tuplen = tuplen + sizeof(tuplen);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->tuple, tuple->tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+}
+
+static void
+readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ unsigned int tuplen = len - sizeof(unsigned int);
+
+ /*
+ * Allocate space for the BRIN sort tuple, which is BrinTuple with an
+ * extra length field.
+ */
+ BrinSortTuple *tuple
+ = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
+
+ tuple->tuplen = tuplen;
+
+ LogicalTapeReadExact(tape, &tuple->tuple, tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
+ stup->tuple = (void *) tuple;
+
+ /* set up first-column key value, which is block number */
+ stup->datum1 = tuple->tuple.bt_blkno;
+}
+
/*
* Routines specialized for DatumTuple case
*/
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d51..3451ecb211f 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index d03360eac04..72a20d882f5 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -79,6 +79,7 @@ typedef struct ParallelBlockTableScanDescData
BlockNumber phs_nblocks; /* # blocks in relation at start of scan */
slock_t phs_mutex; /* mutual exclusion for setting startblock */
BlockNumber phs_startblock; /* starting block number */
+ BlockNumber phs_chunk_factor; /* chunks need to be a multiple of this */
pg_atomic_uint64 phs_nallocated; /* number of blocks allocated to
* workers so far. */
} ParallelBlockTableScanDescData;
diff --git a/src/include/access/tableam.h b/src/include/access/tableam.h
index dbb709b56ce..d94e4d32aa1 100644
--- a/src/include/access/tableam.h
+++ b/src/include/access/tableam.h
@@ -390,7 +390,8 @@ typedef struct TableAmRoutine
* relation.
*/
Size (*parallelscan_initialize) (Relation rel,
- ParallelTableScanDesc pscan);
+ ParallelTableScanDesc pscan,
+ BlockNumber chunk_factor);
/*
* Reinitialize `pscan` for a new scan. `rel` will be the same relation as
@@ -1148,7 +1149,8 @@ extern Size table_parallelscan_estimate(Relation rel, Snapshot snapshot);
*/
extern void table_parallelscan_initialize(Relation rel,
ParallelTableScanDesc pscan,
- Snapshot snapshot);
+ Snapshot snapshot,
+ BlockNumber chunk_factor);
/*
* Begin a parallel scan. `pscan` needs to have been initialized with
@@ -2064,7 +2066,8 @@ extern void simple_table_tuple_update(Relation rel, ItemPointer otid,
extern Size table_block_parallelscan_estimate(Relation rel);
extern Size table_block_parallelscan_initialize(Relation rel,
- ParallelTableScanDesc pscan);
+ ParallelTableScanDesc pscan,
+ BlockNumber chunk_factor);
extern void table_block_parallelscan_reinitialize(Relation rel,
ParallelTableScanDesc pscan);
extern BlockNumber table_block_parallelscan_nextpage(Relation rel,
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index 9ed2de76cd6..357eb35311d 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -21,6 +21,7 @@
#ifndef TUPLESORT_H
#define TUPLESORT_H
+#include "access/brin_tuple.h"
#include "access/itup.h"
#include "executor/tuptable.h"
#include "storage/dsm.h"
@@ -282,6 +283,9 @@ typedef struct
* The "index_hash" API is similar to index_btree, but the tuples are
* actually sorted by their hash codes not the raw data.
*
+ * The "index_brin" API is similar to index_btree, but the tuples are
+ * BrinTuple and are sorted by their block number not the raw data.
+ *
* Parallel sort callers are required to coordinate multiple tuplesort states
* in a leader process and one or more worker processes. The leader process
* must launch workers, and have each perform an independent "partial"
@@ -426,6 +430,10 @@ extern Tuplesortstate *tuplesort_begin_index_gist(Relation heapRel,
Relation indexRel,
int workMem, SortCoordinate coordinate,
int sortopt);
+extern Tuplesortstate *tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem, SortCoordinate coordinate,
+ int sortopt);
extern Tuplesortstate *tuplesort_begin_datum(Oid datumType,
Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag,
@@ -438,6 +446,7 @@ extern void tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup);
extern void tuplesort_putindextuplevalues(Tuplesortstate *state,
Relation rel, ItemPointer self,
const Datum *values, const bool *isnull);
+extern void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tup, Size len);
extern void tuplesort_putdatum(Tuplesortstate *state, Datum val,
bool isNull);
@@ -445,6 +454,8 @@ extern bool tuplesort_gettupleslot(Tuplesortstate *state, bool forward,
bool copy, TupleTableSlot *slot, Datum *abbrev);
extern HeapTuple tuplesort_getheaptuple(Tuplesortstate *state, bool forward);
extern IndexTuple tuplesort_getindextuple(Tuplesortstate *state, bool forward);
+extern BrinTuple *tuplesort_getbrintuple(Tuplesortstate *state, Size *len,
+ bool forward);
extern bool tuplesort_getdatum(Tuplesortstate *state, bool forward, bool copy,
Datum *val, bool *isNull, Datum *abbrev);
--
2.41.0
v3-0002-fix-review-comments.patchtext/x-patch; charset=UTF-8; name=v3-0002-fix-review-comments.patchDownload
From dcb0058540f46fa5f9cbb21e1cff640874418401 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 7 Nov 2023 18:15:20 +0100
Subject: [PATCH v3 2/4] fix review comments
---
src/backend/access/brin/brin.c | 44 +++++++++-------------
src/backend/utils/sort/tuplesort.c | 3 --
src/backend/utils/sort/tuplesortvariants.c | 11 +++---
3 files changed, 23 insertions(+), 35 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index b7cd29c5968..d54d39a535e 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -162,7 +162,7 @@ typedef struct BrinBuildState
{
Relation bs_irel;
int bs_numtuples;
- int bs_reltuples;
+ double bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
@@ -172,13 +172,12 @@ typedef struct BrinBuildState
/*
* bs_leader is only present when a parallel index build is performed, and
- * only in the leader process. (Actually, only the leader has a
+ * only in the leader process. (Actually, only the leader process has a
* BrinBuildState.)
*/
BrinLeader *bs_leader;
int bs_worker_id;
BrinSpool *bs_spool;
- BrinSpool *bs_spool_out;
} BrinBuildState;
/*
@@ -961,7 +960,7 @@ brinbuildCallback(Relation index,
/*
* A version of the callback, used by parallel index builds. The main difference
* is that instead of writing the BRIN tuples into the index, we write them into
- * a shared tuplestore file, and leave the insertion up to the leader (which may
+ * a shared tuplestore, and leave the insertion up to the leader (which may
* reorder them a bit etc.). The callback also does not generate empty ranges,
* those may be added by the leader when merging results from workers.
*/
@@ -981,10 +980,10 @@ brinbuildCallbackParallel(Relation index,
/*
* If we're in a block that belongs to a future range, summarize what
* we've got and start afresh. Note the scan might have skipped many
- * pages, if they were devoid of live tuples; make sure to insert index
- * tuples for those too.
+ * pages, if they were devoid of live tuples; we do not create emptry
+ * BRIN ranges here - the leader is responsible for filling them in.
*/
- while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ if (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
{
BRIN_elog((DEBUG2,
@@ -996,19 +995,15 @@ brinbuildCallbackParallel(Relation index,
form_and_spill_tuple(state);
/*
- * set state to correspond to the next range
+ * Set state to correspond to the next range (for this block).
*
- * XXX This has the issue that it skips ranges summarized by other
- * workers, but it also skips empty ranges that should have been
- * summarized. We'd need to either make the workers aware which
- * chunk they are actually processing (which is currently known
- * only in the ParallelBlockTableScan bit). Or we could ignore it
- * here, and then decide it while "merging" results from workers
- * (if there's no entry for the range, it had to be empty so we
- * just add an empty one).
+ * This skips ranges that are either empty (and so we don't get any
+ * tuples to summarize), or processes by other workers. We can't
+ * differentiate those cases here easily, so we leave it up to the
+ * leader to fill empty ranges where needed.
*/
- while (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
- state->bs_currRangeStart += state->bs_pagesPerRange;
+ state->bs_currRangeStart
+ = state->bs_pagesPerRange * (thisblock / state->bs_pagesPerRange);
/* re-initialize state for it */
brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
@@ -1137,11 +1132,7 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
* on the amount of memory used by a CREATE INDEX operation, regardless of
* the use of parallelism or any other factor.
*/
- state->bs_spool_out = (BrinSpool *) palloc0(sizeof(BrinSpool));
- state->bs_spool_out->heap = heap;
- state->bs_spool_out->index = index;
-
- state->bs_spool_out->sortstate =
+ state->bs_spool->sortstate =
tuplesort_begin_index_brin(heap, index,
maintenance_work_mem, coordinate,
TUPLESORT_NONE);
@@ -2427,6 +2418,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
BlockNumber prevblkno = InvalidBlockNumber;
BrinTuple *emptyTuple = NULL;
Size emptySize;
+ BrinSpool *spool = state->bs_spool;
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
@@ -2438,7 +2430,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
state->bs_reltuples = brinshared->reltuples;
state->bs_numtuples = brinshared->indtuples;
- tuplesort_performsort(state->bs_spool_out->sortstate);
+ tuplesort_performsort(spool->sortstate);
/*
* Read the BRIN tuples from the shared tuplesort, sorted by block number.
@@ -2452,7 +2444,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
* XXX We can't quite free the BrinTuple, though, because that's a field
* in BrinSortTuple.
*/
- while ((btup = tuplesort_getbrintuple(state->bs_spool_out->sortstate, &tuplen, true)) != NULL)
+ while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
{
/*
* We should not get two summaries for the same range. The workers
@@ -2494,7 +2486,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
prevblkno = btup->bt_blkno;
}
- tuplesort_end(state->bs_spool_out->sortstate);
+ tuplesort_end(spool->sortstate);
/*
* Next, accumulate WAL usage. (This must wait for the workers to finish,
diff --git a/src/backend/utils/sort/tuplesort.c b/src/backend/utils/sort/tuplesort.c
index 4c6b396a8a8..ab6353bdcd1 100644
--- a/src/backend/utils/sort/tuplesort.c
+++ b/src/backend/utils/sort/tuplesort.c
@@ -1328,7 +1328,6 @@ tuplesort_puttuple_common(Tuplesortstate *state, SortTuple *tuple, bool useAbbre
break;
default:
- Assert(false);
elog(ERROR, "invalid tuplesort state");
break;
}
@@ -1463,7 +1462,6 @@ tuplesort_performsort(Tuplesortstate *state)
break;
default:
- Assert(false);
elog(ERROR, "invalid tuplesort state");
break;
}
@@ -1720,7 +1718,6 @@ tuplesort_gettuple_common(Tuplesortstate *state, bool forward,
return false;
default:
- Assert(false);
elog(ERROR, "invalid tuplesort state");
return false; /* keep compiler quiet */
}
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 343ed4bbc54..525cc01b474 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -161,9 +161,8 @@ typedef struct
/*
* Computing BrinTuple size with only the tuple is difficult, so we want to track
- * the length for r referenced by SortTuple. That's what BrinSortTuple is meant
- * to do - it's essentially a BrinTuple prefixed by length. We only write the
- * BrinTuple to the logtapes, though.
+ * the length referenced by the SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by its length.
*/
typedef struct BrinSortTuple
{
@@ -796,11 +795,11 @@ tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
/* allocate space for the whole BRIN sort tuple */
bstup = palloc(BRINSORTTUPLE_SIZE(size));
- stup.tuple = bstup;
bstup->tuplen = size;
memcpy(&bstup->tuple, tuple, size);
+ stup.tuple = bstup;
stup.datum1 = tuple->bt_blkno;
stup.isnull1 = false;
@@ -1723,8 +1722,8 @@ comparetup_index_brin(const SortTuple *a, const SortTuple *b,
BrinTuple *tuple1;
BrinTuple *tuple2;
- tuple1 = &((BrinSortTuple *) a)->tuple;
- tuple2 = &((BrinSortTuple *) b)->tuple;
+ tuple1 = &((BrinSortTuple *) (a->tuple))->tuple;
+ tuple2 = &((BrinSortTuple *) (b->tuple))->tuple;
if (tuple1->bt_blkno > tuple2->bt_blkno)
return 1;
--
2.41.0
v3-0003-simplify-comparetup_index_brin.patchtext/x-patch; charset=UTF-8; name=v3-0003-simplify-comparetup_index_brin.patchDownload
From f5a5bf18e3049e21f2b2cc62f7cab7fb892bf3a6 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 7 Nov 2023 19:07:48 +0100
Subject: [PATCH v3 3/4] simplify comparetup_index_brin
---
src/backend/utils/sort/tuplesortvariants.c | 11 ++++-------
1 file changed, 4 insertions(+), 7 deletions(-)
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 525cc01b474..9b3a70e6ccf 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -1719,15 +1719,12 @@ static int
comparetup_index_brin(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state)
{
- BrinTuple *tuple1;
- BrinTuple *tuple2;
+ Assert(TuplesortstateGetPublic(state)->haveDatum1);
- tuple1 = &((BrinSortTuple *) (a->tuple))->tuple;
- tuple2 = &((BrinSortTuple *) (b->tuple))->tuple;
-
- if (tuple1->bt_blkno > tuple2->bt_blkno)
+ if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1))
return 1;
- else if (tuple1->bt_blkno < tuple2->bt_blkno)
+
+ if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1))
return -1;
/* silence compilers */
--
2.41.0
v3-0004-remove-tableam-changes.patchtext/x-patch; charset=UTF-8; name=v3-0004-remove-tableam-changes.patchDownload
From 7c9be5a2a71a0eead626c8f9ac248008bc6db3e9 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 7 Nov 2023 20:02:03 +0100
Subject: [PATCH v3 4/4] remove tableam changes
---
src/backend/access/brin/brin.c | 115 +++++++++++++++++++++++++---
src/backend/access/nbtree/nbtsort.c | 2 +-
src/backend/access/table/tableam.c | 49 ++----------
src/backend/executor/nodeSeqscan.c | 3 +-
src/include/access/relscan.h | 1 -
src/include/access/tableam.h | 9 +--
6 files changed, 115 insertions(+), 64 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index d54d39a535e..daadd051e30 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -2334,7 +2334,7 @@ _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
table_parallelscan_initialize(heap,
ParallelTableScanFromBrinShared(brinshared),
- snapshot, brinshared->pagesPerRange);
+ snapshot);
/*
* Store shared tuplesort-private state, for which we reserved space.
@@ -2413,6 +2413,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
{
int i;
BrinTuple *btup;
+ BrinMemTuple *memtuple = NULL;
Size tuplen;
BrinShared *brinshared = brinleader->brinshared;
BlockNumber prevblkno = InvalidBlockNumber;
@@ -2432,13 +2433,19 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
tuplesort_performsort(spool->sortstate);
+ /*
+ * Initialize BrinMemTuple we'll use to union summaries from workers
+ * (in case they happened to produce parts of the same paga range).
+ */
+ memtuple = brin_new_memtuple(state->bs_bdesc);
+
/*
* Read the BRIN tuples from the shared tuplesort, sorted by block number.
* That probably gives us an index that is cheaper to scan, thanks to mostly
* getting data from the same index page as before.
*
* FIXME This probably needs some memory management fixes - we're reading
- * tuples from the tuplesort, we're allocating an emty tuple, and so on.
+ * tuples from the tuplesort, we're allocating an empty tuple, and so on.
* Probably better to release this memory.
*
* XXX We can't quite free the BrinTuple, though, because that's a field
@@ -2446,14 +2453,65 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
*/
while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
{
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
/*
- * We should not get two summaries for the same range. The workers
- * are producing ranges for non-overlapping sections of the table.
+ * Do we need to union summaries for the same page range?
+ *
+ * If this is the first brin tuple we read, then just deform it into
+ * the memtuple, and continue with the next one from tuplesort. We
+ * however may need to insert empty summaries into the index.
+ *
+ * If it's the same block as the last we saw, we simply union the
+ * brin tuple into it, and we're done - we don't even need to insert
+ * empty ranges, because that was done earlier when we saw the first
+ * brin tuple (for this range).
+ *
+ * Finally, if it's not the first brin tuple, and it's not the same
+ * page range, we need to do the insert and then deform the tuple
+ * into the memtuple. Then we'll insert empty ranges before the
+ * new brin tuple, if needed.
*/
- Assert(btup->bt_blkno != prevblkno);
+ if (prevblkno == InvalidBlockNumber)
+ {
+ /* First brin tuples, just deform into memtuple. */
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
- /* Ranges should be multiples of pages_per_range for the index. */
- Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+ /* continue to insert empty pages before thisblock */
+ }
+ else if (memtuple->bt_blkno == btup->bt_blkno)
+ {
+ /*
+ * Not the first brin tuple, but same page range as the previous
+ * one, so we can merge it into the memtuple.
+ */
+ union_tuples(state->bs_bdesc, memtuple, btup);
+ continue;
+ }
+ else
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ /*
+ * We got brin tuple for a different page range, so form a brin
+ * tuple from the memtuple, insert it, and re-init the memtuple
+ * from the new brin tuple.
+ */
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ /* free the formed on-disk tuple */
+ pfree(tmp);
+
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
/* Fill empty ranges for all ranges missing in the tuplesort. */
prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
@@ -2480,14 +2538,51 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
emptyTuple->bt_blkno, emptyTuple, emptySize);
}
- brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
- &state->bs_currentInsertBuf, btup->bt_blkno, btup, tuplen);
-
prevblkno = btup->bt_blkno;
}
tuplesort_end(spool->sortstate);
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ /**/
+ if (prevblkno != InvalidBlockNumber)
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ pfree(tmp);
+ }
+
/*
* Next, accumulate WAL usage. (This must wait for the workers to finish,
* or we might get incomplete data.)
diff --git a/src/backend/access/nbtree/nbtsort.c b/src/backend/access/nbtree/nbtsort.c
index 6241baeea86..c2665fce411 100644
--- a/src/backend/access/nbtree/nbtsort.c
+++ b/src/backend/access/nbtree/nbtsort.c
@@ -1575,7 +1575,7 @@ _bt_begin_parallel(BTBuildState *buildstate, bool isconcurrent, int request)
btshared->brokenhotchain = false;
table_parallelscan_initialize(btspool->heap,
ParallelTableScanFromBTShared(btshared),
- snapshot, InvalidBlockNumber);
+ snapshot);
/*
* Store shared tuplesort-private state, for which we reserved space.
diff --git a/src/backend/access/table/tableam.c b/src/backend/access/table/tableam.c
index 4af0d433e9d..c6bdb7e1c68 100644
--- a/src/backend/access/table/tableam.c
+++ b/src/backend/access/table/tableam.c
@@ -153,9 +153,9 @@ table_parallelscan_estimate(Relation rel, Snapshot snapshot)
void
table_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan,
- Snapshot snapshot, BlockNumber chunk_factor)
+ Snapshot snapshot)
{
- Size snapshot_off = rel->rd_tableam->parallelscan_initialize(rel, pscan, chunk_factor);
+ Size snapshot_off = rel->rd_tableam->parallelscan_initialize(rel, pscan);
pscan->phs_snapshot_off = snapshot_off;
@@ -395,21 +395,16 @@ table_block_parallelscan_estimate(Relation rel)
}
Size
-table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan, BlockNumber chunk_factor)
+table_block_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan)
{
ParallelBlockTableScanDesc bpscan = (ParallelBlockTableScanDesc) pscan;
bpscan->base.phs_relid = RelationGetRelid(rel);
bpscan->phs_nblocks = RelationGetNumberOfBlocks(rel);
- bpscan->phs_chunk_factor = chunk_factor;
- /* compare phs_syncscan initialization to similar logic in initscan
- *
- * Disable sync scans if the chunk factor is set (valid block number).
- */
+ /* compare phs_syncscan initialization to similar logic in initscan */
bpscan->base.phs_syncscan = synchronize_seqscans &&
!RelationUsesLocalBuffers(rel) &&
- (bpscan->phs_nblocks > NBuffers / 4) &&
- !BlockNumberIsValid(bpscan->phs_chunk_factor);
+ bpscan->phs_nblocks > NBuffers / 4;
SpinLockInit(&bpscan->phs_mutex);
bpscan->phs_startblock = InvalidBlockNumber;
pg_atomic_init_u64(&bpscan->phs_nallocated, 0);
@@ -464,25 +459,6 @@ table_block_parallelscan_startblock_init(Relation rel,
pbscanwork->phsw_chunk_size = Min(pbscanwork->phsw_chunk_size,
PARALLEL_SEQSCAN_MAX_CHUNK_SIZE);
- /*
- * If the chunk size factor is set, we need to make sure the chunk size is
- * a multiple of that value. We round the chunk size to the nearest chunk
- * factor multiple, at least one chunk_factor.
- *
- * XXX Note this may override PARALLEL_SEQSCAN_MAX_CHUNK_SIZE, in case the
- * chunk factor (e.g. BRIN pages_per_range) is larger.
- */
- if (pbscan->phs_chunk_factor != InvalidBlockNumber)
- {
- /* nearest (smaller) multiple of chunk_factor */
- pbscanwork->phsw_chunk_size
- = pbscan->phs_chunk_factor * (pbscanwork->phsw_chunk_size / pbscan->phs_chunk_factor);
-
- /* but at least one chunk_factor */
- pbscanwork->phsw_chunk_size = Max(pbscanwork->phsw_chunk_size,
- pbscan->phs_chunk_factor);
- }
-
retry:
/* Grab the spinlock. */
SpinLockAcquire(&pbscan->phs_mutex);
@@ -599,21 +575,6 @@ table_block_parallelscan_nextpage(Relation rel,
(pbscanwork->phsw_chunk_size * PARALLEL_SEQSCAN_RAMPDOWN_CHUNKS))
pbscanwork->phsw_chunk_size >>= 1;
- /*
- * We need to make sure the new chunk_size is still a suitable multiple
- * of chunk_factor.
- */
- if (pbscan->phs_chunk_factor != InvalidBlockNumber)
- {
- /* nearest (smaller) multiple of chunk_factor */
- pbscanwork->phsw_chunk_size
- = pbscan->phs_chunk_factor * (pbscanwork->phsw_chunk_size / pbscan->phs_chunk_factor);
-
- /* but at least one chunk_factor */
- pbscanwork->phsw_chunk_size = Max(pbscanwork->phsw_chunk_size,
- pbscan->phs_chunk_factor);
- }
-
nallocated = pbscanwork->phsw_nallocated =
pg_atomic_fetch_add_u64(&pbscan->phs_nallocated,
pbscanwork->phsw_chunk_size);
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 529a7ed3284..49a5933aff6 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -262,8 +262,7 @@ ExecSeqScanInitializeDSM(SeqScanState *node,
pscan = shm_toc_allocate(pcxt->toc, node->pscan_len);
table_parallelscan_initialize(node->ss.ss_currentRelation,
pscan,
- estate->es_snapshot,
- InvalidBlockNumber);
+ estate->es_snapshot);
shm_toc_insert(pcxt->toc, node->ss.ps.plan->plan_node_id, pscan);
node->ss.ss_currentScanDesc =
table_beginscan_parallel(node->ss.ss_currentRelation, pscan);
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 72a20d882f5..d03360eac04 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -79,7 +79,6 @@ typedef struct ParallelBlockTableScanDescData
BlockNumber phs_nblocks; /* # blocks in relation at start of scan */
slock_t phs_mutex; /* mutual exclusion for setting startblock */
BlockNumber phs_startblock; /* starting block number */
- BlockNumber phs_chunk_factor; /* chunks need to be a multiple of this */
pg_atomic_uint64 phs_nallocated; /* number of blocks allocated to
* workers so far. */
} ParallelBlockTableScanDescData;
diff --git a/src/include/access/tableam.h b/src/include/access/tableam.h
index d94e4d32aa1..dbb709b56ce 100644
--- a/src/include/access/tableam.h
+++ b/src/include/access/tableam.h
@@ -390,8 +390,7 @@ typedef struct TableAmRoutine
* relation.
*/
Size (*parallelscan_initialize) (Relation rel,
- ParallelTableScanDesc pscan,
- BlockNumber chunk_factor);
+ ParallelTableScanDesc pscan);
/*
* Reinitialize `pscan` for a new scan. `rel` will be the same relation as
@@ -1149,8 +1148,7 @@ extern Size table_parallelscan_estimate(Relation rel, Snapshot snapshot);
*/
extern void table_parallelscan_initialize(Relation rel,
ParallelTableScanDesc pscan,
- Snapshot snapshot,
- BlockNumber chunk_factor);
+ Snapshot snapshot);
/*
* Begin a parallel scan. `pscan` needs to have been initialized with
@@ -2066,8 +2064,7 @@ extern void simple_table_tuple_update(Relation rel, ItemPointer otid,
extern Size table_block_parallelscan_estimate(Relation rel);
extern Size table_block_parallelscan_initialize(Relation rel,
- ParallelTableScanDesc pscan,
- BlockNumber chunk_factor);
+ ParallelTableScanDesc pscan);
extern void table_block_parallelscan_reinitialize(Relation rel,
ParallelTableScanDesc pscan);
extern BlockNumber table_block_parallelscan_nextpage(Relation rel,
--
2.41.0
On Wed, 8 Nov 2023 at 12:03, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
here's an updated patch, addressing the review comments, and reworking
how the work is divided between the workers & leader etc.
Thanks!
In general I'm quite happy with the current state, and I believe it's
fairly close to be committable.
Are you planning on committing the patches separately, or squashed? I
won't have much time this week for reviewing the patch, and it seems
like these patches are incremental, so some guidance on what you want
to be reviewed would be appreciated.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech)
On 11/12/23 10:38, Matthias van de Meent wrote:
On Wed, 8 Nov 2023 at 12:03, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
here's an updated patch, addressing the review comments, and reworking
how the work is divided between the workers & leader etc.Thanks!
In general I'm quite happy with the current state, and I believe it's
fairly close to be committable.Are you planning on committing the patches separately, or squashed? I
won't have much time this week for reviewing the patch, and it seems
like these patches are incremental, so some guidance on what you want
to be reviewed would be appreciated.
Definitely squashed. I only kept them separate to make it more obvious
what the changes are.
If you need more time for a review, I can certainly wait. No rush.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On Wed, 8 Nov 2023 at 12:03, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
here's an updated patch, addressing the review comments, and reworking
how the work is divided between the workers & leader etc.0001 is just v2, rebased to current master
0002 and 0003 address most of the issues, in particular it
- removes the unnecessary spool
- fixes bs_reltuples type to double
- a couple comments are reworded to be clearer
- changes loop/condition in brinbuildCallbackParallel
- removes asserts added for debugging
- fixes cast in comparetup_index_brin
- 0003 then simplifies comparetup_index_brin
- I haven't inlined the tuplesort_puttuple_common parameter
(didn't seem worth it)
OK, thanks
0004 Reworks how the work is divided between workers and combined by the
leader. It undoes the tableam.c changes that attempted to divide the
relation into chunks matching the BRIN ranges, and instead merges the
results in the leader (using the BRIN "union" function).
That's OK.
I haven't done any indentation fixes yet.
I did fairly extensive testing, using pageinspect to compare indexes
built with/without parallelism. More testing is needed, but it seems to
work fine (with other opclasses and so on).
After code-only review, here are some comments:
+++ b/src/backend/access/brin/brin.c [...] +/* Magic numbers for parallel state sharing */ +#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001) +#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)
These shm keys use the same constants also in use in
access/nbtree/nbtsort.c. While this shouldn't be an issue in normal
operations, I'd prefer if we didn't actively introduce conflicting
identifiers when we still have significant amounts of unused values
remaining.
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000003)
This is the fourth definition of a PARALLEL%_KEY_QUERY_TEXT, the
others being in access/nbtree/nbtsort.c (value 0xA000000000000004, one
more than brin's), backend/executor/execParallel.c
(0xE000000000000008), and PARALLEL_VACUUM_KEY_QUERY_TEXT (0x3) (though
I've not checked that their uses are exactly the same, I'd expect at
least btree to match mostly, if not fully, 1:1).
I think we could probably benefit from a less ad-hoc sharing of query
texts. I don't think that needs to happen specifically in this patch,
but I think it's something to keep in mind in future efforts.
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state) [...] + BrinSpool *spool = state->bs_spool; [...] + if (!state) + return;
I think the assignment to spool should be moved to below this
condition, as _brin_begin_parallel calls this with state=NULL when it
can't launch parallel workers, which will cause issues here.
+ state->bs_numtuples = brinshared->indtuples;
My IDE complains about bs_numtuples being an integer. This is a
pre-existing issue, but still valid: we can hit an overflow on tables
with pages_per_range=1 and relsize >= 2^31 pages. Extremely unlikely,
but problematic nonetheless.
+ * FIXME This probably needs some memory management fixes - we're reading + * tuples from the tuplesort, we're allocating an emty tuple, and so on. + * Probably better to release this memory.
This should probably be resolved.
I also noticed that this is likely to execute `union_tuples` many
times when pages_per_range is coprime with the parallel table scan's
block stride (or when we for other reasons have many tuples returned
for each range); and this `union_tuples` internally allocates and
deletes its own memory context for its deserialization of the 'b'
tuple. I think we should just pass a scratch context instead, so that
we don't have the overhead of continously creating then deleting the
same memory context.
+++ b/src/backend/catalog/index.c [...] - indexRelation->rd_rel->relam == BTREE_AM_OID) + (indexRelation->rd_rel->relam == BTREE_AM_OID || + indexRelation->rd_rel->relam == BRIN_AM_OID))
I think this needs some more effort. I imagine a new
IndexAmRoutine->amcanbuildparallel is more appropriate than this
hard-coded list - external indexes may want to utilize the parallel
index creation planner facility, too.
Some notes:
As a project PostgreSQL seems to be trying to move away from
hardcoding heap into everything in favour of the more AM-agnostic
'table'. I suggest replacing all mentions of "heap" in the arguments
with "table", to reduce the work future maintainers need to do to fix
this. Even when this AM is mostly targetted towards the heap AM, other
AMs (such as those I've heard of that were developed internally at
EDB) use the same block-addressing that heap does, and should thus be
compatible with BRIN. Thus, "heap" is not a useful name here.
There are 2 new mentions of "tuplestore" in the patch, while the
structure used is tuplesort: one on form_and_spill_tuple, and one on
brinbuildCallbackParallel. Please update those comments.
That's it for code review. I'll do some performance comparisons and
testing soon, too.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech)
On 11/20/23 20:48, Matthias van de Meent wrote:
On Wed, 8 Nov 2023 at 12:03, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
here's an updated patch, addressing the review comments, and reworking
how the work is divided between the workers & leader etc.0001 is just v2, rebased to current master
0002 and 0003 address most of the issues, in particular it
- removes the unnecessary spool
- fixes bs_reltuples type to double
- a couple comments are reworded to be clearer
- changes loop/condition in brinbuildCallbackParallel
- removes asserts added for debugging
- fixes cast in comparetup_index_brin
- 0003 then simplifies comparetup_index_brin
- I haven't inlined the tuplesort_puttuple_common parameter
(didn't seem worth it)OK, thanks
0004 Reworks how the work is divided between workers and combined by the
leader. It undoes the tableam.c changes that attempted to divide the
relation into chunks matching the BRIN ranges, and instead merges the
results in the leader (using the BRIN "union" function).That's OK.
I haven't done any indentation fixes yet.
I did fairly extensive testing, using pageinspect to compare indexes
built with/without parallelism. More testing is needed, but it seems to
work fine (with other opclasses and so on).After code-only review, here are some comments:
+++ b/src/backend/access/brin/brin.c [...] +/* Magic numbers for parallel state sharing */ +#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001) +#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)These shm keys use the same constants also in use in
access/nbtree/nbtsort.c. While this shouldn't be an issue in normal
operations, I'd prefer if we didn't actively introduce conflicting
identifiers when we still have significant amounts of unused values
remaining.
Hmmm. Is there some rule of thumb how to pick these key values? I see
nbtsort.c uses 0xA prefix, execParallel.c uses 0xE, while parallel.c
ended up using 0xFFFFFFFFFFFF as prefix. I've user 0xB, simply because
BRIN also starts with B.
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000003)
This is the fourth definition of a PARALLEL%_KEY_QUERY_TEXT, the
others being in access/nbtree/nbtsort.c (value 0xA000000000000004, one
more than brin's), backend/executor/execParallel.c
(0xE000000000000008), and PARALLEL_VACUUM_KEY_QUERY_TEXT (0x3) (though
I've not checked that their uses are exactly the same, I'd expect at
least btree to match mostly, if not fully, 1:1).
I think we could probably benefit from a less ad-hoc sharing of query
texts. I don't think that needs to happen specifically in this patch,
but I think it's something to keep in mind in future efforts.
I'm afraid I don't quite get what you mean by "ad hoc sharing of query
texts". Are you saying we shouldn't propagate the query text to the
parallel workers? Why? Or what's the proper solution?
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state) [...] + BrinSpool *spool = state->bs_spool; [...] + if (!state) + return;I think the assignment to spool should be moved to below this
condition, as _brin_begin_parallel calls this with state=NULL when it
can't launch parallel workers, which will cause issues here.
Good catch! I wonder if we have tests that might trigger this, say by
setting max_parallel_maintenance_workers > 0 while no workers allowed.
+ state->bs_numtuples = brinshared->indtuples;
My IDE complains about bs_numtuples being an integer. This is a
pre-existing issue, but still valid: we can hit an overflow on tables
with pages_per_range=1 and relsize >= 2^31 pages. Extremely unlikely,
but problematic nonetheless.
True. I think I've been hesitant to make this a double because it seems
a bit weird to do +1 with a double, and at some point (d == d+1). But
this seems safe, we're guaranteed to be far away from that threshold.
+ * FIXME This probably needs some memory management fixes - we're reading + * tuples from the tuplesort, we're allocating an emty tuple, and so on. + * Probably better to release this memory.This should probably be resolved.
AFAICS that comment is actually inaccurate/stale, sorry about that. The
code actually allocates (and resets) a single memtuple, and also
emptyTuple. So I think that's OK, I've removed the comment.
I also noticed that this is likely to execute `union_tuples` many
times when pages_per_range is coprime with the parallel table scan's
block stride (or when we for other reasons have many tuples returned
for each range); and this `union_tuples` internally allocates and
deletes its own memory context for its deserialization of the 'b'
tuple. I think we should just pass a scratch context instead, so that
we don't have the overhead of continously creating then deleting the
same memory context
Perhaps. Looking at the code, isn't it a bit strange how union_tuples
uses the context? It creates the context, calls brin_deform_tuple in
that context, but then the rest of the function (including datumCopy and
similar stuff) happens in the caller's context ...
However, I don't think the number of union_tuples calls is likely to be
very high, especially for large tables. Because we split the table into
2048 chunks, and then cap the chunk size by 8192. For large tables
(where this matters) we're likely close to 8192.
+++ b/src/backend/catalog/index.c [...] - indexRelation->rd_rel->relam == BTREE_AM_OID) + (indexRelation->rd_rel->relam == BTREE_AM_OID || + indexRelation->rd_rel->relam == BRIN_AM_OID))I think this needs some more effort. I imagine a new
IndexAmRoutine->amcanbuildparallel is more appropriate than this
hard-coded list - external indexes may want to utilize the parallel
index creation planner facility, too.
Good idea. I added the IndexAmRoutine flag and used it here.
Some notes:
As a project PostgreSQL seems to be trying to move away from
hardcoding heap into everything in favour of the more AM-agnostic
'table'. I suggest replacing all mentions of "heap" in the arguments
with "table", to reduce the work future maintainers need to do to fix
this. Even when this AM is mostly targetted towards the heap AM, other
AMs (such as those I've heard of that were developed internally at
EDB) use the same block-addressing that heap does, and should thus be
compatible with BRIN. Thus, "heap" is not a useful name here.
I'm not against doing that, but I'd prefer to do that in a separate
patch. There's a bunch of preexisting heap references, so and I don't
want to introduce inconsistency (patch using table, old code heap) nor
do I want to tweak unrelated code.
There are 2 new mentions of "tuplestore" in the patch, while the
structure used is tuplesort: one on form_and_spill_tuple, and one on
brinbuildCallbackParallel. Please update those comments.That's it for code review. I'll do some performance comparisons and
testing soon, too.
Thanks! Attached is a patch squashing the previous version into a single
v3 commit, with fixes for your review in a separate commit.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
v4-0001-parallel-CREATE-INDEX-for-BRIN-v3.patchtext/x-patch; charset=UTF-8; name=v4-0001-parallel-CREATE-INDEX-for-BRIN-v3.patchDownload
From dc59165200834ce9a9756055e7dd7f492977e223 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 7 Nov 2023 17:04:28 +0100
Subject: [PATCH v4 1/2] parallel CREATE INDEX for BRIN v3
---
src/backend/access/brin/brin.c | 868 ++++++++++++++++++++-
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 3 +-
src/backend/utils/sort/tuplesortvariants.c | 207 +++++
src/include/access/brin.h | 3 +
src/include/utils/tuplesort.h | 11 +
6 files changed, 1091 insertions(+), 5 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index f0eac078e0b..0d3d728c9bf 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -33,6 +33,7 @@
#include "postmaster/autovacuum.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -40,7 +41,118 @@
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+#include "utils/tuplesort.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xA000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xA000000000000005)
+
+/*
+ * Status record for spooling/sorting phase.
+ */
+typedef struct BrinSpool
+{
+ Tuplesortstate *sortstate; /* state data for tuplesort.c */
+ Relation heap;
+ Relation index;
+} BrinSpool;
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist for
+ * the benefit of worker processes that need to create state corresponding
+ * to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+ int scantuplesortstates;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN 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 the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
+} BrinShared;
+
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* 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).
+ *
+ * brinshared is the shared state for entire build. sharedsort is the
+ * shared, tuplesort-managed state passed to each process tuplesort.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is required.
+ */
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -50,12 +162,22 @@ typedef struct BrinBuildState
{
Relation bs_irel;
int bs_numtuples;
+ double bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader process has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BrinSpool *bs_spool;
} BrinBuildState;
/*
@@ -76,6 +198,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -83,6 +206,20 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinSpool *brinspool,
+ BrinShared *brinshared,
+ Sharedsort *sharedsort,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -820,6 +957,63 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a shared tuplestore, and leave the insertion up to the leader (which may
+ * reorder them a bit etc.). The callback also does not generate empty ranges,
+ * those may be added by the leader when merging results from workers.
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; we do not create emptry
+ * BRIN ranges here - the leader is responsible for filling them in.
+ */
+ if (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the tuplesort */
+ form_and_spill_tuple(state);
+
+ /*
+ * Set state to correspond to the next range (for this block).
+ *
+ * This skips ranges that are either empty (and so we don't get any
+ * tuples to summarize), or processes by other workers. We can't
+ * differentiate those cases here easily, so we leave it up to the
+ * leader to fill empty ranges where needed.
+ */
+ state->bs_currRangeStart
+ = state->bs_pagesPerRange * (thisblock / state->bs_pagesPerRange);
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -881,18 +1075,89 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
revmap = brinRevmapInitialize(index, &pagesPerRange);
state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool->heap = heap;
+ state->bs_spool->index = index;
+
+ /*
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
+ */
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
+
/*
* Now scan the relation. No syncscan allowed here because we want the
* heap blocks in physical order.
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
- form_and_insert_tuple(state);
+ /*
+ * If parallel build requested and at least one worker process was
+ * successfully launched, set up coordination state
+ */
+ if (state->bs_leader)
+ {
+ SortCoordinate coordinate;
+
+ coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
+ coordinate->isWorker = false;
+ coordinate->nParticipants =
+ state->bs_leader->nparticipanttuplesorts;
+ coordinate->sharedsort = state->bs_leader->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.
+ */
+ state->bs_spool->sortstate =
+ tuplesort_begin_index_brin(heap, index,
+ maintenance_work_mem, coordinate,
+ TUPLESORT_NONE);
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read all
+ * tuples from the shared tuplesort and insert them into the index.
+ */
+ _brin_end_parallel(state->bs_leader, state);
+ }
+ else /* no parallel index build, just do the usual thing */
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
+ form_and_insert_tuple(state);
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
+ }
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1312,12 +1577,16 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
+ state->bs_spool = NULL;
return state;
}
@@ -1609,6 +1878,32 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a (shared) tuplestore (the leader will insert it
+ * into the index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ /* don't insert empty tuples in parallel build */
+ if (state->bs_dtuple->bt_empty_range)
+ return;
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ /* write the BRIN tuple to the tuplesort */
+ tuplesort_putbrintuple(state->bs_spool->sortstate, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -1928,3 +2223,568 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ int scantuplesortstates;
+ Snapshot snapshot;
+ Size estbrinshared;
+ Size estsort;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ request);
+
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+ estsort = tuplesort_estimate_shared(scantuplesortstates);
+ shm_toc_estimate_chunk(&pcxt->estimator, estsort);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->scantuplesortstates = scantuplesortstates;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ 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);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipanttuplesorts = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipanttuplesorts++;
+ brinleader->brinshared = brinshared;
+ brinleader->sharedsort = sharedsort;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinTuple *btup;
+ BrinMemTuple *memtuple = NULL;
+ Size tuplen;
+ BrinShared *brinshared = brinleader->brinshared;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+ BrinSpool *spool = state->bs_spool;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ tuplesort_performsort(spool->sortstate);
+
+ /*
+ * Initialize BrinMemTuple we'll use to union summaries from workers
+ * (in case they happened to produce parts of the same paga range).
+ */
+ memtuple = brin_new_memtuple(state->bs_bdesc);
+
+ /*
+ * Read the BRIN tuples from the shared tuplesort, sorted by block number.
+ * That probably gives us an index that is cheaper to scan, thanks to mostly
+ * getting data from the same index page as before.
+ *
+ * FIXME This probably needs some memory management fixes - we're reading
+ * tuples from the tuplesort, we're allocating an empty tuple, and so on.
+ * Probably better to release this memory.
+ *
+ * XXX We can't quite free the BrinTuple, though, because that's a field
+ * in BrinSortTuple.
+ */
+ while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
+ {
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
+ /*
+ * Do we need to union summaries for the same page range?
+ *
+ * If this is the first brin tuple we read, then just deform it into
+ * the memtuple, and continue with the next one from tuplesort. We
+ * however may need to insert empty summaries into the index.
+ *
+ * If it's the same block as the last we saw, we simply union the
+ * brin tuple into it, and we're done - we don't even need to insert
+ * empty ranges, because that was done earlier when we saw the first
+ * brin tuple (for this range).
+ *
+ * Finally, if it's not the first brin tuple, and it's not the same
+ * page range, we need to do the insert and then deform the tuple
+ * into the memtuple. Then we'll insert empty ranges before the
+ * new brin tuple, if needed.
+ */
+ if (prevblkno == InvalidBlockNumber)
+ {
+ /* First brin tuples, just deform into memtuple. */
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+ else if (memtuple->bt_blkno == btup->bt_blkno)
+ {
+ /*
+ * Not the first brin tuple, but same page range as the previous
+ * one, so we can merge it into the memtuple.
+ */
+ union_tuples(state->bs_bdesc, memtuple, btup);
+ continue;
+ }
+ else
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ /*
+ * We got brin tuple for a different page range, so form a brin
+ * tuple from the memtuple, insert it, and re-init the memtuple
+ * from the new brin tuple.
+ */
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ /* free the formed on-disk tuple */
+ pfree(tmp);
+
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ prevblkno = btup->bt_blkno;
+ }
+
+ tuplesort_end(spool->sortstate);
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ /**/
+ if (prevblkno != InvalidBlockNumber)
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ pfree(tmp);
+ }
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /* Allocate memory and initialize private spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = buildstate->bs_spool->heap;
+ buildstate->bs_spool->index = buildstate->bs_spool->index;
+
+ /*
+ * 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 / brinleader->nparticipanttuplesorts;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool, brinleader->brinshared,
+ brinleader->sharedsort, heap, index, sortmem, true);
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
+ BrinShared *brinshared, Sharedsort *sharedsort,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ SortCoordinate coordinate;
+ TableScanDesc 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 */
+ brinspool->sortstate = tuplesort_begin_index_brin(brinspool->heap,
+ brinspool->index,
+ sortmem, coordinate,
+ TUPLESORT_NONE);
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* insert the last item */
+ form_and_spill_tuple(state);
+
+ /* sort the BRIN ranges built by this worker */
+ tuplesort_performsort(brinspool->sortstate);
+
+ state->bs_reltuples += reltuples;
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+
+ tuplesort_end(brinspool->sortstate);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL, brinshared->pagesPerRange);
+
+ /* Initialize worker's own spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = heapRel;
+ buildstate->bs_spool->index = indexRel;
+
+ /* Look up shared state private to tuplesort.c */
+ sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
+ tuplesort_attach_shared(sharedsort, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /*
+ * 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 / brinshared->scantuplesortstates;
+
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool,
+ brinshared, sharedsort,
+ heapRel, indexRel, sortmem, false);
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 194a1207be6..d78314062e0 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -145,6 +146,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 143fae01ebd..37e4305d50a 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2982,7 +2982,8 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ (indexRelation->rd_rel->relam == BTREE_AM_OID ||
+ indexRelation->rd_rel->relam == BRIN_AM_OID))
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 2cd508e5130..9b3a70e6ccf 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -19,6 +19,7 @@
#include "postgres.h"
+#include "access/brin_tuple.h"
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -43,6 +44,8 @@ static void removeabbrev_cluster(Tuplesortstate *state, SortTuple *stups,
int count);
static void removeabbrev_index(Tuplesortstate *state, SortTuple *stups,
int count);
+static void removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups,
+ int count);
static void removeabbrev_datum(Tuplesortstate *state, SortTuple *stups,
int count);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
@@ -69,10 +72,16 @@ static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash_tiebreak(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
+static int comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state);
static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len);
+static void writetup_index_brin(Tuplesortstate *state, LogicalTape *tape,
+ SortTuple *stup);
+static void readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_datum_tiebreak(const SortTuple *a, const SortTuple *b,
@@ -128,6 +137,16 @@ typedef struct
uint32 max_buckets;
} TuplesortIndexHashArg;
+/*
+ * Data struture pointed by "TuplesortPublic.arg" for the index_brin subcase.
+ */
+typedef struct
+{
+ TuplesortIndexArg index;
+
+ /* XXX do we need something here? */
+} TuplesortIndexBrinArg;
+
/*
* Data struture pointed by "TuplesortPublic.arg" for the Datum case.
* Set by tuplesort_begin_datum and used only by the DatumTuple routines.
@@ -140,6 +159,21 @@ typedef struct
int datumTypeLen;
} TuplesortDatumArg;
+/*
+ * Computing BrinTuple size with only the tuple is difficult, so we want to track
+ * the length referenced by the SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by its length.
+ */
+typedef struct BrinSortTuple
+{
+ Size tuplen;
+ BrinTuple tuple;
+} BrinSortTuple;
+
+/* Size of the BrinSortTuple, given length of the BrinTuple. */
+#define BRINSORTTUPLE_SIZE(len) (offsetof(BrinSortTuple, tuple) + (len))
+
+
Tuplesortstate *
tuplesort_begin_heap(TupleDesc tupDesc,
int nkeys, AttrNumber *attNums,
@@ -527,6 +561,47 @@ tuplesort_begin_index_gist(Relation heapRel,
return state;
}
+Tuplesortstate *
+tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem,
+ SortCoordinate coordinate,
+ int sortopt)
+{
+ Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
+ sortopt);
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext;
+ TuplesortIndexBrinArg *arg;
+
+ oldcontext = MemoryContextSwitchTo(base->maincontext);
+ arg = (TuplesortIndexBrinArg *) palloc(sizeof(TuplesortIndexBrinArg));
+
+#ifdef TRACE_SORT
+ if (trace_sort)
+ elog(LOG,
+ "begin index sort: workMem = %d, randomAccess = %c",
+ workMem,
+ sortopt & TUPLESORT_RANDOMACCESS ? 't' : 'f');
+#endif
+
+ base->nKeys = 1; /* Only one sort column, the block number */
+
+ base->removeabbrev = removeabbrev_index_brin;
+ base->comparetup = comparetup_index_brin;
+ base->writetup = writetup_index_brin;
+ base->readtup = readtup_index_brin;
+ base->haveDatum1 = true;
+ base->arg = arg;
+
+ arg->index.heapRel = heapRel;
+ arg->index.indexRel = indexRel;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return state;
+}
+
Tuplesortstate *
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag, int workMem,
@@ -707,6 +782,35 @@ tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
!stup.isnull1);
}
+/*
+ * Collect one BRIN tuple while collecting input data for sort.
+ */
+void
+tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
+{
+ SortTuple stup;
+ BrinSortTuple *bstup;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->tuplecontext);
+
+ /* allocate space for the whole BRIN sort tuple */
+ bstup = palloc(BRINSORTTUPLE_SIZE(size));
+
+ bstup->tuplen = size;
+ memcpy(&bstup->tuple, tuple, size);
+
+ stup.tuple = bstup;
+ stup.datum1 = tuple->bt_blkno;
+ stup.isnull1 = false;
+
+ tuplesort_puttuple_common(state, &stup,
+ base->sortKeys &&
+ base->sortKeys->abbrev_converter &&
+ !stup.isnull1);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
/*
* Accept one Datum while collecting input data for sort.
*
@@ -850,6 +954,35 @@ tuplesort_getindextuple(Tuplesortstate *state, bool forward)
return (IndexTuple) stup.tuple;
}
+/*
+ * Fetch the next BRIN tuple in either forward or back direction.
+ * Returns NULL if no more tuples. Returned tuple belongs to tuplesort memory
+ * context, and must not be freed by caller. Caller may not rely on tuple
+ * remaining valid after any further manipulation of tuplesort.
+ */
+BrinTuple *
+tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
+ SortTuple stup;
+ BrinSortTuple *btup;
+
+ if (!tuplesort_gettuple_common(state, forward, &stup))
+ stup.tuple = NULL;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!stup.tuple)
+ return NULL;
+
+ btup = (BrinSortTuple *) stup.tuple;
+
+ *len = btup->tuplen;
+
+ return &btup->tuple;
+}
+
/*
* Fetch the next Datum in either forward or back direction.
* Returns false if no more datums.
@@ -1564,6 +1697,80 @@ readtup_index(Tuplesortstate *state, SortTuple *stup,
&stup->isnull1);
}
+/*
+ * Routines specialized for BrinTuple case
+ */
+
+static void
+removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
+{
+ int i;
+
+ for (i = 0; i < count; i++)
+ {
+ BrinSortTuple *tuple;
+
+ tuple = stups[i].tuple;
+ stups[i].datum1 = tuple->tuple.bt_blkno;
+ }
+}
+
+static int
+comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+{
+ Assert(TuplesortstateGetPublic(state)->haveDatum1);
+
+ if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1))
+ return 1;
+
+ if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1))
+ return -1;
+
+ /* silence compilers */
+ return 0;
+}
+
+static void
+writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
+
+ tuplen = tuplen + sizeof(tuplen);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->tuple, tuple->tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+}
+
+static void
+readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ unsigned int tuplen = len - sizeof(unsigned int);
+
+ /*
+ * Allocate space for the BRIN sort tuple, which is BrinTuple with an
+ * extra length field.
+ */
+ BrinSortTuple *tuple
+ = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
+
+ tuple->tuplen = tuplen;
+
+ LogicalTapeReadExact(tape, &tuple->tuple, tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
+ stup->tuple = (void *) tuple;
+
+ /* set up first-column key value, which is block number */
+ stup->datum1 = tuple->tuple.bt_blkno;
+}
+
/*
* Routines specialized for DatumTuple case
*/
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d51..3451ecb211f 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index 9ed2de76cd6..357eb35311d 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -21,6 +21,7 @@
#ifndef TUPLESORT_H
#define TUPLESORT_H
+#include "access/brin_tuple.h"
#include "access/itup.h"
#include "executor/tuptable.h"
#include "storage/dsm.h"
@@ -282,6 +283,9 @@ typedef struct
* The "index_hash" API is similar to index_btree, but the tuples are
* actually sorted by their hash codes not the raw data.
*
+ * The "index_brin" API is similar to index_btree, but the tuples are
+ * BrinTuple and are sorted by their block number not the raw data.
+ *
* Parallel sort callers are required to coordinate multiple tuplesort states
* in a leader process and one or more worker processes. The leader process
* must launch workers, and have each perform an independent "partial"
@@ -426,6 +430,10 @@ extern Tuplesortstate *tuplesort_begin_index_gist(Relation heapRel,
Relation indexRel,
int workMem, SortCoordinate coordinate,
int sortopt);
+extern Tuplesortstate *tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem, SortCoordinate coordinate,
+ int sortopt);
extern Tuplesortstate *tuplesort_begin_datum(Oid datumType,
Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag,
@@ -438,6 +446,7 @@ extern void tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup);
extern void tuplesort_putindextuplevalues(Tuplesortstate *state,
Relation rel, ItemPointer self,
const Datum *values, const bool *isnull);
+extern void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tup, Size len);
extern void tuplesort_putdatum(Tuplesortstate *state, Datum val,
bool isNull);
@@ -445,6 +454,8 @@ extern bool tuplesort_gettupleslot(Tuplesortstate *state, bool forward,
bool copy, TupleTableSlot *slot, Datum *abbrev);
extern HeapTuple tuplesort_getheaptuple(Tuplesortstate *state, bool forward);
extern IndexTuple tuplesort_getindextuple(Tuplesortstate *state, bool forward);
+extern BrinTuple *tuplesort_getbrintuple(Tuplesortstate *state, Size *len,
+ bool forward);
extern bool tuplesort_getdatum(Tuplesortstate *state, bool forward, bool copy,
Datum *val, bool *isNull, Datum *abbrev);
--
2.41.0
v4-0002-review-fixes.patchtext/x-patch; charset=UTF-8; name=v4-0002-review-fixes.patchDownload
From 625880de662181fd433eb583a00af0f16d57c420 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Wed, 22 Nov 2023 19:58:10 +0100
Subject: [PATCH v4 2/2] review fixes
---
contrib/bloom/blutils.c | 1 +
src/backend/access/brin/brin.c | 28 ++++++++-----------
src/backend/access/gin/ginutil.c | 1 +
src/backend/access/gist/gist.c | 1 +
src/backend/access/hash/hash.c | 1 +
src/backend/access/nbtree/nbtree.c | 1 +
src/backend/access/spgist/spgutils.c | 1 +
src/backend/catalog/index.c | 3 +-
src/include/access/amapi.h | 2 ++
.../modules/dummy_index_am/dummy_index_am.c | 1 +
10 files changed, 22 insertions(+), 18 deletions(-)
diff --git a/contrib/bloom/blutils.c b/contrib/bloom/blutils.c
index f23fbb1d9e0..7451fb1b3bb 100644
--- a/contrib/bloom/blutils.c
+++ b/contrib/bloom/blutils.c
@@ -122,6 +122,7 @@ blhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amparallelvacuumoptions =
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 0d3d728c9bf..e836cbaad0b 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -44,11 +44,11 @@
#include "utils/tuplesort.h"
/* Magic numbers for parallel state sharing */
-#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001)
-#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)
-#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000003)
-#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xA000000000000004)
-#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xA000000000000005)
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xB000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xB000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xB000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xB000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xB000000000000005)
/*
* Status record for spooling/sorting phase.
@@ -161,7 +161,7 @@ typedef struct BrinLeader
typedef struct BrinBuildState
{
Relation bs_irel;
- int bs_numtuples;
+ double bs_numtuples;
double bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
@@ -244,6 +244,7 @@ brinhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = true;
@@ -960,7 +961,7 @@ brinbuildCallback(Relation index,
/*
* A version of the callback, used by parallel index builds. The main difference
* is that instead of writing the BRIN tuples into the index, we write them into
- * a shared tuplestore, and leave the insertion up to the leader (which may
+ * a shared tuplesort, and leave the insertion up to the leader (which may
* reorder them a bit etc.). The callback also does not generate empty ranges,
* those may be added by the leader when merging results from workers.
*/
@@ -1880,7 +1881,7 @@ form_and_insert_tuple(BrinBuildState *state)
/*
* Given a deformed tuple in the build state, convert it into the on-disk
- * format and write it to a (shared) tuplestore (the leader will insert it
+ * format and write it to a (shared) tuplesort (the leader will insert it
* into the index later).
*/
static void
@@ -2419,7 +2420,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
BlockNumber prevblkno = InvalidBlockNumber;
BrinTuple *emptyTuple = NULL;
Size emptySize;
- BrinSpool *spool = state->bs_spool;
+ BrinSpool *spool;
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
@@ -2431,6 +2432,8 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
state->bs_reltuples = brinshared->reltuples;
state->bs_numtuples = brinshared->indtuples;
+ /* do the actual sort in the leader */
+ spool = state->bs_spool;
tuplesort_performsort(spool->sortstate);
/*
@@ -2443,13 +2446,6 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
* Read the BRIN tuples from the shared tuplesort, sorted by block number.
* That probably gives us an index that is cheaper to scan, thanks to mostly
* getting data from the same index page as before.
- *
- * FIXME This probably needs some memory management fixes - we're reading
- * tuples from the tuplesort, we're allocating an empty tuple, and so on.
- * Probably better to release this memory.
- *
- * XXX We can't quite free the BrinTuple, though, because that's a field
- * in BrinSortTuple.
*/
while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
{
diff --git a/src/backend/access/gin/ginutil.c b/src/backend/access/gin/ginutil.c
index 7a4cd93f301..d4c9d678223 100644
--- a/src/backend/access/gin/ginutil.c
+++ b/src/backend/access/gin/ginutil.c
@@ -54,6 +54,7 @@ ginhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = true;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/gist/gist.c b/src/backend/access/gist/gist.c
index 8ef5fa03290..acec490912d 100644
--- a/src/backend/access/gist/gist.c
+++ b/src/backend/access/gist/gist.c
@@ -76,6 +76,7 @@ gisthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 7a025f33cfe..74592c7d428 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -73,6 +73,7 @@ hashhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index a88b36a589a..2aba6f5b91e 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -112,6 +112,7 @@ bthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = true;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/spgist/spgutils.c b/src/backend/access/spgist/spgutils.c
index c112e1e5dd4..77b6af694fb 100644
--- a/src/backend/access/spgist/spgutils.c
+++ b/src/backend/access/spgist/spgutils.c
@@ -60,6 +60,7 @@ spghandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 37e4305d50a..40abbaf476b 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2982,8 +2982,7 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- (indexRelation->rd_rel->relam == BTREE_AM_OID ||
- indexRelation->rd_rel->relam == BRIN_AM_OID))
+ indexRelation->rd_indam->amcanbuildparallel)
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/include/access/amapi.h b/src/include/access/amapi.h
index 995725502a6..408bb7595bb 100644
--- a/src/include/access/amapi.h
+++ b/src/include/access/amapi.h
@@ -240,6 +240,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
diff --git a/src/test/modules/dummy_index_am/dummy_index_am.c b/src/test/modules/dummy_index_am/dummy_index_am.c
index cbdae7ab7a5..eaa0c483b7e 100644
--- a/src/test/modules/dummy_index_am/dummy_index_am.c
+++ b/src/test/modules/dummy_index_am/dummy_index_am.c
@@ -294,6 +294,7 @@ dihandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
--
2.41.0
Hi,
On Wed, 22 Nov 2023 at 20:16, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:
On 11/20/23 20:48, Matthias van de Meent wrote:
On Wed, 8 Nov 2023 at 12:03, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
here's an updated patch, addressing the review comments, and reworking
how the work is divided between the workers & leader etc.After code-only review, here are some comments:
+++ b/src/backend/access/brin/brin.c [...] +/* Magic numbers for parallel state sharing */ +#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001) +#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)These shm keys use the same constants also in use in
access/nbtree/nbtsort.c. While this shouldn't be an issue in normal
operations, I'd prefer if we didn't actively introduce conflicting
identifiers when we still have significant amounts of unused values
remaining.Hmmm. Is there some rule of thumb how to pick these key values?
None that I know of.
There is a warning in various places that define these constants that
they take care to not conflict with plan node's node_id: parallel plan
execution uses plain plan node IDs as keys, and as node_id is
int-sized, any other key value that's created manually of value < 2^32
should be sure that it can't be executed in a parallel backend.
But apart from that one case, I can't find a convention, no.
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000003)
This is the fourth definition of a PARALLEL%_KEY_QUERY_TEXT, the
others being in access/nbtree/nbtsort.c (value 0xA000000000000004, one
more than brin's), backend/executor/execParallel.c
(0xE000000000000008), and PARALLEL_VACUUM_KEY_QUERY_TEXT (0x3) (though
I've not checked that their uses are exactly the same, I'd expect at
least btree to match mostly, if not fully, 1:1).
I think we could probably benefit from a less ad-hoc sharing of query
texts. I don't think that needs to happen specifically in this patch,
but I think it's something to keep in mind in future efforts.I'm afraid I don't quite get what you mean by "ad hoc sharing of query
texts". Are you saying we shouldn't propagate the query text to the
parallel workers? Why? Or what's the proper solution?
What I mean is that we have several different keys that all look like
they contain the debug query string, and always for the same debugging
purposes. For debugging, I think it'd be useful to use one well-known
key, rather than N well-known keys in each of the N parallel
subsystems.
But as mentioned, it doesn't need to happen in this patch, as that'd
increase scope beyond brin/index ams.
+ state->bs_numtuples = brinshared->indtuples;
My IDE complains about bs_numtuples being an integer. This is a
pre-existing issue, but still valid: we can hit an overflow on tables
with pages_per_range=1 and relsize >= 2^31 pages. Extremely unlikely,
but problematic nonetheless.True. I think I've been hesitant to make this a double because it seems
a bit weird to do +1 with a double, and at some point (d == d+1). But
this seems safe, we're guaranteed to be far away from that threshold.
Yes, ignoring practical constraints like page space, we "only" have
bitspace for 2^48 tuples in each (non-partitioned) relation, so
double's 56 significant bits should be more than enough to count
tuples.
I also noticed that this is likely to execute `union_tuples` many
times when pages_per_range is coprime with the parallel table scan's
block stride (or when we for other reasons have many tuples returned
for each range); and this `union_tuples` internally allocates and
deletes its own memory context for its deserialization of the 'b'
tuple. I think we should just pass a scratch context instead, so that
we don't have the overhead of continously creating then deleting the
same memory contextPerhaps. Looking at the code, isn't it a bit strange how union_tuples
uses the context? It creates the context, calls brin_deform_tuple in
that context, but then the rest of the function (including datumCopy and
similar stuff) happens in the caller's context ...
The union operator may leak (lots of) memory, so I think it makes
sense to keep a context around that can be reset after we've extracted
the merge result.
However, I don't think the number of union_tuples calls is likely to be
very high, especially for large tables. Because we split the table into
2048 chunks, and then cap the chunk size by 8192. For large tables
(where this matters) we're likely close to 8192.
I agree that the merging part of the index creation is the last part,
and usually has no high impact on the total performance of the reindex
operation, but in memory-constrained environments releasing and then
requesting the same chunk of memory over and over again just isn't
great.
Also note that parallel scan chunk sizes decrease when we're about to
hit the end of the table, and that a different AM may have different
ideas about scanning a table in parallel; it could very well decide to
use striped assignments exclusively, as opposed to on-demand chunk
allocations; both increasing the chance that brin's page ranges are
processed by more than one backend.
As a project PostgreSQL seems to be trying to move away from
hardcoding heap into everything in favour of the more AM-agnostic
'table'. I suggest replacing all mentions of "heap" in the arguments
with "table", to reduce the work future maintainers need to do to fix
this.I'm not against doing that, but I'd prefer to do that in a separate
patch. There's a bunch of preexisting heap references, so and I don't
want to introduce inconsistency (patch using table, old code heap) nor
do I want to tweak unrelated code.
Sure.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech)
On 11/23/23 13:33, Matthias van de Meent wrote:
Hi,
On Wed, 22 Nov 2023 at 20:16, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:On 11/20/23 20:48, Matthias van de Meent wrote:
On Wed, 8 Nov 2023 at 12:03, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
Hi,
here's an updated patch, addressing the review comments, and reworking
how the work is divided between the workers & leader etc.After code-only review, here are some comments:
+++ b/src/backend/access/brin/brin.c [...] +/* Magic numbers for parallel state sharing */ +#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xA000000000000001) +#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xA000000000000002)These shm keys use the same constants also in use in
access/nbtree/nbtsort.c. While this shouldn't be an issue in normal
operations, I'd prefer if we didn't actively introduce conflicting
identifiers when we still have significant amounts of unused values
remaining.Hmmm. Is there some rule of thumb how to pick these key values?
None that I know of.
There is a warning in various places that define these constants that
they take care to not conflict with plan node's node_id: parallel plan
execution uses plain plan node IDs as keys, and as node_id is
int-sized, any other key value that's created manually of value < 2^32
should be sure that it can't be executed in a parallel backend.
But apart from that one case, I can't find a convention, no.
OK, in that case 0xB is fine.
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xA000000000000003)
This is the fourth definition of a PARALLEL%_KEY_QUERY_TEXT, the
others being in access/nbtree/nbtsort.c (value 0xA000000000000004, one
more than brin's), backend/executor/execParallel.c
(0xE000000000000008), and PARALLEL_VACUUM_KEY_QUERY_TEXT (0x3) (though
I've not checked that their uses are exactly the same, I'd expect at
least btree to match mostly, if not fully, 1:1).
I think we could probably benefit from a less ad-hoc sharing of query
texts. I don't think that needs to happen specifically in this patch,
but I think it's something to keep in mind in future efforts.I'm afraid I don't quite get what you mean by "ad hoc sharing of query
texts". Are you saying we shouldn't propagate the query text to the
parallel workers? Why? Or what's the proper solution?What I mean is that we have several different keys that all look like
they contain the debug query string, and always for the same debugging
purposes. For debugging, I think it'd be useful to use one well-known
key, rather than N well-known keys in each of the N parallel
subsystems.But as mentioned, it doesn't need to happen in this patch, as that'd
increase scope beyond brin/index ams.
Agreed.
I also noticed that this is likely to execute `union_tuples` many
times when pages_per_range is coprime with the parallel table scan's
block stride (or when we for other reasons have many tuples returned
for each range); and this `union_tuples` internally allocates and
deletes its own memory context for its deserialization of the 'b'
tuple. I think we should just pass a scratch context instead, so that
we don't have the overhead of continously creating then deleting the
same memory contextPerhaps. Looking at the code, isn't it a bit strange how union_tuples
uses the context? It creates the context, calls brin_deform_tuple in
that context, but then the rest of the function (including datumCopy and
similar stuff) happens in the caller's context ...The union operator may leak (lots of) memory, so I think it makes
sense to keep a context around that can be reset after we've extracted
the merge result.
But does the current code actually achieve that? It does create a "brin
union" context, but then it only does this:
/* Use our own memory context to avoid retail pfree */
cxt = AllocSetContextCreate(CurrentMemoryContext,
"brin union",
ALLOCSET_DEFAULT_SIZES);
oldcxt = MemoryContextSwitchTo(cxt);
db = brin_deform_tuple(bdesc, b, NULL);
MemoryContextSwitchTo(oldcxt);
Surely that does not limit the amount of memory used by the actual union
functions in any way?
However, I don't think the number of union_tuples calls is likely to be
very high, especially for large tables. Because we split the table into
2048 chunks, and then cap the chunk size by 8192. For large tables
(where this matters) we're likely close to 8192.I agree that the merging part of the index creation is the last part,
and usually has no high impact on the total performance of the reindex
operation, but in memory-constrained environments releasing and then
requesting the same chunk of memory over and over again just isn't
great.
OK, I'll take a look at the scratch context you suggested.
My point however was we won't actually do that very often, because on
large tables the BRIN ranges are likely smaller than the parallel scan
chunk size, so few overlaps. OTOH if the table is small, or if the BRIN
ranges are large, there'll be few of them.
Also note that parallel scan chunk sizes decrease when we're about to
hit the end of the table, and that a different AM may have different
ideas about scanning a table in parallel; it could very well decide to
use striped assignments exclusively, as opposed to on-demand chunk
allocations; both increasing the chance that brin's page ranges are
processed by more than one backend.
Yeah, but the ramp-up and ramp-down should have negligible impact, IMO.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On Thu, 23 Nov 2023 at 14:35, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:
On 11/23/23 13:33, Matthias van de Meent wrote:
The union operator may leak (lots of) memory, so I think it makes
sense to keep a context around that can be reset after we've extracted
the merge result.But does the current code actually achieve that? It does create a "brin
union" context, but then it only does this:/* Use our own memory context to avoid retail pfree */
cxt = AllocSetContextCreate(CurrentMemoryContext,
"brin union",
ALLOCSET_DEFAULT_SIZES);
oldcxt = MemoryContextSwitchTo(cxt);
db = brin_deform_tuple(bdesc, b, NULL);
MemoryContextSwitchTo(oldcxt);Surely that does not limit the amount of memory used by the actual union
functions in any way?
Oh, yes, of course. For some reason I thought that covered the calls
to the union operator function too, but it indeed only covers
deserialization. I do think it is still worthwhile to not do the
create/delete cycle, but won't hold the patch back for that.
However, I don't think the number of union_tuples calls is likely to be
very high, especially for large tables. Because we split the table into
2048 chunks, and then cap the chunk size by 8192. For large tables
(where this matters) we're likely close to 8192.I agree that the merging part of the index creation is the last part,
and usually has no high impact on the total performance of the reindex
operation, but in memory-constrained environments releasing and then
requesting the same chunk of memory over and over again just isn't
great.OK, I'll take a look at the scratch context you suggested.
My point however was we won't actually do that very often, because on
large tables the BRIN ranges are likely smaller than the parallel scan
chunk size, so few overlaps. OTOH if the table is small, or if the BRIN
ranges are large, there'll be few of them.
That's true, so maybe I'm concerned about something that amounts to
only marginal gains.
I noticed that the v4 patch doesn't yet update the documentation in
indexam.sgml with am->amcanbuildparallel.
Once that is included and reviewed I think this will be ready, unless
you want to address any of my comments upthread (that I marked with
'not in this patch') in this patch.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech)
On 11/28/23 16:39, Matthias van de Meent wrote:
On Thu, 23 Nov 2023 at 14:35, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:On 11/23/23 13:33, Matthias van de Meent wrote:
The union operator may leak (lots of) memory, so I think it makes
sense to keep a context around that can be reset after we've extracted
the merge result.But does the current code actually achieve that? It does create a "brin
union" context, but then it only does this:/* Use our own memory context to avoid retail pfree */
cxt = AllocSetContextCreate(CurrentMemoryContext,
"brin union",
ALLOCSET_DEFAULT_SIZES);
oldcxt = MemoryContextSwitchTo(cxt);
db = brin_deform_tuple(bdesc, b, NULL);
MemoryContextSwitchTo(oldcxt);Surely that does not limit the amount of memory used by the actual union
functions in any way?Oh, yes, of course. For some reason I thought that covered the calls
to the union operator function too, but it indeed only covers
deserialization. I do think it is still worthwhile to not do the
create/delete cycle, but won't hold the patch back for that.
I think the union_tuples() changes are better left for a separate patch.
However, I don't think the number of union_tuples calls is likely to be
very high, especially for large tables. Because we split the table into
2048 chunks, and then cap the chunk size by 8192. For large tables
(where this matters) we're likely close to 8192.I agree that the merging part of the index creation is the last part,
and usually has no high impact on the total performance of the reindex
operation, but in memory-constrained environments releasing and then
requesting the same chunk of memory over and over again just isn't
great.OK, I'll take a look at the scratch context you suggested.
My point however was we won't actually do that very often, because on
large tables the BRIN ranges are likely smaller than the parallel scan
chunk size, so few overlaps. OTOH if the table is small, or if the BRIN
ranges are large, there'll be few of them.That's true, so maybe I'm concerned about something that amounts to
only marginal gains.
However, after thinking about this a bit more, I think we actually do
need to do something about the memory management when merging tuples.
AFAIK the general assumption was that union_tuple() only runs for a
single range, and then the whole context gets freed. But the way the
merging was implemented, it all runs in a single context. And while a
single union_tuple() may not need a lot memory, in total it may be
annoying. I just added a palloc(1MB) into union_tuples and ended up with
~160MB allocated in the PortalContext on just 2GB table. In practice the
memory will grow more slowly, but not great :-/
The attached 0003 patch adds a memory context that's reset after
producing a merged BRIN tuple for each page range.
I noticed that the v4 patch doesn't yet update the documentation in
indexam.sgml with am->amcanbuildparallel.
Should be fixed by 0002. I decided to add a simple note to ambuild(),
not sure if something more is needed.
Once that is included and reviewed I think this will be ready, unless
you want to address any of my comments upthread (that I marked with
'not in this patch') in this patch.
Thanks. I believe the attached version addresses it. There's also 0004
with some indentation tweaks per pgindent.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
v20231128-0001-parallel-CREATE-INDEX-for-BRIN-v3.patchtext/x-patch; charset=UTF-8; name=v20231128-0001-parallel-CREATE-INDEX-for-BRIN-v3.patchDownload
From 219a451250cbbbc1235c60f4776c2ee6bc84432d Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 7 Nov 2023 17:04:28 +0100
Subject: [PATCH v20231128 1/4] parallel CREATE INDEX for BRIN v3
---
contrib/bloom/blutils.c | 1 +
src/backend/access/brin/brin.c | 866 +++++++++++++++++-
src/backend/access/gin/ginutil.c | 1 +
src/backend/access/gist/gist.c | 1 +
src/backend/access/hash/hash.c | 1 +
src/backend/access/nbtree/nbtree.c | 1 +
src/backend/access/spgist/spgutils.c | 1 +
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 2 +-
src/backend/utils/sort/tuplesortvariants.c | 207 +++++
src/include/access/amapi.h | 2 +
src/include/access/brin.h | 3 +
src/include/utils/tuplesort.h | 11 +
.../modules/dummy_index_am/dummy_index_am.c | 1 +
14 files changed, 1096 insertions(+), 6 deletions(-)
diff --git a/contrib/bloom/blutils.c b/contrib/bloom/blutils.c
index 4830cb3fee6..a781c5d98d6 100644
--- a/contrib/bloom/blutils.c
+++ b/contrib/bloom/blutils.c
@@ -122,6 +122,7 @@ blhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amparallelvacuumoptions =
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 4f2dfdd17b9..0c6ca1ac18c 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -33,6 +33,7 @@
#include "postmaster/autovacuum.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -40,7 +41,118 @@
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+#include "utils/tuplesort.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xB000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xB000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xB000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xB000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xB000000000000005)
+
+/*
+ * Status record for spooling/sorting phase.
+ */
+typedef struct BrinSpool
+{
+ Tuplesortstate *sortstate; /* state data for tuplesort.c */
+ Relation heap;
+ Relation index;
+} BrinSpool;
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist for
+ * the benefit of worker processes that need to create state corresponding
+ * to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+ int scantuplesortstates;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN 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 the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
+} BrinShared;
+
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* 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).
+ *
+ * brinshared is the shared state for entire build. sharedsort is the
+ * shared, tuplesort-managed state passed to each process tuplesort.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is required.
+ */
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -49,13 +161,23 @@
typedef struct BrinBuildState
{
Relation bs_irel;
- int bs_numtuples;
+ double bs_numtuples;
+ double bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader process has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BrinSpool *bs_spool;
} BrinBuildState;
/*
@@ -88,6 +210,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -95,6 +218,20 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinSpool *brinspool,
+ BrinShared *brinshared,
+ Sharedsort *sharedsort,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -119,6 +256,7 @@ brinhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = true;
@@ -874,6 +1012,63 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a shared tuplesort, and leave the insertion up to the leader (which may
+ * reorder them a bit etc.). The callback also does not generate empty ranges,
+ * those may be added by the leader when merging results from workers.
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; we do not create emptry
+ * BRIN ranges here - the leader is responsible for filling them in.
+ */
+ if (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the tuplesort */
+ form_and_spill_tuple(state);
+
+ /*
+ * Set state to correspond to the next range (for this block).
+ *
+ * This skips ranges that are either empty (and so we don't get any
+ * tuples to summarize), or processes by other workers. We can't
+ * differentiate those cases here easily, so we leave it up to the
+ * leader to fill empty ranges where needed.
+ */
+ state->bs_currRangeStart
+ = state->bs_pagesPerRange * (thisblock / state->bs_pagesPerRange);
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -935,18 +1130,89 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
revmap = brinRevmapInitialize(index, &pagesPerRange);
state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool->heap = heap;
+ state->bs_spool->index = index;
+
+ /*
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
+ */
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
+
/*
* Now scan the relation. No syncscan allowed here because we want the
* heap blocks in physical order.
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
- form_and_insert_tuple(state);
+ /*
+ * If parallel build requested and at least one worker process was
+ * successfully launched, set up coordination state
+ */
+ if (state->bs_leader)
+ {
+ SortCoordinate coordinate;
+
+ coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
+ coordinate->isWorker = false;
+ coordinate->nParticipants =
+ state->bs_leader->nparticipanttuplesorts;
+ coordinate->sharedsort = state->bs_leader->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.
+ */
+ state->bs_spool->sortstate =
+ tuplesort_begin_index_brin(heap, index,
+ maintenance_work_mem, coordinate,
+ TUPLESORT_NONE);
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read all
+ * tuples from the shared tuplesort and insert them into the index.
+ */
+ _brin_end_parallel(state->bs_leader, state);
+ }
+ else /* no parallel index build, just do the usual thing */
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
+ form_and_insert_tuple(state);
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
+ }
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1366,12 +1632,16 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
+ state->bs_spool = NULL;
return state;
}
@@ -1663,6 +1933,32 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a (shared) tuplesort (the leader will insert it
+ * into the index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ /* don't insert empty tuples in parallel build */
+ if (state->bs_dtuple->bt_empty_range)
+ return;
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ /* write the BRIN tuple to the tuplesort */
+ tuplesort_putbrintuple(state->bs_spool->sortstate, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -1982,3 +2278,563 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ int scantuplesortstates;
+ Snapshot snapshot;
+ Size estbrinshared;
+ Size estsort;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ request);
+
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+ estsort = tuplesort_estimate_shared(scantuplesortstates);
+ shm_toc_estimate_chunk(&pcxt->estimator, estsort);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->scantuplesortstates = scantuplesortstates;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ 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);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipanttuplesorts = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipanttuplesorts++;
+ brinleader->brinshared = brinshared;
+ brinleader->sharedsort = sharedsort;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinTuple *btup;
+ BrinMemTuple *memtuple = NULL;
+ Size tuplen;
+ BrinShared *brinshared = brinleader->brinshared;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+ BrinSpool *spool;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ /* do the actual sort in the leader */
+ spool = state->bs_spool;
+ tuplesort_performsort(spool->sortstate);
+
+ /*
+ * Initialize BrinMemTuple we'll use to union summaries from workers
+ * (in case they happened to produce parts of the same paga range).
+ */
+ memtuple = brin_new_memtuple(state->bs_bdesc);
+
+ /*
+ * Read the BRIN tuples from the shared tuplesort, sorted by block number.
+ * That probably gives us an index that is cheaper to scan, thanks to mostly
+ * getting data from the same index page as before.
+ */
+ while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
+ {
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
+ /*
+ * Do we need to union summaries for the same page range?
+ *
+ * If this is the first brin tuple we read, then just deform it into
+ * the memtuple, and continue with the next one from tuplesort. We
+ * however may need to insert empty summaries into the index.
+ *
+ * If it's the same block as the last we saw, we simply union the
+ * brin tuple into it, and we're done - we don't even need to insert
+ * empty ranges, because that was done earlier when we saw the first
+ * brin tuple (for this range).
+ *
+ * Finally, if it's not the first brin tuple, and it's not the same
+ * page range, we need to do the insert and then deform the tuple
+ * into the memtuple. Then we'll insert empty ranges before the
+ * new brin tuple, if needed.
+ */
+ if (prevblkno == InvalidBlockNumber)
+ {
+ /* First brin tuples, just deform into memtuple. */
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+ else if (memtuple->bt_blkno == btup->bt_blkno)
+ {
+ /*
+ * Not the first brin tuple, but same page range as the previous
+ * one, so we can merge it into the memtuple.
+ */
+ union_tuples(state->bs_bdesc, memtuple, btup);
+ continue;
+ }
+ else
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ /*
+ * We got brin tuple for a different page range, so form a brin
+ * tuple from the memtuple, insert it, and re-init the memtuple
+ * from the new brin tuple.
+ */
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ /* free the formed on-disk tuple */
+ pfree(tmp);
+
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ prevblkno = btup->bt_blkno;
+ }
+
+ tuplesort_end(spool->sortstate);
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ /**/
+ if (prevblkno != InvalidBlockNumber)
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ pfree(tmp);
+ }
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /* Allocate memory and initialize private spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = buildstate->bs_spool->heap;
+ buildstate->bs_spool->index = buildstate->bs_spool->index;
+
+ /*
+ * 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 / brinleader->nparticipanttuplesorts;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool, brinleader->brinshared,
+ brinleader->sharedsort, heap, index, sortmem, true);
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
+ BrinShared *brinshared, Sharedsort *sharedsort,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ SortCoordinate coordinate;
+ TableScanDesc 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 */
+ brinspool->sortstate = tuplesort_begin_index_brin(brinspool->heap,
+ brinspool->index,
+ sortmem, coordinate,
+ TUPLESORT_NONE);
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* insert the last item */
+ form_and_spill_tuple(state);
+
+ /* sort the BRIN ranges built by this worker */
+ tuplesort_performsort(brinspool->sortstate);
+
+ state->bs_reltuples += reltuples;
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+
+ tuplesort_end(brinspool->sortstate);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL, brinshared->pagesPerRange);
+
+ /* Initialize worker's own spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = heapRel;
+ buildstate->bs_spool->index = indexRel;
+
+ /* Look up shared state private to tuplesort.c */
+ sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
+ tuplesort_attach_shared(sharedsort, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /*
+ * 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 / brinshared->scantuplesortstates;
+
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool,
+ brinshared, sharedsort,
+ heapRel, indexRel, sortmem, false);
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/gin/ginutil.c b/src/backend/access/gin/ginutil.c
index a875c5d3d7a..9b1a0ac345d 100644
--- a/src/backend/access/gin/ginutil.c
+++ b/src/backend/access/gin/ginutil.c
@@ -54,6 +54,7 @@ ginhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = true;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/gist/gist.c b/src/backend/access/gist/gist.c
index 9a1bf8f66cb..e052ba8bda2 100644
--- a/src/backend/access/gist/gist.c
+++ b/src/backend/access/gist/gist.c
@@ -76,6 +76,7 @@ gisthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 6443ff21bda..905519692c6 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -73,6 +73,7 @@ hashhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 0930f9b37e3..6c8cd93fa0a 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -112,6 +112,7 @@ bthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = true;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/spgist/spgutils.c b/src/backend/access/spgist/spgutils.c
index 30c00876a56..fd4b6157101 100644
--- a/src/backend/access/spgist/spgutils.c
+++ b/src/backend/access/spgist/spgutils.c
@@ -60,6 +60,7 @@ spghandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 194a1207be6..d78314062e0 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -145,6 +146,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 143fae01ebd..40abbaf476b 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2982,7 +2982,7 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ indexRelation->rd_indam->amcanbuildparallel)
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 2cd508e5130..9b3a70e6ccf 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -19,6 +19,7 @@
#include "postgres.h"
+#include "access/brin_tuple.h"
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -43,6 +44,8 @@ static void removeabbrev_cluster(Tuplesortstate *state, SortTuple *stups,
int count);
static void removeabbrev_index(Tuplesortstate *state, SortTuple *stups,
int count);
+static void removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups,
+ int count);
static void removeabbrev_datum(Tuplesortstate *state, SortTuple *stups,
int count);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
@@ -69,10 +72,16 @@ static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash_tiebreak(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
+static int comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state);
static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len);
+static void writetup_index_brin(Tuplesortstate *state, LogicalTape *tape,
+ SortTuple *stup);
+static void readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_datum_tiebreak(const SortTuple *a, const SortTuple *b,
@@ -128,6 +137,16 @@ typedef struct
uint32 max_buckets;
} TuplesortIndexHashArg;
+/*
+ * Data struture pointed by "TuplesortPublic.arg" for the index_brin subcase.
+ */
+typedef struct
+{
+ TuplesortIndexArg index;
+
+ /* XXX do we need something here? */
+} TuplesortIndexBrinArg;
+
/*
* Data struture pointed by "TuplesortPublic.arg" for the Datum case.
* Set by tuplesort_begin_datum and used only by the DatumTuple routines.
@@ -140,6 +159,21 @@ typedef struct
int datumTypeLen;
} TuplesortDatumArg;
+/*
+ * Computing BrinTuple size with only the tuple is difficult, so we want to track
+ * the length referenced by the SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by its length.
+ */
+typedef struct BrinSortTuple
+{
+ Size tuplen;
+ BrinTuple tuple;
+} BrinSortTuple;
+
+/* Size of the BrinSortTuple, given length of the BrinTuple. */
+#define BRINSORTTUPLE_SIZE(len) (offsetof(BrinSortTuple, tuple) + (len))
+
+
Tuplesortstate *
tuplesort_begin_heap(TupleDesc tupDesc,
int nkeys, AttrNumber *attNums,
@@ -527,6 +561,47 @@ tuplesort_begin_index_gist(Relation heapRel,
return state;
}
+Tuplesortstate *
+tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem,
+ SortCoordinate coordinate,
+ int sortopt)
+{
+ Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
+ sortopt);
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext;
+ TuplesortIndexBrinArg *arg;
+
+ oldcontext = MemoryContextSwitchTo(base->maincontext);
+ arg = (TuplesortIndexBrinArg *) palloc(sizeof(TuplesortIndexBrinArg));
+
+#ifdef TRACE_SORT
+ if (trace_sort)
+ elog(LOG,
+ "begin index sort: workMem = %d, randomAccess = %c",
+ workMem,
+ sortopt & TUPLESORT_RANDOMACCESS ? 't' : 'f');
+#endif
+
+ base->nKeys = 1; /* Only one sort column, the block number */
+
+ base->removeabbrev = removeabbrev_index_brin;
+ base->comparetup = comparetup_index_brin;
+ base->writetup = writetup_index_brin;
+ base->readtup = readtup_index_brin;
+ base->haveDatum1 = true;
+ base->arg = arg;
+
+ arg->index.heapRel = heapRel;
+ arg->index.indexRel = indexRel;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return state;
+}
+
Tuplesortstate *
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag, int workMem,
@@ -707,6 +782,35 @@ tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
!stup.isnull1);
}
+/*
+ * Collect one BRIN tuple while collecting input data for sort.
+ */
+void
+tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
+{
+ SortTuple stup;
+ BrinSortTuple *bstup;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->tuplecontext);
+
+ /* allocate space for the whole BRIN sort tuple */
+ bstup = palloc(BRINSORTTUPLE_SIZE(size));
+
+ bstup->tuplen = size;
+ memcpy(&bstup->tuple, tuple, size);
+
+ stup.tuple = bstup;
+ stup.datum1 = tuple->bt_blkno;
+ stup.isnull1 = false;
+
+ tuplesort_puttuple_common(state, &stup,
+ base->sortKeys &&
+ base->sortKeys->abbrev_converter &&
+ !stup.isnull1);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
/*
* Accept one Datum while collecting input data for sort.
*
@@ -850,6 +954,35 @@ tuplesort_getindextuple(Tuplesortstate *state, bool forward)
return (IndexTuple) stup.tuple;
}
+/*
+ * Fetch the next BRIN tuple in either forward or back direction.
+ * Returns NULL if no more tuples. Returned tuple belongs to tuplesort memory
+ * context, and must not be freed by caller. Caller may not rely on tuple
+ * remaining valid after any further manipulation of tuplesort.
+ */
+BrinTuple *
+tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
+ SortTuple stup;
+ BrinSortTuple *btup;
+
+ if (!tuplesort_gettuple_common(state, forward, &stup))
+ stup.tuple = NULL;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!stup.tuple)
+ return NULL;
+
+ btup = (BrinSortTuple *) stup.tuple;
+
+ *len = btup->tuplen;
+
+ return &btup->tuple;
+}
+
/*
* Fetch the next Datum in either forward or back direction.
* Returns false if no more datums.
@@ -1564,6 +1697,80 @@ readtup_index(Tuplesortstate *state, SortTuple *stup,
&stup->isnull1);
}
+/*
+ * Routines specialized for BrinTuple case
+ */
+
+static void
+removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
+{
+ int i;
+
+ for (i = 0; i < count; i++)
+ {
+ BrinSortTuple *tuple;
+
+ tuple = stups[i].tuple;
+ stups[i].datum1 = tuple->tuple.bt_blkno;
+ }
+}
+
+static int
+comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+{
+ Assert(TuplesortstateGetPublic(state)->haveDatum1);
+
+ if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1))
+ return 1;
+
+ if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1))
+ return -1;
+
+ /* silence compilers */
+ return 0;
+}
+
+static void
+writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
+
+ tuplen = tuplen + sizeof(tuplen);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->tuple, tuple->tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+}
+
+static void
+readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ unsigned int tuplen = len - sizeof(unsigned int);
+
+ /*
+ * Allocate space for the BRIN sort tuple, which is BrinTuple with an
+ * extra length field.
+ */
+ BrinSortTuple *tuple
+ = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
+
+ tuple->tuplen = tuplen;
+
+ LogicalTapeReadExact(tape, &tuple->tuple, tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
+ stup->tuple = (void *) tuple;
+
+ /* set up first-column key value, which is block number */
+ stup->datum1 = tuple->tuple.bt_blkno;
+}
+
/*
* Routines specialized for DatumTuple case
*/
diff --git a/src/include/access/amapi.h b/src/include/access/amapi.h
index 244459587fc..df85ae3aace 100644
--- a/src/include/access/amapi.h
+++ b/src/include/access/amapi.h
@@ -243,6 +243,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d51..3451ecb211f 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index 9ed2de76cd6..357eb35311d 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -21,6 +21,7 @@
#ifndef TUPLESORT_H
#define TUPLESORT_H
+#include "access/brin_tuple.h"
#include "access/itup.h"
#include "executor/tuptable.h"
#include "storage/dsm.h"
@@ -282,6 +283,9 @@ typedef struct
* The "index_hash" API is similar to index_btree, but the tuples are
* actually sorted by their hash codes not the raw data.
*
+ * The "index_brin" API is similar to index_btree, but the tuples are
+ * BrinTuple and are sorted by their block number not the raw data.
+ *
* Parallel sort callers are required to coordinate multiple tuplesort states
* in a leader process and one or more worker processes. The leader process
* must launch workers, and have each perform an independent "partial"
@@ -426,6 +430,10 @@ extern Tuplesortstate *tuplesort_begin_index_gist(Relation heapRel,
Relation indexRel,
int workMem, SortCoordinate coordinate,
int sortopt);
+extern Tuplesortstate *tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem, SortCoordinate coordinate,
+ int sortopt);
extern Tuplesortstate *tuplesort_begin_datum(Oid datumType,
Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag,
@@ -438,6 +446,7 @@ extern void tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup);
extern void tuplesort_putindextuplevalues(Tuplesortstate *state,
Relation rel, ItemPointer self,
const Datum *values, const bool *isnull);
+extern void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tup, Size len);
extern void tuplesort_putdatum(Tuplesortstate *state, Datum val,
bool isNull);
@@ -445,6 +454,8 @@ extern bool tuplesort_gettupleslot(Tuplesortstate *state, bool forward,
bool copy, TupleTableSlot *slot, Datum *abbrev);
extern HeapTuple tuplesort_getheaptuple(Tuplesortstate *state, bool forward);
extern IndexTuple tuplesort_getindextuple(Tuplesortstate *state, bool forward);
+extern BrinTuple *tuplesort_getbrintuple(Tuplesortstate *state, Size *len,
+ bool forward);
extern bool tuplesort_getdatum(Tuplesortstate *state, bool forward, bool copy,
Datum *val, bool *isNull, Datum *abbrev);
diff --git a/src/test/modules/dummy_index_am/dummy_index_am.c b/src/test/modules/dummy_index_am/dummy_index_am.c
index cbdae7ab7a5..eaa0c483b7e 100644
--- a/src/test/modules/dummy_index_am/dummy_index_am.c
+++ b/src/test/modules/dummy_index_am/dummy_index_am.c
@@ -294,6 +294,7 @@ dihandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
--
2.42.0
v20231128-0002-add-docs-for-amcanbuildparallel.patchtext/x-patch; charset=UTF-8; name=v20231128-0002-add-docs-for-amcanbuildparallel.patchDownload
From e3f42c711c0426b85edf3a88a538e58bc0588b6e Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 28 Nov 2023 18:37:42 +0100
Subject: [PATCH v20231128 2/4] add docs for amcanbuildparallel
---
doc/src/sgml/indexam.sgml | 7 +++++++
1 file changed, 7 insertions(+)
diff --git a/doc/src/sgml/indexam.sgml b/doc/src/sgml/indexam.sgml
index f107c43d6a6..cc4135e3940 100644
--- a/doc/src/sgml/indexam.sgml
+++ b/doc/src/sgml/indexam.sgml
@@ -123,6 +123,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
@@ -286,6 +288,11 @@ ambuild (Relation heapRelation,
and compute the keys that need to be inserted into the index.
The function must return a palloc'd struct containing statistics about
the new index.
+ The <structfield>amcanbuildparallel</structfield> flag indicates whether
+ the access method supports parallel index builds. When set to <literal>true</literal>,
+ the system will attempt to allocate parallel workers for the build.
+ Access methods supporting only non-parallel index builds should leave
+ this flag set to <literal>false</literal>.
</para>
<para>
--
2.42.0
v20231128-0003-use-per-range-memory-context-for-merging-i.patchtext/x-patch; charset=UTF-8; name=v20231128-0003-use-per-range-memory-context-for-merging-i.patchDownload
From 0b431ec837f00f852e28b7b5e8e59dbdf61af27f Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 28 Nov 2023 18:38:32 +0100
Subject: [PATCH v20231128 3/4] use per-range memory context for merging in
leader
---
src/backend/access/brin/brin.c | 34 ++++++++++++++++++++++++++++++----
1 file changed, 30 insertions(+), 4 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 0c6ca1ac18c..8d96d2ac9be 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -2475,6 +2475,8 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
BrinTuple *emptyTuple = NULL;
Size emptySize;
BrinSpool *spool;
+ MemoryContext rangeCxt,
+ oldCxt;
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
@@ -2496,6 +2498,19 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
*/
memtuple = brin_new_memtuple(state->bs_bdesc);
+ /*
+ * Create a memory context we'll reset to combine results for a single
+ * page range (received from the workers). We don't expect huge number
+ * of overlaps under regular circumstances, because for large tables
+ * the chunk size is likely larger than the BRIN page range), but it
+ * can happen, and the union functions may do all kinds of stuff. So
+ * we better reset the context once in a while.
+ */
+ rangeCxt = AllocSetContextCreate(CurrentMemoryContext,
+ "brin union",
+ ALLOCSET_DEFAULT_SIZES);
+ oldCxt = MemoryContextSwitchTo(rangeCxt);
+
/*
* Read the BRIN tuples from the shared tuplesort, sorted by block number.
* That probably gives us an index that is cheaper to scan, thanks to mostly
@@ -2555,8 +2570,12 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
- /* free the formed on-disk tuple */
- pfree(tmp);
+ /*
+ * Reset the per-output-range context. This frees all the memory
+ * possibly allocated by the union functions, and also the BRIN
+ * tuple we just formed and inserted.
+ */
+ MemoryContextReset(rangeCxt);
memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
@@ -2593,7 +2612,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
tuplesort_end(spool->sortstate);
- /* Fill empty ranges for all ranges missing in the tuplesort. */
+ /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
{
@@ -2618,7 +2637,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
emptyTuple->bt_blkno, emptyTuple, emptySize);
}
- /**/
+ /* Fill the BRIN tuple for the last page range. */
if (prevblkno != InvalidBlockNumber)
{
BrinTuple *tmp;
@@ -2633,6 +2652,13 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
pfree(tmp);
}
+ /*
+ * Switch back to the originam memory context, and destroy the one we
+ * created to isolate the union_tuple calls.
+ */
+ MemoryContextSwitchTo(oldCxt);
+ MemoryContextDelete(rangeCxt);
+
/*
* Next, accumulate WAL usage. (This must wait for the workers to finish,
* or we might get incomplete data.)
--
2.42.0
v20231128-0004-pgindent.patchtext/x-patch; charset=UTF-8; name=v20231128-0004-pgindent.patchDownload
From d6c08a8426907292a352d85b8fecc12d3adc8cbc Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Tue, 28 Nov 2023 18:48:21 +0100
Subject: [PATCH v20231128 4/4] pgindent
---
src/backend/access/brin/brin.c | 96 +++++++++++-----------
src/backend/utils/sort/tuplesortvariants.c | 16 ++--
src/tools/pgindent/typedefs.list | 5 ++
3 files changed, 62 insertions(+), 55 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 8d96d2ac9be..001cf04aac5 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -67,14 +67,14 @@ typedef struct BrinSpool
typedef struct BrinShared
{
/*
- * These fields are not modified during the build. They primarily exist for
- * the benefit of worker processes that need to create state corresponding
- * to that used by the leader.
+ * These fields are not modified during the build. They primarily exist
+ * for the benefit of worker processes that need to create state
+ * corresponding to that used by the leader.
*/
Oid heaprelid;
Oid indexrelid;
bool isconcurrent;
- BlockNumber pagesPerRange;
+ BlockNumber pagesPerRange;
int scantuplesortstates;
/*
@@ -145,9 +145,10 @@ typedef struct BrinLeader
*
* brinshared is the shared state for entire build. sharedsort is the
* shared, tuplesort-managed state passed to each process tuplesort.
- * snapshot is the snapshot used by the scan iff an MVCC snapshot is required.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is
+ * required.
*/
- BrinShared *brinshared;
+ BrinShared *brinshared;
Sharedsort *sharedsort;
Snapshot snapshot;
WalUsage *walusage;
@@ -1035,8 +1036,8 @@ brinbuildCallbackParallel(Relation index,
/*
* If we're in a block that belongs to a future range, summarize what
* we've got and start afresh. Note the scan might have skipped many
- * pages, if they were devoid of live tuples; we do not create emptry
- * BRIN ranges here - the leader is responsible for filling them in.
+ * pages, if they were devoid of live tuples; we do not create emptry BRIN
+ * ranges here - the leader is responsible for filling them in.
*/
if (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
{
@@ -1169,23 +1170,24 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
/*
* 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.
+ * 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.
+ * 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.
*/
state->bs_spool->sortstate =
tuplesort_begin_index_brin(heap, index,
@@ -1198,7 +1200,7 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
*/
_brin_end_parallel(state->bs_leader, state);
}
- else /* no parallel index build, just do the usual thing */
+ else /* no parallel index build */
{
reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
brinbuildCallback, (void *) state, NULL);
@@ -2288,9 +2290,9 @@ _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
Snapshot snapshot;
Size estbrinshared;
Size estsort;
- BrinShared *brinshared;
- Sharedsort *sharedsort;
- BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
WalUsage *walusage;
BufferUsage *bufferusage;
bool leaderparticipates = true;
@@ -2471,12 +2473,12 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
BrinMemTuple *memtuple = NULL;
Size tuplen;
BrinShared *brinshared = brinleader->brinshared;
- BlockNumber prevblkno = InvalidBlockNumber;
+ BlockNumber prevblkno = InvalidBlockNumber;
BrinTuple *emptyTuple = NULL;
Size emptySize;
BrinSpool *spool;
- MemoryContext rangeCxt,
- oldCxt;
+ MemoryContext rangeCxt,
+ oldCxt;
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
@@ -2493,18 +2495,18 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
tuplesort_performsort(spool->sortstate);
/*
- * Initialize BrinMemTuple we'll use to union summaries from workers
- * (in case they happened to produce parts of the same paga range).
+ * Initialize BrinMemTuple we'll use to union summaries from workers (in
+ * case they happened to produce parts of the same paga range).
*/
memtuple = brin_new_memtuple(state->bs_bdesc);
/*
* Create a memory context we'll reset to combine results for a single
- * page range (received from the workers). We don't expect huge number
- * of overlaps under regular circumstances, because for large tables
- * the chunk size is likely larger than the BRIN page range), but it
- * can happen, and the union functions may do all kinds of stuff. So
- * we better reset the context once in a while.
+ * page range (received from the workers). We don't expect huge number of
+ * overlaps under regular circumstances, because for large tables the
+ * chunk size is likely larger than the BRIN page range), but it can
+ * happen, and the union functions may do all kinds of stuff. So we better
+ * reset the context once in a while.
*/
rangeCxt = AllocSetContextCreate(CurrentMemoryContext,
"brin union",
@@ -2513,8 +2515,8 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
/*
* Read the BRIN tuples from the shared tuplesort, sorted by block number.
- * That probably gives us an index that is cheaper to scan, thanks to mostly
- * getting data from the same index page as before.
+ * That probably gives us an index that is cheaper to scan, thanks to
+ * mostly getting data from the same index page as before.
*/
while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
{
@@ -2528,15 +2530,15 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
* the memtuple, and continue with the next one from tuplesort. We
* however may need to insert empty summaries into the index.
*
- * If it's the same block as the last we saw, we simply union the
- * brin tuple into it, and we're done - we don't even need to insert
- * empty ranges, because that was done earlier when we saw the first
- * brin tuple (for this range).
+ * If it's the same block as the last we saw, we simply union the brin
+ * tuple into it, and we're done - we don't even need to insert empty
+ * ranges, because that was done earlier when we saw the first brin
+ * tuple (for this range).
*
* Finally, if it's not the first brin tuple, and it's not the same
- * page range, we need to do the insert and then deform the tuple
- * into the memtuple. Then we'll insert empty ranges before the
- * new brin tuple, if needed.
+ * page range, we need to do the insert and then deform the tuple into
+ * the memtuple. Then we'll insert empty ranges before the new brin
+ * tuple, if needed.
*/
if (prevblkno == InvalidBlockNumber)
{
@@ -2730,7 +2732,7 @@ _brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
bool progress)
{
SortCoordinate coordinate;
- TableScanDesc scan;
+ TableScanDesc scan;
double reltuples;
IndexInfo *indexInfo;
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 9b3a70e6ccf..90fc605f1ca 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -965,8 +965,8 @@ tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
{
TuplesortPublic *base = TuplesortstateGetPublic(state);
MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
- SortTuple stup;
- BrinSortTuple *btup;
+ SortTuple stup;
+ BrinSortTuple *btup;
if (!tuplesort_gettuple_common(state, forward, &stup))
stup.tuple = NULL;
@@ -1708,7 +1708,7 @@ removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
for (i = 0; i < count; i++)
{
- BrinSortTuple *tuple;
+ BrinSortTuple *tuple;
tuple = stups[i].tuple;
stups[i].datum1 = tuple->tuple.bt_blkno;
@@ -1735,8 +1735,8 @@ static void
writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
{
TuplesortPublic *base = TuplesortstateGetPublic(state);
- BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
- unsigned int tuplen = tuple->tuplen;
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
tuplen = tuplen + sizeof(tuplen);
LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
@@ -1749,6 +1749,7 @@ static void
readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len)
{
+ BrinSortTuple *tuple;
TuplesortPublic *base = TuplesortstateGetPublic(state);
unsigned int tuplen = len - sizeof(unsigned int);
@@ -1756,9 +1757,8 @@ readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
* Allocate space for the BRIN sort tuple, which is BrinTuple with an
* extra length field.
*/
- BrinSortTuple *tuple
- = (BrinSortTuple *) tuplesort_readtup_alloc(state,
- BRINSORTTUPLE_SIZE(tuplen));
+ tuple = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
tuple->tuplen = tuplen;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 86a9886d4f7..001fef58652 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -297,13 +297,17 @@ BpChar
BrinBuildState
BrinDesc
BrinInsertState
+BrinLeader
BrinMemTuple
BrinMetaPageData
BrinOpaque
BrinOpcInfo
BrinOptions
BrinRevmap
+BrinShared
+BrinSortTuple
BrinSpecialSpace
+BrinSpool
BrinStatsData
BrinTuple
BrinValues
@@ -2879,6 +2883,7 @@ TupleTableSlotOps
TuplesortClusterArg
TuplesortDatumArg
TuplesortIndexArg
+TuplesortIndexBrinArg
TuplesortIndexBTreeArg
TuplesortIndexHashArg
TuplesortInstrumentation
--
2.42.0
On Tue, 28 Nov 2023 at 18:59, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:
On 11/28/23 16:39, Matthias van de Meent wrote:
On Thu, 23 Nov 2023 at 14:35, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:On 11/23/23 13:33, Matthias van de Meent wrote:
The union operator may leak (lots of) memory, so I think it makes
sense to keep a context around that can be reset after we've extracted
the merge result.But does the current code actually achieve that? It does create a "brin
union" context, but then it only does this:/* Use our own memory context to avoid retail pfree */
cxt = AllocSetContextCreate(CurrentMemoryContext,
"brin union",
ALLOCSET_DEFAULT_SIZES);
oldcxt = MemoryContextSwitchTo(cxt);
db = brin_deform_tuple(bdesc, b, NULL);
MemoryContextSwitchTo(oldcxt);Surely that does not limit the amount of memory used by the actual union
functions in any way?Oh, yes, of course. For some reason I thought that covered the calls
to the union operator function too, but it indeed only covers
deserialization. I do think it is still worthwhile to not do the
create/delete cycle, but won't hold the patch back for that.I think the union_tuples() changes are better left for a separate patch.
However, I don't think the number of union_tuples calls is likely to be
very high, especially for large tables. Because we split the table into
2048 chunks, and then cap the chunk size by 8192. For large tables
(where this matters) we're likely close to 8192.I agree that the merging part of the index creation is the last part,
and usually has no high impact on the total performance of the reindex
operation, but in memory-constrained environments releasing and then
requesting the same chunk of memory over and over again just isn't
great.OK, I'll take a look at the scratch context you suggested.
My point however was we won't actually do that very often, because on
large tables the BRIN ranges are likely smaller than the parallel scan
chunk size, so few overlaps. OTOH if the table is small, or if the BRIN
ranges are large, there'll be few of them.That's true, so maybe I'm concerned about something that amounts to
only marginal gains.However, after thinking about this a bit more, I think we actually do
need to do something about the memory management when merging tuples.
AFAIK the general assumption was that union_tuple() only runs for a
single range, and then the whole context gets freed.
Correct, but it is also is (or should be) assumed that union_tuple
will be called several times in the same context to fix repeat
concurrent updates. Presumably, that only happens rarely, but it's
something that should be kept in mind regardless.
But the way the
merging was implemented, it all runs in a single context. And while a
single union_tuple() may not need a lot memory, in total it may be
annoying. I just added a palloc(1MB) into union_tuples and ended up with
~160MB allocated in the PortalContext on just 2GB table. In practice the
memory will grow more slowly, but not great :-/The attached 0003 patch adds a memory context that's reset after
producing a merged BRIN tuple for each page range.
Looks good.
This also made me think a bit more about how we're working with the
tuples. With your latest patch, we always deserialize and re-serialize
the sorted brin tuples, just in case the next tuple will also be a
BRIN tuple of the same page range. Could we save some of that
deserialization time by optimistically expecting that we're not going
to need to merge the tuple and only store a local copy of it locally?
See attached 0002; this saves some cycles in common cases.
The v20231128 version of the patchset (as squashed, attached v5-0001)
looks good to me.
Kind regards,
Matthias van de Meent
Neon (http://neon.tech)
Attachments:
v5-0002-Reduce-de-forming-of-BRIN-tuples-in-parallel-BRIN.patchapplication/octet-stream; name=v5-0002-Reduce-de-forming-of-BRIN-tuples-in-parallel-BRIN.patchDownload
From 40536a80563b5b12e9123989af3d94c264a824ee Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Wed, 29 Nov 2023 15:09:17 +0100
Subject: [PATCH v5 2/2] Reduce de-/forming of BRIN tuples in parallel BRIN
build
De-/forming of the BRIN tuple is actually quite expensive,
so in this commit we introduce a mechanism where we don't
do that if only one BRIN tuple is stored in the shared
sort by moving the serialized tuple into the local
context, and only deserialize it once we need to use the
memtuple.
---
src/backend/access/brin/brin.c | 58 ++++++++++++++++++++++++++--------
1 file changed, 44 insertions(+), 14 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 001cf04aac..909074d430 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -2471,6 +2471,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
int i;
BrinTuple *btup;
BrinMemTuple *memtuple = NULL;
+ BrinMemTuple *memtup_holder = NULL;
Size tuplen;
BrinShared *brinshared = brinleader->brinshared;
BlockNumber prevblkno = InvalidBlockNumber;
@@ -2479,6 +2480,8 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
BrinSpool *spool;
MemoryContext rangeCxt,
oldCxt;
+ BrinTuple *prevbtup;
+ Size prevtuplen;
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
@@ -2498,7 +2501,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
* Initialize BrinMemTuple we'll use to union summaries from workers (in
* case they happened to produce parts of the same paga range).
*/
- memtuple = brin_new_memtuple(state->bs_bdesc);
+ memtup_holder = brin_new_memtuple(state->bs_bdesc);
/*
* Create a memory context we'll reset to combine results for a single
@@ -2542,17 +2545,25 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
*/
if (prevblkno == InvalidBlockNumber)
{
- /* First brin tuples, just deform into memtuple. */
- memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+ /* First brin tuple, store it in the local context. */
+ prevbtup = palloc0(tuplen);
+ memcpy(prevbtup, btup, tuplen);
+ prevtuplen = tuplen;
/* continue to insert empty pages before thisblock */
}
- else if (memtuple->bt_blkno == btup->bt_blkno)
+ else if (prevbtup->bt_blkno == btup->bt_blkno)
{
/*
* Not the first brin tuple, but same page range as the previous
- * one, so we can merge it into the memtuple.
+ * one, so we can merge it into the memtuple. If this is the first
+ * merge for this block number, we still have to deform the
+ * original tuple before we can merge them.
*/
+ if (memtuple == NULL)
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup,
+ memtup_holder);
+
union_tuples(state->bs_bdesc, memtuple, btup);
continue;
}
@@ -2563,23 +2574,36 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
/*
* We got brin tuple for a different page range, so form a brin
- * tuple from the memtuple, insert it, and re-init the memtuple
- * from the new brin tuple.
+ * tuple from the memtuple (if required), insert it, and store a
+ * copy of the new tuple for future use.
*/
- tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
- memtuple, &len);
+ if (memtuple != NULL)
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+ else
+ {
+ tmp = prevbtup;
+ len = prevtuplen;
+ }
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
/*
* Reset the per-output-range context. This frees all the memory
- * possibly allocated by the union functions, and also the BRIN
- * tuple we just formed and inserted.
+ * possibly allocated by the union functions, the previous cached
+ * btuple, and the btup we just may have formed from the memtuple.
*/
MemoryContextReset(rangeCxt);
- memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+ /*
+ * Prepare for the next iteration by storing the new range's tuple
+ * in the current context, and resetting memtuple.
+ */
+ prevbtup = palloc0(tuplen);
+ memcpy(prevbtup, btup, tuplen);
+ prevtuplen = tuplen;
+ memtuple = NULL;
/* continue to insert empty pages before thisblock */
}
@@ -2645,8 +2669,14 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
BrinTuple *tmp;
Size len;
- tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
- memtuple, &len);
+ if (memtuple != NULL)
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+ else
+ {
+ tmp = prevbtup;
+ len = prevtuplen;
+ }
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
--
2.40.1
v5-0001-Allow-BRIN-to-build-its-index-in-parallel.patchapplication/octet-stream; name=v5-0001-Allow-BRIN-to-build-its-index-in-parallel.patchDownload
From 48ea20bc3d991ea57475ead179c510a420d5e406 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Wed, 29 Nov 2023 14:35:10 +0100
Subject: [PATCH v5 1/2] Allow BRIN to build its index in parallel
---
contrib/bloom/blutils.c | 1 +
doc/src/sgml/indexam.sgml | 7 +
src/backend/access/brin/brin.c | 894 +++++++++++++++++-
src/backend/access/gin/ginutil.c | 1 +
src/backend/access/gist/gist.c | 1 +
src/backend/access/hash/hash.c | 1 +
src/backend/access/nbtree/nbtree.c | 1 +
src/backend/access/spgist/spgutils.c | 1 +
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 2 +-
src/backend/utils/sort/tuplesortvariants.c | 207 ++++
src/include/access/amapi.h | 2 +
src/include/access/brin.h | 3 +
src/include/utils/tuplesort.h | 11 +
.../modules/dummy_index_am/dummy_index_am.c | 1 +
src/tools/pgindent/typedefs.list | 5 +
16 files changed, 1136 insertions(+), 6 deletions(-)
diff --git a/contrib/bloom/blutils.c b/contrib/bloom/blutils.c
index 4830cb3fee..a781c5d98d 100644
--- a/contrib/bloom/blutils.c
+++ b/contrib/bloom/blutils.c
@@ -122,6 +122,7 @@ blhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amparallelvacuumoptions =
diff --git a/doc/src/sgml/indexam.sgml b/doc/src/sgml/indexam.sgml
index f107c43d6a..cc4135e394 100644
--- a/doc/src/sgml/indexam.sgml
+++ b/doc/src/sgml/indexam.sgml
@@ -123,6 +123,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
@@ -286,6 +288,11 @@ ambuild (Relation heapRelation,
and compute the keys that need to be inserted into the index.
The function must return a palloc'd struct containing statistics about
the new index.
+ The <structfield>amcanbuildparallel</structfield> flag indicates whether
+ the access method supports parallel index builds. When set to <literal>true</literal>,
+ the system will attempt to allocate parallel workers for the build.
+ Access methods supporting only non-parallel index builds should leave
+ this flag set to <literal>false</literal>.
</para>
<para>
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 4f2dfdd17b..001cf04aac 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -33,6 +33,7 @@
#include "postmaster/autovacuum.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -40,7 +41,119 @@
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+#include "utils/tuplesort.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xB000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xB000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xB000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xB000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xB000000000000005)
+
+/*
+ * Status record for spooling/sorting phase.
+ */
+typedef struct BrinSpool
+{
+ Tuplesortstate *sortstate; /* state data for tuplesort.c */
+ Relation heap;
+ Relation index;
+} BrinSpool;
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist
+ * for the benefit of worker processes that need to create state
+ * corresponding to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+ int scantuplesortstates;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN 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 the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
+} BrinShared;
+
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* 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).
+ *
+ * brinshared is the shared state for entire build. sharedsort is the
+ * shared, tuplesort-managed state passed to each process tuplesort.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is
+ * required.
+ */
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -49,13 +162,23 @@
typedef struct BrinBuildState
{
Relation bs_irel;
- int bs_numtuples;
+ double bs_numtuples;
+ double bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader process has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BrinSpool *bs_spool;
} BrinBuildState;
/*
@@ -88,6 +211,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -95,6 +219,20 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinSpool *brinspool,
+ BrinShared *brinshared,
+ Sharedsort *sharedsort,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -119,6 +257,7 @@ brinhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = true;
@@ -874,6 +1013,63 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a shared tuplesort, and leave the insertion up to the leader (which may
+ * reorder them a bit etc.). The callback also does not generate empty ranges,
+ * those may be added by the leader when merging results from workers.
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; we do not create emptry BRIN
+ * ranges here - the leader is responsible for filling them in.
+ */
+ if (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the tuplesort */
+ form_and_spill_tuple(state);
+
+ /*
+ * Set state to correspond to the next range (for this block).
+ *
+ * This skips ranges that are either empty (and so we don't get any
+ * tuples to summarize), or processes by other workers. We can't
+ * differentiate those cases here easily, so we leave it up to the
+ * leader to fill empty ranges where needed.
+ */
+ state->bs_currRangeStart
+ = state->bs_pagesPerRange * (thisblock / state->bs_pagesPerRange);
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -935,18 +1131,90 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
revmap = brinRevmapInitialize(index, &pagesPerRange);
state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool->heap = heap;
+ state->bs_spool->index = index;
+
+ /*
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
+ */
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
+
/*
* Now scan the relation. No syncscan allowed here because we want the
* heap blocks in physical order.
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
- form_and_insert_tuple(state);
+ /*
+ * If parallel build requested and at least one worker process was
+ * successfully launched, set up coordination state
+ */
+ if (state->bs_leader)
+ {
+ SortCoordinate coordinate;
+
+ coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
+ coordinate->isWorker = false;
+ coordinate->nParticipants =
+ state->bs_leader->nparticipanttuplesorts;
+ coordinate->sharedsort = state->bs_leader->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.
+ */
+ state->bs_spool->sortstate =
+ tuplesort_begin_index_brin(heap, index,
+ maintenance_work_mem, coordinate,
+ TUPLESORT_NONE);
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read all
+ * tuples from the shared tuplesort and insert them into the index.
+ */
+ _brin_end_parallel(state->bs_leader, state);
+ }
+ else /* no parallel index build */
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
+ form_and_insert_tuple(state);
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
+ }
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1366,12 +1634,16 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
+ state->bs_spool = NULL;
return state;
}
@@ -1663,6 +1935,32 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a (shared) tuplesort (the leader will insert it
+ * into the index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ /* don't insert empty tuples in parallel build */
+ if (state->bs_dtuple->bt_empty_range)
+ return;
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ /* write the BRIN tuple to the tuplesort */
+ tuplesort_putbrintuple(state->bs_spool->sortstate, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -1982,3 +2280,589 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ int scantuplesortstates;
+ Snapshot snapshot;
+ Size estbrinshared;
+ Size estsort;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ request);
+
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+ estsort = tuplesort_estimate_shared(scantuplesortstates);
+ shm_toc_estimate_chunk(&pcxt->estimator, estsort);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->scantuplesortstates = scantuplesortstates;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ 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);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipanttuplesorts = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipanttuplesorts++;
+ brinleader->brinshared = brinshared;
+ brinleader->sharedsort = sharedsort;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinTuple *btup;
+ BrinMemTuple *memtuple = NULL;
+ Size tuplen;
+ BrinShared *brinshared = brinleader->brinshared;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+ BrinSpool *spool;
+ MemoryContext rangeCxt,
+ oldCxt;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ /* do the actual sort in the leader */
+ spool = state->bs_spool;
+ tuplesort_performsort(spool->sortstate);
+
+ /*
+ * Initialize BrinMemTuple we'll use to union summaries from workers (in
+ * case they happened to produce parts of the same paga range).
+ */
+ memtuple = brin_new_memtuple(state->bs_bdesc);
+
+ /*
+ * Create a memory context we'll reset to combine results for a single
+ * page range (received from the workers). We don't expect huge number of
+ * overlaps under regular circumstances, because for large tables the
+ * chunk size is likely larger than the BRIN page range), but it can
+ * happen, and the union functions may do all kinds of stuff. So we better
+ * reset the context once in a while.
+ */
+ rangeCxt = AllocSetContextCreate(CurrentMemoryContext,
+ "brin union",
+ ALLOCSET_DEFAULT_SIZES);
+ oldCxt = MemoryContextSwitchTo(rangeCxt);
+
+ /*
+ * Read the BRIN tuples from the shared tuplesort, sorted by block number.
+ * That probably gives us an index that is cheaper to scan, thanks to
+ * mostly getting data from the same index page as before.
+ */
+ while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
+ {
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
+ /*
+ * Do we need to union summaries for the same page range?
+ *
+ * If this is the first brin tuple we read, then just deform it into
+ * the memtuple, and continue with the next one from tuplesort. We
+ * however may need to insert empty summaries into the index.
+ *
+ * If it's the same block as the last we saw, we simply union the brin
+ * tuple into it, and we're done - we don't even need to insert empty
+ * ranges, because that was done earlier when we saw the first brin
+ * tuple (for this range).
+ *
+ * Finally, if it's not the first brin tuple, and it's not the same
+ * page range, we need to do the insert and then deform the tuple into
+ * the memtuple. Then we'll insert empty ranges before the new brin
+ * tuple, if needed.
+ */
+ if (prevblkno == InvalidBlockNumber)
+ {
+ /* First brin tuples, just deform into memtuple. */
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+ else if (memtuple->bt_blkno == btup->bt_blkno)
+ {
+ /*
+ * Not the first brin tuple, but same page range as the previous
+ * one, so we can merge it into the memtuple.
+ */
+ union_tuples(state->bs_bdesc, memtuple, btup);
+ continue;
+ }
+ else
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ /*
+ * We got brin tuple for a different page range, so form a brin
+ * tuple from the memtuple, insert it, and re-init the memtuple
+ * from the new brin tuple.
+ */
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ /*
+ * Reset the per-output-range context. This frees all the memory
+ * possibly allocated by the union functions, and also the BRIN
+ * tuple we just formed and inserted.
+ */
+ MemoryContextReset(rangeCxt);
+
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ prevblkno = btup->bt_blkno;
+ }
+
+ tuplesort_end(spool->sortstate);
+
+ /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ /* Fill the BRIN tuple for the last page range. */
+ if (prevblkno != InvalidBlockNumber)
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ pfree(tmp);
+ }
+
+ /*
+ * Switch back to the originam memory context, and destroy the one we
+ * created to isolate the union_tuple calls.
+ */
+ MemoryContextSwitchTo(oldCxt);
+ MemoryContextDelete(rangeCxt);
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /* Allocate memory and initialize private spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = buildstate->bs_spool->heap;
+ buildstate->bs_spool->index = buildstate->bs_spool->index;
+
+ /*
+ * 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 / brinleader->nparticipanttuplesorts;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool, brinleader->brinshared,
+ brinleader->sharedsort, heap, index, sortmem, true);
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
+ BrinShared *brinshared, Sharedsort *sharedsort,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ SortCoordinate coordinate;
+ TableScanDesc 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 */
+ brinspool->sortstate = tuplesort_begin_index_brin(brinspool->heap,
+ brinspool->index,
+ sortmem, coordinate,
+ TUPLESORT_NONE);
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* insert the last item */
+ form_and_spill_tuple(state);
+
+ /* sort the BRIN ranges built by this worker */
+ tuplesort_performsort(brinspool->sortstate);
+
+ state->bs_reltuples += reltuples;
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+
+ tuplesort_end(brinspool->sortstate);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL, brinshared->pagesPerRange);
+
+ /* Initialize worker's own spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = heapRel;
+ buildstate->bs_spool->index = indexRel;
+
+ /* Look up shared state private to tuplesort.c */
+ sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
+ tuplesort_attach_shared(sharedsort, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /*
+ * 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 / brinshared->scantuplesortstates;
+
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool,
+ brinshared, sharedsort,
+ heapRel, indexRel, sortmem, false);
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/gin/ginutil.c b/src/backend/access/gin/ginutil.c
index a875c5d3d7..9b1a0ac345 100644
--- a/src/backend/access/gin/ginutil.c
+++ b/src/backend/access/gin/ginutil.c
@@ -54,6 +54,7 @@ ginhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = true;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/gist/gist.c b/src/backend/access/gist/gist.c
index 9a1bf8f66c..e052ba8bda 100644
--- a/src/backend/access/gist/gist.c
+++ b/src/backend/access/gist/gist.c
@@ -76,6 +76,7 @@ gisthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 6443ff21bd..905519692c 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -73,6 +73,7 @@ hashhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 0930f9b37e..6c8cd93fa0 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -112,6 +112,7 @@ bthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = true;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/spgist/spgutils.c b/src/backend/access/spgist/spgutils.c
index 30c00876a5..fd4b615710 100644
--- a/src/backend/access/spgist/spgutils.c
+++ b/src/backend/access/spgist/spgutils.c
@@ -60,6 +60,7 @@ spghandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 194a1207be..d78314062e 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -145,6 +146,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 143fae01eb..40abbaf476 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2982,7 +2982,7 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ indexRelation->rd_indam->amcanbuildparallel)
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 2cd508e513..90fc605f1c 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -19,6 +19,7 @@
#include "postgres.h"
+#include "access/brin_tuple.h"
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -43,6 +44,8 @@ static void removeabbrev_cluster(Tuplesortstate *state, SortTuple *stups,
int count);
static void removeabbrev_index(Tuplesortstate *state, SortTuple *stups,
int count);
+static void removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups,
+ int count);
static void removeabbrev_datum(Tuplesortstate *state, SortTuple *stups,
int count);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
@@ -69,10 +72,16 @@ static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash_tiebreak(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
+static int comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state);
static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len);
+static void writetup_index_brin(Tuplesortstate *state, LogicalTape *tape,
+ SortTuple *stup);
+static void readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_datum_tiebreak(const SortTuple *a, const SortTuple *b,
@@ -128,6 +137,16 @@ typedef struct
uint32 max_buckets;
} TuplesortIndexHashArg;
+/*
+ * Data struture pointed by "TuplesortPublic.arg" for the index_brin subcase.
+ */
+typedef struct
+{
+ TuplesortIndexArg index;
+
+ /* XXX do we need something here? */
+} TuplesortIndexBrinArg;
+
/*
* Data struture pointed by "TuplesortPublic.arg" for the Datum case.
* Set by tuplesort_begin_datum and used only by the DatumTuple routines.
@@ -140,6 +159,21 @@ typedef struct
int datumTypeLen;
} TuplesortDatumArg;
+/*
+ * Computing BrinTuple size with only the tuple is difficult, so we want to track
+ * the length referenced by the SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by its length.
+ */
+typedef struct BrinSortTuple
+{
+ Size tuplen;
+ BrinTuple tuple;
+} BrinSortTuple;
+
+/* Size of the BrinSortTuple, given length of the BrinTuple. */
+#define BRINSORTTUPLE_SIZE(len) (offsetof(BrinSortTuple, tuple) + (len))
+
+
Tuplesortstate *
tuplesort_begin_heap(TupleDesc tupDesc,
int nkeys, AttrNumber *attNums,
@@ -527,6 +561,47 @@ tuplesort_begin_index_gist(Relation heapRel,
return state;
}
+Tuplesortstate *
+tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem,
+ SortCoordinate coordinate,
+ int sortopt)
+{
+ Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
+ sortopt);
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext;
+ TuplesortIndexBrinArg *arg;
+
+ oldcontext = MemoryContextSwitchTo(base->maincontext);
+ arg = (TuplesortIndexBrinArg *) palloc(sizeof(TuplesortIndexBrinArg));
+
+#ifdef TRACE_SORT
+ if (trace_sort)
+ elog(LOG,
+ "begin index sort: workMem = %d, randomAccess = %c",
+ workMem,
+ sortopt & TUPLESORT_RANDOMACCESS ? 't' : 'f');
+#endif
+
+ base->nKeys = 1; /* Only one sort column, the block number */
+
+ base->removeabbrev = removeabbrev_index_brin;
+ base->comparetup = comparetup_index_brin;
+ base->writetup = writetup_index_brin;
+ base->readtup = readtup_index_brin;
+ base->haveDatum1 = true;
+ base->arg = arg;
+
+ arg->index.heapRel = heapRel;
+ arg->index.indexRel = indexRel;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return state;
+}
+
Tuplesortstate *
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag, int workMem,
@@ -707,6 +782,35 @@ tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
!stup.isnull1);
}
+/*
+ * Collect one BRIN tuple while collecting input data for sort.
+ */
+void
+tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
+{
+ SortTuple stup;
+ BrinSortTuple *bstup;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->tuplecontext);
+
+ /* allocate space for the whole BRIN sort tuple */
+ bstup = palloc(BRINSORTTUPLE_SIZE(size));
+
+ bstup->tuplen = size;
+ memcpy(&bstup->tuple, tuple, size);
+
+ stup.tuple = bstup;
+ stup.datum1 = tuple->bt_blkno;
+ stup.isnull1 = false;
+
+ tuplesort_puttuple_common(state, &stup,
+ base->sortKeys &&
+ base->sortKeys->abbrev_converter &&
+ !stup.isnull1);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
/*
* Accept one Datum while collecting input data for sort.
*
@@ -850,6 +954,35 @@ tuplesort_getindextuple(Tuplesortstate *state, bool forward)
return (IndexTuple) stup.tuple;
}
+/*
+ * Fetch the next BRIN tuple in either forward or back direction.
+ * Returns NULL if no more tuples. Returned tuple belongs to tuplesort memory
+ * context, and must not be freed by caller. Caller may not rely on tuple
+ * remaining valid after any further manipulation of tuplesort.
+ */
+BrinTuple *
+tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
+ SortTuple stup;
+ BrinSortTuple *btup;
+
+ if (!tuplesort_gettuple_common(state, forward, &stup))
+ stup.tuple = NULL;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!stup.tuple)
+ return NULL;
+
+ btup = (BrinSortTuple *) stup.tuple;
+
+ *len = btup->tuplen;
+
+ return &btup->tuple;
+}
+
/*
* Fetch the next Datum in either forward or back direction.
* Returns false if no more datums.
@@ -1564,6 +1697,80 @@ readtup_index(Tuplesortstate *state, SortTuple *stup,
&stup->isnull1);
}
+/*
+ * Routines specialized for BrinTuple case
+ */
+
+static void
+removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
+{
+ int i;
+
+ for (i = 0; i < count; i++)
+ {
+ BrinSortTuple *tuple;
+
+ tuple = stups[i].tuple;
+ stups[i].datum1 = tuple->tuple.bt_blkno;
+ }
+}
+
+static int
+comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+{
+ Assert(TuplesortstateGetPublic(state)->haveDatum1);
+
+ if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1))
+ return 1;
+
+ if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1))
+ return -1;
+
+ /* silence compilers */
+ return 0;
+}
+
+static void
+writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
+
+ tuplen = tuplen + sizeof(tuplen);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->tuple, tuple->tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+}
+
+static void
+readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len)
+{
+ BrinSortTuple *tuple;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ unsigned int tuplen = len - sizeof(unsigned int);
+
+ /*
+ * Allocate space for the BRIN sort tuple, which is BrinTuple with an
+ * extra length field.
+ */
+ tuple = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
+
+ tuple->tuplen = tuplen;
+
+ LogicalTapeReadExact(tape, &tuple->tuple, tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
+ stup->tuple = (void *) tuple;
+
+ /* set up first-column key value, which is block number */
+ stup->datum1 = tuple->tuple.bt_blkno;
+}
+
/*
* Routines specialized for DatumTuple case
*/
diff --git a/src/include/access/amapi.h b/src/include/access/amapi.h
index 244459587f..df85ae3aac 100644
--- a/src/include/access/amapi.h
+++ b/src/include/access/amapi.h
@@ -243,6 +243,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d5..3451ecb211 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index 9ed2de76cd..357eb35311 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -21,6 +21,7 @@
#ifndef TUPLESORT_H
#define TUPLESORT_H
+#include "access/brin_tuple.h"
#include "access/itup.h"
#include "executor/tuptable.h"
#include "storage/dsm.h"
@@ -282,6 +283,9 @@ typedef struct
* The "index_hash" API is similar to index_btree, but the tuples are
* actually sorted by their hash codes not the raw data.
*
+ * The "index_brin" API is similar to index_btree, but the tuples are
+ * BrinTuple and are sorted by their block number not the raw data.
+ *
* Parallel sort callers are required to coordinate multiple tuplesort states
* in a leader process and one or more worker processes. The leader process
* must launch workers, and have each perform an independent "partial"
@@ -426,6 +430,10 @@ extern Tuplesortstate *tuplesort_begin_index_gist(Relation heapRel,
Relation indexRel,
int workMem, SortCoordinate coordinate,
int sortopt);
+extern Tuplesortstate *tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem, SortCoordinate coordinate,
+ int sortopt);
extern Tuplesortstate *tuplesort_begin_datum(Oid datumType,
Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag,
@@ -438,6 +446,7 @@ extern void tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup);
extern void tuplesort_putindextuplevalues(Tuplesortstate *state,
Relation rel, ItemPointer self,
const Datum *values, const bool *isnull);
+extern void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tup, Size len);
extern void tuplesort_putdatum(Tuplesortstate *state, Datum val,
bool isNull);
@@ -445,6 +454,8 @@ extern bool tuplesort_gettupleslot(Tuplesortstate *state, bool forward,
bool copy, TupleTableSlot *slot, Datum *abbrev);
extern HeapTuple tuplesort_getheaptuple(Tuplesortstate *state, bool forward);
extern IndexTuple tuplesort_getindextuple(Tuplesortstate *state, bool forward);
+extern BrinTuple *tuplesort_getbrintuple(Tuplesortstate *state, Size *len,
+ bool forward);
extern bool tuplesort_getdatum(Tuplesortstate *state, bool forward, bool copy,
Datum *val, bool *isNull, Datum *abbrev);
diff --git a/src/test/modules/dummy_index_am/dummy_index_am.c b/src/test/modules/dummy_index_am/dummy_index_am.c
index cbdae7ab7a..eaa0c483b7 100644
--- a/src/test/modules/dummy_index_am/dummy_index_am.c
+++ b/src/test/modules/dummy_index_am/dummy_index_am.c
@@ -294,6 +294,7 @@ dihandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 86a9886d4f..001fef5865 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -297,13 +297,17 @@ BpChar
BrinBuildState
BrinDesc
BrinInsertState
+BrinLeader
BrinMemTuple
BrinMetaPageData
BrinOpaque
BrinOpcInfo
BrinOptions
BrinRevmap
+BrinShared
+BrinSortTuple
BrinSpecialSpace
+BrinSpool
BrinStatsData
BrinTuple
BrinValues
@@ -2879,6 +2883,7 @@ TupleTableSlotOps
TuplesortClusterArg
TuplesortDatumArg
TuplesortIndexArg
+TuplesortIndexBrinArg
TuplesortIndexBTreeArg
TuplesortIndexHashArg
TuplesortInstrumentation
--
2.40.1
On 11/29/23 15:42, Matthias van de Meent wrote:
On Tue, 28 Nov 2023 at 18:59, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:On 11/28/23 16:39, Matthias van de Meent wrote:
On Thu, 23 Nov 2023 at 14:35, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:On 11/23/23 13:33, Matthias van de Meent wrote:
The union operator may leak (lots of) memory, so I think it makes
sense to keep a context around that can be reset after we've extracted
the merge result.But does the current code actually achieve that? It does create a "brin
union" context, but then it only does this:/* Use our own memory context to avoid retail pfree */
cxt = AllocSetContextCreate(CurrentMemoryContext,
"brin union",
ALLOCSET_DEFAULT_SIZES);
oldcxt = MemoryContextSwitchTo(cxt);
db = brin_deform_tuple(bdesc, b, NULL);
MemoryContextSwitchTo(oldcxt);Surely that does not limit the amount of memory used by the actual union
functions in any way?Oh, yes, of course. For some reason I thought that covered the calls
to the union operator function too, but it indeed only covers
deserialization. I do think it is still worthwhile to not do the
create/delete cycle, but won't hold the patch back for that.I think the union_tuples() changes are better left for a separate patch.
However, I don't think the number of union_tuples calls is likely to be
very high, especially for large tables. Because we split the table into
2048 chunks, and then cap the chunk size by 8192. For large tables
(where this matters) we're likely close to 8192.I agree that the merging part of the index creation is the last part,
and usually has no high impact on the total performance of the reindex
operation, but in memory-constrained environments releasing and then
requesting the same chunk of memory over and over again just isn't
great.OK, I'll take a look at the scratch context you suggested.
My point however was we won't actually do that very often, because on
large tables the BRIN ranges are likely smaller than the parallel scan
chunk size, so few overlaps. OTOH if the table is small, or if the BRIN
ranges are large, there'll be few of them.That's true, so maybe I'm concerned about something that amounts to
only marginal gains.However, after thinking about this a bit more, I think we actually do
need to do something about the memory management when merging tuples.
AFAIK the general assumption was that union_tuple() only runs for a
single range, and then the whole context gets freed.Correct, but it is also is (or should be) assumed that union_tuple
will be called several times in the same context to fix repeat
concurrent updates. Presumably, that only happens rarely, but it's
something that should be kept in mind regardless.
In theory, yes. But union_tuples() is used only in summarize_range(),
and that only processes a single page range.
But the way the
merging was implemented, it all runs in a single context. And while a
single union_tuple() may not need a lot memory, in total it may be
annoying. I just added a palloc(1MB) into union_tuples and ended up with
~160MB allocated in the PortalContext on just 2GB table. In practice the
memory will grow more slowly, but not great :-/The attached 0003 patch adds a memory context that's reset after
producing a merged BRIN tuple for each page range.Looks good.
This also made me think a bit more about how we're working with the
tuples. With your latest patch, we always deserialize and re-serialize
the sorted brin tuples, just in case the next tuple will also be a
BRIN tuple of the same page range. Could we save some of that
deserialization time by optimistically expecting that we're not going
to need to merge the tuple and only store a local copy of it locally?
See attached 0002; this saves some cycles in common cases.
Good idea!
The v20231128 version of the patchset (as squashed, attached v5-0001)
looks good to me.
Cool. I'll put this through a bit more stress testing, and then I'll get
it pushed.
Thanks for the reviews!
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On 11/29/23 15:52, Tomas Vondra wrote:
...
This also made me think a bit more about how we're working with the
tuples. With your latest patch, we always deserialize and re-serialize
the sorted brin tuples, just in case the next tuple will also be a
BRIN tuple of the same page range. Could we save some of that
deserialization time by optimistically expecting that we're not going
to need to merge the tuple and only store a local copy of it locally?
See attached 0002; this saves some cycles in common cases.Good idea!
FWIW there's a bug, in this part of the optimization:
------------------
+ if (memtuple == NULL)
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup,
+ memtup_holder);
+
union_tuples(state->bs_bdesc, memtuple, btup);
continue;
------------------
The deforming should use prevbtup, otherwise union_tuples() jut combines
two copies of the same tuple.
Which however brings me to the bigger issue with this - my stress test
found this issue pretty quickly, but then I spent quite a bit of time
trying to find what went wrong. I find this reworked code pretty hard to
understand, and not necessarily because of how it's written. The problem
is it the same loop tries to juggle multiple pieces of information with
different lifespans, and so on. I find it really hard to reason about
how it behaves ...
I did try to measure how much it actually saves, but none of the tests I
did actually found measurable improvement. So I'm tempted to just not
include this part, and accept that we may deserialize some of the tuples
unnecessarily.
Did you actually observe measurable improvements in some cases?
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On Wed, 29 Nov 2023 at 18:55, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:
On 11/29/23 15:52, Tomas Vondra wrote:
...
This also made me think a bit more about how we're working with the
tuples. With your latest patch, we always deserialize and re-serialize
the sorted brin tuples, just in case the next tuple will also be a
BRIN tuple of the same page range. Could we save some of that
deserialization time by optimistically expecting that we're not going
to need to merge the tuple and only store a local copy of it locally?
See attached 0002; this saves some cycles in common cases.Good idea!
FWIW there's a bug, in this part of the optimization:
------------------ + if (memtuple == NULL) + memtuple = brin_deform_tuple(state->bs_bdesc, btup, + memtup_holder); + union_tuples(state->bs_bdesc, memtuple, btup); continue; ------------------The deforming should use prevbtup, otherwise union_tuples() jut combines
two copies of the same tuple.
Good point. There were some more issues as well, fixes are attached.
Which however brings me to the bigger issue with this - my stress test
found this issue pretty quickly, but then I spent quite a bit of time
trying to find what went wrong. I find this reworked code pretty hard to
understand, and not necessarily because of how it's written. The problem
is it the same loop tries to juggle multiple pieces of information with
different lifespans, and so on. I find it really hard to reason about
how it behaves ...
Yeah, it'd be nice if we had a peek option for sortsupport, that'd
improve context handling.
I did try to measure how much it actually saves, but none of the tests I
did actually found measurable improvement. So I'm tempted to just not
include this part, and accept that we may deserialize some of the tuples
unnecessarily.Did you actually observe measurable improvements in some cases?
The improvements would mostly stem from brin indexes with multiple
(potentially compressed) by-ref types, as they go through more complex
and expensive code to deserialize, requiring separate palloc() and
memcpy() calls each.
For single-column and by-value types the improvements are expected to
be negligible, because there is no meaningful difference between
copying a single by-ref value and copying its container; the
additional work done for each tuple is marginal for those.
For an 8-column BRIN index ((sha256((id)::text::bytea)::text),
(sha256((id+1)::text::bytea)::text),
(sha256((id+2)::text::bytea)::text), ...) instrumented with 0003 I
measured a difference of 10x less time spent in the main loop of
_brin_end_parallel, from ~30ms to 3ms when dealing with 55k 1-block
ranges. It's not a lot, but worth at least something, I guess?
The attached patch fixes the issue that you called out .
It also further updates _brin_end_parallel: the final 'write empty
tuples' loop is never hit and is thus removed, because if there were
any tuples in the spool we'd have filled the empty ranges at the end
of the main loop, and if there were no tuples in the spool then the
memtuple would still be at its original initialized value of 0 thus
resulting in a constant false condition. I also updated some comments.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech)
Attachments:
v6-0003-NOCOMMIT-Instrumentation-for-time-spent-in-_brin_.patchapplication/octet-stream; name=v6-0003-NOCOMMIT-Instrumentation-for-time-spent-in-_brin_.patchDownload
From 97707510215456cf8db0f720b1c9a96e6f4dc073 Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Wed, 29 Nov 2023 20:35:22 +0100
Subject: [PATCH v6 3/3] NOCOMMIT: Instrumentation for time spent in
_brin_end_parallel
---
src/backend/access/brin/brin.c | 17 +++++++++++++++++
1 file changed, 17 insertions(+)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index e7ec7c7b53..119c23cb09 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -2482,6 +2482,9 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
oldCxt;
BrinTuple *prevbtup = NULL;
Size prevtuplen;
+ instr_time time_spent;
+
+ INSTR_TIME_SET_ZERO(time_spent);
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
@@ -2523,6 +2526,11 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
*/
while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
{
+ instr_time opr_start;
+ instr_time opr_end;
+
+ INSTR_TIME_SET_CURRENT(opr_start);
+
/* Ranges should be multiples of pages_per_range for the index. */
Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
@@ -2579,6 +2587,8 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
}
union_tuples(state->bs_bdesc, memtuple, btup);
+ INSTR_TIME_SET_CURRENT(opr_end);
+ INSTR_TIME_ACCUM_DIFF(time_spent, opr_end, opr_start);
continue;
}
else
@@ -2602,8 +2612,11 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
len = prevtuplen;
}
+ INSTR_TIME_SET_CURRENT(opr_end);
+ INSTR_TIME_ACCUM_DIFF(time_spent, opr_end, opr_start);
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+ INSTR_TIME_SET_CURRENT(opr_start);
/*
* Reset the per-output-range context. This frees all the memory
@@ -2627,6 +2640,9 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
/* continue to insert empty pages before thisblock */
}
+ INSTR_TIME_SET_CURRENT(opr_end);
+ INSTR_TIME_ACCUM_DIFF(time_spent, opr_end, opr_start);
+
/* Fill empty ranges for all ranges missing in the tuplesort. */
prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
@@ -2698,6 +2714,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
UnregisterSnapshot(brinleader->snapshot);
DestroyParallelContext(brinleader->pcxt);
ExitParallelMode();
+ elog(LOG, "Spent %ld nanos on non-IO tasks", INSTR_TIME_GET_NANOSEC(time_spent));
}
/*
--
2.40.1
v6-0002-Reduce-de-forming-of-BRIN-tuples-in-parallel-BRIN.patchapplication/octet-stream; name=v6-0002-Reduce-de-forming-of-BRIN-tuples-in-parallel-BRIN.patchDownload
From 2a1de622d4a83b940fba3ad6f9f1d1af334d2552 Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Wed, 29 Nov 2023 15:09:17 +0100
Subject: [PATCH v6 2/3] Reduce de-/forming of BRIN tuples in parallel BRIN
build
De-/forming of the BRIN tuple is actually quite expensive,
so in this commit we introduce a mechanism where we don't
do that if only one BRIN tuple is stored in the shared
sort by moving the serialized tuple into the local
context, and only deserialize it once we need to use the
memtuple.
---
src/backend/access/brin/brin.c | 115 ++++++++++++++++++++-------------
1 file changed, 70 insertions(+), 45 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 001cf04aac..e7ec7c7b53 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -2471,6 +2471,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
int i;
BrinTuple *btup;
BrinMemTuple *memtuple = NULL;
+ BrinMemTuple *memtup_holder = NULL;
Size tuplen;
BrinShared *brinshared = brinleader->brinshared;
BlockNumber prevblkno = InvalidBlockNumber;
@@ -2479,6 +2480,8 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
BrinSpool *spool;
MemoryContext rangeCxt,
oldCxt;
+ BrinTuple *prevbtup = NULL;
+ Size prevtuplen;
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
@@ -2498,7 +2501,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
* Initialize BrinMemTuple we'll use to union summaries from workers (in
* case they happened to produce parts of the same paga range).
*/
- memtuple = brin_new_memtuple(state->bs_bdesc);
+ memtup_holder = brin_new_memtuple(state->bs_bdesc);
/*
* Create a memory context we'll reset to combine results for a single
@@ -2526,9 +2529,10 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
/*
* Do we need to union summaries for the same page range?
*
- * If this is the first brin tuple we read, then just deform it into
- * the memtuple, and continue with the next one from tuplesort. We
- * however may need to insert empty summaries into the index.
+ * If this is the first brin tuple we read, then cache it and continue
+ * with the next one from tuplesort. We however may need to insert
+ * empty summaries into the index for the page ranges < the first
+ * page.
*
* If it's the same block as the last we saw, we simply union the brin
* tuple into it, and we're done - we don't even need to insert empty
@@ -2539,20 +2543,41 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
* page range, we need to do the insert and then deform the tuple into
* the memtuple. Then we'll insert empty ranges before the new brin
* tuple, if needed.
+ *
+ * Because it is likely we only get one tuple for each page range, we
+ * only deform the tuple when we actually need to merge the data. This
+ * means we'll only deform the tuple if we detect duplicates, and the
+ * deformed tuple is stored as memtuple. `memtuple_holder` is the
+ * reference to the BRIN memory tuple, memtuple refers to the same
+ * tuple if and when in use for merging data.
+ *
+ * When writing data to disk, we take care to use the cached copy if
+ * memtuple is not set, otherwise we form a new tuple and write that
+ * out to disk.
*/
if (prevblkno == InvalidBlockNumber)
{
- /* First brin tuples, just deform into memtuple. */
- memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+ /* First brin tuple, store it in the local context. */
+ prevbtup = palloc0(tuplen);
+ memcpy(prevbtup, btup, tuplen);
+ prevtuplen = tuplen;
/* continue to insert empty pages before thisblock */
}
- else if (memtuple->bt_blkno == btup->bt_blkno)
+ else if (prevbtup->bt_blkno == btup->bt_blkno)
{
/*
* Not the first brin tuple, but same page range as the previous
- * one, so we can merge it into the memtuple.
+ * one, so we can merge it into the memtuple. If this is the first
+ * merge for this block number, we still have to deform the
+ * original tuple before we can merge them.
*/
+ if (memtuple == NULL)
+ {
+ memtuple = brin_deform_tuple(state->bs_bdesc, prevbtup,
+ memtup_holder);
+ }
+
union_tuples(state->bs_bdesc, memtuple, btup);
continue;
}
@@ -2563,23 +2588,41 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
/*
* We got brin tuple for a different page range, so form a brin
- * tuple from the memtuple, insert it, and re-init the memtuple
- * from the new brin tuple.
+ * tuple from the memtuple (if required), insert it, and store a
+ * copy of the new tuple for future use.
*/
- tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
- memtuple, &len);
+ if (memtuple != NULL)
+ {
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+ }
+ else
+ {
+ tmp = prevbtup;
+ len = prevtuplen;
+ }
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
/*
* Reset the per-output-range context. This frees all the memory
- * possibly allocated by the union functions, and also the BRIN
- * tuple we just formed and inserted.
+ * possibly allocated by the union functions, the previous cached
+ * btuple, and the btup we just may have formed from the memtuple.
*/
MemoryContextReset(rangeCxt);
- memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+ /* clear any potential pointers to the cleared context */
+ memtuple = NULL;
+ prevbtup = NULL;
+
+ /*
+ * Prepare for the next iteration by storing the new range's tuple
+ * in the current context, and resetting memtuple.
+ */
+ prevbtup = palloc0(tuplen);
+ memcpy(prevbtup, btup, tuplen);
+ prevtuplen = tuplen;
/* continue to insert empty pages before thisblock */
}
@@ -2614,44 +2657,26 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
tuplesort_end(spool->sortstate);
- /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
- prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
- while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ /*
+ * If we had any result ranges, then we need to finish writing the cached
+ * data.
+ */
+ if (prevbtup != NULL)
{
- /* the missing range */
- prevblkno += state->bs_pagesPerRange;
-
- /* Did we already build the empty range? If not, do it now. */
- if (emptyTuple == NULL)
- {
- BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+ BrinTuple *tmp;
+ Size len;
- emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
- }
+ if (memtuple != NULL)
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
else
{
- /* we already have am "empty range" tuple, just set the block */
- emptyTuple->bt_blkno = prevblkno;
+ tmp = prevbtup;
+ len = prevtuplen;
}
- brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
- &state->bs_currentInsertBuf,
- emptyTuple->bt_blkno, emptyTuple, emptySize);
- }
-
- /* Fill the BRIN tuple for the last page range. */
- if (prevblkno != InvalidBlockNumber)
- {
- BrinTuple *tmp;
- Size len;
-
- tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
- memtuple, &len);
-
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
-
- pfree(tmp);
}
/*
--
2.40.1
v6-0001-Allow-BRIN-to-build-its-index-in-parallel.patchapplication/octet-stream; name=v6-0001-Allow-BRIN-to-build-its-index-in-parallel.patchDownload
From 48ea20bc3d991ea57475ead179c510a420d5e406 Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Wed, 29 Nov 2023 14:35:10 +0100
Subject: [PATCH v6 1/3] Allow BRIN to build its index in parallel
---
contrib/bloom/blutils.c | 1 +
doc/src/sgml/indexam.sgml | 7 +
src/backend/access/brin/brin.c | 894 +++++++++++++++++-
src/backend/access/gin/ginutil.c | 1 +
src/backend/access/gist/gist.c | 1 +
src/backend/access/hash/hash.c | 1 +
src/backend/access/nbtree/nbtree.c | 1 +
src/backend/access/spgist/spgutils.c | 1 +
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 2 +-
src/backend/utils/sort/tuplesortvariants.c | 207 ++++
src/include/access/amapi.h | 2 +
src/include/access/brin.h | 3 +
src/include/utils/tuplesort.h | 11 +
.../modules/dummy_index_am/dummy_index_am.c | 1 +
src/tools/pgindent/typedefs.list | 5 +
16 files changed, 1136 insertions(+), 6 deletions(-)
diff --git a/contrib/bloom/blutils.c b/contrib/bloom/blutils.c
index 4830cb3fee..a781c5d98d 100644
--- a/contrib/bloom/blutils.c
+++ b/contrib/bloom/blutils.c
@@ -122,6 +122,7 @@ blhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amparallelvacuumoptions =
diff --git a/doc/src/sgml/indexam.sgml b/doc/src/sgml/indexam.sgml
index f107c43d6a..cc4135e394 100644
--- a/doc/src/sgml/indexam.sgml
+++ b/doc/src/sgml/indexam.sgml
@@ -123,6 +123,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
@@ -286,6 +288,11 @@ ambuild (Relation heapRelation,
and compute the keys that need to be inserted into the index.
The function must return a palloc'd struct containing statistics about
the new index.
+ The <structfield>amcanbuildparallel</structfield> flag indicates whether
+ the access method supports parallel index builds. When set to <literal>true</literal>,
+ the system will attempt to allocate parallel workers for the build.
+ Access methods supporting only non-parallel index builds should leave
+ this flag set to <literal>false</literal>.
</para>
<para>
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 4f2dfdd17b..001cf04aac 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -33,6 +33,7 @@
#include "postmaster/autovacuum.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -40,7 +41,119 @@
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+#include "utils/tuplesort.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xB000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xB000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xB000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xB000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xB000000000000005)
+
+/*
+ * Status record for spooling/sorting phase.
+ */
+typedef struct BrinSpool
+{
+ Tuplesortstate *sortstate; /* state data for tuplesort.c */
+ Relation heap;
+ Relation index;
+} BrinSpool;
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist
+ * for the benefit of worker processes that need to create state
+ * corresponding to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+ int scantuplesortstates;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN 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 the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
+} BrinShared;
+
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* 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).
+ *
+ * brinshared is the shared state for entire build. sharedsort is the
+ * shared, tuplesort-managed state passed to each process tuplesort.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is
+ * required.
+ */
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -49,13 +162,23 @@
typedef struct BrinBuildState
{
Relation bs_irel;
- int bs_numtuples;
+ double bs_numtuples;
+ double bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader process has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BrinSpool *bs_spool;
} BrinBuildState;
/*
@@ -88,6 +211,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -95,6 +219,20 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinSpool *brinspool,
+ BrinShared *brinshared,
+ Sharedsort *sharedsort,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -119,6 +257,7 @@ brinhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = true;
@@ -874,6 +1013,63 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a shared tuplesort, and leave the insertion up to the leader (which may
+ * reorder them a bit etc.). The callback also does not generate empty ranges,
+ * those may be added by the leader when merging results from workers.
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; we do not create emptry BRIN
+ * ranges here - the leader is responsible for filling them in.
+ */
+ if (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the tuplesort */
+ form_and_spill_tuple(state);
+
+ /*
+ * Set state to correspond to the next range (for this block).
+ *
+ * This skips ranges that are either empty (and so we don't get any
+ * tuples to summarize), or processes by other workers. We can't
+ * differentiate those cases here easily, so we leave it up to the
+ * leader to fill empty ranges where needed.
+ */
+ state->bs_currRangeStart
+ = state->bs_pagesPerRange * (thisblock / state->bs_pagesPerRange);
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -935,18 +1131,90 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
revmap = brinRevmapInitialize(index, &pagesPerRange);
state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool->heap = heap;
+ state->bs_spool->index = index;
+
+ /*
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
+ */
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
+
/*
* Now scan the relation. No syncscan allowed here because we want the
* heap blocks in physical order.
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
- form_and_insert_tuple(state);
+ /*
+ * If parallel build requested and at least one worker process was
+ * successfully launched, set up coordination state
+ */
+ if (state->bs_leader)
+ {
+ SortCoordinate coordinate;
+
+ coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
+ coordinate->isWorker = false;
+ coordinate->nParticipants =
+ state->bs_leader->nparticipanttuplesorts;
+ coordinate->sharedsort = state->bs_leader->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.
+ */
+ state->bs_spool->sortstate =
+ tuplesort_begin_index_brin(heap, index,
+ maintenance_work_mem, coordinate,
+ TUPLESORT_NONE);
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read all
+ * tuples from the shared tuplesort and insert them into the index.
+ */
+ _brin_end_parallel(state->bs_leader, state);
+ }
+ else /* no parallel index build */
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
+ form_and_insert_tuple(state);
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
+ }
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1366,12 +1634,16 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
+ state->bs_spool = NULL;
return state;
}
@@ -1663,6 +1935,32 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a (shared) tuplesort (the leader will insert it
+ * into the index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ /* don't insert empty tuples in parallel build */
+ if (state->bs_dtuple->bt_empty_range)
+ return;
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ /* write the BRIN tuple to the tuplesort */
+ tuplesort_putbrintuple(state->bs_spool->sortstate, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -1982,3 +2280,589 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ int scantuplesortstates;
+ Snapshot snapshot;
+ Size estbrinshared;
+ Size estsort;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ request);
+
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+ estsort = tuplesort_estimate_shared(scantuplesortstates);
+ shm_toc_estimate_chunk(&pcxt->estimator, estsort);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->scantuplesortstates = scantuplesortstates;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ 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);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipanttuplesorts = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipanttuplesorts++;
+ brinleader->brinshared = brinshared;
+ brinleader->sharedsort = sharedsort;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinTuple *btup;
+ BrinMemTuple *memtuple = NULL;
+ Size tuplen;
+ BrinShared *brinshared = brinleader->brinshared;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+ BrinSpool *spool;
+ MemoryContext rangeCxt,
+ oldCxt;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ /* do the actual sort in the leader */
+ spool = state->bs_spool;
+ tuplesort_performsort(spool->sortstate);
+
+ /*
+ * Initialize BrinMemTuple we'll use to union summaries from workers (in
+ * case they happened to produce parts of the same paga range).
+ */
+ memtuple = brin_new_memtuple(state->bs_bdesc);
+
+ /*
+ * Create a memory context we'll reset to combine results for a single
+ * page range (received from the workers). We don't expect huge number of
+ * overlaps under regular circumstances, because for large tables the
+ * chunk size is likely larger than the BRIN page range), but it can
+ * happen, and the union functions may do all kinds of stuff. So we better
+ * reset the context once in a while.
+ */
+ rangeCxt = AllocSetContextCreate(CurrentMemoryContext,
+ "brin union",
+ ALLOCSET_DEFAULT_SIZES);
+ oldCxt = MemoryContextSwitchTo(rangeCxt);
+
+ /*
+ * Read the BRIN tuples from the shared tuplesort, sorted by block number.
+ * That probably gives us an index that is cheaper to scan, thanks to
+ * mostly getting data from the same index page as before.
+ */
+ while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
+ {
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
+ /*
+ * Do we need to union summaries for the same page range?
+ *
+ * If this is the first brin tuple we read, then just deform it into
+ * the memtuple, and continue with the next one from tuplesort. We
+ * however may need to insert empty summaries into the index.
+ *
+ * If it's the same block as the last we saw, we simply union the brin
+ * tuple into it, and we're done - we don't even need to insert empty
+ * ranges, because that was done earlier when we saw the first brin
+ * tuple (for this range).
+ *
+ * Finally, if it's not the first brin tuple, and it's not the same
+ * page range, we need to do the insert and then deform the tuple into
+ * the memtuple. Then we'll insert empty ranges before the new brin
+ * tuple, if needed.
+ */
+ if (prevblkno == InvalidBlockNumber)
+ {
+ /* First brin tuples, just deform into memtuple. */
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+ else if (memtuple->bt_blkno == btup->bt_blkno)
+ {
+ /*
+ * Not the first brin tuple, but same page range as the previous
+ * one, so we can merge it into the memtuple.
+ */
+ union_tuples(state->bs_bdesc, memtuple, btup);
+ continue;
+ }
+ else
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ /*
+ * We got brin tuple for a different page range, so form a brin
+ * tuple from the memtuple, insert it, and re-init the memtuple
+ * from the new brin tuple.
+ */
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ /*
+ * Reset the per-output-range context. This frees all the memory
+ * possibly allocated by the union functions, and also the BRIN
+ * tuple we just formed and inserted.
+ */
+ MemoryContextReset(rangeCxt);
+
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ prevblkno = btup->bt_blkno;
+ }
+
+ tuplesort_end(spool->sortstate);
+
+ /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ /* Fill the BRIN tuple for the last page range. */
+ if (prevblkno != InvalidBlockNumber)
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ pfree(tmp);
+ }
+
+ /*
+ * Switch back to the originam memory context, and destroy the one we
+ * created to isolate the union_tuple calls.
+ */
+ MemoryContextSwitchTo(oldCxt);
+ MemoryContextDelete(rangeCxt);
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /* Allocate memory and initialize private spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = buildstate->bs_spool->heap;
+ buildstate->bs_spool->index = buildstate->bs_spool->index;
+
+ /*
+ * 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 / brinleader->nparticipanttuplesorts;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool, brinleader->brinshared,
+ brinleader->sharedsort, heap, index, sortmem, true);
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
+ BrinShared *brinshared, Sharedsort *sharedsort,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ SortCoordinate coordinate;
+ TableScanDesc 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 */
+ brinspool->sortstate = tuplesort_begin_index_brin(brinspool->heap,
+ brinspool->index,
+ sortmem, coordinate,
+ TUPLESORT_NONE);
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* insert the last item */
+ form_and_spill_tuple(state);
+
+ /* sort the BRIN ranges built by this worker */
+ tuplesort_performsort(brinspool->sortstate);
+
+ state->bs_reltuples += reltuples;
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+
+ tuplesort_end(brinspool->sortstate);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL, brinshared->pagesPerRange);
+
+ /* Initialize worker's own spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = heapRel;
+ buildstate->bs_spool->index = indexRel;
+
+ /* Look up shared state private to tuplesort.c */
+ sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
+ tuplesort_attach_shared(sharedsort, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /*
+ * 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 / brinshared->scantuplesortstates;
+
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool,
+ brinshared, sharedsort,
+ heapRel, indexRel, sortmem, false);
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/gin/ginutil.c b/src/backend/access/gin/ginutil.c
index a875c5d3d7..9b1a0ac345 100644
--- a/src/backend/access/gin/ginutil.c
+++ b/src/backend/access/gin/ginutil.c
@@ -54,6 +54,7 @@ ginhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = true;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/gist/gist.c b/src/backend/access/gist/gist.c
index 9a1bf8f66c..e052ba8bda 100644
--- a/src/backend/access/gist/gist.c
+++ b/src/backend/access/gist/gist.c
@@ -76,6 +76,7 @@ gisthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 6443ff21bd..905519692c 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -73,6 +73,7 @@ hashhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 0930f9b37e..6c8cd93fa0 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -112,6 +112,7 @@ bthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = true;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/spgist/spgutils.c b/src/backend/access/spgist/spgutils.c
index 30c00876a5..fd4b615710 100644
--- a/src/backend/access/spgist/spgutils.c
+++ b/src/backend/access/spgist/spgutils.c
@@ -60,6 +60,7 @@ spghandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 194a1207be..d78314062e 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -145,6 +146,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 143fae01eb..40abbaf476 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2982,7 +2982,7 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ indexRelation->rd_indam->amcanbuildparallel)
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 2cd508e513..90fc605f1c 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -19,6 +19,7 @@
#include "postgres.h"
+#include "access/brin_tuple.h"
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -43,6 +44,8 @@ static void removeabbrev_cluster(Tuplesortstate *state, SortTuple *stups,
int count);
static void removeabbrev_index(Tuplesortstate *state, SortTuple *stups,
int count);
+static void removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups,
+ int count);
static void removeabbrev_datum(Tuplesortstate *state, SortTuple *stups,
int count);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
@@ -69,10 +72,16 @@ static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash_tiebreak(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
+static int comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state);
static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len);
+static void writetup_index_brin(Tuplesortstate *state, LogicalTape *tape,
+ SortTuple *stup);
+static void readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_datum_tiebreak(const SortTuple *a, const SortTuple *b,
@@ -128,6 +137,16 @@ typedef struct
uint32 max_buckets;
} TuplesortIndexHashArg;
+/*
+ * Data struture pointed by "TuplesortPublic.arg" for the index_brin subcase.
+ */
+typedef struct
+{
+ TuplesortIndexArg index;
+
+ /* XXX do we need something here? */
+} TuplesortIndexBrinArg;
+
/*
* Data struture pointed by "TuplesortPublic.arg" for the Datum case.
* Set by tuplesort_begin_datum and used only by the DatumTuple routines.
@@ -140,6 +159,21 @@ typedef struct
int datumTypeLen;
} TuplesortDatumArg;
+/*
+ * Computing BrinTuple size with only the tuple is difficult, so we want to track
+ * the length referenced by the SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by its length.
+ */
+typedef struct BrinSortTuple
+{
+ Size tuplen;
+ BrinTuple tuple;
+} BrinSortTuple;
+
+/* Size of the BrinSortTuple, given length of the BrinTuple. */
+#define BRINSORTTUPLE_SIZE(len) (offsetof(BrinSortTuple, tuple) + (len))
+
+
Tuplesortstate *
tuplesort_begin_heap(TupleDesc tupDesc,
int nkeys, AttrNumber *attNums,
@@ -527,6 +561,47 @@ tuplesort_begin_index_gist(Relation heapRel,
return state;
}
+Tuplesortstate *
+tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem,
+ SortCoordinate coordinate,
+ int sortopt)
+{
+ Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
+ sortopt);
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext;
+ TuplesortIndexBrinArg *arg;
+
+ oldcontext = MemoryContextSwitchTo(base->maincontext);
+ arg = (TuplesortIndexBrinArg *) palloc(sizeof(TuplesortIndexBrinArg));
+
+#ifdef TRACE_SORT
+ if (trace_sort)
+ elog(LOG,
+ "begin index sort: workMem = %d, randomAccess = %c",
+ workMem,
+ sortopt & TUPLESORT_RANDOMACCESS ? 't' : 'f');
+#endif
+
+ base->nKeys = 1; /* Only one sort column, the block number */
+
+ base->removeabbrev = removeabbrev_index_brin;
+ base->comparetup = comparetup_index_brin;
+ base->writetup = writetup_index_brin;
+ base->readtup = readtup_index_brin;
+ base->haveDatum1 = true;
+ base->arg = arg;
+
+ arg->index.heapRel = heapRel;
+ arg->index.indexRel = indexRel;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return state;
+}
+
Tuplesortstate *
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag, int workMem,
@@ -707,6 +782,35 @@ tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
!stup.isnull1);
}
+/*
+ * Collect one BRIN tuple while collecting input data for sort.
+ */
+void
+tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
+{
+ SortTuple stup;
+ BrinSortTuple *bstup;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->tuplecontext);
+
+ /* allocate space for the whole BRIN sort tuple */
+ bstup = palloc(BRINSORTTUPLE_SIZE(size));
+
+ bstup->tuplen = size;
+ memcpy(&bstup->tuple, tuple, size);
+
+ stup.tuple = bstup;
+ stup.datum1 = tuple->bt_blkno;
+ stup.isnull1 = false;
+
+ tuplesort_puttuple_common(state, &stup,
+ base->sortKeys &&
+ base->sortKeys->abbrev_converter &&
+ !stup.isnull1);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
/*
* Accept one Datum while collecting input data for sort.
*
@@ -850,6 +954,35 @@ tuplesort_getindextuple(Tuplesortstate *state, bool forward)
return (IndexTuple) stup.tuple;
}
+/*
+ * Fetch the next BRIN tuple in either forward or back direction.
+ * Returns NULL if no more tuples. Returned tuple belongs to tuplesort memory
+ * context, and must not be freed by caller. Caller may not rely on tuple
+ * remaining valid after any further manipulation of tuplesort.
+ */
+BrinTuple *
+tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
+ SortTuple stup;
+ BrinSortTuple *btup;
+
+ if (!tuplesort_gettuple_common(state, forward, &stup))
+ stup.tuple = NULL;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!stup.tuple)
+ return NULL;
+
+ btup = (BrinSortTuple *) stup.tuple;
+
+ *len = btup->tuplen;
+
+ return &btup->tuple;
+}
+
/*
* Fetch the next Datum in either forward or back direction.
* Returns false if no more datums.
@@ -1564,6 +1697,80 @@ readtup_index(Tuplesortstate *state, SortTuple *stup,
&stup->isnull1);
}
+/*
+ * Routines specialized for BrinTuple case
+ */
+
+static void
+removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
+{
+ int i;
+
+ for (i = 0; i < count; i++)
+ {
+ BrinSortTuple *tuple;
+
+ tuple = stups[i].tuple;
+ stups[i].datum1 = tuple->tuple.bt_blkno;
+ }
+}
+
+static int
+comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+{
+ Assert(TuplesortstateGetPublic(state)->haveDatum1);
+
+ if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1))
+ return 1;
+
+ if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1))
+ return -1;
+
+ /* silence compilers */
+ return 0;
+}
+
+static void
+writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
+
+ tuplen = tuplen + sizeof(tuplen);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->tuple, tuple->tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+}
+
+static void
+readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len)
+{
+ BrinSortTuple *tuple;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ unsigned int tuplen = len - sizeof(unsigned int);
+
+ /*
+ * Allocate space for the BRIN sort tuple, which is BrinTuple with an
+ * extra length field.
+ */
+ tuple = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
+
+ tuple->tuplen = tuplen;
+
+ LogicalTapeReadExact(tape, &tuple->tuple, tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
+ stup->tuple = (void *) tuple;
+
+ /* set up first-column key value, which is block number */
+ stup->datum1 = tuple->tuple.bt_blkno;
+}
+
/*
* Routines specialized for DatumTuple case
*/
diff --git a/src/include/access/amapi.h b/src/include/access/amapi.h
index 244459587f..df85ae3aac 100644
--- a/src/include/access/amapi.h
+++ b/src/include/access/amapi.h
@@ -243,6 +243,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d5..3451ecb211 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index 9ed2de76cd..357eb35311 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -21,6 +21,7 @@
#ifndef TUPLESORT_H
#define TUPLESORT_H
+#include "access/brin_tuple.h"
#include "access/itup.h"
#include "executor/tuptable.h"
#include "storage/dsm.h"
@@ -282,6 +283,9 @@ typedef struct
* The "index_hash" API is similar to index_btree, but the tuples are
* actually sorted by their hash codes not the raw data.
*
+ * The "index_brin" API is similar to index_btree, but the tuples are
+ * BrinTuple and are sorted by their block number not the raw data.
+ *
* Parallel sort callers are required to coordinate multiple tuplesort states
* in a leader process and one or more worker processes. The leader process
* must launch workers, and have each perform an independent "partial"
@@ -426,6 +430,10 @@ extern Tuplesortstate *tuplesort_begin_index_gist(Relation heapRel,
Relation indexRel,
int workMem, SortCoordinate coordinate,
int sortopt);
+extern Tuplesortstate *tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem, SortCoordinate coordinate,
+ int sortopt);
extern Tuplesortstate *tuplesort_begin_datum(Oid datumType,
Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag,
@@ -438,6 +446,7 @@ extern void tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup);
extern void tuplesort_putindextuplevalues(Tuplesortstate *state,
Relation rel, ItemPointer self,
const Datum *values, const bool *isnull);
+extern void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tup, Size len);
extern void tuplesort_putdatum(Tuplesortstate *state, Datum val,
bool isNull);
@@ -445,6 +454,8 @@ extern bool tuplesort_gettupleslot(Tuplesortstate *state, bool forward,
bool copy, TupleTableSlot *slot, Datum *abbrev);
extern HeapTuple tuplesort_getheaptuple(Tuplesortstate *state, bool forward);
extern IndexTuple tuplesort_getindextuple(Tuplesortstate *state, bool forward);
+extern BrinTuple *tuplesort_getbrintuple(Tuplesortstate *state, Size *len,
+ bool forward);
extern bool tuplesort_getdatum(Tuplesortstate *state, bool forward, bool copy,
Datum *val, bool *isNull, Datum *abbrev);
diff --git a/src/test/modules/dummy_index_am/dummy_index_am.c b/src/test/modules/dummy_index_am/dummy_index_am.c
index cbdae7ab7a..eaa0c483b7 100644
--- a/src/test/modules/dummy_index_am/dummy_index_am.c
+++ b/src/test/modules/dummy_index_am/dummy_index_am.c
@@ -294,6 +294,7 @@ dihandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 86a9886d4f..001fef5865 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -297,13 +297,17 @@ BpChar
BrinBuildState
BrinDesc
BrinInsertState
+BrinLeader
BrinMemTuple
BrinMetaPageData
BrinOpaque
BrinOpcInfo
BrinOptions
BrinRevmap
+BrinShared
+BrinSortTuple
BrinSpecialSpace
+BrinSpool
BrinStatsData
BrinTuple
BrinValues
@@ -2879,6 +2883,7 @@ TupleTableSlotOps
TuplesortClusterArg
TuplesortDatumArg
TuplesortIndexArg
+TuplesortIndexBrinArg
TuplesortIndexBTreeArg
TuplesortIndexHashArg
TuplesortInstrumentation
--
2.40.1
On 11/29/23 21:30, Matthias van de Meent wrote:
On Wed, 29 Nov 2023 at 18:55, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:On 11/29/23 15:52, Tomas Vondra wrote:
...
This also made me think a bit more about how we're working with the
tuples. With your latest patch, we always deserialize and re-serialize
the sorted brin tuples, just in case the next tuple will also be a
BRIN tuple of the same page range. Could we save some of that
deserialization time by optimistically expecting that we're not going
to need to merge the tuple and only store a local copy of it locally?
See attached 0002; this saves some cycles in common cases.Good idea!
FWIW there's a bug, in this part of the optimization:
------------------ + if (memtuple == NULL) + memtuple = brin_deform_tuple(state->bs_bdesc, btup, + memtup_holder); + union_tuples(state->bs_bdesc, memtuple, btup); continue; ------------------The deforming should use prevbtup, otherwise union_tuples() jut combines
two copies of the same tuple.Good point. There were some more issues as well, fixes are attached.
Which however brings me to the bigger issue with this - my stress test
found this issue pretty quickly, but then I spent quite a bit of time
trying to find what went wrong. I find this reworked code pretty hard to
understand, and not necessarily because of how it's written. The problem
is it the same loop tries to juggle multiple pieces of information with
different lifespans, and so on. I find it really hard to reason about
how it behaves ...Yeah, it'd be nice if we had a peek option for sortsupport, that'd
improve context handling.I did try to measure how much it actually saves, but none of the tests I
did actually found measurable improvement. So I'm tempted to just not
include this part, and accept that we may deserialize some of the tuples
unnecessarily.Did you actually observe measurable improvements in some cases?
The improvements would mostly stem from brin indexes with multiple
(potentially compressed) by-ref types, as they go through more complex
and expensive code to deserialize, requiring separate palloc() and
memcpy() calls each.
For single-column and by-value types the improvements are expected to
be negligible, because there is no meaningful difference between
copying a single by-ref value and copying its container; the
additional work done for each tuple is marginal for those.For an 8-column BRIN index ((sha256((id)::text::bytea)::text),
(sha256((id+1)::text::bytea)::text),
(sha256((id+2)::text::bytea)::text), ...) instrumented with 0003 I
measured a difference of 10x less time spent in the main loop of
_brin_end_parallel, from ~30ms to 3ms when dealing with 55k 1-block
ranges. It's not a lot, but worth at least something, I guess?
It is something, but I can't really convince myself it's worth the extra
code complexity. It's a somewhat extreme example, and the parallelism
certainly saves much more than this.
The attached patch fixes the issue that you called out .
It also further updates _brin_end_parallel: the final 'write empty
tuples' loop is never hit and is thus removed, because if there were
any tuples in the spool we'd have filled the empty ranges at the end
of the main loop, and if there were no tuples in the spool then the
memtuple would still be at its original initialized value of 0 thus
resulting in a constant false condition. I also updated some comments.
Ah, right. I'll take a look tomorrow, but I guess I didn't realize we
insert the empty ranges in the main loop, because we're already looking
at the *next* summary.
But I think the idea was to insert empty ranges if there's a chunk of
empty ranges at the end of the table, after the last tuple the index
build reads. But I'm not sure that can actually happen ...
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On Wed, 29 Nov 2023 at 21:56, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:
On 11/29/23 21:30, Matthias van de Meent wrote:
On Wed, 29 Nov 2023 at 18:55, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:I did try to measure how much it actually saves, but none of the tests I
did actually found measurable improvement. So I'm tempted to just not
include this part, and accept that we may deserialize some of the tuples
unnecessarily.Did you actually observe measurable improvements in some cases?
The improvements would mostly stem from brin indexes with multiple
(potentially compressed) by-ref types, as they go through more complex
and expensive code to deserialize, requiring separate palloc() and
memcpy() calls each.
For single-column and by-value types the improvements are expected to
be negligible, because there is no meaningful difference between
copying a single by-ref value and copying its container; the
additional work done for each tuple is marginal for those.For an 8-column BRIN index ((sha256((id)::text::bytea)::text),
(sha256((id+1)::text::bytea)::text),
(sha256((id+2)::text::bytea)::text), ...) instrumented with 0003 I
measured a difference of 10x less time spent in the main loop of
_brin_end_parallel, from ~30ms to 3ms when dealing with 55k 1-block
ranges. It's not a lot, but worth at least something, I guess?It is something, but I can't really convince myself it's worth the extra
code complexity. It's a somewhat extreme example, and the parallelism
certainly saves much more than this.
True. For this, I usually keep in mind that the docs on multi-column
indexes still indicate to use 1 N-column brin index over N 1-column
brin indexes (assuming the same storage parameters), so multi-column
BRIN indexes should not be considered to be uncommon:
"The only reason to have multiple BRIN indexes instead of one
multicolumn BRIN index on a single table is to have a different
pages_per_range storage parameter."
Note that most of the time in my example index is spent in creating
the actual tuples due to the use of hashing for data generation; for
index or plain to-text formatting the improvement is much more
pronounced: If I use an 8-column index (id::text, id, ...), index
creation takes ~500ms with 4+ workers. Of this, deforming takes some
20ms, though when skipping the deforming step (i.e.,with my patch) it
takes ~3.5ms. That's a 3% shaved off the build time when the index
shape is beneficial.
The attached patch fixes the issue that you called out .
It also further updates _brin_end_parallel: the final 'write empty
tuples' loop is never hit and is thus removed, because if there were
any tuples in the spool we'd have filled the empty ranges at the end
of the main loop, and if there were no tuples in the spool then the
memtuple would still be at its original initialized value of 0 thus
resulting in a constant false condition. I also updated some comments.Ah, right. I'll take a look tomorrow, but I guess I didn't realize we
insert the empty ranges in the main loop, because we're already looking
at the *next* summary.
Yes, merging adds some significant complexity here. I don't think we
can easily get around that though...
But I think the idea was to insert empty ranges if there's a chunk of
empty ranges at the end of the table, after the last tuple the index
build reads. But I'm not sure that can actually happen ...
This would be trivial to construct with partial indexes; e.g. WHERE
(my_pk IS NULL) would consist of exclusively empty ranges.
I don't see a lot of value in partial BRIN indexes, but I may be
overlooking something.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech)
On 11/29/23 23:59, Matthias van de Meent wrote:
On Wed, 29 Nov 2023 at 21:56, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:On 11/29/23 21:30, Matthias van de Meent wrote:
On Wed, 29 Nov 2023 at 18:55, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:I did try to measure how much it actually saves, but none of the tests I
did actually found measurable improvement. So I'm tempted to just not
include this part, and accept that we may deserialize some of the tuples
unnecessarily.Did you actually observe measurable improvements in some cases?
The improvements would mostly stem from brin indexes with multiple
(potentially compressed) by-ref types, as they go through more complex
and expensive code to deserialize, requiring separate palloc() and
memcpy() calls each.
For single-column and by-value types the improvements are expected to
be negligible, because there is no meaningful difference between
copying a single by-ref value and copying its container; the
additional work done for each tuple is marginal for those.For an 8-column BRIN index ((sha256((id)::text::bytea)::text),
(sha256((id+1)::text::bytea)::text),
(sha256((id+2)::text::bytea)::text), ...) instrumented with 0003 I
measured a difference of 10x less time spent in the main loop of
_brin_end_parallel, from ~30ms to 3ms when dealing with 55k 1-block
ranges. It's not a lot, but worth at least something, I guess?It is something, but I can't really convince myself it's worth the extra
code complexity. It's a somewhat extreme example, and the parallelism
certainly saves much more than this.True. For this, I usually keep in mind that the docs on multi-column
indexes still indicate to use 1 N-column brin index over N 1-column
brin indexes (assuming the same storage parameters), so multi-column
BRIN indexes should not be considered to be uncommon:"The only reason to have multiple BRIN indexes instead of one
multicolumn BRIN index on a single table is to have a different
pages_per_range storage parameter."Note that most of the time in my example index is spent in creating
the actual tuples due to the use of hashing for data generation; for
index or plain to-text formatting the improvement is much more
pronounced: If I use an 8-column index (id::text, id, ...), index
creation takes ~500ms with 4+ workers. Of this, deforming takes some
20ms, though when skipping the deforming step (i.e.,with my patch) it
takes ~3.5ms. That's a 3% shaved off the build time when the index
shape is beneficial.
That's all true, and while 3.5% is not something to ignore, my POV is
that the parallelism speeds this up from ~2000ms to ~500ms. Yes, it
would be great to shave off the extra 1% (relative to the original
duration). But I don't have a great idea how to do code that in a way
that is readable, and I don't want to stall the patch indefinitely
because of a comparatively small improvement.
Therefore I propose we get the simpler code committed and leave this as
a future improvement.
The attached patch fixes the issue that you called out .
It also further updates _brin_end_parallel: the final 'write empty
tuples' loop is never hit and is thus removed, because if there were
any tuples in the spool we'd have filled the empty ranges at the end
of the main loop, and if there were no tuples in the spool then the
memtuple would still be at its original initialized value of 0 thus
resulting in a constant false condition. I also updated some comments.Ah, right. I'll take a look tomorrow, but I guess I didn't realize we
insert the empty ranges in the main loop, because we're already looking
at the *next* summary.Yes, merging adds some significant complexity here. I don't think we
can easily get around that though...But I think the idea was to insert empty ranges if there's a chunk of
empty ranges at the end of the table, after the last tuple the index
build reads. But I'm not sure that can actually happen ...This would be trivial to construct with partial indexes; e.g. WHERE
(my_pk IS NULL) would consist of exclusively empty ranges.
I don't see a lot of value in partial BRIN indexes, but I may be
overlooking something.
Oh, I haven't even thought about partial BRIN indexes! I'm sure for
those it's even more important to actually fill-in the empty ranges,
otherwise we end up scanning the whole supposedly filtered-out part of
the table. I'll do some testing with that.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On Thu, 30 Nov 2023 at 01:10, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:
On 11/29/23 23:59, Matthias van de Meent wrote:
On Wed, 29 Nov 2023 at 21:56, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:On 11/29/23 21:30, Matthias van de Meent wrote:
On Wed, 29 Nov 2023 at 18:55, Tomas Vondra
<tomas.vondra@enterprisedb.com> wrote:I did try to measure how much it actually saves, but none of the tests I
did actually found measurable improvement. So I'm tempted to just not
include this part, and accept that we may deserialize some of the tuples
unnecessarily.Did you actually observe measurable improvements in some cases?
The improvements would mostly stem from brin indexes with multiple
(potentially compressed) by-ref types, as they go through more complex
and expensive code to deserialize, requiring separate palloc() and
memcpy() calls each.
For single-column and by-value types the improvements are expected to
be negligible, because there is no meaningful difference between
copying a single by-ref value and copying its container; the
additional work done for each tuple is marginal for those.For an 8-column BRIN index ((sha256((id)::text::bytea)::text),
(sha256((id+1)::text::bytea)::text),
(sha256((id+2)::text::bytea)::text), ...) instrumented with 0003 I
measured a difference of 10x less time spent in the main loop of
_brin_end_parallel, from ~30ms to 3ms when dealing with 55k 1-block
ranges. It's not a lot, but worth at least something, I guess?It is something, but I can't really convince myself it's worth the extra
code complexity. It's a somewhat extreme example, and the parallelism
certainly saves much more than this.True. For this, I usually keep in mind that the docs on multi-column
indexes still indicate to use 1 N-column brin index over N 1-column
brin indexes (assuming the same storage parameters), so multi-column
BRIN indexes should not be considered to be uncommon:"The only reason to have multiple BRIN indexes instead of one
multicolumn BRIN index on a single table is to have a different
pages_per_range storage parameter."Note that most of the time in my example index is spent in creating
the actual tuples due to the use of hashing for data generation; for
index or plain to-text formatting the improvement is much more
pronounced: If I use an 8-column index (id::text, id, ...), index
creation takes ~500ms with 4+ workers. Of this, deforming takes some
20ms, though when skipping the deforming step (i.e.,with my patch) it
takes ~3.5ms. That's a 3% shaved off the build time when the index
shape is beneficial.That's all true, and while 3.5% is not something to ignore, my POV is
that the parallelism speeds this up from ~2000ms to ~500ms. Yes, it
would be great to shave off the extra 1% (relative to the original
duration). But I don't have a great idea how to do code that in a way
that is readable, and I don't want to stall the patch indefinitely
because of a comparatively small improvement.Therefore I propose we get the simpler code committed and leave this as
a future improvement.
That's fine with me, it is one reason why I kept it as a separate patch file.
The attached patch fixes the issue that you called out .
It also further updates _brin_end_parallel: the final 'write empty
tuples' loop is never hit and is thus removed, because if there were
any tuples in the spool we'd have filled the empty ranges at the end
of the main loop, and if there were no tuples in the spool then the
memtuple would still be at its original initialized value of 0 thus
resulting in a constant false condition. I also updated some comments.Ah, right. I'll take a look tomorrow, but I guess I didn't realize we
insert the empty ranges in the main loop, because we're already looking
at the *next* summary.Yes, merging adds some significant complexity here. I don't think we
can easily get around that though...But I think the idea was to insert empty ranges if there's a chunk of
empty ranges at the end of the table, after the last tuple the index
build reads. But I'm not sure that can actually happen ...This would be trivial to construct with partial indexes; e.g. WHERE
(my_pk IS NULL) would consist of exclusively empty ranges.
I don't see a lot of value in partial BRIN indexes, but I may be
overlooking something.Oh, I haven't even thought about partial BRIN indexes! I'm sure for
those it's even more important to actually fill-in the empty ranges,
otherwise we end up scanning the whole supposedly filtered-out part of
the table. I'll do some testing with that.
I just ran some more tests in less favorable environments, and it
looks like I hit a bug:
% SET max_parallel_workers = 0;
% CREATE INDEX ... USING brin (...);
ERROR: cannot update tuples during a parallel operation
Fix attached in 0002.
In 0003 I add the mentioned backfilling of empty ranges at the end of
the table. I added it for both normal and parallel index builds, as
normal builds apparently also didn't yet have this yet.
Kind regards,
Matthias van de Meent
Attachments:
v7-0002-BRIN-Exit-parallel-mode-when-not-starting-paralle.patchapplication/octet-stream; name=v7-0002-BRIN-Exit-parallel-mode-when-not-starting-paralle.patchDownload
From 8303469748a412525ad55aa9bba0de9ead3c26bf Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Thu, 30 Nov 2023 18:19:40 +0100
Subject: [PATCH v7 2/3] BRIN: Exit parallel mode when not starting parallel
create index
---
src/backend/access/brin/brin.c | 11 +++++++++++
1 file changed, 11 insertions(+)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 001cf04aac..251350fde2 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -2483,8 +2483,19 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
+ /*
+ * If we didn't actually launch workers, we still have to make sure to exit
+ * parallel mode.
+ */
if (!state)
+ {
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
return;
+ }
/* copy the data into leader state (we have to wait for the workers ) */
state->bs_reltuples = brinshared->reltuples;
--
2.40.1
v7-0003-BRIN-Backfill-empty-ranges.patchapplication/octet-stream; name=v7-0003-BRIN-Backfill-empty-ranges.patchDownload
From 922460a1d56c0365770bfb08bdd8e2b291164362 Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Thu, 30 Nov 2023 18:12:16 +0100
Subject: [PATCH v7 3/3] BRIN: Backfill empty ranges
If we don't, then an index with WHERE (practically_nonnull IS NULL)
would be a full table scan due to missing entries in the ranges table.
The issue is fixed for both normal and parallel index creation.
---
src/backend/access/brin/brin.c | 69 ++++++++++++++++++++++++----------
1 file changed, 49 insertions(+), 20 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 251350fde2..9d2179e6d7 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -165,6 +165,7 @@ typedef struct BrinBuildState
double bs_numtuples;
double bs_reltuples;
Buffer bs_currentInsertBuf;
+ BlockNumber bs_tablePages;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
@@ -1134,6 +1135,7 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
state->bs_spool->heap = heap;
state->bs_spool->index = index;
+ state->bs_tablePages = RelationGetNumberOfBlocks(heap);
/*
* Attempt to launch parallel worker scan when required
@@ -1208,6 +1210,23 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
/* process the final batch */
form_and_insert_tuple(state);
+ state->bs_currRangeStart += state->bs_pagesPerRange;
+ /*
+ * Backfill the final ranges with empty data.
+ *
+ * This saves us from doing what amounts to full table scans when the
+ * index is built on stupid index quals like WHERE (nonnull_column IS
+ * NULL).
+ */
+ while (state->bs_currRangeStart + state->bs_pagesPerRange - 1 < state->bs_tablePages)
+ {
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+
+ form_and_insert_tuple(state);
+
+ state->bs_currRangeStart += state->bs_pagesPerRange;
+ }
+
/* track the number of relation tuples */
state->bs_reltuples = reltuples;
}
@@ -2625,13 +2644,34 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
tuplesort_end(spool->sortstate);
- /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
- prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
- while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ /* Fill the BRIN tuple for the last page range with data. */
+ if (prevblkno != InvalidBlockNumber)
{
- /* the missing range */
+ BrinTuple *tmp;
+ Size len;
+
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ pfree(tmp);
+ }
+
+ /*
+ * Fill empty ranges at the end, for all ranges missing in the tuplesort.
+ *
+ * Starting from here, prevblkno is the to-be-inserted range's start block
+ * number. Note that we don't fill in the relation's last page range.
+ */
+ if (prevblkno == InvalidBlockNumber)
+ prevblkno = 0;
+ else
prevblkno += state->bs_pagesPerRange;
+ while (prevblkno + state->bs_pagesPerRange < state->bs_tablePages)
+ {
/* Did we already build the empty range? If not, do it now. */
if (emptyTuple == NULL)
{
@@ -2641,32 +2681,21 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
}
else
{
- /* we already have am "empty range" tuple, just set the block */
+ /* we already have an "empty range" tuple, just set the block */
emptyTuple->bt_blkno = prevblkno;
}
+ /* Insert the missing range */
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf,
emptyTuple->bt_blkno, emptyTuple, emptySize);
- }
- /* Fill the BRIN tuple for the last page range. */
- if (prevblkno != InvalidBlockNumber)
- {
- BrinTuple *tmp;
- Size len;
-
- tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
- memtuple, &len);
-
- brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
- &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
-
- pfree(tmp);
+ /* ... and update to the next range's block number */
+ prevblkno += state->bs_pagesPerRange;
}
/*
- * Switch back to the originam memory context, and destroy the one we
+ * Switch back to the original memory context, and destroy the one we
* created to isolate the union_tuple calls.
*/
MemoryContextSwitchTo(oldCxt);
--
2.40.1
v7-0001-Allow-BRIN-to-build-its-index-in-parallel.patchapplication/octet-stream; name=v7-0001-Allow-BRIN-to-build-its-index-in-parallel.patchDownload
From 48ea20bc3d991ea57475ead179c510a420d5e406 Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Wed, 29 Nov 2023 14:35:10 +0100
Subject: [PATCH v7 1/3] Allow BRIN to build its index in parallel
---
contrib/bloom/blutils.c | 1 +
doc/src/sgml/indexam.sgml | 7 +
src/backend/access/brin/brin.c | 894 +++++++++++++++++-
src/backend/access/gin/ginutil.c | 1 +
src/backend/access/gist/gist.c | 1 +
src/backend/access/hash/hash.c | 1 +
src/backend/access/nbtree/nbtree.c | 1 +
src/backend/access/spgist/spgutils.c | 1 +
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 2 +-
src/backend/utils/sort/tuplesortvariants.c | 207 ++++
src/include/access/amapi.h | 2 +
src/include/access/brin.h | 3 +
src/include/utils/tuplesort.h | 11 +
.../modules/dummy_index_am/dummy_index_am.c | 1 +
src/tools/pgindent/typedefs.list | 5 +
16 files changed, 1136 insertions(+), 6 deletions(-)
diff --git a/contrib/bloom/blutils.c b/contrib/bloom/blutils.c
index 4830cb3fee..a781c5d98d 100644
--- a/contrib/bloom/blutils.c
+++ b/contrib/bloom/blutils.c
@@ -122,6 +122,7 @@ blhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amparallelvacuumoptions =
diff --git a/doc/src/sgml/indexam.sgml b/doc/src/sgml/indexam.sgml
index f107c43d6a..cc4135e394 100644
--- a/doc/src/sgml/indexam.sgml
+++ b/doc/src/sgml/indexam.sgml
@@ -123,6 +123,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
@@ -286,6 +288,11 @@ ambuild (Relation heapRelation,
and compute the keys that need to be inserted into the index.
The function must return a palloc'd struct containing statistics about
the new index.
+ The <structfield>amcanbuildparallel</structfield> flag indicates whether
+ the access method supports parallel index builds. When set to <literal>true</literal>,
+ the system will attempt to allocate parallel workers for the build.
+ Access methods supporting only non-parallel index builds should leave
+ this flag set to <literal>false</literal>.
</para>
<para>
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 4f2dfdd17b..001cf04aac 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -33,6 +33,7 @@
#include "postmaster/autovacuum.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -40,7 +41,119 @@
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+#include "utils/tuplesort.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xB000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xB000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xB000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xB000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xB000000000000005)
+
+/*
+ * Status record for spooling/sorting phase.
+ */
+typedef struct BrinSpool
+{
+ Tuplesortstate *sortstate; /* state data for tuplesort.c */
+ Relation heap;
+ Relation index;
+} BrinSpool;
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist
+ * for the benefit of worker processes that need to create state
+ * corresponding to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+ int scantuplesortstates;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN 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 the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
+} BrinShared;
+
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* 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).
+ *
+ * brinshared is the shared state for entire build. sharedsort is the
+ * shared, tuplesort-managed state passed to each process tuplesort.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is
+ * required.
+ */
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -49,13 +162,23 @@
typedef struct BrinBuildState
{
Relation bs_irel;
- int bs_numtuples;
+ double bs_numtuples;
+ double bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader process has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BrinSpool *bs_spool;
} BrinBuildState;
/*
@@ -88,6 +211,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -95,6 +219,20 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinSpool *brinspool,
+ BrinShared *brinshared,
+ Sharedsort *sharedsort,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -119,6 +257,7 @@ brinhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = true;
@@ -874,6 +1013,63 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a shared tuplesort, and leave the insertion up to the leader (which may
+ * reorder them a bit etc.). The callback also does not generate empty ranges,
+ * those may be added by the leader when merging results from workers.
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; we do not create emptry BRIN
+ * ranges here - the leader is responsible for filling them in.
+ */
+ if (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the tuplesort */
+ form_and_spill_tuple(state);
+
+ /*
+ * Set state to correspond to the next range (for this block).
+ *
+ * This skips ranges that are either empty (and so we don't get any
+ * tuples to summarize), or processes by other workers. We can't
+ * differentiate those cases here easily, so we leave it up to the
+ * leader to fill empty ranges where needed.
+ */
+ state->bs_currRangeStart
+ = state->bs_pagesPerRange * (thisblock / state->bs_pagesPerRange);
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -935,18 +1131,90 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
revmap = brinRevmapInitialize(index, &pagesPerRange);
state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool->heap = heap;
+ state->bs_spool->index = index;
+
+ /*
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
+ */
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
+
/*
* Now scan the relation. No syncscan allowed here because we want the
* heap blocks in physical order.
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
- form_and_insert_tuple(state);
+ /*
+ * If parallel build requested and at least one worker process was
+ * successfully launched, set up coordination state
+ */
+ if (state->bs_leader)
+ {
+ SortCoordinate coordinate;
+
+ coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
+ coordinate->isWorker = false;
+ coordinate->nParticipants =
+ state->bs_leader->nparticipanttuplesorts;
+ coordinate->sharedsort = state->bs_leader->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.
+ */
+ state->bs_spool->sortstate =
+ tuplesort_begin_index_brin(heap, index,
+ maintenance_work_mem, coordinate,
+ TUPLESORT_NONE);
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read all
+ * tuples from the shared tuplesort and insert them into the index.
+ */
+ _brin_end_parallel(state->bs_leader, state);
+ }
+ else /* no parallel index build */
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
+ form_and_insert_tuple(state);
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
+ }
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1366,12 +1634,16 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
+ state->bs_spool = NULL;
return state;
}
@@ -1663,6 +1935,32 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a (shared) tuplesort (the leader will insert it
+ * into the index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ /* don't insert empty tuples in parallel build */
+ if (state->bs_dtuple->bt_empty_range)
+ return;
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ /* write the BRIN tuple to the tuplesort */
+ tuplesort_putbrintuple(state->bs_spool->sortstate, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -1982,3 +2280,589 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ int scantuplesortstates;
+ Snapshot snapshot;
+ Size estbrinshared;
+ Size estsort;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ request);
+
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+ estsort = tuplesort_estimate_shared(scantuplesortstates);
+ shm_toc_estimate_chunk(&pcxt->estimator, estsort);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->scantuplesortstates = scantuplesortstates;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ 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);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipanttuplesorts = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipanttuplesorts++;
+ brinleader->brinshared = brinshared;
+ brinleader->sharedsort = sharedsort;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinTuple *btup;
+ BrinMemTuple *memtuple = NULL;
+ Size tuplen;
+ BrinShared *brinshared = brinleader->brinshared;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+ BrinSpool *spool;
+ MemoryContext rangeCxt,
+ oldCxt;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ /* do the actual sort in the leader */
+ spool = state->bs_spool;
+ tuplesort_performsort(spool->sortstate);
+
+ /*
+ * Initialize BrinMemTuple we'll use to union summaries from workers (in
+ * case they happened to produce parts of the same paga range).
+ */
+ memtuple = brin_new_memtuple(state->bs_bdesc);
+
+ /*
+ * Create a memory context we'll reset to combine results for a single
+ * page range (received from the workers). We don't expect huge number of
+ * overlaps under regular circumstances, because for large tables the
+ * chunk size is likely larger than the BRIN page range), but it can
+ * happen, and the union functions may do all kinds of stuff. So we better
+ * reset the context once in a while.
+ */
+ rangeCxt = AllocSetContextCreate(CurrentMemoryContext,
+ "brin union",
+ ALLOCSET_DEFAULT_SIZES);
+ oldCxt = MemoryContextSwitchTo(rangeCxt);
+
+ /*
+ * Read the BRIN tuples from the shared tuplesort, sorted by block number.
+ * That probably gives us an index that is cheaper to scan, thanks to
+ * mostly getting data from the same index page as before.
+ */
+ while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
+ {
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
+ /*
+ * Do we need to union summaries for the same page range?
+ *
+ * If this is the first brin tuple we read, then just deform it into
+ * the memtuple, and continue with the next one from tuplesort. We
+ * however may need to insert empty summaries into the index.
+ *
+ * If it's the same block as the last we saw, we simply union the brin
+ * tuple into it, and we're done - we don't even need to insert empty
+ * ranges, because that was done earlier when we saw the first brin
+ * tuple (for this range).
+ *
+ * Finally, if it's not the first brin tuple, and it's not the same
+ * page range, we need to do the insert and then deform the tuple into
+ * the memtuple. Then we'll insert empty ranges before the new brin
+ * tuple, if needed.
+ */
+ if (prevblkno == InvalidBlockNumber)
+ {
+ /* First brin tuples, just deform into memtuple. */
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+ else if (memtuple->bt_blkno == btup->bt_blkno)
+ {
+ /*
+ * Not the first brin tuple, but same page range as the previous
+ * one, so we can merge it into the memtuple.
+ */
+ union_tuples(state->bs_bdesc, memtuple, btup);
+ continue;
+ }
+ else
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ /*
+ * We got brin tuple for a different page range, so form a brin
+ * tuple from the memtuple, insert it, and re-init the memtuple
+ * from the new brin tuple.
+ */
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ /*
+ * Reset the per-output-range context. This frees all the memory
+ * possibly allocated by the union functions, and also the BRIN
+ * tuple we just formed and inserted.
+ */
+ MemoryContextReset(rangeCxt);
+
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ prevblkno = btup->bt_blkno;
+ }
+
+ tuplesort_end(spool->sortstate);
+
+ /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ /* Fill the BRIN tuple for the last page range. */
+ if (prevblkno != InvalidBlockNumber)
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ pfree(tmp);
+ }
+
+ /*
+ * Switch back to the originam memory context, and destroy the one we
+ * created to isolate the union_tuple calls.
+ */
+ MemoryContextSwitchTo(oldCxt);
+ MemoryContextDelete(rangeCxt);
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /* Allocate memory and initialize private spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = buildstate->bs_spool->heap;
+ buildstate->bs_spool->index = buildstate->bs_spool->index;
+
+ /*
+ * 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 / brinleader->nparticipanttuplesorts;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool, brinleader->brinshared,
+ brinleader->sharedsort, heap, index, sortmem, true);
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
+ BrinShared *brinshared, Sharedsort *sharedsort,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ SortCoordinate coordinate;
+ TableScanDesc 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 */
+ brinspool->sortstate = tuplesort_begin_index_brin(brinspool->heap,
+ brinspool->index,
+ sortmem, coordinate,
+ TUPLESORT_NONE);
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* insert the last item */
+ form_and_spill_tuple(state);
+
+ /* sort the BRIN ranges built by this worker */
+ tuplesort_performsort(brinspool->sortstate);
+
+ state->bs_reltuples += reltuples;
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+
+ tuplesort_end(brinspool->sortstate);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL, brinshared->pagesPerRange);
+
+ /* Initialize worker's own spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = heapRel;
+ buildstate->bs_spool->index = indexRel;
+
+ /* Look up shared state private to tuplesort.c */
+ sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
+ tuplesort_attach_shared(sharedsort, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /*
+ * 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 / brinshared->scantuplesortstates;
+
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool,
+ brinshared, sharedsort,
+ heapRel, indexRel, sortmem, false);
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/gin/ginutil.c b/src/backend/access/gin/ginutil.c
index a875c5d3d7..9b1a0ac345 100644
--- a/src/backend/access/gin/ginutil.c
+++ b/src/backend/access/gin/ginutil.c
@@ -54,6 +54,7 @@ ginhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = true;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/gist/gist.c b/src/backend/access/gist/gist.c
index 9a1bf8f66c..e052ba8bda 100644
--- a/src/backend/access/gist/gist.c
+++ b/src/backend/access/gist/gist.c
@@ -76,6 +76,7 @@ gisthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 6443ff21bd..905519692c 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -73,6 +73,7 @@ hashhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 0930f9b37e..6c8cd93fa0 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -112,6 +112,7 @@ bthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = true;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/spgist/spgutils.c b/src/backend/access/spgist/spgutils.c
index 30c00876a5..fd4b615710 100644
--- a/src/backend/access/spgist/spgutils.c
+++ b/src/backend/access/spgist/spgutils.c
@@ -60,6 +60,7 @@ spghandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 194a1207be..d78314062e 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -145,6 +146,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 143fae01eb..40abbaf476 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2982,7 +2982,7 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ indexRelation->rd_indam->amcanbuildparallel)
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 2cd508e513..90fc605f1c 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -19,6 +19,7 @@
#include "postgres.h"
+#include "access/brin_tuple.h"
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -43,6 +44,8 @@ static void removeabbrev_cluster(Tuplesortstate *state, SortTuple *stups,
int count);
static void removeabbrev_index(Tuplesortstate *state, SortTuple *stups,
int count);
+static void removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups,
+ int count);
static void removeabbrev_datum(Tuplesortstate *state, SortTuple *stups,
int count);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
@@ -69,10 +72,16 @@ static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash_tiebreak(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
+static int comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state);
static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len);
+static void writetup_index_brin(Tuplesortstate *state, LogicalTape *tape,
+ SortTuple *stup);
+static void readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_datum_tiebreak(const SortTuple *a, const SortTuple *b,
@@ -128,6 +137,16 @@ typedef struct
uint32 max_buckets;
} TuplesortIndexHashArg;
+/*
+ * Data struture pointed by "TuplesortPublic.arg" for the index_brin subcase.
+ */
+typedef struct
+{
+ TuplesortIndexArg index;
+
+ /* XXX do we need something here? */
+} TuplesortIndexBrinArg;
+
/*
* Data struture pointed by "TuplesortPublic.arg" for the Datum case.
* Set by tuplesort_begin_datum and used only by the DatumTuple routines.
@@ -140,6 +159,21 @@ typedef struct
int datumTypeLen;
} TuplesortDatumArg;
+/*
+ * Computing BrinTuple size with only the tuple is difficult, so we want to track
+ * the length referenced by the SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by its length.
+ */
+typedef struct BrinSortTuple
+{
+ Size tuplen;
+ BrinTuple tuple;
+} BrinSortTuple;
+
+/* Size of the BrinSortTuple, given length of the BrinTuple. */
+#define BRINSORTTUPLE_SIZE(len) (offsetof(BrinSortTuple, tuple) + (len))
+
+
Tuplesortstate *
tuplesort_begin_heap(TupleDesc tupDesc,
int nkeys, AttrNumber *attNums,
@@ -527,6 +561,47 @@ tuplesort_begin_index_gist(Relation heapRel,
return state;
}
+Tuplesortstate *
+tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem,
+ SortCoordinate coordinate,
+ int sortopt)
+{
+ Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
+ sortopt);
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext;
+ TuplesortIndexBrinArg *arg;
+
+ oldcontext = MemoryContextSwitchTo(base->maincontext);
+ arg = (TuplesortIndexBrinArg *) palloc(sizeof(TuplesortIndexBrinArg));
+
+#ifdef TRACE_SORT
+ if (trace_sort)
+ elog(LOG,
+ "begin index sort: workMem = %d, randomAccess = %c",
+ workMem,
+ sortopt & TUPLESORT_RANDOMACCESS ? 't' : 'f');
+#endif
+
+ base->nKeys = 1; /* Only one sort column, the block number */
+
+ base->removeabbrev = removeabbrev_index_brin;
+ base->comparetup = comparetup_index_brin;
+ base->writetup = writetup_index_brin;
+ base->readtup = readtup_index_brin;
+ base->haveDatum1 = true;
+ base->arg = arg;
+
+ arg->index.heapRel = heapRel;
+ arg->index.indexRel = indexRel;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return state;
+}
+
Tuplesortstate *
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag, int workMem,
@@ -707,6 +782,35 @@ tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
!stup.isnull1);
}
+/*
+ * Collect one BRIN tuple while collecting input data for sort.
+ */
+void
+tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
+{
+ SortTuple stup;
+ BrinSortTuple *bstup;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->tuplecontext);
+
+ /* allocate space for the whole BRIN sort tuple */
+ bstup = palloc(BRINSORTTUPLE_SIZE(size));
+
+ bstup->tuplen = size;
+ memcpy(&bstup->tuple, tuple, size);
+
+ stup.tuple = bstup;
+ stup.datum1 = tuple->bt_blkno;
+ stup.isnull1 = false;
+
+ tuplesort_puttuple_common(state, &stup,
+ base->sortKeys &&
+ base->sortKeys->abbrev_converter &&
+ !stup.isnull1);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
/*
* Accept one Datum while collecting input data for sort.
*
@@ -850,6 +954,35 @@ tuplesort_getindextuple(Tuplesortstate *state, bool forward)
return (IndexTuple) stup.tuple;
}
+/*
+ * Fetch the next BRIN tuple in either forward or back direction.
+ * Returns NULL if no more tuples. Returned tuple belongs to tuplesort memory
+ * context, and must not be freed by caller. Caller may not rely on tuple
+ * remaining valid after any further manipulation of tuplesort.
+ */
+BrinTuple *
+tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
+ SortTuple stup;
+ BrinSortTuple *btup;
+
+ if (!tuplesort_gettuple_common(state, forward, &stup))
+ stup.tuple = NULL;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!stup.tuple)
+ return NULL;
+
+ btup = (BrinSortTuple *) stup.tuple;
+
+ *len = btup->tuplen;
+
+ return &btup->tuple;
+}
+
/*
* Fetch the next Datum in either forward or back direction.
* Returns false if no more datums.
@@ -1564,6 +1697,80 @@ readtup_index(Tuplesortstate *state, SortTuple *stup,
&stup->isnull1);
}
+/*
+ * Routines specialized for BrinTuple case
+ */
+
+static void
+removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
+{
+ int i;
+
+ for (i = 0; i < count; i++)
+ {
+ BrinSortTuple *tuple;
+
+ tuple = stups[i].tuple;
+ stups[i].datum1 = tuple->tuple.bt_blkno;
+ }
+}
+
+static int
+comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+{
+ Assert(TuplesortstateGetPublic(state)->haveDatum1);
+
+ if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1))
+ return 1;
+
+ if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1))
+ return -1;
+
+ /* silence compilers */
+ return 0;
+}
+
+static void
+writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
+
+ tuplen = tuplen + sizeof(tuplen);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->tuple, tuple->tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+}
+
+static void
+readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len)
+{
+ BrinSortTuple *tuple;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ unsigned int tuplen = len - sizeof(unsigned int);
+
+ /*
+ * Allocate space for the BRIN sort tuple, which is BrinTuple with an
+ * extra length field.
+ */
+ tuple = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
+
+ tuple->tuplen = tuplen;
+
+ LogicalTapeReadExact(tape, &tuple->tuple, tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
+ stup->tuple = (void *) tuple;
+
+ /* set up first-column key value, which is block number */
+ stup->datum1 = tuple->tuple.bt_blkno;
+}
+
/*
* Routines specialized for DatumTuple case
*/
diff --git a/src/include/access/amapi.h b/src/include/access/amapi.h
index 244459587f..df85ae3aac 100644
--- a/src/include/access/amapi.h
+++ b/src/include/access/amapi.h
@@ -243,6 +243,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d5..3451ecb211 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index 9ed2de76cd..357eb35311 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -21,6 +21,7 @@
#ifndef TUPLESORT_H
#define TUPLESORT_H
+#include "access/brin_tuple.h"
#include "access/itup.h"
#include "executor/tuptable.h"
#include "storage/dsm.h"
@@ -282,6 +283,9 @@ typedef struct
* The "index_hash" API is similar to index_btree, but the tuples are
* actually sorted by their hash codes not the raw data.
*
+ * The "index_brin" API is similar to index_btree, but the tuples are
+ * BrinTuple and are sorted by their block number not the raw data.
+ *
* Parallel sort callers are required to coordinate multiple tuplesort states
* in a leader process and one or more worker processes. The leader process
* must launch workers, and have each perform an independent "partial"
@@ -426,6 +430,10 @@ extern Tuplesortstate *tuplesort_begin_index_gist(Relation heapRel,
Relation indexRel,
int workMem, SortCoordinate coordinate,
int sortopt);
+extern Tuplesortstate *tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem, SortCoordinate coordinate,
+ int sortopt);
extern Tuplesortstate *tuplesort_begin_datum(Oid datumType,
Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag,
@@ -438,6 +446,7 @@ extern void tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup);
extern void tuplesort_putindextuplevalues(Tuplesortstate *state,
Relation rel, ItemPointer self,
const Datum *values, const bool *isnull);
+extern void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tup, Size len);
extern void tuplesort_putdatum(Tuplesortstate *state, Datum val,
bool isNull);
@@ -445,6 +454,8 @@ extern bool tuplesort_gettupleslot(Tuplesortstate *state, bool forward,
bool copy, TupleTableSlot *slot, Datum *abbrev);
extern HeapTuple tuplesort_getheaptuple(Tuplesortstate *state, bool forward);
extern IndexTuple tuplesort_getindextuple(Tuplesortstate *state, bool forward);
+extern BrinTuple *tuplesort_getbrintuple(Tuplesortstate *state, Size *len,
+ bool forward);
extern bool tuplesort_getdatum(Tuplesortstate *state, bool forward, bool copy,
Datum *val, bool *isNull, Datum *abbrev);
diff --git a/src/test/modules/dummy_index_am/dummy_index_am.c b/src/test/modules/dummy_index_am/dummy_index_am.c
index cbdae7ab7a..eaa0c483b7 100644
--- a/src/test/modules/dummy_index_am/dummy_index_am.c
+++ b/src/test/modules/dummy_index_am/dummy_index_am.c
@@ -294,6 +294,7 @@ dihandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 86a9886d4f..001fef5865 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -297,13 +297,17 @@ BpChar
BrinBuildState
BrinDesc
BrinInsertState
+BrinLeader
BrinMemTuple
BrinMetaPageData
BrinOpaque
BrinOpcInfo
BrinOptions
BrinRevmap
+BrinShared
+BrinSortTuple
BrinSpecialSpace
+BrinSpool
BrinStatsData
BrinTuple
BrinValues
@@ -2879,6 +2883,7 @@ TupleTableSlotOps
TuplesortClusterArg
TuplesortDatumArg
TuplesortIndexArg
+TuplesortIndexBrinArg
TuplesortIndexBTreeArg
TuplesortIndexHashArg
TuplesortInstrumentation
--
2.40.1
On 11/30/23 18:47, Matthias van de Meent wrote:
...
I just ran some more tests in less favorable environments, and it
looks like I hit a bug:% SET max_parallel_workers = 0;
% CREATE INDEX ... USING brin (...);
ERROR: cannot update tuples during a parallel operationFix attached in 0002.
Yeah, that's a bug, thanks for the fix. Yeah Just jumping to a "cleanup"
label seems a bit cleaner (if that can be said about using goto), so I
tweaked the patch to do that instead.
In 0003 I add the mentioned backfilling of empty ranges at the end of
the table. I added it for both normal and parallel index builds, as
normal builds apparently also didn't yet have this yet.
Right. I was thinking about doing that to, but you beat me to it. I
don't want to bury this in the main patch adding parallel builds, it's
not really related to parallel CREATE INDEX. And it'd be weird to have
this for parallel builds first, so I rebased it as 0001.
As for the backfilling, I think we need to simplify the code a bit. We
have three places doing essentially the same thing (one for serial
builds, two for parallel builds). That's unnecessarily verbose, and
makes it harder to understand the code. But more importantly, the three
places are not doing exactly the same - some increment the current range
before, some do it at the end of the loop, etc. I got confused by this
multiple times.
So 0004 simplifies this - the backfilling is done by a function called
from all the places. The main complexity is in ensuring all three places
have the same concept of how to specify the range (of ranges) to fill.
Note: The serial might have two places too, but the main loop in
brinbuildCallback() does it range by range. It's a bit less efficient as
it can't use the pre-built empty tuple easily, but that's fine IMO.
skipping the last page range?
-----------------------------
I noticed you explicitly skipped backfilling empty tuple for the last
page range. Can you explain? I suspect the idea was that the user
activity would trigger building the tuple once that page range is
filled, but we don't really know if the table receives any changes. It
might easily be just a static table, in which case the last range would
remain unsummarized. If this is the right thing to do, the serial build
should do that too probably ...
But I don't think that's the correct thing to do - I think CREATE INDEX
is expected to always build a complete index, so my version always
builds an index for all table pages.
BlockNumber overflows
---------------------
The one thing that I'm not quite sure is correct is whether this handles
overflows/underflows correctly. I mean, imagine you have a huge table
that's almost 0xFFFFFFFF blocks, pages_per_range is prime, and the last
range ends less than pages_per_range from 0xFFFFFFFF. Then this
blkno += pages_per_range;
can overflow, and might start inserting index tuples again (so we'd end
up with a duplicate).
I do think the current patch does this correctly, but AFAICS this is a
pre-existing issue ...
Anyway, while working on this / stress-testing it, I realized there's a
bug in how we allocate the emptyTuple. It's allocated lazily, but if can
easily happen in the per-range context we introduced last week. It needs
to be allocated in the context covering the whole index build.
I think the best way to do that is per 0006, i.e. allocate it in the
BrinBuildState, along with the appropriate memory context.
Obviously, all of this (0002-0006) should be squashed into a single
commit, I only keep it separate to make it clearer what changed.
stress-testing script
---------------------
I'm also attaching the bash script I use to stress test this - it's just
a loop that creates somewhat random table (different number of rows,
distinct values, ...), maybe deletes some of it, creates an index
(possibly partial), and then does various checks on it (checks number of
ranges, queries the table, etc.). It's somewhat primitive but it turned
out to be very capable in triggering bugs in BlockNumber arithmetic,
emptyTuple allocations, etc.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
v8-0005-simplify-filling-empty-ranges.patchtext/x-patch; charset=UTF-8; name=v8-0005-simplify-filling-empty-ranges.patchDownload
From 7411a50e5789886f257934df74e89466773cbcdd Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Sat, 2 Dec 2023 20:02:09 +0100
Subject: [PATCH v8 5/6] simplify filling empty ranges
---
src/backend/access/brin/brin.c | 151 +++++++++++++++++++++------------
1 file changed, 97 insertions(+), 54 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 00b437d9c1f..84bce1220a0 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -165,7 +165,7 @@ typedef struct BrinBuildState
double bs_numtuples;
double bs_reltuples;
Buffer bs_currentInsertBuf;
- BlockNumber bs_tablePages;
+ BlockNumber bs_maxRangeStart;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
@@ -223,6 +223,10 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
static BrinTuple *brin_init_empty_tuple(BrinBuildState *state, BlockNumber blkno,
BrinTuple *emptyTuple, Size *emptySize);
+static void brin_fill_empty_ranges(BrinBuildState *state,
+ BlockNumber prevRange, BlockNumber maxRange,
+ BrinTuple **emptyTuple, Size *emptySize);
+static BlockNumber brin_next_range(BrinBuildState *state, BlockNumber blkno);
/* parallel index builds */
static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
@@ -1087,8 +1091,7 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
BrinRevmap *revmap;
BrinBuildState *state;
Buffer meta;
- BlockNumber pagesPerRange,
- tablePages;
+ BlockNumber pagesPerRange;
/*
* We expect to be called exactly once for any index relation.
@@ -1135,8 +1138,8 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
* Initialize our state, including the deformed tuple state.
*/
revmap = brinRevmapInitialize(index, &pagesPerRange);
- tablePages = RelationGetNumberOfBlocks(heap);
- state = initialize_brin_buildstate(index, revmap, pagesPerRange, tablePages);
+ state = initialize_brin_buildstate(index, revmap, pagesPerRange,
+ RelationGetNumberOfBlocks(heap));
state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
state->bs_spool->heap = heap;
@@ -1207,14 +1210,19 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
}
else /* no parallel index build */
{
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+
reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
brinbuildCallback, (void *) state, NULL);
- /* process the final batch */
+ /* process the final batch
+ *
+ * XXX Note this does not update state->bs_currRangeStart, i.e.
+ * it stays set to the last range added to the index.
+ */
form_and_insert_tuple(state);
- /* XXX shouldn't this happen in the brinbuildCallback? */
- state->bs_currRangeStart += state->bs_pagesPerRange;
/*
* Backfill the final ranges with empty data.
*
@@ -1222,14 +1230,10 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
* index is built on stupid index quals like WHERE (nonnull_column IS
* NULL).
*/
- while (state->bs_currRangeStart + state->bs_pagesPerRange - 1 < state->bs_tablePages)
- {
- brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
-
- form_and_insert_tuple(state);
-
- state->bs_currRangeStart += state->bs_pagesPerRange;
- }
+ brin_fill_empty_ranges(state,
+ state->bs_currRangeStart,
+ state->bs_maxRangeStart,
+ &emptyTuple, &emptySize);
/* track the number of relation tuples */
state->bs_reltuples = reltuples;
@@ -1660,7 +1664,6 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
- state->bs_tablePages = tablePages;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
@@ -1669,6 +1672,19 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_worker_id = 0;
state->bs_spool = NULL;
+ /*
+ * Calculate the start of the last page range. Page numbers are 0-based,
+ * so to get the index of the last page we need to subtract one. Then the
+ * integer division gives us the proper 0-based range index.
+ */
+ state->bs_maxRangeStart = ((tablePages - 1) / pagesPerRange) * pagesPerRange;
+
+ /*
+ * But, we actually need the start of the next range, or InvalidBlockNumber
+ * if it would overflow.
+ */
+ state->bs_maxRangeStart = brin_next_range(state, state->bs_maxRangeStart);
+
return state;
}
@@ -2614,19 +2630,8 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
}
/* Fill empty ranges for all ranges missing in the tuplesort. */
- prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
- while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
- {
- /* the missing range */
- prevblkno += state->bs_pagesPerRange;
-
- /* Did we already build the empty range? If not, do it now. */
- emptyTuple = brin_init_empty_tuple(state, prevblkno, emptyTuple, &emptySize);
-
- brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
- &state->bs_currentInsertBuf,
- emptyTuple->bt_blkno, emptyTuple, emptySize);
- }
+ brin_fill_empty_ranges(state, prevblkno, btup->bt_blkno,
+ &emptyTuple, &emptySize);
prevblkno = btup->bt_blkno;
}
@@ -2648,30 +2653,9 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
pfree(tmp);
}
- /*
- * Fill empty ranges at the end, for all ranges missing in the tuplesort.
- *
- * Starting from here, prevblkno is the to-be-inserted range's start block
- * number. Note that we don't fill in the relation's last page range.
- */
- if (prevblkno == InvalidBlockNumber)
- prevblkno = 0;
- else
- prevblkno += state->bs_pagesPerRange;
-
- while (prevblkno + state->bs_pagesPerRange < state->bs_tablePages)
- {
- /* Did we already build the empty range? If not, do it now. */
- emptyTuple = brin_init_empty_tuple(state, prevblkno, emptyTuple, &emptySize);
-
- /* Insert the missing range */
- brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
- &state->bs_currentInsertBuf,
- emptyTuple->bt_blkno, emptyTuple, emptySize);
-
- /* ... and update to the next range's block number */
- prevblkno += state->bs_pagesPerRange;
- }
+ /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
+ brin_fill_empty_ranges(state, prevblkno, state->bs_maxRangeStart,
+ &emptyTuple, &emptySize);
/*
* Switch back to the original memory context, and destroy the one we
@@ -2917,3 +2901,62 @@ brin_init_empty_tuple(BrinBuildState *state, BlockNumber blkno,
return emptyTuple;
}
+
+/*
+ * brin_fill_empty_ranges
+ * Add BRIN index tuples representing empty page ranges.
+ *
+ * prevRange/nextRange determine the for which page ranges to add the empty
+ * summaries, and both are exclusive. That is, only ranges starting at blkno
+ * for which (prevRange < blkno < nextRange) will be added to the index.
+ *
+ * Both values may be InvalidBlockNumber. For prevRange this means there is
+ * no previous range, so the first range inserted should be for blkno=0. When
+ * nextRange is InvalidBlockNumber, it means the table is large enough for
+ * the blkno to overflow.
+ *
+ * The empty tuple is built only once when needed, and then kept and reused
+ * for all future calls.
+ */
+static void
+brin_fill_empty_ranges(BrinBuildState *state,
+ BlockNumber prevRange, BlockNumber maxRange,
+ BrinTuple **emptyTuple, Size *emptySize)
+{
+ BlockNumber blkno;
+
+ /*
+ * If we already summarized some ranges, we need to start with the next one.
+ * Otherwise we need to start from the first range of the table.
+ */
+ blkno = (prevRange == InvalidBlockNumber) ? 0 : brin_next_range(state, prevRange);
+
+ /*
+ * Generate empty ranges until we hit the next non-empty range or summarize
+ * the last range of the table.
+ */
+ while (blkno < maxRange)
+ {
+ /* Did we already build the empty tuple? If not, do it now. */
+ *emptyTuple = brin_init_empty_tuple(state, blkno, *emptyTuple, emptySize);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ (*emptyTuple)->bt_blkno, *emptyTuple, *emptySize);
+
+ /* try next page range */
+ blkno = brin_next_range(state, blkno);
+ }
+}
+
+static BlockNumber
+brin_next_range(BrinBuildState *state, BlockNumber blkno)
+{
+ BlockNumber ret = (blkno + state->bs_pagesPerRange);
+
+ /* overflow */
+ if (ret < blkno)
+ ret = InvalidBlockNumber;
+
+ return ret;
+}
--
2.41.0
v8-0006-memcontext-emptyTuple.patchtext/x-patch; charset=UTF-8; name=v8-0006-memcontext-emptyTuple.patchDownload
From b93f75f8954c2e6b1170a61a52fae35cb9dbcf95 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Sun, 3 Dec 2023 12:59:25 +0100
Subject: [PATCH v8 6/6] memcontext emptyTuple
---
src/backend/access/brin/brin.c | 60 ++++++++++++++++++----------------
1 file changed, 31 insertions(+), 29 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 84bce1220a0..af0453dbdbc 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -172,6 +172,10 @@ typedef struct BrinBuildState
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+ BrinTuple *bs_emptyTuple;
+ Size bs_emptyTupleLen;
+ MemoryContext bs_context;
+
/*
* bs_leader is only present when a parallel index build is performed, and
* only in the leader process. (Actually, only the leader process has a
@@ -221,11 +225,8 @@ static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
-static BrinTuple *brin_init_empty_tuple(BrinBuildState *state, BlockNumber blkno,
- BrinTuple *emptyTuple, Size *emptySize);
static void brin_fill_empty_ranges(BrinBuildState *state,
- BlockNumber prevRange, BlockNumber maxRange,
- BrinTuple **emptyTuple, Size *emptySize);
+ BlockNumber prevRange, BlockNumber maxRange);
static BlockNumber brin_next_range(BrinBuildState *state, BlockNumber blkno);
/* parallel index builds */
@@ -1210,9 +1211,6 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
}
else /* no parallel index build */
{
- BrinTuple *emptyTuple = NULL;
- Size emptySize;
-
reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
brinbuildCallback, (void *) state, NULL);
@@ -1232,8 +1230,7 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
*/
brin_fill_empty_ranges(state,
state->bs_currRangeStart,
- state->bs_maxRangeStart,
- &emptyTuple, &emptySize);
+ state->bs_maxRangeStart);
/* track the number of relation tuples */
state->bs_reltuples = reltuples;
@@ -1671,6 +1668,9 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_leader = NULL;
state->bs_worker_id = 0;
state->bs_spool = NULL;
+ state->bs_context = CurrentMemoryContext;
+ state->bs_emptyTuple = NULL;
+ state->bs_emptyTupleLen = 0;
/*
* Calculate the start of the last page range. Page numbers are 0-based,
@@ -2515,8 +2515,6 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
Size tuplen;
BrinShared *brinshared = brinleader->brinshared;
BlockNumber prevblkno = InvalidBlockNumber;
- BrinTuple *emptyTuple = NULL;
- Size emptySize;
BrinSpool *spool;
MemoryContext rangeCxt,
oldCxt;
@@ -2630,8 +2628,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
}
/* Fill empty ranges for all ranges missing in the tuplesort. */
- brin_fill_empty_ranges(state, prevblkno, btup->bt_blkno,
- &emptyTuple, &emptySize);
+ brin_fill_empty_ranges(state, prevblkno, btup->bt_blkno);
prevblkno = btup->bt_blkno;
}
@@ -2654,8 +2651,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
}
/* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
- brin_fill_empty_ranges(state, prevblkno, state->bs_maxRangeStart,
- &emptyTuple, &emptySize);
+ brin_fill_empty_ranges(state, prevblkno, state->bs_maxRangeStart);
/*
* Switch back to the original memory context, and destroy the one we
@@ -2875,31 +2871,38 @@ _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
}
/*
- * brin_init_empty_tuple
+ * brin_build_empty_tuple
* Maybe initialize a BRIN tuple representing empty range.
*
* If emptyTuple is NULL, initializes new tuple representing empty range at
* block blkno. Otherwise the tuple is reused, and only the bt_blkno field
* is updated.
*/
-static BrinTuple *
-brin_init_empty_tuple(BrinBuildState *state, BlockNumber blkno,
- BrinTuple *emptyTuple, Size *emptySize)
+static void
+brin_build_empty_tuple(BrinBuildState *state, BlockNumber blkno)
{
/* Did we already build the empty range? If not, do it now. */
- if (emptyTuple == NULL)
+ if (state->bs_emptyTuple == NULL)
{
- BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+ MemoryContext oldcxt;
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ /*
+ * Make sure to allocate the tuple in context that lasts for the
+ * whole index build.
+ */
+ oldcxt = MemoryContextSwitchTo(state->bs_context);
- emptyTuple = brin_form_tuple(state->bs_bdesc, blkno, dtuple, emptySize);
+ state->bs_emptyTuple = brin_form_tuple(state->bs_bdesc, blkno, dtuple,
+ &state->bs_emptyTupleLen);
+
+ MemoryContextSwitchTo(oldcxt);
}
else
{
/* we already have an "empty range" tuple, just set the block */
- emptyTuple->bt_blkno = blkno;
+ state->bs_emptyTuple->bt_blkno = blkno;
}
-
- return emptyTuple;
}
/*
@@ -2920,8 +2923,7 @@ brin_init_empty_tuple(BrinBuildState *state, BlockNumber blkno,
*/
static void
brin_fill_empty_ranges(BrinBuildState *state,
- BlockNumber prevRange, BlockNumber maxRange,
- BrinTuple **emptyTuple, Size *emptySize)
+ BlockNumber prevRange, BlockNumber maxRange)
{
BlockNumber blkno;
@@ -2938,11 +2940,11 @@ brin_fill_empty_ranges(BrinBuildState *state,
while (blkno < maxRange)
{
/* Did we already build the empty tuple? If not, do it now. */
- *emptyTuple = brin_init_empty_tuple(state, blkno, *emptyTuple, emptySize);
+ brin_build_empty_tuple(state, blkno);
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf,
- (*emptyTuple)->bt_blkno, *emptyTuple, *emptySize);
+ blkno, state->bs_emptyTuple, state->bs_emptyTupleLen);
/* try next page range */
blkno = brin_next_range(state, blkno);
--
2.41.0
v8-0001-backfill-pages-in-serial-build.patchtext/x-patch; charset=UTF-8; name=v8-0001-backfill-pages-in-serial-build.patchDownload
From db38b4c3ff1c8ac3a451da67bde41f125a9922f0 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas.vondra@postgresql.org>
Date: Sat, 2 Dec 2023 17:55:10 +0100
Subject: [PATCH v8 1/6] backfill pages in serial build
---
src/backend/access/brin/brin.c | 35 +++++++++++++++++++++++++++++-----
1 file changed, 30 insertions(+), 5 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 4f2dfdd17b9..b4bedbdc53c 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -51,6 +51,7 @@ typedef struct BrinBuildState
Relation bs_irel;
int bs_numtuples;
Buffer bs_currentInsertBuf;
+ BlockNumber bs_tablePages;
BlockNumber bs_pagesPerRange;
BlockNumber bs_currRangeStart;
BrinRevmap *bs_rmAccess;
@@ -82,7 +83,9 @@ typedef struct BrinOpaque
#define BRIN_ALL_BLOCKRANGES InvalidBlockNumber
static BrinBuildState *initialize_brin_buildstate(Relation idxRel,
- BrinRevmap *revmap, BlockNumber pagesPerRange);
+ BrinRevmap *revmap,
+ BlockNumber pagesPerRange,
+ BlockNumber tablePages);
static BrinInsertState *initialize_brin_insertstate(Relation idxRel, IndexInfo *indexInfo);
static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
@@ -886,7 +889,8 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
BrinRevmap *revmap;
BrinBuildState *state;
Buffer meta;
- BlockNumber pagesPerRange;
+ BlockNumber pagesPerRange,
+ tablePages;
/*
* We expect to be called exactly once for any index relation.
@@ -933,7 +937,8 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
* Initialize our state, including the deformed tuple state.
*/
revmap = brinRevmapInitialize(index, &pagesPerRange);
- state = initialize_brin_buildstate(index, revmap, pagesPerRange);
+ tablePages = RelationGetNumberOfBlocks(heap);
+ state = initialize_brin_buildstate(index, revmap, pagesPerRange, tablePages);
/*
* Now scan the relation. No syncscan allowed here because we want the
@@ -945,6 +950,24 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
/* process the final batch */
form_and_insert_tuple(state);
+ /* XXX shouldn't this happen in the brinbuildCallback? */
+ state->bs_currRangeStart += state->bs_pagesPerRange;
+ /*
+ * Backfill the final ranges with empty data.
+ *
+ * This saves us from doing what amounts to full table scans when the
+ * index is built on stupid index quals like WHERE (nonnull_column IS
+ * NULL).
+ */
+ while (state->bs_currRangeStart + state->bs_pagesPerRange - 1 < state->bs_tablePages)
+ {
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+
+ form_and_insert_tuple(state);
+
+ state->bs_currRangeStart += state->bs_pagesPerRange;
+ }
+
/* release resources */
idxtuples = state->bs_numtuples;
brinRevmapTerminate(state->bs_rmAccess);
@@ -1358,7 +1381,7 @@ brinGetStats(Relation index, BrinStatsData *stats)
*/
static BrinBuildState *
initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
- BlockNumber pagesPerRange)
+ BlockNumber pagesPerRange, BlockNumber tablePages)
{
BrinBuildState *state;
@@ -1368,6 +1391,7 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_numtuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
+ state->bs_tablePages = tablePages;
state->bs_currRangeStart = 0;
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
@@ -1612,7 +1636,8 @@ brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
/* first time through */
Assert(!indexInfo);
state = initialize_brin_buildstate(index, revmap,
- pagesPerRange);
+ pagesPerRange,
+ InvalidBlockNumber);
indexInfo = BuildIndexInfo(index);
}
summarize_range(indexInfo, state, heapRel, startBlk, heapNumBlocks);
--
2.41.0
v8-0002-Allow-BRIN-to-build-its-index-in-parallel.patchtext/x-patch; charset=UTF-8; name=v8-0002-Allow-BRIN-to-build-its-index-in-parallel.patchDownload
From 206f3420b5b406573e6a040ae2954854a0404bf8 Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Wed, 29 Nov 2023 14:35:10 +0100
Subject: [PATCH v8 2/6] Allow BRIN to build its index in parallel
---
contrib/bloom/blutils.c | 1 +
doc/src/sgml/indexam.sgml | 7 +
src/backend/access/brin/brin.c | 916 +++++++++++++++++-
src/backend/access/gin/ginutil.c | 1 +
src/backend/access/gist/gist.c | 1 +
src/backend/access/hash/hash.c | 1 +
src/backend/access/nbtree/nbtree.c | 1 +
src/backend/access/spgist/spgutils.c | 1 +
src/backend/access/transam/parallel.c | 4 +
src/backend/catalog/index.c | 2 +-
src/backend/utils/sort/tuplesortvariants.c | 207 ++++
src/include/access/amapi.h | 2 +
src/include/access/brin.h | 3 +
src/include/utils/tuplesort.h | 11 +
.../modules/dummy_index_am/dummy_index_am.c | 1 +
src/tools/pgindent/typedefs.list | 5 +
16 files changed, 1147 insertions(+), 17 deletions(-)
diff --git a/contrib/bloom/blutils.c b/contrib/bloom/blutils.c
index 4830cb3fee6..a781c5d98d6 100644
--- a/contrib/bloom/blutils.c
+++ b/contrib/bloom/blutils.c
@@ -122,6 +122,7 @@ blhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amparallelvacuumoptions =
diff --git a/doc/src/sgml/indexam.sgml b/doc/src/sgml/indexam.sgml
index f107c43d6a6..cc4135e3940 100644
--- a/doc/src/sgml/indexam.sgml
+++ b/doc/src/sgml/indexam.sgml
@@ -123,6 +123,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
@@ -286,6 +288,11 @@ ambuild (Relation heapRelation,
and compute the keys that need to be inserted into the index.
The function must return a palloc'd struct containing statistics about
the new index.
+ The <structfield>amcanbuildparallel</structfield> flag indicates whether
+ the access method supports parallel index builds. When set to <literal>true</literal>,
+ the system will attempt to allocate parallel workers for the build.
+ Access methods supporting only non-parallel index builds should leave
+ this flag set to <literal>false</literal>.
</para>
<para>
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index b4bedbdc53c..edf2daad0c3 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -33,6 +33,7 @@
#include "postmaster/autovacuum.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "tcop/tcopprot.h" /* pgrminclude ignore */
#include "utils/acl.h"
#include "utils/builtins.h"
#include "utils/datum.h"
@@ -40,7 +41,119 @@
#include "utils/index_selfuncs.h"
#include "utils/memutils.h"
#include "utils/rel.h"
+#include "utils/tuplesort.h"
+/* Magic numbers for parallel state sharing */
+#define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xB000000000000001)
+#define PARALLEL_KEY_TUPLESORT UINT64CONST(0xB000000000000002)
+#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xB000000000000003)
+#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xB000000000000004)
+#define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xB000000000000005)
+
+/*
+ * Status record for spooling/sorting phase.
+ */
+typedef struct BrinSpool
+{
+ Tuplesortstate *sortstate; /* state data for tuplesort.c */
+ Relation heap;
+ Relation index;
+} BrinSpool;
+
+/*
+ * Status for index builds performed in parallel. This is allocated in a
+ * dynamic shared memory segment.
+ */
+typedef struct BrinShared
+{
+ /*
+ * These fields are not modified during the build. They primarily exist
+ * for the benefit of worker processes that need to create state
+ * corresponding to that used by the leader.
+ */
+ Oid heaprelid;
+ Oid indexrelid;
+ bool isconcurrent;
+ BlockNumber pagesPerRange;
+ int scantuplesortstates;
+
+ /*
+ * workersdonecv is used to monitor the progress of workers. All parallel
+ * participants must indicate that they are done before leader can use
+ * results built by the workers (and before leader can write the data into
+ * the index).
+ */
+ ConditionVariable workersdonecv;
+
+ /*
+ * mutex protects all fields before heapdesc.
+ *
+ * These fields contain status information of interest to BRIN 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 the scans.
+ *
+ * nparticipantsdone is number of worker processes finished.
+ *
+ * reltuples is the total number of input heap tuples.
+ *
+ * indtuples is the total number of tuples that made it into the index.
+ */
+ int nparticipantsdone;
+ double reltuples;
+ double indtuples;
+
+ /*
+ * ParallelTableScanDescData data follows. Can't directly embed here, as
+ * implementations of the parallel table scan desc interface might need
+ * stronger alignment.
+ */
+} BrinShared;
+
+/*
+ * Return pointer to a BrinShared's parallel table scan.
+ *
+ * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
+ * MAXALIGN.
+ */
+#define ParallelTableScanFromBrinShared(shared) \
+ (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
+
+/*
+ * Status for leader in parallel index build.
+ */
+typedef struct BrinLeader
+{
+ /* 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).
+ *
+ * brinshared is the shared state for entire build. sharedsort is the
+ * shared, tuplesort-managed state passed to each process tuplesort.
+ * snapshot is the snapshot used by the scan iff an MVCC snapshot is
+ * required.
+ */
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ Snapshot snapshot;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+} BrinLeader;
/*
* We use a BrinBuildState during initial construction of a BRIN index.
@@ -49,7 +162,8 @@
typedef struct BrinBuildState
{
Relation bs_irel;
- int bs_numtuples;
+ double bs_numtuples;
+ double bs_reltuples;
Buffer bs_currentInsertBuf;
BlockNumber bs_tablePages;
BlockNumber bs_pagesPerRange;
@@ -57,6 +171,15 @@ typedef struct BrinBuildState
BrinRevmap *bs_rmAccess;
BrinDesc *bs_bdesc;
BrinMemTuple *bs_dtuple;
+
+ /*
+ * bs_leader is only present when a parallel index build is performed, and
+ * only in the leader process. (Actually, only the leader process has a
+ * BrinBuildState.)
+ */
+ BrinLeader *bs_leader;
+ int bs_worker_id;
+ BrinSpool *bs_spool;
} BrinBuildState;
/*
@@ -91,6 +214,7 @@ static void terminate_brin_buildstate(BrinBuildState *state);
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange,
bool include_partial, double *numSummarized, double *numExisting);
static void form_and_insert_tuple(BrinBuildState *state);
+static void form_and_spill_tuple(BrinBuildState *state);
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a,
BrinTuple *b);
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
@@ -98,6 +222,20 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+/* parallel index builds */
+static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request);
+static void _brin_end_parallel(BrinLeader *btleader, BrinBuildState *state);
+static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
+ Relation heap, Relation index);
+static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
+ BrinSpool *brinspool,
+ BrinShared *brinshared,
+ Sharedsort *sharedsort,
+ Relation heap, Relation index,
+ int sortmem, bool progress);
+
/*
* BRIN handler function: return IndexAmRoutine with access method parameters
* and callbacks.
@@ -122,6 +260,7 @@ brinhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = true;
@@ -877,6 +1016,63 @@ brinbuildCallback(Relation index,
values, isnull);
}
+/*
+ * A version of the callback, used by parallel index builds. The main difference
+ * is that instead of writing the BRIN tuples into the index, we write them into
+ * a shared tuplesort, and leave the insertion up to the leader (which may
+ * reorder them a bit etc.). The callback also does not generate empty ranges,
+ * those may be added by the leader when merging results from workers.
+ */
+static void
+brinbuildCallbackParallel(Relation index,
+ ItemPointer tid,
+ Datum *values,
+ bool *isnull,
+ bool tupleIsAlive,
+ void *brstate)
+{
+ BrinBuildState *state = (BrinBuildState *) brstate;
+ BlockNumber thisblock;
+
+ thisblock = ItemPointerGetBlockNumber(tid);
+
+ /*
+ * If we're in a block that belongs to a future range, summarize what
+ * we've got and start afresh. Note the scan might have skipped many
+ * pages, if they were devoid of live tuples; we do not create emptry BRIN
+ * ranges here - the leader is responsible for filling them in.
+ */
+ if (thisblock > state->bs_currRangeStart + state->bs_pagesPerRange - 1)
+ {
+
+ BRIN_elog((DEBUG2,
+ "brinbuildCallback: completed a range: %u--%u",
+ state->bs_currRangeStart,
+ state->bs_currRangeStart + state->bs_pagesPerRange));
+
+ /* create the index tuple and write it into the tuplesort */
+ form_and_spill_tuple(state);
+
+ /*
+ * Set state to correspond to the next range (for this block).
+ *
+ * This skips ranges that are either empty (and so we don't get any
+ * tuples to summarize), or processes by other workers. We can't
+ * differentiate those cases here easily, so we leave it up to the
+ * leader to fill empty ranges where needed.
+ */
+ state->bs_currRangeStart
+ = state->bs_pagesPerRange * (thisblock / state->bs_pagesPerRange);
+
+ /* re-initialize state for it */
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ }
+
+ /* Accumulate the current tuple into the running state */
+ (void) add_values_to_range(index, state->bs_bdesc, state->bs_dtuple,
+ values, isnull);
+}
+
/*
* brinbuild() -- build a new BRIN index.
*/
@@ -940,36 +1136,106 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
tablePages = RelationGetNumberOfBlocks(heap);
state = initialize_brin_buildstate(index, revmap, pagesPerRange, tablePages);
+ state->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ state->bs_spool->heap = heap;
+ state->bs_spool->index = index;
+
/*
- * Now scan the relation. No syncscan allowed here because we want the
- * heap blocks in physical order.
+ * Attempt to launch parallel worker scan when required
+ *
+ * XXX plan_create_index_workers makes the number of workers dependent on
+ * maintenance_work_mem, requiring 32MB for each worker. That makes sense
+ * for btree, but not for BRIN, which can do away with much less memory.
+ * So maybe make that somehow less strict, optionally?
*/
- reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
- brinbuildCallback, (void *) state, NULL);
-
- /* process the final batch */
- form_and_insert_tuple(state);
+ if (indexInfo->ii_ParallelWorkers > 0)
+ _brin_begin_parallel(state, heap, index, indexInfo->ii_Concurrent,
+ indexInfo->ii_ParallelWorkers);
- /* XXX shouldn't this happen in the brinbuildCallback? */
- state->bs_currRangeStart += state->bs_pagesPerRange;
/*
- * Backfill the final ranges with empty data.
+ * Now scan the relation. No syncscan allowed here because we want the
+ * heap blocks in physical order.
*
- * This saves us from doing what amounts to full table scans when the
- * index is built on stupid index quals like WHERE (nonnull_column IS
- * NULL).
+ * If parallel build requested and at least one worker process was
+ * successfully launched, set up coordination state
*/
- while (state->bs_currRangeStart + state->bs_pagesPerRange - 1 < state->bs_tablePages)
+ if (state->bs_leader)
{
- brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+ SortCoordinate coordinate;
+ coordinate = (SortCoordinate) palloc0(sizeof(SortCoordinateData));
+ coordinate->isWorker = false;
+ coordinate->nParticipants =
+ state->bs_leader->nparticipanttuplesorts;
+ coordinate->sharedsort = state->bs_leader->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.
+ */
+ state->bs_spool->sortstate =
+ tuplesort_begin_index_brin(heap, index,
+ maintenance_work_mem, coordinate,
+ TUPLESORT_NONE);
+
+ /*
+ * In parallel mode, wait for workers to complete, and then read all
+ * tuples from the shared tuplesort and insert them into the index.
+ */
+ _brin_end_parallel(state->bs_leader, state);
+ }
+ else /* no parallel index build */
+ {
+ reltuples = table_index_build_scan(heap, index, indexInfo, false, true,
+ brinbuildCallback, (void *) state, NULL);
+
+ /* process the final batch */
form_and_insert_tuple(state);
+ /* XXX shouldn't this happen in the brinbuildCallback? */
state->bs_currRangeStart += state->bs_pagesPerRange;
+ /*
+ * Backfill the final ranges with empty data.
+ *
+ * This saves us from doing what amounts to full table scans when the
+ * index is built on stupid index quals like WHERE (nonnull_column IS
+ * NULL).
+ */
+ while (state->bs_currRangeStart + state->bs_pagesPerRange - 1 < state->bs_tablePages)
+ {
+ brin_memtuple_initialize(state->bs_dtuple, state->bs_bdesc);
+
+ form_and_insert_tuple(state);
+
+ state->bs_currRangeStart += state->bs_pagesPerRange;
+ }
+
+ /* track the number of relation tuples */
+ state->bs_reltuples = reltuples;
}
/* release resources */
idxtuples = state->bs_numtuples;
+ reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -1389,6 +1655,7 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_irel = idxRel;
state->bs_numtuples = 0;
+ state->bs_reltuples = 0;
state->bs_currentInsertBuf = InvalidBuffer;
state->bs_pagesPerRange = pagesPerRange;
state->bs_tablePages = tablePages;
@@ -1396,6 +1663,9 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_rmAccess = revmap;
state->bs_bdesc = brin_build_desc(idxRel);
state->bs_dtuple = brin_new_memtuple(state->bs_bdesc);
+ state->bs_leader = NULL;
+ state->bs_worker_id = 0;
+ state->bs_spool = NULL;
return state;
}
@@ -1688,6 +1958,32 @@ form_and_insert_tuple(BrinBuildState *state)
pfree(tup);
}
+/*
+ * Given a deformed tuple in the build state, convert it into the on-disk
+ * format and write it to a (shared) tuplesort (the leader will insert it
+ * into the index later).
+ */
+static void
+form_and_spill_tuple(BrinBuildState *state)
+{
+ BrinTuple *tup;
+ Size size;
+
+ /* don't insert empty tuples in parallel build */
+ if (state->bs_dtuple->bt_empty_range)
+ return;
+
+ tup = brin_form_tuple(state->bs_bdesc, state->bs_currRangeStart,
+ state->bs_dtuple, &size);
+
+ /* write the BRIN tuple to the tuplesort */
+ tuplesort_putbrintuple(state->bs_spool->sortstate, tup, size);
+
+ state->bs_numtuples++;
+
+ pfree(tup);
+}
+
/*
* Given two deformed tuples, adjust the first one so that it's consistent
* with the summary values in both.
@@ -2007,3 +2303,591 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+
+static void
+_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
+ bool isconcurrent, int request)
+{
+ ParallelContext *pcxt;
+ int scantuplesortstates;
+ Snapshot snapshot;
+ Size estbrinshared;
+ Size estsort;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinLeader *brinleader = (BrinLeader *) palloc0(sizeof(BrinLeader));
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ bool leaderparticipates = true;
+ int querylen;
+
+#ifdef DISABLE_LEADER_PARTICIPATION
+ leaderparticipates = false;
+#endif
+
+ /*
+ * Enter parallel mode, and create context for parallel build of brin
+ * index
+ */
+ EnterParallelMode();
+ Assert(request > 0);
+ pcxt = CreateParallelContext("postgres", "_brin_parallel_build_main",
+ request);
+
+ 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 our own PARALLEL_KEY_BRIN_SHARED workspace.
+ */
+ estbrinshared = _brin_parallel_estimate_shared(heap, snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, estbrinshared);
+ estsort = tuplesort_estimate_shared(scantuplesortstates);
+ shm_toc_estimate_chunk(&pcxt->estimator, estsort);
+
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+
+ /*
+ * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
+ * and PARALLEL_KEY_BUFFER_USAGE.
+ *
+ * If there are no extensions loaded that care, we could skip this. We
+ * have no way of knowing whether anyone's looking at pgWalUsage or
+ * pgBufferUsage, so do it unconditionally.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
+ if (debug_query_string)
+ {
+ querylen = strlen(debug_query_string);
+ shm_toc_estimate_chunk(&pcxt->estimator, querylen + 1);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ else
+ querylen = 0; /* keep compiler quiet */
+
+ /* Everyone's had a chance to ask for space, so now create the DSM */
+ InitializeParallelDSM(pcxt);
+
+ /* If no DSM segment was available, back out (do serial build) */
+ if (pcxt->seg == NULL)
+ {
+ if (IsMVCCSnapshot(snapshot))
+ UnregisterSnapshot(snapshot);
+ DestroyParallelContext(pcxt);
+ ExitParallelMode();
+ return;
+ }
+
+ /* Store shared build state, for which we reserved space */
+ brinshared = (BrinShared *) shm_toc_allocate(pcxt->toc, estbrinshared);
+ /* Initialize immutable state */
+ brinshared->heaprelid = RelationGetRelid(heap);
+ brinshared->indexrelid = RelationGetRelid(index);
+ brinshared->isconcurrent = isconcurrent;
+ brinshared->scantuplesortstates = scantuplesortstates;
+ brinshared->pagesPerRange = buildstate->bs_pagesPerRange;
+ ConditionVariableInit(&brinshared->workersdonecv);
+ SpinLockInit(&brinshared->mutex);
+
+ /* Initialize mutable state */
+ brinshared->nparticipantsdone = 0;
+ brinshared->reltuples = 0.0;
+ brinshared->indtuples = 0.0;
+
+ table_parallelscan_initialize(heap,
+ ParallelTableScanFromBrinShared(brinshared),
+ 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);
+
+ /*
+ * Store shared tuplesort-private state, for which we reserved space.
+ * Then, initialize opaque state using tuplesort routine.
+ */
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BRIN_SHARED, brinshared);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLESORT, sharedsort);
+
+ /* Store query string for workers */
+ if (debug_query_string)
+ {
+ char *sharedquery;
+
+ sharedquery = (char *) shm_toc_allocate(pcxt->toc, querylen + 1);
+ memcpy(sharedquery, debug_query_string, querylen + 1);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUERY_TEXT, sharedquery);
+ }
+
+ /*
+ * Allocate space for each worker's WalUsage and BufferUsage; no need to
+ * initialize.
+ */
+ walusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(WalUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_WAL_USAGE, walusage);
+ bufferusage = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(BufferUsage), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFFER_USAGE, bufferusage);
+
+ /* Launch workers, saving status for leader/caller */
+ LaunchParallelWorkers(pcxt);
+ brinleader->pcxt = pcxt;
+ brinleader->nparticipanttuplesorts = pcxt->nworkers_launched;
+ if (leaderparticipates)
+ brinleader->nparticipanttuplesorts++;
+ brinleader->brinshared = brinshared;
+ brinleader->sharedsort = sharedsort;
+ brinleader->snapshot = snapshot;
+ brinleader->walusage = walusage;
+ brinleader->bufferusage = bufferusage;
+
+ /* If no workers were successfully launched, back out (do serial build) */
+ if (pcxt->nworkers_launched == 0)
+ {
+ _brin_end_parallel(brinleader, NULL);
+ return;
+ }
+
+ /* Save leader state now that it's clear build will be parallel */
+ buildstate->bs_leader = brinleader;
+
+ /* Join heap scan ourselves */
+ if (leaderparticipates)
+ _brin_leader_participate_as_worker(buildstate, heap, index);
+
+ /*
+ * 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
+_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
+{
+ int i;
+ BrinTuple *btup;
+ BrinMemTuple *memtuple = NULL;
+ Size tuplen;
+ BrinShared *brinshared = brinleader->brinshared;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ BrinTuple *emptyTuple = NULL;
+ Size emptySize;
+ BrinSpool *spool;
+ MemoryContext rangeCxt,
+ oldCxt;
+
+ /* Shutdown worker processes */
+ WaitForParallelWorkersToFinish(brinleader->pcxt);
+
+ if (!state)
+ return;
+
+ /* copy the data into leader state (we have to wait for the workers ) */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
+
+ /* do the actual sort in the leader */
+ spool = state->bs_spool;
+ tuplesort_performsort(spool->sortstate);
+
+ /*
+ * Initialize BrinMemTuple we'll use to union summaries from workers (in
+ * case they happened to produce parts of the same paga range).
+ */
+ memtuple = brin_new_memtuple(state->bs_bdesc);
+
+ /*
+ * Create a memory context we'll reset to combine results for a single
+ * page range (received from the workers). We don't expect huge number of
+ * overlaps under regular circumstances, because for large tables the
+ * chunk size is likely larger than the BRIN page range), but it can
+ * happen, and the union functions may do all kinds of stuff. So we better
+ * reset the context once in a while.
+ */
+ rangeCxt = AllocSetContextCreate(CurrentMemoryContext,
+ "brin union",
+ ALLOCSET_DEFAULT_SIZES);
+ oldCxt = MemoryContextSwitchTo(rangeCxt);
+
+ /*
+ * Read the BRIN tuples from the shared tuplesort, sorted by block number.
+ * That probably gives us an index that is cheaper to scan, thanks to
+ * mostly getting data from the same index page as before.
+ */
+ while ((btup = tuplesort_getbrintuple(spool->sortstate, &tuplen, true)) != NULL)
+ {
+ /* Ranges should be multiples of pages_per_range for the index. */
+ Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+
+ /*
+ * Do we need to union summaries for the same page range?
+ *
+ * If this is the first brin tuple we read, then just deform it into
+ * the memtuple, and continue with the next one from tuplesort. We
+ * however may need to insert empty summaries into the index.
+ *
+ * If it's the same block as the last we saw, we simply union the brin
+ * tuple into it, and we're done - we don't even need to insert empty
+ * ranges, because that was done earlier when we saw the first brin
+ * tuple (for this range).
+ *
+ * Finally, if it's not the first brin tuple, and it's not the same
+ * page range, we need to do the insert and then deform the tuple into
+ * the memtuple. Then we'll insert empty ranges before the new brin
+ * tuple, if needed.
+ */
+ if (prevblkno == InvalidBlockNumber)
+ {
+ /* First brin tuples, just deform into memtuple. */
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+ else if (memtuple->bt_blkno == btup->bt_blkno)
+ {
+ /*
+ * Not the first brin tuple, but same page range as the previous
+ * one, so we can merge it into the memtuple.
+ */
+ union_tuples(state->bs_bdesc, memtuple, btup);
+ continue;
+ }
+ else
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ /*
+ * We got brin tuple for a different page range, so form a brin
+ * tuple from the memtuple, insert it, and re-init the memtuple
+ * from the new brin tuple.
+ */
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ /*
+ * Reset the per-output-range context. This frees all the memory
+ * possibly allocated by the union functions, and also the BRIN
+ * tuple we just formed and inserted.
+ */
+ MemoryContextReset(rangeCxt);
+
+ memtuple = brin_deform_tuple(state->bs_bdesc, btup, memtuple);
+
+ /* continue to insert empty pages before thisblock */
+ }
+
+ /* Fill empty ranges for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < btup->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ prevblkno = btup->bt_blkno;
+ }
+
+ tuplesort_end(spool->sortstate);
+
+ /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
+ prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
+ while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
+ {
+ /* the missing range */
+ prevblkno += state->bs_pagesPerRange;
+
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
+ }
+ else
+ {
+ /* we already have am "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = prevblkno;
+ }
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+ }
+
+ /* Fill the BRIN tuple for the last page range. */
+ if (prevblkno != InvalidBlockNumber)
+ {
+ BrinTuple *tmp;
+ Size len;
+
+ tmp = brin_form_tuple(state->bs_bdesc, memtuple->bt_blkno,
+ memtuple, &len);
+
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf, tmp->bt_blkno, tmp, len);
+
+ pfree(tmp);
+ }
+
+ /*
+ * Switch back to the originam memory context, and destroy the one we
+ * created to isolate the union_tuple calls.
+ */
+ MemoryContextSwitchTo(oldCxt);
+ MemoryContextDelete(rangeCxt);
+
+ /*
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
+ */
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Returns size of shared memory required to store state for a parallel
+ * brin index build based on the snapshot its parallel scan will use.
+ */
+static Size
+_brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
+{
+ /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
+ return add_size(BUFFERALIGN(sizeof(BrinShared)),
+ table_parallelscan_estimate(heap, snapshot));
+}
+
+/*
+ * Within leader, participate as a parallel worker.
+ */
+static void
+_brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
+{
+ BrinLeader *brinleader = buildstate->bs_leader;
+ int sortmem;
+
+ /* Allocate memory and initialize private spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = buildstate->bs_spool->heap;
+ buildstate->bs_spool->index = buildstate->bs_spool->index;
+
+ /*
+ * 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 / brinleader->nparticipanttuplesorts;
+
+ /* Perform work common to all participants */
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool, brinleader->brinshared,
+ brinleader->sharedsort, heap, index, sortmem, true);
+}
+
+/*
+ * 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
+_brin_parallel_scan_and_build(BrinBuildState *state, BrinSpool *brinspool,
+ BrinShared *brinshared, Sharedsort *sharedsort,
+ Relation heap, Relation index, int sortmem,
+ bool progress)
+{
+ SortCoordinate coordinate;
+ TableScanDesc 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 */
+ brinspool->sortstate = tuplesort_begin_index_brin(brinspool->heap,
+ brinspool->index,
+ sortmem, coordinate,
+ TUPLESORT_NONE);
+
+ /* Join parallel scan */
+ indexInfo = BuildIndexInfo(index);
+ indexInfo->ii_Concurrent = brinshared->isconcurrent;
+
+ scan = table_beginscan_parallel(heap,
+ ParallelTableScanFromBrinShared(brinshared));
+
+ reltuples = table_index_build_scan(heap, index, indexInfo, true, true,
+ brinbuildCallbackParallel, state, scan);
+
+ /* insert the last item */
+ form_and_spill_tuple(state);
+
+ /* sort the BRIN ranges built by this worker */
+ tuplesort_performsort(brinspool->sortstate);
+
+ state->bs_reltuples += reltuples;
+
+ /*
+ * Done. Record ambuild statistics.
+ */
+ SpinLockAcquire(&brinshared->mutex);
+ brinshared->nparticipantsdone++;
+ brinshared->reltuples += state->bs_reltuples;
+ brinshared->indtuples += state->bs_numtuples;
+ SpinLockRelease(&brinshared->mutex);
+
+ /* Notify leader */
+ ConditionVariableSignal(&brinshared->workersdonecv);
+
+ tuplesort_end(brinspool->sortstate);
+}
+
+/*
+ * Perform work within a launched parallel process.
+ */
+void
+_brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
+{
+ char *sharedquery;
+ BrinShared *brinshared;
+ Sharedsort *sharedsort;
+ BrinBuildState *buildstate;
+ Relation heapRel;
+ Relation indexRel;
+ LOCKMODE heapLockmode;
+ LOCKMODE indexLockmode;
+ WalUsage *walusage;
+ BufferUsage *bufferusage;
+ int sortmem;
+
+ /*
+ * The only possible status flag that can be set to the parallel worker is
+ * PROC_IN_SAFE_IC.
+ */
+ Assert((MyProc->statusFlags == 0) ||
+ (MyProc->statusFlags == PROC_IN_SAFE_IC));
+
+ /* Set debug_query_string for individual workers first */
+ sharedquery = shm_toc_lookup(toc, PARALLEL_KEY_QUERY_TEXT, true);
+ debug_query_string = sharedquery;
+
+ /* Report the query string from leader */
+ pgstat_report_activity(STATE_RUNNING, debug_query_string);
+
+ /* Look up brin shared state */
+ brinshared = shm_toc_lookup(toc, PARALLEL_KEY_BRIN_SHARED, false);
+
+ /* Open relations using lock modes known to be obtained by index.c */
+ if (!brinshared->isconcurrent)
+ {
+ heapLockmode = ShareLock;
+ indexLockmode = AccessExclusiveLock;
+ }
+ else
+ {
+ heapLockmode = ShareUpdateExclusiveLock;
+ indexLockmode = RowExclusiveLock;
+ }
+
+ /* Open relations within worker */
+ heapRel = table_open(brinshared->heaprelid, heapLockmode);
+ indexRel = index_open(brinshared->indexrelid, indexLockmode);
+
+ buildstate = initialize_brin_buildstate(indexRel, NULL,
+ brinshared->pagesPerRange,
+ InvalidBlockNumber);
+
+ /* Initialize worker's own spool */
+ buildstate->bs_spool = (BrinSpool *) palloc0(sizeof(BrinSpool));
+ buildstate->bs_spool->heap = heapRel;
+ buildstate->bs_spool->index = indexRel;
+
+ /* Look up shared state private to tuplesort.c */
+ sharedsort = shm_toc_lookup(toc, PARALLEL_KEY_TUPLESORT, false);
+ tuplesort_attach_shared(sharedsort, seg);
+
+ /* Prepare to track buffer usage during parallel execution */
+ InstrStartParallelQuery();
+
+ /*
+ * 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 / brinshared->scantuplesortstates;
+
+ _brin_parallel_scan_and_build(buildstate, buildstate->bs_spool,
+ brinshared, sharedsort,
+ heapRel, indexRel, sortmem, false);
+
+ /* Report WAL/buffer usage during parallel execution */
+ bufferusage = shm_toc_lookup(toc, PARALLEL_KEY_BUFFER_USAGE, false);
+ walusage = shm_toc_lookup(toc, PARALLEL_KEY_WAL_USAGE, false);
+ InstrEndParallelQuery(&bufferusage[ParallelWorkerNumber],
+ &walusage[ParallelWorkerNumber]);
+
+ index_close(indexRel, indexLockmode);
+ table_close(heapRel, heapLockmode);
+}
diff --git a/src/backend/access/gin/ginutil.c b/src/backend/access/gin/ginutil.c
index a875c5d3d7a..9b1a0ac345d 100644
--- a/src/backend/access/gin/ginutil.c
+++ b/src/backend/access/gin/ginutil.c
@@ -54,6 +54,7 @@ ginhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = true;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/gist/gist.c b/src/backend/access/gist/gist.c
index 9a1bf8f66cb..e052ba8bda2 100644
--- a/src/backend/access/gist/gist.c
+++ b/src/backend/access/gist/gist.c
@@ -76,6 +76,7 @@ gisthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/hash/hash.c b/src/backend/access/hash/hash.c
index 6443ff21bda..905519692c6 100644
--- a/src/backend/access/hash/hash.c
+++ b/src/backend/access/hash/hash.c
@@ -73,6 +73,7 @@ hashhandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = true;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/nbtree/nbtree.c b/src/backend/access/nbtree/nbtree.c
index 0930f9b37e3..6c8cd93fa0a 100644
--- a/src/backend/access/nbtree/nbtree.c
+++ b/src/backend/access/nbtree/nbtree.c
@@ -112,6 +112,7 @@ bthandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = true;
amroutine->ampredlocks = true;
amroutine->amcanparallel = true;
+ amroutine->amcanbuildparallel = true;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/spgist/spgutils.c b/src/backend/access/spgist/spgutils.c
index 30c00876a56..fd4b6157101 100644
--- a/src/backend/access/spgist/spgutils.c
+++ b/src/backend/access/spgist/spgutils.c
@@ -60,6 +60,7 @@ spghandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 194a1207be6..d78314062e0 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -14,6 +14,7 @@
#include "postgres.h"
+#include "access/brin.h"
#include "access/nbtree.h"
#include "access/parallel.h"
#include "access/session.h"
@@ -145,6 +146,9 @@ static const struct
{
"_bt_parallel_build_main", _bt_parallel_build_main
},
+ {
+ "_brin_parallel_build_main", _brin_parallel_build_main
+ },
{
"parallel_vacuum_main", parallel_vacuum_main
}
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 143fae01ebd..40abbaf476b 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -2982,7 +2982,7 @@ index_build(Relation heapRelation,
* Note that planner considers parallel safety for us.
*/
if (parallel && IsNormalProcessingMode() &&
- indexRelation->rd_rel->relam == BTREE_AM_OID)
+ indexRelation->rd_indam->amcanbuildparallel)
indexInfo->ii_ParallelWorkers =
plan_create_index_workers(RelationGetRelid(heapRelation),
RelationGetRelid(indexRelation));
diff --git a/src/backend/utils/sort/tuplesortvariants.c b/src/backend/utils/sort/tuplesortvariants.c
index 2cd508e5130..90fc605f1ca 100644
--- a/src/backend/utils/sort/tuplesortvariants.c
+++ b/src/backend/utils/sort/tuplesortvariants.c
@@ -19,6 +19,7 @@
#include "postgres.h"
+#include "access/brin_tuple.h"
#include "access/hash.h"
#include "access/htup_details.h"
#include "access/nbtree.h"
@@ -43,6 +44,8 @@ static void removeabbrev_cluster(Tuplesortstate *state, SortTuple *stups,
int count);
static void removeabbrev_index(Tuplesortstate *state, SortTuple *stups,
int count);
+static void removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups,
+ int count);
static void removeabbrev_datum(Tuplesortstate *state, SortTuple *stups,
int count);
static int comparetup_heap(const SortTuple *a, const SortTuple *b,
@@ -69,10 +72,16 @@ static int comparetup_index_hash(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_index_hash_tiebreak(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
+static int comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state);
static void writetup_index(Tuplesortstate *state, LogicalTape *tape,
SortTuple *stup);
static void readtup_index(Tuplesortstate *state, SortTuple *stup,
LogicalTape *tape, unsigned int len);
+static void writetup_index_brin(Tuplesortstate *state, LogicalTape *tape,
+ SortTuple *stup);
+static void readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len);
static int comparetup_datum(const SortTuple *a, const SortTuple *b,
Tuplesortstate *state);
static int comparetup_datum_tiebreak(const SortTuple *a, const SortTuple *b,
@@ -128,6 +137,16 @@ typedef struct
uint32 max_buckets;
} TuplesortIndexHashArg;
+/*
+ * Data struture pointed by "TuplesortPublic.arg" for the index_brin subcase.
+ */
+typedef struct
+{
+ TuplesortIndexArg index;
+
+ /* XXX do we need something here? */
+} TuplesortIndexBrinArg;
+
/*
* Data struture pointed by "TuplesortPublic.arg" for the Datum case.
* Set by tuplesort_begin_datum and used only by the DatumTuple routines.
@@ -140,6 +159,21 @@ typedef struct
int datumTypeLen;
} TuplesortDatumArg;
+/*
+ * Computing BrinTuple size with only the tuple is difficult, so we want to track
+ * the length referenced by the SortTuple. That's what BrinSortTuple is meant
+ * to do - it's essentially a BrinTuple prefixed by its length.
+ */
+typedef struct BrinSortTuple
+{
+ Size tuplen;
+ BrinTuple tuple;
+} BrinSortTuple;
+
+/* Size of the BrinSortTuple, given length of the BrinTuple. */
+#define BRINSORTTUPLE_SIZE(len) (offsetof(BrinSortTuple, tuple) + (len))
+
+
Tuplesortstate *
tuplesort_begin_heap(TupleDesc tupDesc,
int nkeys, AttrNumber *attNums,
@@ -527,6 +561,47 @@ tuplesort_begin_index_gist(Relation heapRel,
return state;
}
+Tuplesortstate *
+tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem,
+ SortCoordinate coordinate,
+ int sortopt)
+{
+ Tuplesortstate *state = tuplesort_begin_common(workMem, coordinate,
+ sortopt);
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext;
+ TuplesortIndexBrinArg *arg;
+
+ oldcontext = MemoryContextSwitchTo(base->maincontext);
+ arg = (TuplesortIndexBrinArg *) palloc(sizeof(TuplesortIndexBrinArg));
+
+#ifdef TRACE_SORT
+ if (trace_sort)
+ elog(LOG,
+ "begin index sort: workMem = %d, randomAccess = %c",
+ workMem,
+ sortopt & TUPLESORT_RANDOMACCESS ? 't' : 'f');
+#endif
+
+ base->nKeys = 1; /* Only one sort column, the block number */
+
+ base->removeabbrev = removeabbrev_index_brin;
+ base->comparetup = comparetup_index_brin;
+ base->writetup = writetup_index_brin;
+ base->readtup = readtup_index_brin;
+ base->haveDatum1 = true;
+ base->arg = arg;
+
+ arg->index.heapRel = heapRel;
+ arg->index.indexRel = indexRel;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ return state;
+}
+
Tuplesortstate *
tuplesort_begin_datum(Oid datumType, Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag, int workMem,
@@ -707,6 +782,35 @@ tuplesort_putindextuplevalues(Tuplesortstate *state, Relation rel,
!stup.isnull1);
}
+/*
+ * Collect one BRIN tuple while collecting input data for sort.
+ */
+void
+tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
+{
+ SortTuple stup;
+ BrinSortTuple *bstup;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->tuplecontext);
+
+ /* allocate space for the whole BRIN sort tuple */
+ bstup = palloc(BRINSORTTUPLE_SIZE(size));
+
+ bstup->tuplen = size;
+ memcpy(&bstup->tuple, tuple, size);
+
+ stup.tuple = bstup;
+ stup.datum1 = tuple->bt_blkno;
+ stup.isnull1 = false;
+
+ tuplesort_puttuple_common(state, &stup,
+ base->sortKeys &&
+ base->sortKeys->abbrev_converter &&
+ !stup.isnull1);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
/*
* Accept one Datum while collecting input data for sort.
*
@@ -850,6 +954,35 @@ tuplesort_getindextuple(Tuplesortstate *state, bool forward)
return (IndexTuple) stup.tuple;
}
+/*
+ * Fetch the next BRIN tuple in either forward or back direction.
+ * Returns NULL if no more tuples. Returned tuple belongs to tuplesort memory
+ * context, and must not be freed by caller. Caller may not rely on tuple
+ * remaining valid after any further manipulation of tuplesort.
+ */
+BrinTuple *
+tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ MemoryContext oldcontext = MemoryContextSwitchTo(base->sortcontext);
+ SortTuple stup;
+ BrinSortTuple *btup;
+
+ if (!tuplesort_gettuple_common(state, forward, &stup))
+ stup.tuple = NULL;
+
+ MemoryContextSwitchTo(oldcontext);
+
+ if (!stup.tuple)
+ return NULL;
+
+ btup = (BrinSortTuple *) stup.tuple;
+
+ *len = btup->tuplen;
+
+ return &btup->tuple;
+}
+
/*
* Fetch the next Datum in either forward or back direction.
* Returns false if no more datums.
@@ -1564,6 +1697,80 @@ readtup_index(Tuplesortstate *state, SortTuple *stup,
&stup->isnull1);
}
+/*
+ * Routines specialized for BrinTuple case
+ */
+
+static void
+removeabbrev_index_brin(Tuplesortstate *state, SortTuple *stups, int count)
+{
+ int i;
+
+ for (i = 0; i < count; i++)
+ {
+ BrinSortTuple *tuple;
+
+ tuple = stups[i].tuple;
+ stups[i].datum1 = tuple->tuple.bt_blkno;
+ }
+}
+
+static int
+comparetup_index_brin(const SortTuple *a, const SortTuple *b,
+ Tuplesortstate *state)
+{
+ Assert(TuplesortstateGetPublic(state)->haveDatum1);
+
+ if (DatumGetUInt32(a->datum1) > DatumGetUInt32(b->datum1))
+ return 1;
+
+ if (DatumGetUInt32(a->datum1) < DatumGetUInt32(b->datum1))
+ return -1;
+
+ /* silence compilers */
+ return 0;
+}
+
+static void
+writetup_index_brin(Tuplesortstate *state, LogicalTape *tape, SortTuple *stup)
+{
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ BrinSortTuple *tuple = (BrinSortTuple *) stup->tuple;
+ unsigned int tuplen = tuple->tuplen;
+
+ tuplen = tuplen + sizeof(tuplen);
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+ LogicalTapeWrite(tape, &tuple->tuple, tuple->tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeWrite(tape, &tuplen, sizeof(tuplen));
+}
+
+static void
+readtup_index_brin(Tuplesortstate *state, SortTuple *stup,
+ LogicalTape *tape, unsigned int len)
+{
+ BrinSortTuple *tuple;
+ TuplesortPublic *base = TuplesortstateGetPublic(state);
+ unsigned int tuplen = len - sizeof(unsigned int);
+
+ /*
+ * Allocate space for the BRIN sort tuple, which is BrinTuple with an
+ * extra length field.
+ */
+ tuple = (BrinSortTuple *) tuplesort_readtup_alloc(state,
+ BRINSORTTUPLE_SIZE(tuplen));
+
+ tuple->tuplen = tuplen;
+
+ LogicalTapeReadExact(tape, &tuple->tuple, tuplen);
+ if (base->sortopt & TUPLESORT_RANDOMACCESS) /* need trailing length word? */
+ LogicalTapeReadExact(tape, &tuplen, sizeof(tuplen));
+ stup->tuple = (void *) tuple;
+
+ /* set up first-column key value, which is block number */
+ stup->datum1 = tuple->tuple.bt_blkno;
+}
+
/*
* Routines specialized for DatumTuple case
*/
diff --git a/src/include/access/amapi.h b/src/include/access/amapi.h
index 244459587fc..df85ae3aace 100644
--- a/src/include/access/amapi.h
+++ b/src/include/access/amapi.h
@@ -243,6 +243,8 @@ typedef struct IndexAmRoutine
bool ampredlocks;
/* does AM support parallel scan? */
bool amcanparallel;
+ /* does AM support parallel build? */
+ bool amcanbuildparallel;
/* does AM support columns included with clause INCLUDE? */
bool amcaninclude;
/* does AM use maintenance_work_mem? */
diff --git a/src/include/access/brin.h b/src/include/access/brin.h
index ed66f1b3d51..3451ecb211f 100644
--- a/src/include/access/brin.h
+++ b/src/include/access/brin.h
@@ -11,6 +11,7 @@
#define BRIN_H
#include "nodes/execnodes.h"
+#include "storage/shm_toc.h"
#include "utils/relcache.h"
@@ -52,4 +53,6 @@ typedef struct BrinStatsData
extern void brinGetStats(Relation index, BrinStatsData *stats);
+extern void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc);
+
#endif /* BRIN_H */
diff --git a/src/include/utils/tuplesort.h b/src/include/utils/tuplesort.h
index 9ed2de76cd6..357eb35311d 100644
--- a/src/include/utils/tuplesort.h
+++ b/src/include/utils/tuplesort.h
@@ -21,6 +21,7 @@
#ifndef TUPLESORT_H
#define TUPLESORT_H
+#include "access/brin_tuple.h"
#include "access/itup.h"
#include "executor/tuptable.h"
#include "storage/dsm.h"
@@ -282,6 +283,9 @@ typedef struct
* The "index_hash" API is similar to index_btree, but the tuples are
* actually sorted by their hash codes not the raw data.
*
+ * The "index_brin" API is similar to index_btree, but the tuples are
+ * BrinTuple and are sorted by their block number not the raw data.
+ *
* Parallel sort callers are required to coordinate multiple tuplesort states
* in a leader process and one or more worker processes. The leader process
* must launch workers, and have each perform an independent "partial"
@@ -426,6 +430,10 @@ extern Tuplesortstate *tuplesort_begin_index_gist(Relation heapRel,
Relation indexRel,
int workMem, SortCoordinate coordinate,
int sortopt);
+extern Tuplesortstate *tuplesort_begin_index_brin(Relation heapRel,
+ Relation indexRel,
+ int workMem, SortCoordinate coordinate,
+ int sortopt);
extern Tuplesortstate *tuplesort_begin_datum(Oid datumType,
Oid sortOperator, Oid sortCollation,
bool nullsFirstFlag,
@@ -438,6 +446,7 @@ extern void tuplesort_putheaptuple(Tuplesortstate *state, HeapTuple tup);
extern void tuplesort_putindextuplevalues(Tuplesortstate *state,
Relation rel, ItemPointer self,
const Datum *values, const bool *isnull);
+extern void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tup, Size len);
extern void tuplesort_putdatum(Tuplesortstate *state, Datum val,
bool isNull);
@@ -445,6 +454,8 @@ extern bool tuplesort_gettupleslot(Tuplesortstate *state, bool forward,
bool copy, TupleTableSlot *slot, Datum *abbrev);
extern HeapTuple tuplesort_getheaptuple(Tuplesortstate *state, bool forward);
extern IndexTuple tuplesort_getindextuple(Tuplesortstate *state, bool forward);
+extern BrinTuple *tuplesort_getbrintuple(Tuplesortstate *state, Size *len,
+ bool forward);
extern bool tuplesort_getdatum(Tuplesortstate *state, bool forward, bool copy,
Datum *val, bool *isNull, Datum *abbrev);
diff --git a/src/test/modules/dummy_index_am/dummy_index_am.c b/src/test/modules/dummy_index_am/dummy_index_am.c
index cbdae7ab7a5..eaa0c483b7e 100644
--- a/src/test/modules/dummy_index_am/dummy_index_am.c
+++ b/src/test/modules/dummy_index_am/dummy_index_am.c
@@ -294,6 +294,7 @@ dihandler(PG_FUNCTION_ARGS)
amroutine->amclusterable = false;
amroutine->ampredlocks = false;
amroutine->amcanparallel = false;
+ amroutine->amcanbuildparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
amroutine->amsummarizing = false;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index d659adbfd6c..8d94ef16623 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -297,13 +297,17 @@ BpChar
BrinBuildState
BrinDesc
BrinInsertState
+BrinLeader
BrinMemTuple
BrinMetaPageData
BrinOpaque
BrinOpcInfo
BrinOptions
BrinRevmap
+BrinShared
+BrinSortTuple
BrinSpecialSpace
+BrinSpool
BrinStatsData
BrinTuple
BrinValues
@@ -2882,6 +2886,7 @@ TupleTableSlotOps
TuplesortClusterArg
TuplesortDatumArg
TuplesortIndexArg
+TuplesortIndexBrinArg
TuplesortIndexBTreeArg
TuplesortIndexHashArg
TuplesortInstrumentation
--
2.41.0
v8-0003-BRIN-Exit-parallel-mode-when-not-starting-paralle.patchtext/x-patch; charset=UTF-8; name=v8-0003-BRIN-Exit-parallel-mode-when-not-starting-paralle.patchDownload
From 2242c6719c81552cf7bceaedcdbde759600d0973 Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Thu, 30 Nov 2023 18:19:40 +0100
Subject: [PATCH v8 3/6] BRIN: Exit parallel mode when not starting parallel
create index
---
src/backend/access/brin/brin.c | 8 +++++++-
1 file changed, 7 insertions(+), 1 deletion(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index edf2daad0c3..bc519825f2e 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -2506,8 +2506,12 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
+ /*
+ * If we didn't actually launch workers, we still have to make sure to exit
+ * parallel mode.
+ */
if (!state)
- return;
+ goto cleanup;
/* copy the data into leader state (we have to wait for the workers ) */
state->bs_reltuples = brinshared->reltuples;
@@ -2691,6 +2695,8 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+cleanup:
+
/* Free last reference to MVCC snapshot, if one was used */
if (IsMVCCSnapshot(brinleader->snapshot))
UnregisterSnapshot(brinleader->snapshot);
--
2.41.0
v8-0004-BRIN-Backfill-empty-ranges.patchtext/x-patch; charset=UTF-8; name=v8-0004-BRIN-Backfill-empty-ranges.patchDownload
From 27675f10b328fc6d80331dfa99a1b8e3309a90ff Mon Sep 17 00:00:00 2001
From: Matthias van de Meent <boekewurm+postgres@gmail.com>
Date: Thu, 30 Nov 2023 18:12:16 +0100
Subject: [PATCH v8 4/6] BRIN: Backfill empty ranges
If we don't, then an index with WHERE (practically_nonnull IS NULL)
would be a full table scan due to missing entries in the ranges table.
The issue is fixed for both normal and parallel index creation.
---
src/backend/access/brin/brin.c | 96 ++++++++++++++++++++--------------
1 file changed, 58 insertions(+), 38 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index bc519825f2e..00b437d9c1f 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -221,6 +221,8 @@ static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy);
static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
BrinMemTuple *dtup, const Datum *values, const bool *nulls);
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
+static BrinTuple *brin_init_empty_tuple(BrinBuildState *state, BlockNumber blkno,
+ BrinTuple *emptyTuple, Size *emptySize);
/* parallel index builds */
static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
@@ -2619,17 +2621,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
prevblkno += state->bs_pagesPerRange;
/* Did we already build the empty range? If not, do it now. */
- if (emptyTuple == NULL)
- {
- BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
-
- emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
- }
- else
- {
- /* we already have am "empty range" tuple, just set the block */
- emptyTuple->bt_blkno = prevblkno;
- }
+ emptyTuple = brin_init_empty_tuple(state, prevblkno, emptyTuple, &emptySize);
brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
&state->bs_currentInsertBuf,
@@ -2641,32 +2633,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
tuplesort_end(spool->sortstate);
- /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
- prevblkno = (prevblkno == InvalidBlockNumber) ? 0 : prevblkno;
- while (prevblkno + state->bs_pagesPerRange < memtuple->bt_blkno)
- {
- /* the missing range */
- prevblkno += state->bs_pagesPerRange;
-
- /* Did we already build the empty range? If not, do it now. */
- if (emptyTuple == NULL)
- {
- BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
-
- emptyTuple = brin_form_tuple(state->bs_bdesc, prevblkno, dtuple, &emptySize);
- }
- else
- {
- /* we already have am "empty range" tuple, just set the block */
- emptyTuple->bt_blkno = prevblkno;
- }
-
- brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
- &state->bs_currentInsertBuf,
- emptyTuple->bt_blkno, emptyTuple, emptySize);
- }
-
- /* Fill the BRIN tuple for the last page range. */
+ /* Fill the BRIN tuple for the last page range with data. */
if (prevblkno != InvalidBlockNumber)
{
BrinTuple *tmp;
@@ -2682,7 +2649,32 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
}
/*
- * Switch back to the originam memory context, and destroy the one we
+ * Fill empty ranges at the end, for all ranges missing in the tuplesort.
+ *
+ * Starting from here, prevblkno is the to-be-inserted range's start block
+ * number. Note that we don't fill in the relation's last page range.
+ */
+ if (prevblkno == InvalidBlockNumber)
+ prevblkno = 0;
+ else
+ prevblkno += state->bs_pagesPerRange;
+
+ while (prevblkno + state->bs_pagesPerRange < state->bs_tablePages)
+ {
+ /* Did we already build the empty range? If not, do it now. */
+ emptyTuple = brin_init_empty_tuple(state, prevblkno, emptyTuple, &emptySize);
+
+ /* Insert the missing range */
+ brin_doinsert(state->bs_irel, state->bs_pagesPerRange, state->bs_rmAccess,
+ &state->bs_currentInsertBuf,
+ emptyTuple->bt_blkno, emptyTuple, emptySize);
+
+ /* ... and update to the next range's block number */
+ prevblkno += state->bs_pagesPerRange;
+ }
+
+ /*
+ * Switch back to the original memory context, and destroy the one we
* created to isolate the union_tuple calls.
*/
MemoryContextSwitchTo(oldCxt);
@@ -2897,3 +2889,31 @@ _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
index_close(indexRel, indexLockmode);
table_close(heapRel, heapLockmode);
}
+
+/*
+ * brin_init_empty_tuple
+ * Maybe initialize a BRIN tuple representing empty range.
+ *
+ * If emptyTuple is NULL, initializes new tuple representing empty range at
+ * block blkno. Otherwise the tuple is reused, and only the bt_blkno field
+ * is updated.
+ */
+static BrinTuple *
+brin_init_empty_tuple(BrinBuildState *state, BlockNumber blkno,
+ BrinTuple *emptyTuple, Size *emptySize)
+{
+ /* Did we already build the empty range? If not, do it now. */
+ if (emptyTuple == NULL)
+ {
+ BrinMemTuple *dtuple = brin_new_memtuple(state->bs_bdesc);
+
+ emptyTuple = brin_form_tuple(state->bs_bdesc, blkno, dtuple, emptySize);
+ }
+ else
+ {
+ /* we already have an "empty range" tuple, just set the block */
+ emptyTuple->bt_blkno = blkno;
+ }
+
+ return emptyTuple;
+}
--
2.41.0
On Sun, 3 Dec 2023 at 17:46, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
On 11/30/23 18:47, Matthias van de Meent wrote:
...
I just ran some more tests in less favorable environments, and it
looks like I hit a bug:% SET max_parallel_workers = 0;
% CREATE INDEX ... USING brin (...);
ERROR: cannot update tuples during a parallel operationFix attached in 0002.
Yeah, that's a bug, thanks for the fix. Yeah Just jumping to a "cleanup"
label seems a bit cleaner (if that can be said about using goto), so I
tweaked the patch to do that instead.
Good point, I agree that's cleaner.
In 0003 I add the mentioned backfilling of empty ranges at the end of
the table. I added it for both normal and parallel index builds, as
normal builds apparently also didn't yet have this yet.Right. I was thinking about doing that to, but you beat me to it. I
don't want to bury this in the main patch adding parallel builds, it's
not really related to parallel CREATE INDEX. And it'd be weird to have
this for parallel builds first, so I rebased it as 0001.
OK.
As for the backfilling, I think we need to simplify the code a bit.
So 0004 simplifies this - the backfilling is done by a function called
from all the places. The main complexity is in ensuring all three places
have the same concept of how to specify the range (of ranges) to fill.
Good points, +1. However, the simplification in 0005 breaks that with
an underflow:
@@ -1669,6 +1672,19 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_worker_id = 0;
state->bs_spool = NULL;+ /* + * Calculate the start of the last page range. Page numbers are 0-based, + * so to get the index of the last page we need to subtract one. Then the + * integer division gives us the proper 0-based range index. + */ + state->bs_maxRangeStart = ((tablePages - 1) / pagesPerRange) * pagesPerRange;
When the table is empty, this will try to fill all potential ranges up
to InvalidBlockNo's range, which is obviously invalid. It also breaks
the regression tests, as showin in CFBot.
skipping the last page range?
-----------------------------I noticed you explicitly skipped backfilling empty tuple for the last
page range. Can you explain? I suspect the idea was that the user
activity would trigger building the tuple once that page range is
filled, but we don't really know if the table receives any changes. It
might easily be just a static table, in which case the last range would
remain unsummarized. If this is the right thing to do, the serial build
should do that too probably ...But I don't think that's the correct thing to do - I think CREATE INDEX
is expected to always build a complete index, so my version always
builds an index for all table pages.
Hmm. My idea here is to create an index that is closer to what you get
when you hit the insertion path with aminsert. This isn't 1:1 how the
index builds ranges during (re)index when there is data for that
range, but I thought it to be a close enough analog. Either way, I
don't mind it adding an empty range for the last range if that's
considered useful.
BlockNumber overflows
---------------------The one thing that I'm not quite sure is correct is whether this handles
overflows/underflows correctly. I mean, imagine you have a huge table
that's almost 0xFFFFFFFF blocks, pages_per_range is prime, and the last
range ends less than pages_per_range from 0xFFFFFFFF. Then thisblkno += pages_per_range;
can overflow, and might start inserting index tuples again (so we'd end
up with a duplicate).I do think the current patch does this correctly, but AFAICS this is a
pre-existing issue ...
Yes, I know I've flagged this at least once before. IIRC, the response
back then was that it's a very unlikely issue, as you'd have to extend
the relation to at least the first block of the last range, which
would currently be InvalidBlockNo - 131072 + 1, or just shy of 32TB of
data at 8kB BLCKSZ. That's not exactly a common use case, and BRIN
range ID wraparound is likely the least of your worries at that point.
Anyway, while working on this / stress-testing it, I realized there's a
bug in how we allocate the emptyTuple. It's allocated lazily, but if can
easily happen in the per-range context we introduced last week. It needs
to be allocated in the context covering the whole index build.
Yeah, I hadn't tested with (very) sparse datasets yet.
I think the best way to do that is per 0006, i.e. allocate it in the
BrinBuildState, along with the appropriate memory context.
That fix looks fine to me.
Kind regards,
Matthias van de Meent
Neon (https://neon.tech)
On 12/4/23 16:00, Matthias van de Meent wrote:
On Sun, 3 Dec 2023 at 17:46, Tomas Vondra <tomas.vondra@enterprisedb.com> wrote:
On 11/30/23 18:47, Matthias van de Meent wrote:
...
I just ran some more tests in less favorable environments, and it
looks like I hit a bug:% SET max_parallel_workers = 0;
% CREATE INDEX ... USING brin (...);
ERROR: cannot update tuples during a parallel operationFix attached in 0002.
Yeah, that's a bug, thanks for the fix. Yeah Just jumping to a "cleanup"
label seems a bit cleaner (if that can be said about using goto), so I
tweaked the patch to do that instead.Good point, I agree that's cleaner.
In 0003 I add the mentioned backfilling of empty ranges at the end of
the table. I added it for both normal and parallel index builds, as
normal builds apparently also didn't yet have this yet.Right. I was thinking about doing that to, but you beat me to it. I
don't want to bury this in the main patch adding parallel builds, it's
not really related to parallel CREATE INDEX. And it'd be weird to have
this for parallel builds first, so I rebased it as 0001.OK.
As for the backfilling, I think we need to simplify the code a bit.
So 0004 simplifies this - the backfilling is done by a function called
from all the places. The main complexity is in ensuring all three places
have the same concept of how to specify the range (of ranges) to fill.Good points, +1. However, the simplification in 0005 breaks that with
an underflow:@@ -1669,6 +1672,19 @@ initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap,
state->bs_worker_id = 0;
state->bs_spool = NULL;+ /* + * Calculate the start of the last page range. Page numbers are 0-based, + * so to get the index of the last page we need to subtract one. Then the + * integer division gives us the proper 0-based range index. + */ + state->bs_maxRangeStart = ((tablePages - 1) / pagesPerRange) * pagesPerRange;When the table is empty, this will try to fill all potential ranges up
to InvalidBlockNo's range, which is obviously invalid. It also breaks
the regression tests, as showin in CFBot.
Whoooops! You're right, ofc. If it's empty, we should use 0 instead.
That's what we do now anyway, BRIN will have the first range even for
empty tables.
skipping the last page range?
-----------------------------I noticed you explicitly skipped backfilling empty tuple for the last
page range. Can you explain? I suspect the idea was that the user
activity would trigger building the tuple once that page range is
filled, but we don't really know if the table receives any changes. It
might easily be just a static table, in which case the last range would
remain unsummarized. If this is the right thing to do, the serial build
should do that too probably ...But I don't think that's the correct thing to do - I think CREATE INDEX
is expected to always build a complete index, so my version always
builds an index for all table pages.Hmm. My idea here is to create an index that is closer to what you get
when you hit the insertion path with aminsert. This isn't 1:1 how the
index builds ranges during (re)index when there is data for that
range, but I thought it to be a close enough analog. Either way, I
don't mind it adding an empty range for the last range if that's
considered useful.
I understand, but I'm not sure if keeping this consistency with aminsert
has any material benefit. It's not like we do that now, I think (for
empty tables we already build the first index range).
BlockNumber overflows
---------------------The one thing that I'm not quite sure is correct is whether this handles
overflows/underflows correctly. I mean, imagine you have a huge table
that's almost 0xFFFFFFFF blocks, pages_per_range is prime, and the last
range ends less than pages_per_range from 0xFFFFFFFF. Then thisblkno += pages_per_range;
can overflow, and might start inserting index tuples again (so we'd end
up with a duplicate).I do think the current patch does this correctly, but AFAICS this is a
pre-existing issue ...Yes, I know I've flagged this at least once before. IIRC, the response
back then was that it's a very unlikely issue, as you'd have to extend
the relation to at least the first block of the last range, which
would currently be InvalidBlockNo - 131072 + 1, or just shy of 32TB of
data at 8kB BLCKSZ. That's not exactly a common use case, and BRIN
range ID wraparound is likely the least of your worries at that point.
Probably true, but it seems somewhat careless and untidy ...
Anyway, while working on this / stress-testing it, I realized there's a
bug in how we allocate the emptyTuple. It's allocated lazily, but if can
easily happen in the per-range context we introduced last week. It needs
to be allocated in the context covering the whole index build.Yeah, I hadn't tested with (very) sparse datasets yet.
I haven't actually checked what the failing cases look like, but I don't
think it needs to be particularly sparse. AFAIK it's just that the
script deletes a chunk of the data somewhere in the table and/or it also
creates a partial index.
I think the best way to do that is per 0006, i.e. allocate it in the
BrinBuildState, along with the appropriate memory context.That fix looks fine to me.
Thanks!
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Hi,
I've pushed the first two parts (backfill of empty ranges for serial
builds, allowing parallelism) after a bit more cleanup, adding a simple
pageinspect test to 0001, improving comments and some minor adjustments.
I ended up removing the protections against BlockNumber overflows, and
moved them into a separate WIP patch. I still think we should probably
reconsider the position that we don't need to worry about issues so
close to the 32TB boundary, but it seemed rather weird to fix only the
new bits and leave the existing issues in place.
I'm attaching that as a WIP patch, but I don't know if/when I'll get
back to this.
Thanks for the reviews/reworks/ideas!
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
0001-WIP-Prevent-overflows-for-BRIN-page-ranges.patchtext/x-patch; charset=UTF-8; name=0001-WIP-Prevent-overflows-for-BRIN-page-ranges.patchDownload
From cf53c109c73cfa9264df71763e9ec5712f1c1f7f Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas@2ndquadrant.com>
Date: Thu, 7 Dec 2023 15:07:04 +0100
Subject: [PATCH] WIP: Prevent overflows for BRIN page ranges
Introduces remove brin_next_range that checks if blkno overflows, and
instead sets it to InvalidBlockNumber.
---
src/backend/access/brin/brin.c | 17 +++++++++++++++--
1 file changed, 15 insertions(+), 2 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 23f081389b2..1952142d050 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -227,6 +227,7 @@ static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc,
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys);
static void brin_fill_empty_ranges(BrinBuildState *state,
BlockNumber prevRange, BlockNumber maxRange);
+static BlockNumber brin_next_range(BrinBuildState *state, BlockNumber blkno);
/* parallel index builds */
static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
@@ -2935,7 +2936,7 @@ brin_fill_empty_ranges(BrinBuildState *state,
* If we already summarized some ranges, we need to start with the next
* one. Otherwise start from the first range of the table.
*/
- blkno = (prevRange == InvalidBlockNumber) ? 0 : (prevRange + state->bs_pagesPerRange);
+ blkno = (prevRange == InvalidBlockNumber) ? 0 : brin_next_range(state, prevRange);
/* Generate empty ranges until we hit the next non-empty range. */
while (blkno < nextRange)
@@ -2948,6 +2949,18 @@ brin_fill_empty_ranges(BrinBuildState *state,
blkno, state->bs_emptyTuple, state->bs_emptyTupleLen);
/* try next page range */
- blkno += state->bs_pagesPerRange;
+ blkno = brin_next_range(state, blkno);
}
}
+
+static BlockNumber
+brin_next_range(BrinBuildState *state, BlockNumber blkno)
+{
+ BlockNumber ret = (blkno + state->bs_pagesPerRange);
+
+ /* overflow */
+ if (ret < blkno)
+ ret = InvalidBlockNumber;
+
+ return ret;
+}
--
2.41.0
Hi,
While cleaning up some unnecessary bits of the code and slightly
inaccurate comments, I ran into a failure when the parallel scan (used
by the parallel build) happened to be a synchronized scan. When the scan
did not start on page 0, the parallel callback failed to correctly
handle tuples after wrapping around to the start of the table.
AFAICS the extensive testing I did during development did not detect
this because strictly speaking the index was "correct" (as in not
returning incorrect results in queries), just less efficient (missing
some ranges, and some ranges being "wider" than necessary). Or perhaps
the tests happened to not trigger synchronized scans.
Should be fixed by 1ccab5038eaf261f. It took me ages to realize what the
problem is, and I initially suspected there's some missing coordination
between the workers/leader, or something.
So I started comparing the code to btree, which is where it originated,
and I realized there's indeed one difference - the BRIN code only does
half the work with the workersdonecv variable. The workers do correctly
update the count and notify the leader, but the leader never waits for
the count to be 0. That is, there's nothing like _bt_parallel_heapscan.
I wonder whether this actually is a problem, considering the differences
between the flow in BRIN and BTREE. In particular, the "leader" does the
work in _brin_end_parallel() after WaitForParallelWorkersToFinish(). So
it's not like there might be a worker still processing data, I think.
But now that I think about it, maybe it's not such a great idea to do
this kind of work in _brin_end_parallel(). Maybe it should do just stuff
related to termination of workers etc. and the merging of results should
happen elsewhere - earlier in brinbuild()? Then it'd make sense to have
something like _bt_parallel_heapscan ...
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Hi,
While preparing a differential code coverage report between 16 and HEAD, one
thing that stands out is the parallel brin build code. Neither on
coverage.postgresql.org nor locally is that code reached during our tests.
https://coverage.postgresql.org/src/backend/access/brin/brin.c.gcov.html#2333
Greetings,
Andres Freund
On 4/13/24 10:36, Andres Freund wrote:
Hi,
While preparing a differential code coverage report between 16 and HEAD, one
thing that stands out is the parallel brin build code. Neither on
coverage.postgresql.org nor locally is that code reached during our tests.
Thanks for pointing this out, it's definitely something that I need to
improve (admittedly, should have been part of the patch). I'll also look
into eliminating the difference between BTREE and BRIN parallel builds,
mentioned in my last message in this thread.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On 4/13/24 11:19, Tomas Vondra wrote:
On 4/13/24 10:36, Andres Freund wrote:
Hi,
While preparing a differential code coverage report between 16 and HEAD, one
thing that stands out is the parallel brin build code. Neither on
coverage.postgresql.org nor locally is that code reached during our tests.Thanks for pointing this out, it's definitely something that I need to
improve (admittedly, should have been part of the patch). I'll also look
into eliminating the difference between BTREE and BRIN parallel builds,
mentioned in my last message in this thread.
Here's a couple patches adding a test for the parallel CREATE INDEX with
BRIN. The actual test is 0003/0004 - I added the test to pageinspect,
because that allows cross-checking the index to one built without
parallelism, which I think is better than just doing CREATE INDEX
without properly testing it produces correct results.
It's not entirely trivial because for some opclasses (e.g. minmax-multi)
the results depend on the order in which values are added, and order in
which summaries from different workers are merged.
Funnily enough, while adding the test, I ran into two pre-existing bugs.
One is that brin_bloom_union forgot to update the number of bits set in
the bitmap, another one is that 6bcda4a721 changes PG_DETOAST_DATUM to
the _PACKED version, which however does the wrong thing. Both of which
are mostly harmless - it only affects the output function, which is
unused outside pageinspect. No impact on query correctness etc.
The test needs a bit more work to make sure it works on 32-bit machines
etc. which I think may affect available space on a page, which in turn
might affect the minmax-multi summaries. But I'll take care this early
next week.
Funnily
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
v20240413-0001-Update-nbits_set-in-brin_bloom_union.patchtext/x-patch; charset=UTF-8; name=v20240413-0001-Update-nbits_set-in-brin_bloom_union.patchDownload
From 450f992c12d43c14aa1b4b72cfd9d3ce0251eed1 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Sat, 13 Apr 2024 22:15:26 +0200
Subject: [PATCH v20240413 1/4] Update nbits_set in brin_bloom_union
When merging BRIN bloom summaries, it's not enough to merge the bitmaps,
we need to update the nbits_set counter too.
This is mostly harmless, as it does not affect correctness - as the
counter is used only in the out function, and that's used only in
pageinspect to show basic info about the summary.
Backpatch-through: 14
---
src/backend/access/brin/brin_bloom.c | 10 ++++++++++
1 file changed, 10 insertions(+)
diff --git a/src/backend/access/brin/brin_bloom.c b/src/backend/access/brin/brin_bloom.c
index ebf33016279..9e3bc567303 100644
--- a/src/backend/access/brin/brin_bloom.c
+++ b/src/backend/access/brin/brin_bloom.c
@@ -687,10 +687,20 @@ brin_bloom_union(PG_FUNCTION_ARGS)
nbytes = (filter_a->nbits) / 8;
+ filter_a->nbits_set = 0;
+
/* simply OR the bitmaps */
for (i = 0; i < nbytes; i++)
+ {
filter_a->data[i] |= filter_b->data[i];
+ for (int bit = 0; bit < 8; bit++)
+ {
+ if (filter_a->data[i] & (0x01 << bit))
+ filter_a->nbits_set++;
+ }
+ }
+
PG_RETURN_VOID();
}
--
2.44.0
v20240413-0002-Use-correct-PG_DETOAST_DATUM-macro-in-BRIN.patchtext/x-patch; charset=UTF-8; name=v20240413-0002-Use-correct-PG_DETOAST_DATUM-macro-in-BRIN.patchDownload
From 07739713893bb0f6be5853324e8897159cf2ad07 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Sat, 13 Apr 2024 22:18:37 +0200
Subject: [PATCH v20240413 2/4] Use correct PG_DETOAST_DATUM macro in BRIN
Commit 6bcda4a721 replaced PG_DETOAST_DATUM with PG_DETOAST_DATUM_PACKED
in two places, in output function for minmax-multi and bloom. But this
is incorrect - the structs backing both summaries include 4B header, so
we need to detoast them fully. With _PACKED we may end up with 1B
header, so the fields will be shifted and have incorrect values.
Backpatch-through: 16
---
src/backend/access/brin/brin_bloom.c | 2 +-
src/backend/access/brin/brin_minmax_multi.c | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/src/backend/access/brin/brin_bloom.c b/src/backend/access/brin/brin_bloom.c
index 9e3bc567303..1c2437cef74 100644
--- a/src/backend/access/brin/brin_bloom.c
+++ b/src/backend/access/brin/brin_bloom.c
@@ -807,7 +807,7 @@ brin_bloom_summary_out(PG_FUNCTION_ARGS)
StringInfoData str;
/* detoast the data to get value with a full 4B header */
- filter = (BloomFilter *) PG_DETOAST_DATUM_PACKED(PG_GETARG_DATUM(0));
+ filter = (BloomFilter *) PG_DETOAST_DATUM(PG_GETARG_DATUM(0));
initStringInfo(&str);
appendStringInfoChar(&str, '{');
diff --git a/src/backend/access/brin/brin_minmax_multi.c b/src/backend/access/brin/brin_minmax_multi.c
index c5962c00d64..e5d95de5d84 100644
--- a/src/backend/access/brin/brin_minmax_multi.c
+++ b/src/backend/access/brin/brin_minmax_multi.c
@@ -3023,7 +3023,7 @@ brin_minmax_multi_summary_out(PG_FUNCTION_ARGS)
* Detoast to get value with full 4B header (can't be stored in a toast
* table, but can use 1B header).
*/
- ranges = (SerializedRanges *) PG_DETOAST_DATUM_PACKED(PG_GETARG_DATUM(0));
+ ranges = (SerializedRanges *) PG_DETOAST_DATUM(PG_GETARG_DATUM(0));
/* lookup output func for the type */
getTypeOutputInfo(ranges->typid, &outfunc, &isvarlena);
--
2.44.0
v20240413-0003-Add-regression-tests-for-BRIN-parallel-bui.patchtext/x-patch; charset=UTF-8; name=v20240413-0003-Add-regression-tests-for-BRIN-parallel-bui.patchDownload
From e6c8c6f5d1d9420a5166512af25e7c3407a877d1 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Sat, 13 Apr 2024 22:25:35 +0200
Subject: [PATCH v20240413 3/4] Add regression tests for BRIN parallel builds
Adds regression test for parallel CREATE INDEX with a BRIN index. The
test is added to pageinspect, and which makes it easy to compare the
parallel index to an index built without parallelism.
---
contrib/pageinspect/expected/brin.out | 106 ++++++++++++++++++++++++++
contrib/pageinspect/sql/brin.sql | 85 +++++++++++++++++++++
2 files changed, 191 insertions(+)
diff --git a/contrib/pageinspect/expected/brin.out b/contrib/pageinspect/expected/brin.out
index 3f6e5174bc6..66b199dba8a 100644
--- a/contrib/pageinspect/expected/brin.out
+++ b/contrib/pageinspect/expected/brin.out
@@ -108,3 +108,109 @@ SELECT (COUNT(*) = (SELECT relpages FROM pg_class WHERE relname = 'test2')) AS r
DROP TABLE test1;
DROP TABLE test2;
+-- test that parallel index build produces the same BRIN index as serial build
+CREATE TABLE brin_parallel_test (a int, b text) WITH (fillfactor=25);
+-- a mix of NULLs and non-NULL values
+INSERT INTO brin_parallel_test
+SELECT (CASE WHEN (mod(i,231) = 0) OR (i BETWEEN 3500 AND 4000) THEN NULL ELSE i END),
+ (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END)
+ FROM generate_series(1,10000) S(i);
+-- delete a couple pages, to make a range empty
+DELETE FROM brin_parallel_test WHERE a BETWEEN 1000 and 2000;
+-- vacuum to make the ranges actually empty
+VACUUM brin_parallel_test;
+-- We build index with different BRIN opclasses, but we can't use minmax-multi
+-- because the summary is not deterministic (i.e. the order of values affects
+-- how ranges get merged, and if workers see random subsets between runs, that
+-- affects the resulting summaries too). The summaries are equivalent in the
+-- sense that should give the same query results, but look different enough to
+-- not be very useful/convenient for this test.
+SET max_parallel_maintenance_workers = 0;
+CREATE INDEX brin_test_serial_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+-- XXX we have to set some parameters to force parallel build, both because
+-- there's a requirement for table size and plan_create_index_workers assumes
+-- each worker will use work_mem=32MB for sorting (which works for btree, but
+-- not really for BRIN)
+SET min_parallel_table_scan_size = 0;
+SET max_parallel_maintenance_workers = 4;
+SET maintenance_work_mem = '128MB';
+CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+SELECT relname, relpages FROM pg_class WHERE relname IN ('brin_test_serial_idx', 'brin_test_parallel_idx') ORDER BY relname;
+ relname | relpages
+------------------------+----------
+ brin_test_parallel_idx | 4
+ brin_test_serial_idx | 4
+(2 rows)
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 2), 'brin_test_parallel_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 2), 'brin_test_serial_idx');
+ itemoffset | blknum | attnum | allnulls | hasnulls | placeholder | empty | value
+------------+--------+--------+----------+----------+-------------+-------+-------
+(0 rows)
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 2), 'brin_test_serial_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 2), 'brin_test_parallel_idx');
+ itemoffset | blknum | attnum | allnulls | hasnulls | placeholder | empty | value
+------------+--------+--------+----------+----------+-------------+-------+-------
+(0 rows)
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_test_parallel_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 3), 'brin_test_serial_idx');
+ itemoffset | blknum | attnum | allnulls | hasnulls | placeholder | empty | value
+------------+--------+--------+----------+----------+-------------+-------+-------
+(0 rows)
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 3), 'brin_test_serial_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_test_parallel_idx');
+ itemoffset | blknum | attnum | allnulls | hasnulls | placeholder | empty | value
+------------+--------+--------+----------+----------+-------------+-------+-------
+(0 rows)
+
+-- now do the same parallel build, but don't allow starting parallel workers
+-- to force fallback to serial build, and repeat the checks
+DROP INDEX brin_test_parallel_idx;
+SET max_parallel_workers = 0;
+CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+SELECT relname, relpages FROM pg_class WHERE relname IN ('brin_test_serial_idx', 'brin_test_parallel_idx') ORDER BY relname;
+ relname | relpages
+------------------------+----------
+ brin_test_parallel_idx | 4
+ brin_test_serial_idx | 4
+(2 rows)
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 2), 'brin_test_parallel_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 2), 'brin_test_serial_idx');
+ itemoffset | blknum | attnum | allnulls | hasnulls | placeholder | empty | value
+------------+--------+--------+----------+----------+-------------+-------+-------
+(0 rows)
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 2), 'brin_test_serial_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 2), 'brin_test_parallel_idx');
+ itemoffset | blknum | attnum | allnulls | hasnulls | placeholder | empty | value
+------------+--------+--------+----------+----------+-------------+-------+-------
+(0 rows)
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_test_parallel_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 3), 'brin_test_serial_idx');
+ itemoffset | blknum | attnum | allnulls | hasnulls | placeholder | empty | value
+------------+--------+--------+----------+----------+-------------+-------+-------
+(0 rows)
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 3), 'brin_test_serial_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_test_parallel_idx');
+ itemoffset | blknum | attnum | allnulls | hasnulls | placeholder | empty | value
+------------+--------+--------+----------+----------+-------------+-------+-------
+(0 rows)
+
+DROP TABLE brin_parallel_test;
+RESET min_parallel_table_scan_size;
+RESET max_parallel_maintenance_workers;
+RESET maintenance_work_mem;
diff --git a/contrib/pageinspect/sql/brin.sql b/contrib/pageinspect/sql/brin.sql
index 50f260b8e1f..affd805bb27 100644
--- a/contrib/pageinspect/sql/brin.sql
+++ b/contrib/pageinspect/sql/brin.sql
@@ -54,3 +54,88 @@ SELECT (COUNT(*) = (SELECT relpages FROM pg_class WHERE relname = 'test2')) AS r
DROP TABLE test1;
DROP TABLE test2;
+
+-- test that parallel index build produces the same BRIN index as serial build
+CREATE TABLE brin_parallel_test (a int, b text) WITH (fillfactor=25);
+
+-- a mix of NULLs and non-NULL values
+INSERT INTO brin_parallel_test
+SELECT (CASE WHEN (mod(i,231) = 0) OR (i BETWEEN 3500 AND 4000) THEN NULL ELSE i END),
+ (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END)
+ FROM generate_series(1,10000) S(i);
+
+-- delete a couple pages, to make a range empty
+DELETE FROM brin_parallel_test WHERE a BETWEEN 1000 and 2000;
+
+-- vacuum to make the ranges actually empty
+VACUUM brin_parallel_test;
+
+-- We build index with different BRIN opclasses, but we can't use minmax-multi
+-- because the summary is not deterministic (i.e. the order of values affects
+-- how ranges get merged, and if workers see random subsets between runs, that
+-- affects the resulting summaries too). The summaries are equivalent in the
+-- sense that should give the same query results, but look different enough to
+-- not be very useful/convenient for this test.
+SET max_parallel_maintenance_workers = 0;
+CREATE INDEX brin_test_serial_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+
+-- XXX we have to set some parameters to force parallel build, both because
+-- there's a requirement for table size and plan_create_index_workers assumes
+-- each worker will use work_mem=32MB for sorting (which works for btree, but
+-- not really for BRIN)
+SET min_parallel_table_scan_size = 0;
+SET max_parallel_maintenance_workers = 4;
+SET maintenance_work_mem = '128MB';
+CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+
+SELECT relname, relpages FROM pg_class WHERE relname IN ('brin_test_serial_idx', 'brin_test_parallel_idx') ORDER BY relname;
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 2), 'brin_test_parallel_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 2), 'brin_test_serial_idx');
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 2), 'brin_test_serial_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 2), 'brin_test_parallel_idx');
+
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_test_parallel_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 3), 'brin_test_serial_idx');
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 3), 'brin_test_serial_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_test_parallel_idx');
+
+-- now do the same parallel build, but don't allow starting parallel workers
+-- to force fallback to serial build, and repeat the checks
+
+DROP INDEX brin_test_parallel_idx;
+
+SET max_parallel_workers = 0;
+CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+
+SELECT relname, relpages FROM pg_class WHERE relname IN ('brin_test_serial_idx', 'brin_test_parallel_idx') ORDER BY relname;
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 2), 'brin_test_parallel_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 2), 'brin_test_serial_idx');
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 2), 'brin_test_serial_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 2), 'brin_test_parallel_idx');
+
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_test_parallel_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 3), 'brin_test_serial_idx');
+
+SELECT * FROM brin_page_items(get_raw_page('brin_test_serial_idx', 3), 'brin_test_serial_idx')
+EXCEPT
+SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_test_parallel_idx');
+
+
+DROP TABLE brin_parallel_test;
+RESET min_parallel_table_scan_size;
+RESET max_parallel_maintenance_workers;
+RESET maintenance_work_mem;
--
2.44.0
v20240413-0004-add-minmax-multi-to-the-regression-test.patchtext/x-patch; charset=UTF-8; name=v20240413-0004-add-minmax-multi-to-the-regression-test.patchDownload
From a79446bdc46c85ae9e9386e55bc3f7da43523106 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tv@fuzzy.cz>
Date: Sat, 13 Apr 2024 22:50:53 +0200
Subject: [PATCH v20240413 4/4] add minmax-multi to the regression test
---
contrib/pageinspect/expected/brin.out | 25 ++++++++++++++-----------
contrib/pageinspect/sql/brin.sql | 25 ++++++++++++++-----------
2 files changed, 28 insertions(+), 22 deletions(-)
diff --git a/contrib/pageinspect/expected/brin.out b/contrib/pageinspect/expected/brin.out
index 66b199dba8a..33e565f8300 100644
--- a/contrib/pageinspect/expected/brin.out
+++ b/contrib/pageinspect/expected/brin.out
@@ -109,24 +109,27 @@ SELECT (COUNT(*) = (SELECT relpages FROM pg_class WHERE relname = 'test2')) AS r
DROP TABLE test1;
DROP TABLE test2;
-- test that parallel index build produces the same BRIN index as serial build
-CREATE TABLE brin_parallel_test (a int, b text) WITH (fillfactor=25);
+CREATE TABLE brin_parallel_test (a int, b text, c bigint) WITH (fillfactor=40);
-- a mix of NULLs and non-NULL values
INSERT INTO brin_parallel_test
SELECT (CASE WHEN (mod(i,231) = 0) OR (i BETWEEN 3500 AND 4000) THEN NULL ELSE i END),
- (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END)
+ (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END),
+ (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE (i/40) + mod(i,8) END)
FROM generate_series(1,10000) S(i);
-- delete a couple pages, to make a range empty
DELETE FROM brin_parallel_test WHERE a BETWEEN 1000 and 2000;
-- vacuum to make the ranges actually empty
VACUUM brin_parallel_test;
--- We build index with different BRIN opclasses, but we can't use minmax-multi
--- because the summary is not deterministic (i.e. the order of values affects
--- how ranges get merged, and if workers see random subsets between runs, that
--- affects the resulting summaries too). The summaries are equivalent in the
--- sense that should give the same query results, but look different enough to
--- not be very useful/convenient for this test.
+-- We build index with different BRIN opclasses - minmax, bloom and minmax-multi.
+-- With minmax-multi we need to be more careful, because the summary is not
+-- entirely deterministic - the order of values affects how ranges get merged,
+-- and if workers see random subsets between runs, that affects the resulting
+-- summaries too). The summaries would be equivalent in the sense that should
+-- give the same query results, but are not equal for EXCEPT to work. To work
+-- around that, we generate only small number of values to keep exact values,
+-- in which case the ordering does not matter.
SET max_parallel_maintenance_workers = 0;
-CREATE INDEX brin_test_serial_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+CREATE INDEX brin_test_serial_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b, c int8_minmax_multi_ops) WITH (pages_per_range=7);
-- XXX we have to set some parameters to force parallel build, both because
-- there's a requirement for table size and plan_create_index_workers assumes
-- each worker will use work_mem=32MB for sorting (which works for btree, but
@@ -134,7 +137,7 @@ CREATE INDEX brin_test_serial_idx ON brin_parallel_test USING brin (a int4_minma
SET min_parallel_table_scan_size = 0;
SET max_parallel_maintenance_workers = 4;
SET maintenance_work_mem = '128MB';
-CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b, c int8_minmax_multi_ops) WITH (pages_per_range=7);
SELECT relname, relpages FROM pg_class WHERE relname IN ('brin_test_serial_idx', 'brin_test_parallel_idx') ORDER BY relname;
relname | relpages
------------------------+----------
@@ -174,7 +177,7 @@ SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_t
-- to force fallback to serial build, and repeat the checks
DROP INDEX brin_test_parallel_idx;
SET max_parallel_workers = 0;
-CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b, c int8_minmax_multi_ops) WITH (pages_per_range=7);
SELECT relname, relpages FROM pg_class WHERE relname IN ('brin_test_serial_idx', 'brin_test_parallel_idx') ORDER BY relname;
relname | relpages
------------------------+----------
diff --git a/contrib/pageinspect/sql/brin.sql b/contrib/pageinspect/sql/brin.sql
index affd805bb27..f113de39c5d 100644
--- a/contrib/pageinspect/sql/brin.sql
+++ b/contrib/pageinspect/sql/brin.sql
@@ -56,12 +56,13 @@ DROP TABLE test1;
DROP TABLE test2;
-- test that parallel index build produces the same BRIN index as serial build
-CREATE TABLE brin_parallel_test (a int, b text) WITH (fillfactor=25);
+CREATE TABLE brin_parallel_test (a int, b text, c bigint) WITH (fillfactor=40);
-- a mix of NULLs and non-NULL values
INSERT INTO brin_parallel_test
SELECT (CASE WHEN (mod(i,231) = 0) OR (i BETWEEN 3500 AND 4000) THEN NULL ELSE i END),
- (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END)
+ (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END),
+ (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE (i/40) + mod(i,8) END)
FROM generate_series(1,10000) S(i);
-- delete a couple pages, to make a range empty
@@ -70,14 +71,16 @@ DELETE FROM brin_parallel_test WHERE a BETWEEN 1000 and 2000;
-- vacuum to make the ranges actually empty
VACUUM brin_parallel_test;
--- We build index with different BRIN opclasses, but we can't use minmax-multi
--- because the summary is not deterministic (i.e. the order of values affects
--- how ranges get merged, and if workers see random subsets between runs, that
--- affects the resulting summaries too). The summaries are equivalent in the
--- sense that should give the same query results, but look different enough to
--- not be very useful/convenient for this test.
+-- We build index with different BRIN opclasses - minmax, bloom and minmax-multi.
+-- With minmax-multi we need to be more careful, because the summary is not
+-- entirely deterministic - the order of values affects how ranges get merged,
+-- and if workers see random subsets between runs, that affects the resulting
+-- summaries too). The summaries would be equivalent in the sense that should
+-- give the same query results, but are not equal for EXCEPT to work. To work
+-- around that, we generate only small number of values to keep exact values,
+-- in which case the ordering does not matter.
SET max_parallel_maintenance_workers = 0;
-CREATE INDEX brin_test_serial_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+CREATE INDEX brin_test_serial_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b, c int8_minmax_multi_ops) WITH (pages_per_range=7);
-- XXX we have to set some parameters to force parallel build, both because
-- there's a requirement for table size and plan_create_index_workers assumes
@@ -86,7 +89,7 @@ CREATE INDEX brin_test_serial_idx ON brin_parallel_test USING brin (a int4_minma
SET min_parallel_table_scan_size = 0;
SET max_parallel_maintenance_workers = 4;
SET maintenance_work_mem = '128MB';
-CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b, c int8_minmax_multi_ops) WITH (pages_per_range=7);
SELECT relname, relpages FROM pg_class WHERE relname IN ('brin_test_serial_idx', 'brin_test_parallel_idx') ORDER BY relname;
@@ -113,7 +116,7 @@ SELECT * FROM brin_page_items(get_raw_page('brin_test_parallel_idx', 3), 'brin_t
DROP INDEX brin_test_parallel_idx;
SET max_parallel_workers = 0;
-CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b) WITH (pages_per_range=7);
+CREATE INDEX brin_test_parallel_idx ON brin_parallel_test USING brin (a int4_minmax_ops, a int4_bloom_ops, b, c int8_minmax_multi_ops) WITH (pages_per_range=7);
SELECT relname, relpages FROM pg_class WHERE relname IN ('brin_test_serial_idx', 'brin_test_parallel_idx') ORDER BY relname;
--
2.44.0
I've pushed this, including backpatching the two fixes. I've reduced the
amount of data needed by the test, and made sure it works on 32-bits too
(I was a bit worried it might be sensitive to that, but that seems not
to be the case).
There's still the question of maybe removing the differences between the
BTREE and BRIN code for parallel builds, I mentioned in [1]/messages/by-id/3733d042-71e1-6ae6-5fac-00c12db62db6@enterprisedb.com. That's more
of a cosmetic issue, but I'll add it as an open item for myself.
regards
[1]: /messages/by-id/3733d042-71e1-6ae6-5fac-00c12db62db6@enterprisedb.com
/messages/by-id/3733d042-71e1-6ae6-5fac-00c12db62db6@enterprisedb.com
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Hello Tomas,
14.04.2024 20:09, Tomas Vondra wrote:
I've pushed this, including backpatching the two fixes. I've reduced the
amount of data needed by the test, and made sure it works on 32-bits too
(I was a bit worried it might be sensitive to that, but that seems not
to be the case).
I've discovered that that test addition brings some instability to the test.
With the following pageinspect/Makefile modification:
-REGRESS = page btree brin gin gist hash checksum oldextversions
+REGRESS = page btree brin $(shell printf 'brin %.0s' `seq 99`) gin gist hash checksum oldextversions
echo "autovacuum_naptime = 1" > /tmp/temp.config
TEMP_CONFIG=/tmp/temp.config make -s check -C contrib/pageinspect
fails for me as below:
...
ok 17 - brin 127 ms
not ok 18 - brin 140 ms
ok 19 - brin 125 ms
...
# 4 of 107 tests failed.
The following change:
-CREATE TABLE brin_parallel_test (a int, b text, c bigint) WITH (fillfactor=40);
+CREATE TEMP TABLE brin_parallel_test (a int, b text, c bigint) WITH (fillfactor=40);
(similar to e2933a6e1) makes the test pass reliably for me.
Best regards,
Alexander
On 4/15/24 08:00, Alexander LAW wrote:
Hello Tomas,
14.04.2024 20:09, Tomas Vondra wrote:
I've pushed this, including backpatching the two fixes. I've reduced the
amount of data needed by the test, and made sure it works on 32-bits too
(I was a bit worried it might be sensitive to that, but that seems not
to be the case).I've discovered that that test addition brings some instability to the test. With the following pageinspect/Makefile modification: -REGRESS = page btree brin gin gist hash checksum oldextversions +REGRESS = page btree brin $(shell printf 'brin %.0s' `seq 99`) gin gist hash checksum oldextversionsecho "autovacuum_naptime = 1" > /tmp/temp.config
TEMP_CONFIG=/tmp/temp.config make -s check -C contrib/pageinspect
fails for me as below:
...
ok 17 - brin 127 ms
not ok 18 - brin 140 ms
ok 19 - brin 125 ms
...
# 4 of 107 tests failed.The following change -CREATE TABLE brin_parallel_test (a int, b text, c bigint) WITH (fillfactor=40); +CREATE TEMP TABLE brin_parallel_test (a int, b text, c bigint) WITH (fillfactor=40); (similar to e2933a6e1) makes the test pass reliably for me.
Thanks! This reproduces the issue for me.
I believe this happens because the test does "DELETE + VACUUM" to
generate "gaps" in the table, to get empty ranges in the BRIN. I guess
what's happening is that something (autovacuum or likely something else)
blocks the explicit VACUUM from cleaning some of the pages with deleted
tuples, but then the cleanup happens shortly after between building the
the serial/parallel indexes. That would explain the differences reported
by the regression test.
When I thought about this while writing the test, my reasoning was that
even if the explicit vacuum occasionally fails to clean something, it
should affect all the indexes equally. Which is why I wrote the test to
compare the results using EXCEPT, not checking the exact output.
I'm not a huge fan of temporary tables in regression tests, because it
disappears at the end, making it impossible to inspect the data after a
failure. But the only other option I could think of is disabling
autovacuum on the table, but that does not seem to prevent the failures.
I'll try a bit more to make this work without the temp table.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Import Notes
Reply to msg id not found: ea1af08e-53b6-640c-efe3-0a5c73044f2d@1o.ru
On 4/15/24 10:18, Tomas Vondra wrote:
...
I'll try a bit more to make this work without the temp table.
Considering the earlier discussion in e2933a6e1, I think making the
table TEMP is the best fix, so I'll do that. Thanks for remembering that
change, Alexander!
Attached is the cleanup I thought about doing earlier in this patch [1]
to make the code more like btree. The diff might make it seem like a big
change, but it really just moves the merge code into a separate function
and makes it use using the conditional variable. I still believe the old
code is correct, but this seems like an improvement so plan to push this
soon and resolve the open item.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
0001-Cleanup-parallel-BRIN-index-build-code.patchtext/x-patch; charset=UTF-8; name=0001-Cleanup-parallel-BRIN-index-build-code.patchDownload
From c37d96c4d3a151b3a30f28b971e6058d20978b76 Mon Sep 17 00:00:00 2001
From: Tomas Vondra <tomas@2ndquadrant.com>
Date: Mon, 15 Apr 2024 17:59:31 +0200
Subject: [PATCH] Cleanup parallel BRIN index build code
Commit b43757171470 introduced parallel builds for BRIN indexes, using
code fairly similar to BTREE. But there happened to be a couple minor
unnecessary differences, particularly in when the leader waits for the
workers and merges the results.
Unlike the BTREE code, the leader never waited on the workersdonecv
condition variable, but simply called WaitForParallelWorkersToFinish()
in _brin_end_parallel() before merging the per-worker results. While
this works correctly, it's probably better to do the merging earlier,
after waiting on the condition variable. This way _brin_end_parallel()
is responsible only for exiting the parallel mode and accumulating WAL
usage data, same as in BTREE.
The mering of per-worker results now happens in _brin_parallel_merge(),
while _brin_parallel_heapscan() is responsible for waiting for the
workers to finish scanning the heap.
Discussion: https://postgr.es/m/3733d042-71e1-6ae6-5fac-00c12db62db6@enterprisedb.com
---
src/backend/access/brin/brin.c | 130 ++++++++++++++++++++++++---------
1 file changed, 97 insertions(+), 33 deletions(-)
diff --git a/src/backend/access/brin/brin.c b/src/backend/access/brin/brin.c
index 041415a40e7..32722f0961b 100644
--- a/src/backend/access/brin/brin.c
+++ b/src/backend/access/brin/brin.c
@@ -229,6 +229,8 @@ static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Rela
bool isconcurrent, int request);
static void _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state);
static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot);
+static double _brin_parallel_heapscan(BrinBuildState *buildstate);
+static double _brin_parallel_merge(BrinBuildState *buildstate);
static void _brin_leader_participate_as_worker(BrinBuildState *buildstate,
Relation heap, Relation index);
static void _brin_parallel_scan_and_build(BrinBuildState *buildstate,
@@ -1201,6 +1203,9 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
tuplesort_begin_index_brin(maintenance_work_mem, coordinate,
TUPLESORT_NONE);
+ /* scan the relation and merge per-worker results */
+ reltuples = _brin_parallel_merge(state);
+
_brin_end_parallel(state->bs_leader, state);
}
else /* no parallel index build */
@@ -1233,14 +1238,10 @@ brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
brin_fill_empty_ranges(state,
state->bs_currRangeStart,
state->bs_maxRangeStart);
-
- /* track the number of relation tuples */
- state->bs_reltuples = reltuples;
}
/* release resources */
idxtuples = state->bs_numtuples;
- reltuples = state->bs_reltuples;
brinRevmapTerminate(state->bs_rmAccess);
terminate_brin_buildstate(state);
@@ -2329,6 +2330,22 @@ check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
return true;
}
+/*
+ * Create parallel context, and launch workers for leader.
+ *
+ * buildstate argument should be initialized (with the exception of the
+ * tuplesort states, 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 BrinLeader, which caller must use to shut down parallel
+ * mode by passing it to _brin_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
_brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index,
bool isconcurrent, int request)
@@ -2517,27 +2534,87 @@ static void
_brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
{
int i;
- BrinTuple *btup;
- BrinMemTuple *memtuple = NULL;
- Size tuplen;
- BrinShared *brinshared = brinleader->brinshared;
- BlockNumber prevblkno = InvalidBlockNumber;
- MemoryContext rangeCxt,
- oldCxt;
/* Shutdown worker processes */
WaitForParallelWorkersToFinish(brinleader->pcxt);
/*
- * If we didn't actually launch workers, we still have to make sure to
- * exit parallel mode.
+ * Next, accumulate WAL usage. (This must wait for the workers to finish,
+ * or we might get incomplete data.)
*/
- if (!state)
- goto cleanup;
+ for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
+ InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
+
+ /* Free last reference to MVCC snapshot, if one was used */
+ if (IsMVCCSnapshot(brinleader->snapshot))
+ UnregisterSnapshot(brinleader->snapshot);
+ DestroyParallelContext(brinleader->pcxt);
+ ExitParallelMode();
+}
+
+/*
+ * Within leader, wait for end of heap scan.
+ *
+ * When called, parallel heap scan started by _brin_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).
+ *
+ * Returns the total number of heap tuples scanned.
+ */
+static double
+_brin_parallel_heapscan(BrinBuildState *state)
+{
+ BrinShared *brinshared = state->bs_leader->brinshared;
+ int nparticipanttuplesorts;
+
+ nparticipanttuplesorts = state->bs_leader->nparticipanttuplesorts;
+ for (;;)
+ {
+ SpinLockAcquire(&brinshared->mutex);
+ if (brinshared->nparticipantsdone == nparticipanttuplesorts)
+ {
+ /* copy the data into leader state */
+ state->bs_reltuples = brinshared->reltuples;
+ state->bs_numtuples = brinshared->indtuples;
- /* copy the data into leader state (we have to wait for the workers ) */
- state->bs_reltuples = brinshared->reltuples;
- state->bs_numtuples = brinshared->indtuples;
+ SpinLockRelease(&brinshared->mutex);
+ break;
+ }
+ SpinLockRelease(&brinshared->mutex);
+
+ ConditionVariableSleep(&brinshared->workersdonecv,
+ WAIT_EVENT_PARALLEL_CREATE_INDEX_SCAN);
+ }
+
+ ConditionVariableCancelSleep();
+
+ return state->bs_reltuples;
+}
+
+/*
+ * Within leader, wait for end of heap scan and merge per-worker results.
+ *
+ * After waiting for all workers to finish, merge the per-worker results into
+ * the complete index. The results from each worker are sorted by block number
+ * (start of the page range). While combinig the per-worker results we merge
+ * summaries for the same page range, and also fill-in empty summaries for
+ * ranges without any tuples.
+ *
+ * Returns the total number of heap tuples scanned.
+ */
+static double
+_brin_parallel_merge(BrinBuildState *state)
+{
+ BrinTuple *btup;
+ BrinMemTuple *memtuple = NULL;
+ Size tuplen;
+ BlockNumber prevblkno = InvalidBlockNumber;
+ MemoryContext rangeCxt,
+ oldCxt;
+ double reltuples;
+
+ /* wait for workers to scan table and produce partial results */
+ reltuples = _brin_parallel_heapscan(state);
/* do the actual sort in the leader */
tuplesort_performsort(state->bs_sortstate);
@@ -2569,7 +2646,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
while ((btup = tuplesort_getbrintuple(state->bs_sortstate, &tuplen, true)) != NULL)
{
/* Ranges should be multiples of pages_per_range for the index. */
- Assert(btup->bt_blkno % brinshared->pagesPerRange == 0);
+ Assert(btup->bt_blkno % state->bs_leader->brinshared->pagesPerRange == 0);
/*
* Do we need to union summaries for the same page range?
@@ -2665,20 +2742,7 @@ _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
MemoryContextSwitchTo(oldCxt);
MemoryContextDelete(rangeCxt);
- /*
- * Next, accumulate WAL usage. (This must wait for the workers to finish,
- * or we might get incomplete data.)
- */
- for (i = 0; i < brinleader->pcxt->nworkers_launched; i++)
- InstrAccumParallelQuery(&brinleader->bufferusage[i], &brinleader->walusage[i]);
-
-cleanup:
-
- /* Free last reference to MVCC snapshot, if one was used */
- if (IsMVCCSnapshot(brinleader->snapshot))
- UnregisterSnapshot(brinleader->snapshot);
- DestroyParallelContext(brinleader->pcxt);
- ExitParallelMode();
+ return reltuples;
}
/*
--
2.44.0
On 4/15/24 20:35, Tomas Vondra wrote:
On 4/15/24 10:18, Tomas Vondra wrote:
...
I'll try a bit more to make this work without the temp table.
Considering the earlier discussion in e2933a6e1, I think making the
table TEMP is the best fix, so I'll do that. Thanks for remembering that
change, Alexander!
D'oh! I pushed this fix to stabilize the test earlier today, but I just
realized it unfortunately makes the test useless. The idea of the test
was to build BRIN indexes with/without parallelism, and check that the
indexes are exactly the same.
The instability comes from deletes, which I added to get "empty" ranges
in the table, which may not be cleaned up in time for the CREATE INDEX
commands, depending on what else is happening. A TEMPORARY table does
not have this issue (as observed in e2933a6e1), but there's the minor
problem that plan_create_index_workers() does this:
/*
* Determine if it's safe to proceed.
*
* Currently, parallel workers can't access the leader's temporary
* tables. Furthermore, any index predicate or index expressions must
* be parallel safe.
*/
if (heap->rd_rel->relpersistence == RELPERSISTENCE_TEMP ||
!is_parallel_safe(root, (Node *) RelationGetIndexExpressions(index)) ||
!is_parallel_safe(root, (Node *) RelationGetIndexPredicate(index)))
{
parallel_workers = 0;
goto done;
}
That is, no parallel index builds on temporary tables. Which means the
test is not actually testing anything :-( Much more stable, but not very
useful for finding issues.
I think the best way to stabilize the test is to just not delete the
rows. That means we won't have any "empty" ranges (runs of pages without
any tuples).
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On 4/16/24 22:33, Tomas Vondra wrote:
On 4/15/24 20:35, Tomas Vondra wrote:
On 4/15/24 10:18, Tomas Vondra wrote:
...
That is, no parallel index builds on temporary tables. Which means the
test is not actually testing anything :-( Much more stable, but not very
useful for finding issues.I think the best way to stabilize the test is to just not delete the
rows. That means we won't have any "empty" ranges (runs of pages without
any tuples).
I just pushed a revert and a patch to stabilize the test in a different
way - Matthias mentioned to me off-list that DELETE is not the only way
to generate empty ranges in a BRIN index, because partial indexes have
the same effect. After playing with that a bit, that seems to work fine
(works with parallel builds, not affected by cleanup), so done that way.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On 4/15/24 20:35, Tomas Vondra wrote:
...
Attached is the cleanup I thought about doing earlier in this patch [1]
to make the code more like btree. The diff might make it seem like a big
change, but it really just moves the merge code into a separate function
and makes it use using the conditional variable. I still believe the old
code is correct, but this seems like an improvement so plan to push this
soon and resolve the open item.
I've now pushed this cleanup patch, after rewording the commit message a
little bit, etc. I believe this resolves the open item tracking this, so
I've moved it to the "resolved" part.
regards
--
Tomas Vondra
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
On 13.04.24 23:04, Tomas Vondra wrote:
While preparing a differential code coverage report between 16 and HEAD, one
thing that stands out is the parallel brin build code. Neither on
coverage.postgresql.org nor locally is that code reached during our tests.Thanks for pointing this out, it's definitely something that I need to
improve (admittedly, should have been part of the patch). I'll also look
into eliminating the difference between BTREE and BRIN parallel builds,
mentioned in my last message in this thread.Here's a couple patches adding a test for the parallel CREATE INDEX with
BRIN. The actual test is 0003/0004 - I added the test to pageinspect,
because that allows cross-checking the index to one built without
parallelism, which I think is better than just doing CREATE INDEX
without properly testing it produces correct results.
These pageinspect tests added a new use of the md5() function. We got
rid of those in the tests for PG17. You could write the test case with
something like
SELECT (CASE WHEN (mod(i,231) = 0) OR (i BETWEEN 3500 AND 4000) THEN NULL ELSE i END),
- (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END),
+ (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE encode(sha256(i::text::bytea), 'hex') END),
(CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3850 AND 4500) THEN NULL ELSE (i/100) + mod(i,8) END)
But this changes the test output slightly and I'm not sure if this gives
you the data distribution that you need for you test. Could your check
this please?
On 8/15/24 15:48, Peter Eisentraut wrote:
On 13.04.24 23:04, Tomas Vondra wrote:
While preparing a differential code coverage report between 16 and
HEAD, one
thing that stands out is the parallel brin build code. Neither on
coverage.postgresql.org nor locally is that code reached during our
tests.Thanks for pointing this out, it's definitely something that I need to
improve (admittedly, should have been part of the patch). I'll also look
into eliminating the difference between BTREE and BRIN parallel builds,
mentioned in my last message in this thread.Here's a couple patches adding a test for the parallel CREATE INDEX with
BRIN. The actual test is 0003/0004 - I added the test to pageinspect,
because that allows cross-checking the index to one built without
parallelism, which I think is better than just doing CREATE INDEX
without properly testing it produces correct results.These pageinspect tests added a new use of the md5() function. We got
rid of those in the tests for PG17. You could write the test case with
something likeSELECT (CASE WHEN (mod(i,231) = 0) OR (i BETWEEN 3500 AND 4000) THEN NULL ELSE i END), - (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END), + (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE encode(sha256(i::text::bytea), 'hex') END), (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3850 AND 4500) THEN NULL ELSE (i/100) + mod(i,8) END)But this changes the test output slightly and I'm not sure if this gives
you the data distribution that you need for you test. Could your check
this please?
I think this is fine. The output only changes because sha256 produces
longer values than md5, so that the summaries are longer the index gets
a page longer. AFAIK that has no impact on the test.
regards
--
Tomas Vondra
On 16.08.24 11:22, Tomas Vondra wrote:
These pageinspect tests added a new use of the md5() function. We got
rid of those in the tests for PG17. You could write the test case with
something likeSELECT (CASE WHEN (mod(i,231) = 0) OR (i BETWEEN 3500 AND 4000) THEN NULL ELSE i END), - (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE md5(i::text) END), + (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3750 AND 4250) THEN NULL ELSE encode(sha256(i::text::bytea), 'hex') END), (CASE WHEN (mod(i,233) = 0) OR (i BETWEEN 3850 AND 4500) THEN NULL ELSE (i/100) + mod(i,8) END)But this changes the test output slightly and I'm not sure if this gives
you the data distribution that you need for you test. Could your check
this please?I think this is fine. The output only changes because sha256 produces
longer values than md5, so that the summaries are longer the index gets
a page longer. AFAIK that has no impact on the test.
Ok, I have committed that. Thanks for checking.