3 * Implementation of BRIN indexes for Postgres
5 * See src/backend/access/brin/README for details.
7 * Portions Copyright (c) 1996-2025, PostgreSQL Global Development Group
8 * Portions Copyright (c) 1994, Regents of the University of California
11 * src/backend/access/brin/brin.c
14 * * ScalarArrayOpExpr (amsearcharray -> SK_SEARCHARRAY)
39#include "utils/fmgrprotos.h"
46/* Magic numbers for parallel state sharing */
47 #define PARALLEL_KEY_BRIN_SHARED UINT64CONST(0xB000000000000001)
48 #define PARALLEL_KEY_TUPLESORT UINT64CONST(0xB000000000000002)
49 #define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xB000000000000003)
50 #define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xB000000000000004)
51 #define PARALLEL_KEY_BUFFER_USAGE UINT64CONST(0xB000000000000005)
54 * Status for index builds performed in parallel. This is allocated in a
55 * dynamic shared memory segment.
60 * These fields are not modified during the build. They primarily exist
61 * for the benefit of worker processes that need to create state
62 * corresponding to that used by the leader.
70 /* Query ID, for report in worker processes */
74 * workersdonecv is used to monitor the progress of workers. All parallel
75 * participants must indicate that they are done before leader can use
76 * results built by the workers (and before leader can write the data into
82 * mutex protects all fields before heapdesc.
84 * These fields contain status information of interest to BRIN index
85 * builds that must work just the same when an index is built in parallel.
90 * Mutable state that is maintained by workers, and reported back to
91 * leader at end of the scans.
93 * nparticipantsdone is number of worker processes finished.
95 * reltuples is the total number of input heap tuples.
97 * indtuples is the total number of tuples that made it into the index.
104 * ParallelTableScanDescData data follows. Can't directly embed here, as
105 * implementations of the parallel table scan desc interface might need
106 * stronger alignment.
111 * Return pointer to a BrinShared's parallel table scan.
113 * c.f. shm_toc_allocate as to why BUFFERALIGN is used, rather than just
116 #define ParallelTableScanFromBrinShared(shared) \
117 (ParallelTableScanDesc) ((char *) (shared) + BUFFERALIGN(sizeof(BrinShared)))
120 * Status for leader in parallel index build.
124 /* parallel context itself */
128 * nparticipanttuplesorts is the exact number of worker processes
129 * successfully launched, plus one leader process if it participates as a
130 * worker (only DISABLE_LEADER_PARTICIPATION builds avoid leader
131 * participating as a worker).
136 * Leader process convenience pointers to shared state (leader avoids TOC
139 * brinshared is the shared state for entire build. sharedsort is the
140 * shared, tuplesort-managed state passed to each process tuplesort.
141 * snapshot is the snapshot used by the scan iff an MVCC snapshot is
152 * We use a BrinBuildState during initial construction of a BRIN index.
153 * The running state is kept in a BrinMemTuple.
173 * bs_leader is only present when a parallel index build is performed, and
174 * only in the leader process. (Actually, only the leader process has a
181 * The sortstate is used by workers (including the leader). It has to be
182 * part of the build state, because that's the only thing passed to the
183 * build callback etc.
189 * We use a BrinInsertState to capture running state spanning multiple
190 * brininsert invocations, within the same command.
200 * Struct used as "opaque" during index scans
209 #define BRIN_ALL_BLOCKRANGES InvalidBlockNumber
218 bool include_partial,
double *numSummarized,
double *numExisting);
230/* parallel index builds */
232 bool isconcurrent,
int request);
246 * BRIN handler function: return IndexAmRoutine with access method parameters
311 * Initialize a BrinInsertState to maintain state to be used across multiple
312 * tuple inserts, within the same command.
332 * A tuple in the heap is being inserted. To keep a brin index up to date,
333 * we need to obtain the relevant index tuple and compare its stored values
334 * with those of the new tuple. If the tuple values are not consistent with
335 * the summary tuple, we need to update the index tuple.
337 * If autosummarization is enabled, check if we need to summarize the previous
340 * If the range is not currently summarized (i.e. the revmap returns NULL for
341 * it), there's nothing to do for this tuple.
362 * If first time through in this statement, initialize the insert state
363 * that we keep for all the inserts in the command.
373 * origHeapBlk is the block number where the insertion occurred. heapBlk
374 * is the first block in the corresponding page range.
377 heapBlk = (origHeapBlk / pagesPerRange) * pagesPerRange;
381 bool need_insert =
false;
389 * If auto-summarization is enabled and we just inserted the first
390 * tuple into the first block of a new non-first page range, request a
391 * summarization run of the previous range.
395 heapBlk == origHeapBlk &&
413 (
errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
414 errmsg(
"request for BRIN range summarization for index \"%s\" page %u was not recorded",
425 /* if range is unsummarized, there's nothing to do */
429 /* First time through in this brininsert call? */
445 * The tuple is consistent with the new values, so there's nothing
461 * Make a copy of the old tuple, so that we can compare it after
462 * re-acquiring the lock.
468 * Before releasing the lock, check if we can attempt a same-page
469 * update. Another process could insert a tuple concurrently in
470 * the same page though, so downstream we must be prepared to cope
471 * if this turns out to not be possible after all.
478 * Try to update the tuple. If this doesn't work for whatever
479 * reason, we need to restart from the top; the revmap might be
480 * pointing at a different tuple for this block now, so we need to
481 * recompute to ensure both our new heap tuple and the other
482 * inserter's are covered by the combined tuple. It might be that
483 * we don't need to update at all.
486 buf, off, origtup, origsz, newtup, newsz,
489 /* no luck; start over */
509 * Callback to clean up the BrinInsertState once all tuple inserts are done.
516 /* bail out if cache not initialized */
520 /* do this first to avoid dangling pointer if we fail partway through */
524 * Clean up the revmap. Note that the brinDesc has already been cleaned up
525 * as part of its own memory context.
532 * Initialize state for a BRIN index scan.
534 * We read the metapage here to determine the pages-per-range number that this
535 * index was built with. Note that since this cannot be changed while we're
536 * holding lock on index, it's not necessary to recompute it during brinrescan.
555 * Execute the index scan.
557 * This works by reading index TIDs from the revmap, and obtaining the index
558 * tuples pointed to by them; the summary values in the index tuples are
559 * compared to the scan keys. We return into the TID bitmap all the pages in
560 * ranges corresponding to index tuples that match the scan keys.
562 * If a TID from the revmap is read as InvalidTID, we know that range is
563 * unsummarized. Pages in those ranges need to be returned regardless of scan
577 int64 totalpages = 0;
599 * We need to know the size of the table so that we know how long to
600 * iterate on the revmap.
608 * Make room for the consistent support procedures of indexed columns. We
609 * don't look them up here; we do that lazily the first time we see a scan
610 * key reference each of them. We rely on zeroing fn_oid to InvalidOid.
615 * Make room for per-attribute lists of scan keys that we'll pass to the
616 * consistent support procedure. We don't know which attributes have scan
617 * keys, so we allocate space for all attributes. That may use more memory
618 * but it's probably cheaper than determining which attributes are used.
620 * We keep null and regular keys separate, so that we can pass just the
621 * regular keys to the consistent function easily.
623 * To reduce the allocation overhead, we allocate one big chunk and then
624 * carve it into smaller arrays ourselves. All the pieces have exactly the
625 * same lifetime, so that's OK.
627 * XXX The widest index can have 32 attributes, so the amount of wasted
628 * memory is negligible. We could invent a more compact approach (with
629 * just space for used attributes) but that would make the matching more
630 * complex so it's not a good trade-off.
652 nnullkeys = (
int *) ptr;
666 /* zero the number of keys */
670 /* Preprocess the scan keys - split them into per-attribute arrays. */
671 for (
int keyno = 0; keyno < scan->
numberOfKeys; keyno++)
677 * The collation of the scan key must match the collation used in the
678 * index column (but only if the search is not IS NULL/ IS NOT NULL).
679 * Otherwise we shouldn't be using this index ...
682 (
key->sk_collation ==
684 keyattno - 1)->attcollation));
687 * First time we see this index attribute, so init as needed.
689 * This is a bit of an overkill - we don't know how many scan keys are
690 * there for this attribute, so we simply allocate the largest number
691 * possible (as if all keys were for this attribute). This may waste a
692 * bit of memory, but we only expect small number of scan keys in
693 * general, so this should be negligible, and repeated repalloc calls
694 * are not free either.
696 if (consistentFn[keyattno - 1].fn_oid ==
InvalidOid)
700 /* First time we see this attribute, so no key/null keys. */
701 Assert(nkeys[keyattno - 1] == 0);
702 Assert(nnullkeys[keyattno - 1] == 0);
710 /* Add key to the proper per-attribute array. */
713 nullkeys[keyattno - 1][nnullkeys[keyattno - 1]] =
key;
714 nnullkeys[keyattno - 1]++;
718 keys[keyattno - 1][nkeys[keyattno - 1]] =
key;
719 nkeys[keyattno - 1]++;
723 /* allocate an initial in-memory tuple, out of the per-range memcxt */
727 * Setup and use a per-range memory context, which is reset every time we
728 * loop below. This avoids having to free the tuples within the loop.
736 * Now scan the revmap. We start by querying for heap page 0,
737 * incrementing by the number of pages per range; this gives us a full
743 bool gottuple =
false;
762 * For page ranges with no indexed tuple, we must return the whole
763 * range; otherwise, compare it to the scan keys.
775 * Placeholder tuples are always returned, regardless of the
776 * values stored in them.
785 * Compare scan keys with summary values stored for the range.
786 * If scan keys are matched, the page range must be added to
787 * the bitmap. We initially assume the range needs to be
788 * added; in particular this serves the case where there are
799 * skip attributes without any scan keys (both regular and
802 if (nkeys[attno - 1] == 0 && nnullkeys[attno - 1] == 0)
808 * If the BRIN tuple indicates that this range is empty,
809 * we can skip it: there's nothing to match. We don't
810 * need to examine the next columns.
819 * First check if there are any IS [NOT] NULL scan keys,
820 * and if we're violating them. In that case we can
821 * terminate early, without invoking the support function.
823 * As there may be more keys, we can only determine
824 * mismatch within this loop.
828 nnullkeys[attno - 1]))
831 * If any of the IS [NOT] NULL keys failed, the page
832 * range as a whole can't pass. So terminate the loop.
839 * So either there are no IS [NOT] NULL keys, or all
840 * passed. If there are no regular scan keys, we're done -
841 * the page range matches. If there are regular keys, but
842 * the page range is marked as 'all nulls' it can't
843 * possibly pass (we're assuming the operators are
847 /* No regular scan keys - page range as a whole passes. */
848 if (!nkeys[attno - 1])
851 Assert((nkeys[attno - 1] > 0) &&
854 /* If it is all nulls, it cannot possibly be consistent. */
862 * Collation from the first key (has to be the same for
863 * all keys for the same attribute).
868 * Check whether the scan key is consistent with the page
869 * range values; if so, have the pages in the range added
870 * to the output bitmap.
872 * The opclass may or may not support processing of
873 * multiple scan keys. We can determine that based on the
874 * number of arguments - functions with extra parameter
875 * (number of scan keys) do support this, otherwise we
876 * have to simply pass the scan keys one by one.
878 if (consistentFn[attno - 1].fn_nargs >= 4)
880 /* Check all keys at once */
892 * Check keys one by one
894 * When there are multiple scan keys, failure to meet
895 * the criteria for a single one of them is enough to
896 * discard the range as a whole, so break out of the
897 * loop as soon as a false return value is obtained.
901 for (keyno = 0; keyno < nkeys[attno - 1]; keyno++)
904 keys[attno - 1][keyno]->sk_collation,
915 * If we found a scan key eliminating the range, no need
916 * to check additional ones.
924 /* add the pages in the range to the output bitmap, if needed */
929 for (pageno = heapBlk;
948 * XXX We have an approximation of the number of *pages* that our scan
949 * returns, but we don't have a precise idea of the number of heap tuples
952 return totalpages * 10;
956 * Re-initialize state for a BRIN index scan
960 ScanKey orderbys,
int norderbys)
963 * Other index AMs preprocess the scan keys at this point, or sometime
964 * early during the scan; this lets them optimize by removing redundant
965 * keys, or doing early returns when they are impossible to satisfy; see
966 * _bt_preprocess_keys for an example. Something like that could be added
975 * Close down a BRIN index scan
988 * Per-heap-tuple callback for table_index_build_scan.
990 * Note we don't worry about the page range at the end of the table here; it is
991 * present in the build state struct after we're called the last time, but not
992 * inserted into the index. Caller must ensure to do so, if appropriate.
1008 * If we're in a block that belongs to a future range, summarize what
1009 * we've got and start afresh. Note the scan might have skipped many
1010 * pages, if they were devoid of live tuples; make sure to insert index
1011 * tuples for those too.
1013 while (thisblock >
state->bs_currRangeStart +
state->bs_pagesPerRange - 1)
1017 "brinbuildCallback: completed a range: %u--%u",
1018 state->bs_currRangeStart,
1019 state->bs_currRangeStart +
state->bs_pagesPerRange));
1021 /* create the index tuple and insert it */
1024 /* set state to correspond to the next range */
1025 state->bs_currRangeStart +=
state->bs_pagesPerRange;
1027 /* re-initialize state for it */
1031 /* Accumulate the current tuple into the running state */
1037 * Per-heap-tuple callback for table_index_build_scan with parallelism.
1039 * A version of the callback used by parallel index builds. The main difference
1040 * is that instead of writing the BRIN tuples into the index, we write them
1041 * into a shared tuplesort, and leave the insertion up to the leader (which may
1042 * reorder them a bit etc.). The callback also does not generate empty ranges,
1043 * those will be added by the leader when merging results from workers.
1059 * If we're in a block that belongs to a different range, summarize what
1060 * we've got and start afresh. Note the scan might have skipped many
1061 * pages, if they were devoid of live tuples; we do not create empty BRIN
1062 * ranges here - the leader is responsible for filling them in.
1064 * Unlike serial builds, parallel index builds allow synchronized seqscans
1065 * (because that's what parallel scans do). This means the block may wrap
1066 * around to the beginning of the relation, so the condition needs to
1067 * check for both future and past ranges.
1069 if ((thisblock < state->bs_currRangeStart) ||
1070 (thisblock >
state->bs_currRangeStart +
state->bs_pagesPerRange - 1))
1074 "brinbuildCallbackParallel: completed a range: %u--%u",
1075 state->bs_currRangeStart,
1076 state->bs_currRangeStart +
state->bs_pagesPerRange));
1078 /* create the index tuple and write it into the tuplesort */
1082 * Set state to correspond to the next range (for this block).
1084 * This skips ranges that are either empty (and so we don't get any
1085 * tuples to summarize), or processed by other workers. We can't
1086 * differentiate those cases here easily, so we leave it up to the
1087 * leader to fill empty ranges where needed.
1089 state->bs_currRangeStart
1090 =
state->bs_pagesPerRange * (thisblock /
state->bs_pagesPerRange);
1092 /* re-initialize state for it */
1096 /* Accumulate the current tuple into the running state */
1102 * brinbuild() -- build a new BRIN index.
1116 * We expect to be called exactly once for any index relation.
1119 elog(
ERROR,
"index \"%s\" already contains data",
1123 * Critical section not required, because on error the creation of the
1124 * whole relation will be rolled back.
1157 * Initialize our state, including the deformed tuple state.
1164 * Attempt to launch parallel worker scan when required
1166 * XXX plan_create_index_workers makes the number of workers dependent on
1167 * maintenance_work_mem, requiring 32MB for each worker. That makes sense
1168 * for btree, but not for BRIN, which can do with much less memory. So
1169 * maybe make that somehow less strict, optionally?
1176 * If parallel build requested and at least one worker process was
1177 * successfully launched, set up coordination state, wait for workers to
1178 * complete. Then read all tuples from the shared tuplesort and insert
1179 * them into the index.
1181 * In serial mode, simply scan the table and build the index one index
1184 if (
state->bs_leader)
1191 state->bs_leader->nparticipanttuplesorts;
1195 * Begin leader tuplesort.
1197 * In cases where parallelism is involved, the leader receives the
1198 * same share of maintenance_work_mem as a serial sort (it is
1199 * generally treated in the same way as a serial sort once we return).
1200 * Parallel worker Tuplesortstates will have received only a fraction
1201 * of maintenance_work_mem, though.
1203 * We rely on the lifetime of the Leader Tuplesortstate almost not
1204 * overlapping with any worker Tuplesortstate's lifetime. There may
1205 * be some small overlap, but that's okay because we rely on leader
1206 * Tuplesortstate only allocating a small, fixed amount of memory
1207 * here. When its tuplesort_performsort() is called (by our caller),
1208 * and significant amounts of memory are likely to be used, all
1209 * workers must have already freed almost all memory held by their
1210 * Tuplesortstates (they are about to go away completely, too). The
1211 * overall effect is that maintenance_work_mem always represents an
1212 * absolute high watermark on the amount of memory used by a CREATE
1213 * INDEX operation, regardless of the use of parallelism or any other
1216 state->bs_sortstate =
1220 /* scan the relation and merge per-worker results */
1225 else /* no parallel index build */
1228 * Now scan the relation. No syncscan allowed here because we want
1229 * the heap blocks in physical order (we want to produce the ranges
1230 * starting from block 0, and the callback also relies on this to not
1231 * generate summary for the same range twice).
1237 * process the final batch
1239 * XXX Note this does not update state->bs_currRangeStart, i.e. it
1240 * stays set to the last range added to the index. This is OK, because
1241 * that's what brin_fill_empty_ranges expects.
1246 * Backfill the final ranges with empty data.
1248 * This saves us from doing what amounts to full table scans when the
1249 * index with a predicate like WHERE (nonnull_column IS NULL), or
1250 * other very selective predicates.
1253 state->bs_currRangeStart,
1254 state->bs_maxRangeStart);
1257 /* release resources */
1258 idxtuples =
state->bs_numtuples;
1278 /* An empty BRIN index has a metapage only. */
1282 /* Initialize and xlog metabuffer. */
1295 * Since there are no per-heap-tuple index tuples in BRIN indexes,
1296 * there's not a lot we can do here.
1298 * XXX we could mark item tuples as "dirty" (when a minimum or maximum heap
1299 * tuple is deleted), meaning the need to re-run summarization on the affected
1300 * range. Would need to add an extra flag in brintuples for that.
1306 /* allocate stats if first time through, else re-use existing struct */
1314 * This routine is in charge of "vacuuming" a BRIN index: we just summarize
1315 * ranges that are currently unsummarized.
1322 /* No-op in ANALYZE ONLY mode */
1329 /* rest of stats is initialized by zeroing */
1345 * reloptions processor for BRIN indexes
1362 * SQL-callable function to scan through an index and summarize all ranges
1363 * that are not currently summarized.
1376 * SQL-callable function to summarize the indicated page range, if not already
1377 * summarized. If the second argument is BRIN_ALL_BLOCKRANGES, all
1378 * unsummarized ranges are summarized.
1390 int save_sec_context;
1392 double numSummarized = 0;
1396 (
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
1397 errmsg(
"recovery is in progress"),
1398 errhint(
"BRIN control functions cannot be executed during recovery.")));
1402 (
errcode(ERRCODE_NUMERIC_VALUE_OUT_OF_RANGE),
1403 errmsg(
"block number out of range: %" PRId64, heapBlk64)));
1407 * We must lock table before index to avoid deadlocks. However, if the
1408 * passed indexoid isn't an index then IndexGetRelation() will fail.
1409 * Rather than emitting a not-very-helpful error message, postpone
1410 * complaining, expecting that the is-it-an-index test below will fail.
1418 * Autovacuum calls us. For its benefit, switch to the table owner's
1419 * userid, so that any index functions are run as that user. Also
1420 * lock down security-restricted operations and arrange to make GUC
1421 * variable changes local to this command. This is harmless, albeit
1422 * unnecessary, when called from SQL, because we fail shortly if the
1423 * user does not own the index.
1434 /* Set these just to suppress "uninitialized variable" warnings */
1436 save_sec_context = -1;
1437 save_nestlevel = -1;
1442 /* Must be a BRIN index */
1443 if (indexRel->
rd_rel->relkind != RELKIND_INDEX ||
1444 indexRel->
rd_rel->relam != BRIN_AM_OID)
1446 (
errcode(ERRCODE_WRONG_OBJECT_TYPE),
1447 errmsg(
"\"%s\" is not a BRIN index",
1450 /* User must own the index (comparable to privileges needed for VACUUM) */
1451 if (heapRel != NULL && !
object_ownercheck(RelationRelationId, indexoid, save_userid))
1456 * Since we did the IndexGetRelation call above without any lock, it's
1457 * barely possible that a race against an index drop/recreation could have
1458 * netted us the wrong table. Recheck.
1463 errmsg(
"could not open parent table of index \"%s\"",
1466 /* see gin_clean_pending_list() */
1467 if (indexRel->
rd_index->indisvalid)
1468 brinsummarize(indexRel, heapRel, heapBlk,
true, &numSummarized, NULL);
1471 (
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
1472 errmsg(
"index \"%s\" is not valid",
1475 /* Roll back any GUC changes executed by index functions */
1478 /* Restore userid and security context */
1488 * SQL-callable interface to mark a range as no longer summarized
1503 (
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
1504 errmsg(
"recovery is in progress"),
1505 errhint(
"BRIN control functions cannot be executed during recovery.")));
1509 (
errcode(ERRCODE_NUMERIC_VALUE_OUT_OF_RANGE),
1510 errmsg(
"block number out of range: %" PRId64,
1515 * We must lock table before index to avoid deadlocks. However, if the
1516 * passed indexoid isn't an index then IndexGetRelation() will fail.
1517 * Rather than emitting a not-very-helpful error message, postpone
1518 * complaining, expecting that the is-it-an-index test below will fail.
1520 * Unlike brin_summarize_range(), autovacuum never calls this. Hence, we
1521 * don't switch userid.
1531 /* Must be a BRIN index */
1532 if (indexRel->
rd_rel->relkind != RELKIND_INDEX ||
1533 indexRel->
rd_rel->relam != BRIN_AM_OID)
1535 (
errcode(ERRCODE_WRONG_OBJECT_TYPE),
1536 errmsg(
"\"%s\" is not a BRIN index",
1539 /* User must own the index (comparable to privileges needed for VACUUM) */
1545 * Since we did the IndexGetRelation call above without any lock, it's
1546 * barely possible that a race against an index drop/recreation could have
1547 * netted us the wrong table. Recheck.
1552 errmsg(
"could not open parent table of index \"%s\"",
1555 /* see gin_clean_pending_list() */
1556 if (indexRel->
rd_index->indisvalid)
1558 /* the revmap does the hard work */
1567 (
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
1568 errmsg(
"index \"%s\" is not valid",
1578 * Build a BrinDesc used to create or scan a BRIN index
1586 int totalstored = 0;
1599 * Obtain BrinOpcInfo for each indexed column. While at it, accumulate
1600 * the number of columns stored, since the number is opclass-defined.
1603 for (keyno = 0; keyno < tupdesc->
natts; keyno++)
1615 /* Allocate our result struct and fill it in */
1616 totalsize = offsetof(
BrinDesc, bd_info) +
1619 bdesc =
palloc(totalsize);
1626 for (keyno = 0; keyno < tupdesc->
natts; keyno++)
1627 bdesc->
bd_info[keyno] = opcinfo[keyno];
1638 /* make sure the tupdesc is still valid */
1640 /* no need for retail pfree */
1645 * Fetch index's statistical data into *stats
1666 * Initialize a BrinBuildState appropriate to create tuples on the given index.
1677 state->bs_irel = idxRel;
1678 state->bs_numtuples = 0;
1679 state->bs_reltuples = 0;
1681 state->bs_pagesPerRange = pagesPerRange;
1682 state->bs_currRangeStart = 0;
1683 state->bs_rmAccess = revmap;
1686 state->bs_leader = NULL;
1687 state->bs_worker_id = 0;
1688 state->bs_sortstate = NULL;
1690 state->bs_emptyTuple = NULL;
1691 state->bs_emptyTupleLen = 0;
1693 /* Remember the memory context to use for an empty tuple, if needed. */
1695 state->bs_emptyTuple = NULL;
1696 state->bs_emptyTupleLen = 0;
1699 * Calculate the start of the last page range. Page numbers are 0-based,
1700 * so to calculate the index we need to subtract one. The integer division
1701 * gives us the index of the page range.
1704 lastRange = ((tablePages - 1) / pagesPerRange) * pagesPerRange;
1706 /* Now calculate the start of the next range. */
1707 state->bs_maxRangeStart = lastRange +
state->bs_pagesPerRange;
1713 * Release resources associated with a BrinBuildState.
1719 * Release the last index buffer used. We might as well ensure that
1720 * whatever free space remains in that page is available in FSM, too.
1742 * On the given BRIN index, summarize the heap page range that corresponds
1743 * to the heap block number given.
1745 * This routine can run in parallel with insertions into the heap. To avoid
1746 * missing those values from the summary tuple, we first insert a placeholder
1747 * index tuple into the index, then execute the heap scan; transactions
1748 * concurrent with the scan update the placeholder tuple. After the scan, we
1749 * union the placeholder tuple with the one computed by this routine. The
1750 * update of the index value happens in a loop, so that if somebody updates
1751 * the placeholder tuple after we read it, we detect the case and try again.
1752 * This ensures that the concurrently inserted tuples are not lost.
1754 * A further corner case is this routine being asked to summarize the partial
1755 * range at the end of the table. heapNumBlocks is the (possibly outdated)
1756 * table size; if we notice that the requested range lies beyond that size,
1757 * we re-compute the table size after inserting the placeholder tuple, to
1758 * avoid missing pages that were appended recently.
1771 * Insert the placeholder tuple
1776 state->bs_rmAccess, &phbuf,
1777 heapBlk, phtup, phsz);
1780 * Compute range end. We hold ShareUpdateExclusive lock on table, so it
1781 * cannot shrink concurrently (but it can grow).
1784 if (heapBlk +
state->bs_pagesPerRange > heapNumBlks)
1787 * If we're asked to scan what we believe to be the final range on the
1788 * table (i.e. a range that might be partial) we need to recompute our
1789 * idea of what the latest page is after inserting the placeholder
1790 * tuple. Anyone that grows the table later will update the
1791 * placeholder tuple, so it doesn't matter that we won't scan these
1792 * pages ourselves. Careful: the table might have been extended
1793 * beyond the current range, so clamp our result.
1795 * Fortunately, this should occur infrequently.
1798 state->bs_pagesPerRange);
1802 /* Easy case: range is known to be complete */
1803 scanNumBlks =
state->bs_pagesPerRange;
1807 * Execute the partial heap scan covering the heap blocks in the specified
1808 * page range, summarizing the heap tuples in it. This scan stops just
1809 * short of brinbuildCallback creating the new index entry.
1811 * Note that it is critical we use the "any visible" mode of
1812 * table_index_build_range_scan here: otherwise, we would miss tuples
1813 * inserted by transactions that are still in progress, among other corner
1816 state->bs_currRangeStart = heapBlk;
1818 heapBlk, scanNumBlks,
1822 * Now we update the values obtained by the scan with the placeholder
1823 * tuple. We do this in a loop which only terminates if we're able to
1824 * update the placeholder tuple successfully; if we are not, this means
1825 * somebody else modified the placeholder tuple after we read it.
1837 * Update the summary tuple and try to update.
1840 heapBlk,
state->bs_dtuple, &newsize);
1844 state->bs_rmAccess, heapBlk, phbuf, offset,
1845 phtup, phsz, newtup, newsize, samepage);
1849 /* If the update succeeded, we're done. */
1854 * If the update didn't work, it might be because somebody updated the
1855 * placeholder tuple concurrently. Extract the new version, union it
1856 * with the values we have from the scan, and start over. (There are
1857 * other reasons for the update to fail, but it's simple to treat them
1862 /* the placeholder tuple must exist */
1864 elog(
ERROR,
"missing placeholder tuple");
1868 /* merge it into the tuple from the heap scan */
1876 * Summarize page ranges that are not already summarized. If pageRange is
1877 * BRIN_ALL_BLOCKRANGES then the whole table is scanned; otherwise, only the
1878 * page range containing the given heap page number is scanned.
1879 * If include_partial is true, then the partial range at the end of the table
1880 * is summarized, otherwise not.
1882 * For each new index tuple inserted, *numSummarized (if not NULL) is
1883 * incremented; for each existing tuple, *numExisting (if not NULL) is
1888 bool include_partial,
double *numSummarized,
double *numExisting)
1900 /* determine range of pages to process */
1906 startBlk = (pageRange / pagesPerRange) * pagesPerRange;
1907 heapNumBlocks =
Min(heapNumBlocks, startBlk + pagesPerRange);
1909 if (startBlk > heapNumBlocks)
1911 /* Nothing to do if start point is beyond end of table */
1917 * Scan the revmap to find unsummarized items.
1920 for (; startBlk < heapNumBlocks; startBlk += pagesPerRange)
1926 * Unless requested to summarize even a partial range, go away now if
1927 * we think the next range is partial. Caller would pass true when it
1928 * is typically run once bulk data loading is done
1929 * (brin_summarize_new_values), and false when it is typically the
1930 * result of arbitrarily-scheduled maintenance command (vacuuming).
1932 if (!include_partial &&
1933 (startBlk + pagesPerRange > heapNumBlocks))
1942 /* no revmap entry for this heap range. Summarize it. */
1945 /* first time through */
1954 /* and re-initialize state for the next range */
1958 *numSummarized += 1.0;
1963 *numExisting += 1.0;
1971 /* free resources */
1981 * Given a deformed tuple in the build state, convert it into the on-disk
1982 * format and insert it into the index, making the revmap point to it.
1991 state->bs_dtuple, &size);
1993 &
state->bs_currentInsertBuf,
state->bs_currRangeStart,
1995 state->bs_numtuples++;
2001 * Given a deformed tuple in the build state, convert it into the on-disk
2002 * format and write it to a (shared) tuplesort (the leader will insert it
2003 * into the index later).
2011 /* don't insert empty tuples in parallel build */
2012 if (
state->bs_dtuple->bt_empty_range)
2016 state->bs_dtuple, &size);
2018 /* write the BRIN tuple to the tuplesort */
2021 state->bs_numtuples++;
2027 * Given two deformed tuples, adjust the first one so that it's consistent
2028 * with the summary values in both.
2038 /* Use our own memory context to avoid retail pfree */
2047 * Check if the ranges are empty.
2049 * If at least one of them is empty, we don't need to call per-key union
2050 * functions at all. If "b" is empty, we just use "a" as the result (it
2051 * might be empty fine, but that's fine). If "a" is empty but "b" is not,
2052 * we use "b" as the result (but we have to copy the data into "a" first).
2054 * Only when both ranges are non-empty, we actually do the per-key merge.
2057 /* If "b" is empty - ignore it and just use "a" (even if it's empty etc.). */
2060 /* skip the per-key merge */
2066 * Now we know "b" is not empty. If "a" is empty, then "b" is the result.
2067 * But we need to copy the data from "b" to "a" first, because that's how
2068 * we pass result out.
2070 * We have to copy all the global/per-key flags etc. too.
2072 if (
a->bt_empty_range)
2084 /* If "b" has no data, we're done. */
2095 /* "a" started empty, but "b" was not empty, so remember that */
2096 a->bt_empty_range =
false;
2098 /* skip the per-key merge */
2103 /* Now we know neither range is empty. */
2113 /* Does the "b" summary represent any NULL values? */
2116 /* Adjust "hasnulls". */
2120 /* If there are no values in B, there's nothing left to do. */
2125 * Adjust "allnulls". If A doesn't have values, just copy the
2126 * values from B into A, and we're done. We cannot run the
2127 * operators in this case, because values in A might contain
2128 * garbage. Note we already established that B contains values.
2130 * Also adjust "hasnulls" in order not to forget the summary
2131 * represents NULL values. This is not redundant with the earlier
2132 * update, because that only happens when allnulls=false.
2165 * Do a complete scan of the index during VACUUM.
2167 * This routine scans the complete index looking for uncataloged index pages,
2168 * i.e. those that might have been lost due to a crash after index extension
2178 * Scan the index in physical order, and clean up any possible mess in
2182 for (blkno = 0; blkno < nblocks; blkno++)
2197 * Update all upper pages in the index's FSM, as well. This ensures not
2198 * only that we propagate leaf-page FSM updates made by brin_page_cleanup,
2199 * but also that any pre-existing damage or out-of-dateness is repaired.
2210 /* If the range starts empty, we're certainly going to modify it. */
2214 * Compare the key values of the new tuple to the stored index values; our
2215 * deformed tuple will get updated if the new tuple doesn't fit the
2216 * original range (note this means we can't break out of the loop early).
2217 * Make a note of whether this happens, so that we know to insert the
2218 * modified tuple later.
2230 * Does the range have actual NULL values? Either of the flags can be
2231 * set, but we ignore the state before adding first row.
2233 * We have to remember this, because we'll modify the flags and we
2234 * need to know if the range started as empty.
2240 * If the value we're adding is NULL, handle it locally. Otherwise
2241 * call the BRIN_PROCNUM_ADDVALUE procedure.
2246 * If the new value is null, we record that we saw it if it's the
2247 * first one; otherwise, there's nothing to do.
2266 /* if that returned true, we need to insert the updated tuple */
2270 * If the range was had actual NULL values (i.e. did not start empty),
2271 * make sure we don't forget about the NULL values. Either the
2272 * allnulls flag is still set to true, or (if the opclass cleared it)
2273 * we need to set hasnulls=true.
2275 * XXX This can only happen when the opclass modified the tuple, so
2276 * the modified flag should be set.
2286 * After updating summaries for all the keys, mark it as not empty.
2288 * If we're actually changing the flag value (i.e. tuple started as
2289 * empty), we should have modified the tuple. So we should not see empty
2290 * range that was not modified.
2304 * First check if there are any IS [NOT] NULL scan keys, and if we're
2307 for (keyno = 0; keyno < nnullkeys; keyno++)
2313 /* Handle only IS NULL/IS NOT NULL tests */
2319 /* IS NULL scan key, but range has no NULLs */
2326 * For IS NOT NULL, we can only skip ranges that are known to have
2335 * Neither IS NULL nor IS NOT NULL was used; assume all indexable
2336 * operators are strict and thus return false with NULL value in
2347 * Create parallel context, and launch workers for leader.
2349 * buildstate argument should be initialized (with the exception of the
2350 * tuplesort states, which may later be created based on shared
2351 * state initially set up here).
2353 * isconcurrent indicates if operation is CREATE INDEX CONCURRENTLY.
2355 * request is the target number of parallel worker processes to launch.
2357 * Sets buildstate's BrinLeader, which caller must use to shut down parallel
2358 * mode by passing it to _brin_end_parallel() at the very end of its index
2359 * build. If not even a single worker process can be launched, this is
2360 * never set, and caller should proceed with a serial index build.
2364 bool isconcurrent,
int request)
2367 int scantuplesortstates;
2376 bool leaderparticipates =
true;
2379#ifdef DISABLE_LEADER_PARTICIPATION
2380 leaderparticipates =
false;
2384 * Enter parallel mode, and create context for parallel build of brin
2392 scantuplesortstates = leaderparticipates ? request + 1 : request;
2395 * Prepare for scan of the base relation. In a normal index build, we use
2396 * SnapshotAny because we must retrieve all tuples and do our own time
2397 * qual checks (because we have to index RECENTLY_DEAD tuples). In a
2398 * concurrent build, we take a regular MVCC snapshot and index whatever's
2399 * live according to that.
2407 * Estimate size for our own PARALLEL_KEY_BRIN_SHARED workspace.
2417 * Estimate space for WalUsage and BufferUsage -- PARALLEL_KEY_WAL_USAGE
2418 * and PARALLEL_KEY_BUFFER_USAGE.
2420 * If there are no extensions loaded that care, we could skip this. We
2421 * have no way of knowing whether anyone's looking at pgWalUsage or
2422 * pgBufferUsage, so do it unconditionally.
2431 /* Finally, estimate PARALLEL_KEY_QUERY_TEXT space */
2439 querylen = 0;
/* keep compiler quiet */
2441 /* Everyone's had a chance to ask for space, so now create the DSM */
2444 /* If no DSM segment was available, back out (do serial build) */
2445 if (pcxt->
seg == NULL)
2454 /* Store shared build state, for which we reserved space */
2456 /* Initialize immutable state */
2466 /* Initialize mutable state */
2476 * Store shared tuplesort-private state, for which we reserved space.
2477 * Then, initialize opaque state using tuplesort routine.
2484 * Store shared tuplesort-private state, for which we reserved space.
2485 * Then, initialize opaque state using tuplesort routine.
2490 /* Store query string for workers */
2501 * Allocate space for each worker's WalUsage and BufferUsage; no need to
2511 /* Launch workers, saving status for leader/caller */
2513 brinleader->
pcxt = pcxt;
2515 if (leaderparticipates)
2523 /* If no workers were successfully launched, back out (do serial build) */
2530 /* Save leader state now that it's clear build will be parallel */
2533 /* Join heap scan ourselves */
2534 if (leaderparticipates)
2538 * Caller needs to wait for all launched workers when we return. Make
2539 * sure that the failure-to-start case will not hang forever.
2545 * Shut down workers, destroy parallel context, and end parallel mode.
2552 /* Shutdown worker processes */
2556 * Next, accumulate WAL usage. (This must wait for the workers to finish,
2557 * or we might get incomplete data.)
2562 /* Free last reference to MVCC snapshot, if one was used */
2570 * Within leader, wait for end of heap scan.
2572 * When called, parallel heap scan started by _brin_begin_parallel() will
2573 * already be underway within worker processes (when leader participates
2574 * as a worker, we should end up here just as workers are finishing).
2576 * Returns the total number of heap tuples scanned.
2582 int nparticipanttuplesorts;
2584 nparticipanttuplesorts =
state->bs_leader->nparticipanttuplesorts;
2590 /* copy the data into leader state */
2600 WAIT_EVENT_PARALLEL_CREATE_INDEX_SCAN);
2605 return state->bs_reltuples;
2609 * Within leader, wait for end of heap scan and merge per-worker results.
2611 * After waiting for all workers to finish, merge the per-worker results into
2612 * the complete index. The results from each worker are sorted by block number
2613 * (start of the page range). While combining the per-worker results we merge
2614 * summaries for the same page range, and also fill-in empty summaries for
2615 * ranges without any tuples.
2617 * Returns the total number of heap tuples scanned.
2630 /* wait for workers to scan table and produce partial results */
2633 /* do the actual sort in the leader */
2637 * Initialize BrinMemTuple we'll use to union summaries from workers (in
2638 * case they happened to produce parts of the same page range).
2643 * Create a memory context we'll reset to combine results for a single
2644 * page range (received from the workers). We don't expect huge number of
2645 * overlaps under regular circumstances, because for large tables the
2646 * chunk size is likely larger than the BRIN page range), but it can
2647 * happen, and the union functions may do all kinds of stuff. So we better
2648 * reset the context once in a while.
2656 * Read the BRIN tuples from the shared tuplesort, sorted by block number.
2657 * That probably gives us an index that is cheaper to scan, thanks to
2658 * mostly getting data from the same index page as before.
2662 /* Ranges should be multiples of pages_per_range for the index. */
2666 * Do we need to union summaries for the same page range?
2668 * If this is the first brin tuple we read, then just deform it into
2669 * the memtuple, and continue with the next one from tuplesort. We
2670 * however may need to insert empty summaries into the index.
2672 * If it's the same block as the last we saw, we simply union the brin
2673 * tuple into it, and we're done - we don't even need to insert empty
2674 * ranges, because that was done earlier when we saw the first brin
2675 * tuple (for this range).
2677 * Finally, if it's not the first brin tuple, and it's not the same
2678 * page range, we need to do the insert and then deform the tuple into
2679 * the memtuple. Then we'll insert empty ranges before the new brin
2684 /* First brin tuples, just deform into memtuple. */
2687 /* continue to insert empty pages before thisblock */
2692 * Not the first brin tuple, but same page range as the previous
2693 * one, so we can merge it into the memtuple.
2704 * We got brin tuple for a different page range, so form a brin
2705 * tuple from the memtuple, insert it, and re-init the memtuple
2706 * from the new brin tuple.
2715 * Reset the per-output-range context. This frees all the memory
2716 * possibly allocated by the union functions, and also the BRIN
2717 * tuple we just formed and inserted.
2723 /* continue to insert empty pages before thisblock */
2726 /* Fill empty ranges for all ranges missing in the tuplesort. */
2734 /* Fill the BRIN tuple for the last page range with data. */
2749 /* Fill empty ranges at the end, for all ranges missing in the tuplesort. */
2753 * Switch back to the original memory context, and destroy the one we
2754 * created to isolate the union_tuple calls.
2763 * Returns size of shared memory required to store state for a parallel
2764 * brin index build based on the snapshot its parallel scan will use.
2769 /* c.f. shm_toc_allocate as to why BUFFERALIGN is used */
2775 * Within leader, participate as a parallel worker.
2784 * Might as well use reliable figure when doling out maintenance_work_mem
2785 * (when requested number of workers were not launched, this will be
2786 * somewhat higher than it is for other workers).
2790 /* Perform work common to all participants */
2796 * Perform a worker's portion of a parallel sort.
2798 * This generates a tuplesort for the worker portion of the table.
2800 * sortmem is the amount of working memory to use within each worker,
2803 * When this returns, workers are done, and need only release resources.
2816 /* Initialize local tuplesort coordination state */
2822 /* Begin "partial" tuplesort */
2826 /* Join parallel scan */
2836 /* insert the last item */
2839 /* sort the BRIN ranges built by this worker */
2842 state->bs_reltuples += reltuples;
2845 * Done. Record ambuild statistics.
2860 * Perform work within a launched parallel process.
2878 * The only possible status flag that can be set to the parallel worker is
2884 /* Set debug_query_string for individual workers first */
2888 /* Report the query string from leader */
2891 /* Look up brin shared state */
2894 /* Open relations using lock modes known to be obtained by index.c */
2906 /* Track query ID */
2909 /* Open relations within worker */
2917 /* Look up shared state private to tuplesort.c */
2921 /* Prepare to track buffer usage during parallel execution */
2925 * Might as well use reliable figure when doling out maintenance_work_mem
2926 * (when requested number of workers were not launched, this will be
2927 * somewhat higher than it is for other workers).
2932 heapRel, indexRel, sortmem,
false);
2934 /* Report WAL/buffer usage during parallel execution */
2945 * brin_build_empty_tuple
2946 * Maybe initialize a BRIN tuple representing empty range.
2948 * Returns a BRIN tuple representing an empty page range starting at the
2949 * specified block number. The empty tuple is initialized only once, when it's
2950 * needed for the first time, stored in the memory context bs_context to ensure
2951 * proper life span, and reused on following calls. All empty tuples are
2952 * exactly the same except for the bt_blkno field, which is set to the value
2953 * in blkno parameter.
2958 /* First time an empty tuple is requested? If yes, initialize it. */
2959 if (
state->bs_emptyTuple == NULL)
2964 /* Allocate the tuple in context for the whole index build. */
2968 &
state->bs_emptyTupleLen);
2974 /* If we already have an empty tuple, just update the block. */
2975 state->bs_emptyTuple->bt_blkno = blkno;
2980 * brin_fill_empty_ranges
2981 * Add BRIN index tuples representing empty page ranges.
2983 * prevRange/nextRange determine for which page ranges to add empty summaries.
2984 * Both boundaries are exclusive, i.e. only ranges starting at blkno for which
2985 * (prevRange < blkno < nextRange) will be added to the index.
2987 * If prevRange is InvalidBlockNumber, this means there was no previous page
2988 * range (i.e. the first empty range to add is for blkno=0).
2990 * The empty tuple is built only once, and then reused for all future calls.
2999 * If we already summarized some ranges, we need to start with the next
3000 * one. Otherwise start from the first range of the table.
3004 /* Generate empty ranges until we hit the next non-empty range. */
3005 while (blkno < nextRange)
3007 /* Did we already build the empty tuple? If not, do it now. */
3011 &
state->bs_currentInsertBuf,
3012 blkno,
state->bs_emptyTuple,
state->bs_emptyTupleLen);
3014 /* try next page range */
3015 blkno +=
state->bs_pagesPerRange;
void aclcheck_error(AclResult aclerr, ObjectType objtype, const char *objectname)
bool object_ownercheck(Oid classid, Oid objectid, Oid roleid)
static bool validate(Port *port, const char *auth)
bool AutoVacuumRequestWork(AutoVacuumWorkItemType type, Oid relationId, BlockNumber blkno)
void InitializeParallelDSM(ParallelContext *pcxt)
void WaitForParallelWorkersToFinish(ParallelContext *pcxt)
void LaunchParallelWorkers(ParallelContext *pcxt)
void DestroyParallelContext(ParallelContext *pcxt)
ParallelContext * CreateParallelContext(const char *library_name, const char *function_name, int nworkers)
void WaitForParallelWorkersToAttach(ParallelContext *pcxt)
void pgstat_report_query_id(int64 query_id, bool force)
int64 pgstat_get_my_query_id(void)
void pgstat_report_activity(BackendState state, const char *cmd_str)
#define InvalidBlockNumber
static Datum values[MAXATTR]
#define PARALLEL_KEY_BUFFER_USAGE
void brininsertcleanup(Relation index, IndexInfo *indexInfo)
static double _brin_parallel_merge(BrinBuildState *state)
static void brin_vacuum_scan(Relation idxrel, BufferAccessStrategy strategy)
Datum brin_desummarize_range(PG_FUNCTION_ARGS)
void brinrescan(IndexScanDesc scan, ScanKey scankey, int nscankeys, ScanKey orderbys, int norderbys)
static void terminate_brin_buildstate(BrinBuildState *state)
#define PARALLEL_KEY_BRIN_SHARED
Datum brin_summarize_range(PG_FUNCTION_ARGS)
IndexBulkDeleteResult * brinbulkdelete(IndexVacuumInfo *info, IndexBulkDeleteResult *stats, IndexBulkDeleteCallback callback, void *callback_state)
static void form_and_spill_tuple(BrinBuildState *state)
#define BRIN_ALL_BLOCKRANGES
struct BrinShared BrinShared
Datum brin_summarize_new_values(PG_FUNCTION_ARGS)
IndexScanDesc brinbeginscan(Relation r, int nkeys, int norderbys)
bytea * brinoptions(Datum reloptions, bool validate)
int64 bringetbitmap(IndexScanDesc scan, TIDBitmap *tbm)
static void brinsummarize(Relation index, Relation heapRel, BlockNumber pageRange, bool include_partial, double *numSummarized, double *numExisting)
static void form_and_insert_tuple(BrinBuildState *state)
void brinbuildempty(Relation index)
void brin_free_desc(BrinDesc *bdesc)
struct BrinInsertState BrinInsertState
static void union_tuples(BrinDesc *bdesc, BrinMemTuple *a, BrinTuple *b)
static void _brin_parallel_scan_and_build(BrinBuildState *state, BrinShared *brinshared, Sharedsort *sharedsort, Relation heap, Relation index, int sortmem, bool progress)
static BrinBuildState * initialize_brin_buildstate(Relation idxRel, BrinRevmap *revmap, BlockNumber pagesPerRange, BlockNumber tablePages)
static void _brin_begin_parallel(BrinBuildState *buildstate, Relation heap, Relation index, bool isconcurrent, int request)
void brinGetStats(Relation index, BrinStatsData *stats)
static void _brin_leader_participate_as_worker(BrinBuildState *buildstate, Relation heap, Relation index)
static bool add_values_to_range(Relation idxRel, BrinDesc *bdesc, BrinMemTuple *dtup, const Datum *values, const bool *nulls)
static void _brin_end_parallel(BrinLeader *brinleader, BrinBuildState *state)
static Size _brin_parallel_estimate_shared(Relation heap, Snapshot snapshot)
struct BrinBuildState BrinBuildState
static void brin_fill_empty_ranges(BrinBuildState *state, BlockNumber prevRange, BlockNumber nextRange)
IndexBuildResult * brinbuild(Relation heap, Relation index, IndexInfo *indexInfo)
IndexBulkDeleteResult * brinvacuumcleanup(IndexVacuumInfo *info, IndexBulkDeleteResult *stats)
struct BrinLeader BrinLeader
struct BrinOpaque BrinOpaque
static void summarize_range(IndexInfo *indexInfo, BrinBuildState *state, Relation heapRel, BlockNumber heapBlk, BlockNumber heapNumBlks)
#define ParallelTableScanFromBrinShared(shared)
#define PARALLEL_KEY_TUPLESORT
static void brinbuildCallbackParallel(Relation index, ItemPointer tid, Datum *values, bool *isnull, bool tupleIsAlive, void *brstate)
bool brininsert(Relation idxRel, Datum *values, bool *nulls, ItemPointer heaptid, Relation heapRel, IndexUniqueCheck checkUnique, bool indexUnchanged, IndexInfo *indexInfo)
#define PARALLEL_KEY_QUERY_TEXT
Datum brinhandler(PG_FUNCTION_ARGS)
BrinDesc * brin_build_desc(Relation rel)
void _brin_parallel_build_main(dsm_segment *seg, shm_toc *toc)
static void brin_build_empty_tuple(BrinBuildState *state, BlockNumber blkno)
#define PARALLEL_KEY_WAL_USAGE
static double _brin_parallel_heapscan(BrinBuildState *state)
static BrinInsertState * initialize_brin_insertstate(Relation idxRel, IndexInfo *indexInfo)
static void brinbuildCallback(Relation index, ItemPointer tid, Datum *values, bool *isnull, bool tupleIsAlive, void *brstate)
void brinendscan(IndexScanDesc scan)
static bool check_null_keys(BrinValues *bval, ScanKey *nullkeys, int nnullkeys)
#define BrinGetPagesPerRange(relation)
#define BrinGetAutoSummarize(relation)
#define BRIN_LAST_OPTIONAL_PROCNUM
#define BRIN_PROCNUM_UNION
#define BRIN_PROCNUM_OPTIONS
#define BRIN_PROCNUM_OPCINFO
#define BRIN_PROCNUM_CONSISTENT
#define BRIN_PROCNUM_ADDVALUE
#define BRIN_CURRENT_VERSION
#define BRIN_METAPAGE_BLKNO
bool brin_doupdate(Relation idxrel, BlockNumber pagesPerRange, BrinRevmap *revmap, BlockNumber heapBlk, Buffer oldbuf, OffsetNumber oldoff, const BrinTuple *origtup, Size origsz, const BrinTuple *newtup, Size newsz, bool samepage)
void brin_page_cleanup(Relation idxrel, Buffer buf)
OffsetNumber brin_doinsert(Relation idxrel, BlockNumber pagesPerRange, BrinRevmap *revmap, Buffer *buffer, BlockNumber heapBlk, BrinTuple *tup, Size itemsz)
void brin_metapage_init(Page page, BlockNumber pagesPerRange, uint16 version)
bool brin_can_do_samepage_update(Buffer buffer, Size origsz, Size newsz)
bool brinRevmapDesummarizeRange(Relation idxrel, BlockNumber heapBlk)
void brinRevmapTerminate(BrinRevmap *revmap)
BrinRevmap * brinRevmapInitialize(Relation idxrel, BlockNumber *pagesPerRange)
BrinTuple * brinGetTupleForHeapBlock(BrinRevmap *revmap, BlockNumber heapBlk, Buffer *buf, OffsetNumber *off, Size *size, int mode)
BrinTuple * brin_copy_tuple(BrinTuple *tuple, Size len, BrinTuple *dest, Size *destsz)
BrinTuple * brin_form_tuple(BrinDesc *brdesc, BlockNumber blkno, BrinMemTuple *tuple, Size *size)
BrinMemTuple * brin_new_memtuple(BrinDesc *brdesc)
void brin_free_tuple(BrinTuple *tuple)
BrinTuple * brin_form_placeholder_tuple(BrinDesc *brdesc, BlockNumber blkno, Size *size)
BrinMemTuple * brin_memtuple_initialize(BrinMemTuple *dtuple, BrinDesc *brdesc)
BrinMemTuple * brin_deform_tuple(BrinDesc *brdesc, BrinTuple *tuple, BrinMemTuple *dMemtuple)
bool brinvalidate(Oid opclassoid)
#define SizeOfBrinCreateIdx
#define XLOG_BRIN_CREATE_INDEX
#define BufferIsInvalid(buffer)
BlockNumber BufferGetBlockNumber(Buffer buffer)
Buffer ExtendBufferedRel(BufferManagerRelation bmr, ForkNumber forkNum, BufferAccessStrategy strategy, uint32 flags)
void ReleaseBuffer(Buffer buffer)
void UnlockReleaseBuffer(Buffer buffer)
void MarkBufferDirty(Buffer buffer)
void LockBuffer(Buffer buffer, int mode)
Buffer ReadBufferExtended(Relation reln, ForkNumber forkNum, BlockNumber blockNum, ReadBufferMode mode, BufferAccessStrategy strategy)
Buffer ReadBuffer(Relation reln, BlockNumber blockNum)
#define BUFFER_LOCK_UNLOCK
#define BUFFER_LOCK_SHARE
#define RelationGetNumberOfBlocks(reln)
static Page BufferGetPage(Buffer buffer)
static bool BufferIsValid(Buffer bufnum)
Size PageGetFreeSpace(const PageData *page)
static char * PageGetContents(Page page)
static ItemId PageGetItemId(Page page, OffsetNumber offsetNumber)
static void PageSetLSN(Page page, XLogRecPtr lsn)
#define PG_USED_FOR_ASSERTS_ONLY
#define OidIsValid(objectId)
bool ConditionVariableCancelSleep(void)
void ConditionVariableInit(ConditionVariable *cv)
void ConditionVariableSleep(ConditionVariable *cv, uint32 wait_event_info)
void ConditionVariableSignal(ConditionVariable *cv)
Datum datumCopy(Datum value, bool typByVal, int typLen)
int errhint(const char *fmt,...)
int errcode(int sqlerrcode)
int errmsg(const char *fmt,...)
#define ereport(elevel,...)
#define palloc_object(type)
#define palloc_array(type, count)
#define palloc0_array(type, count)
#define palloc0_object(type)
Datum FunctionCall4Coll(FmgrInfo *flinfo, Oid collation, Datum arg1, Datum arg2, Datum arg3, Datum arg4)
Datum FunctionCall3Coll(FmgrInfo *flinfo, Oid collation, Datum arg1, Datum arg2, Datum arg3)
void fmgr_info_copy(FmgrInfo *dstinfo, FmgrInfo *srcinfo, MemoryContext destcxt)
#define DirectFunctionCall2(func, arg1, arg2)
#define PG_GETARG_DATUM(n)
#define PG_GETARG_INT64(n)
#define FunctionCall1(flinfo, arg1)
#define PG_RETURN_INT32(x)
#define PG_RETURN_POINTER(x)
void FreeSpaceMapVacuumRange(Relation rel, BlockNumber start, BlockNumber end)
void FreeSpaceMapVacuum(Relation rel)
void RecordPageWithFreeSpace(Relation rel, BlockNumber heapBlk, Size spaceAvail)
IndexScanDesc RelationGetIndexScan(Relation indexRelation, int nkeys, int norderbys)
bool(* IndexBulkDeleteCallback)(ItemPointer itemptr, void *state)
int NewGUCNestLevel(void)
void RestrictSearchPath(void)
void AtEOXact_GUC(bool isCommit, int nestLevel)
Assert(PointerIsAligned(start, uint64))
Oid IndexGetRelation(Oid indexId, bool missing_ok)
IndexInfo * BuildIndexInfo(Relation index)
FmgrInfo * index_getprocinfo(Relation irel, AttrNumber attnum, uint16 procnum)
void index_close(Relation relation, LOCKMODE lockmode)
Relation index_open(Oid relationId, LOCKMODE lockmode)
void InstrAccumParallelQuery(BufferUsage *bufusage, WalUsage *walusage)
void InstrEndParallelQuery(BufferUsage *bufusage, WalUsage *walusage)
void InstrStartParallelQuery(void)
if(TABLE==NULL||TABLE_index==NULL)
#define ItemIdGetLength(itemId)
static OffsetNumber ItemPointerGetOffsetNumber(const ItemPointerData *pointer)
static BlockNumber ItemPointerGetBlockNumber(const ItemPointerData *pointer)
#define AccessExclusiveLock
#define ShareUpdateExclusiveLock
void MemoryContextReset(MemoryContext context)
void pfree(void *pointer)
void * palloc0(Size size)
MemoryContext CurrentMemoryContext
void MemoryContextDelete(MemoryContext context)
#define AllocSetContextCreate
#define ALLOCSET_DEFAULT_SIZES
#define ALLOCSET_SMALL_SIZES
#define SECURITY_RESTRICTED_OPERATION
#define START_CRIT_SECTION()
#define CHECK_FOR_INTERRUPTS()
#define END_CRIT_SECTION()
void GetUserIdAndSecContext(Oid *userid, int *sec_context)
void SetUserIdAndSecContext(Oid userid, int sec_context)
#define FirstOffsetNumber
static MemoryContext MemoryContextSwitchTo(MemoryContext context)
FormData_pg_attribute * Form_pg_attribute
#define ERRCODE_UNDEFINED_TABLE
#define pgstat_count_index_scan(rel)
const char * debug_query_string
static Datum Int64GetDatum(int64 X)
static bool DatumGetBool(Datum X)
static Datum PointerGetDatum(const void *X)
static Datum BoolGetDatum(bool X)
static Datum ObjectIdGetDatum(Oid X)
static Pointer DatumGetPointer(Datum X)
static Datum Int32GetDatum(int32 X)
static void addrange(struct cvec *cv, chr from, chr to)
#define RelationGetRelid(relation)
#define RelationGetDescr(relation)
#define RelationGetRelationName(relation)
#define RelationNeedsWAL(relation)
void * build_reloptions(Datum reloptions, bool validate, relopt_kind kind, Size relopt_struct_size, const relopt_parse_elt *relopt_elems, int num_relopt_elems)
void brincostestimate(PlannerInfo *root, IndexPath *path, double loop_count, Cost *indexStartupCost, Cost *indexTotalCost, Selectivity *indexSelectivity, double *indexCorrelation, double *indexPages)
void * shm_toc_allocate(shm_toc *toc, Size nbytes)
void shm_toc_insert(shm_toc *toc, uint64 key, void *address)
void * shm_toc_lookup(shm_toc *toc, uint64 key, bool noError)
#define shm_toc_estimate_chunk(e, sz)
#define shm_toc_estimate_keys(e, cnt)
Size add_size(Size s1, Size s2)
Size mul_size(Size s1, Size s2)
Snapshot GetTransactionSnapshot(void)
void UnregisterSnapshot(Snapshot snapshot)
Snapshot RegisterSnapshot(Snapshot snapshot)
#define IsMVCCSnapshot(snapshot)
#define SpinLockInit(lock)
#define SpinLockRelease(lock)
#define SpinLockAcquire(lock)
void relation_close(Relation relation, LOCKMODE lockmode)
BlockNumber bs_maxRangeStart
BlockNumber bs_pagesPerRange
Buffer bs_currentInsertBuf
Tuplesortstate * bs_sortstate
BlockNumber bs_currRangeStart
BrinTuple * bs_emptyTuple
BrinOpcInfo * bd_info[FLEXIBLE_ARRAY_MEMBER]
BrinRevmap * bis_rmAccess
BlockNumber bis_pages_per_range
int nparticipanttuplesorts
BufferUsage * bufferusage
BrinValues bt_columns[FLEXIBLE_ARRAY_MEMBER]
BlockNumber lastRevmapPage
BlockNumber pagesPerRange
BlockNumber bo_pagesPerRange
TypeCacheEntry * oi_typcache[FLEXIBLE_ARRAY_MEMBER]
BlockNumber pagesPerRange
ConditionVariable workersdonecv
BlockNumber revmapNumPages
BlockNumber pagesPerRange
ambuildphasename_function ambuildphasename
ambuildempty_function ambuildempty
amvacuumcleanup_function amvacuumcleanup
amoptions_function amoptions
amestimateparallelscan_function amestimateparallelscan
amrestrpos_function amrestrpos
aminsert_function aminsert
amendscan_function amendscan
amtranslate_strategy_function amtranslatestrategy
amparallelrescan_function amparallelrescan
bool amconsistentordering
amtranslate_cmptype_function amtranslatecmptype
amcostestimate_function amcostestimate
amadjustmembers_function amadjustmembers
amgettuple_function amgettuple
amcanreturn_function amcanreturn
amgetbitmap_function amgetbitmap
amproperty_function amproperty
ambulkdelete_function ambulkdelete
amvalidate_function amvalidate
ammarkpos_function ammarkpos
bool amusemaintenanceworkmem
ambeginscan_function ambeginscan
amrescan_function amrescan
aminitparallelscan_function aminitparallelscan
uint8 amparallelvacuumoptions
aminsertcleanup_function aminsertcleanup
amgettreeheight_function amgettreeheight
bool amconsistentequality
struct ScanKeyData * keyData
struct IndexScanInstrumentation * instrument
BufferAccessStrategy strategy
shm_toc_estimator estimator
BlockNumber pagesPerRange
void table_close(Relation relation, LOCKMODE lockmode)
Relation table_open(Oid relationId, LOCKMODE lockmode)
TableScanDesc table_beginscan_parallel(Relation relation, ParallelTableScanDesc pscan)
Size table_parallelscan_estimate(Relation rel, Snapshot snapshot)
void table_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan, Snapshot snapshot)
static double table_index_build_range_scan(Relation table_rel, Relation index_rel, IndexInfo *index_info, bool allow_sync, bool anyvisible, bool progress, BlockNumber start_blockno, BlockNumber numblocks, IndexBuildCallback callback, void *callback_state, TableScanDesc scan)
static double table_index_build_scan(Relation table_rel, Relation index_rel, IndexInfo *index_info, bool allow_sync, bool progress, IndexBuildCallback callback, void *callback_state, TableScanDesc scan)
static void callback(struct sockaddr *addr, struct sockaddr *mask, void *unused)
void tbm_add_page(TIDBitmap *tbm, BlockNumber pageno)
static FormData_pg_attribute * TupleDescAttr(TupleDesc tupdesc, int i)
void tuplesort_performsort(Tuplesortstate *state)
void tuplesort_initialize_shared(Sharedsort *shared, int nWorkers, dsm_segment *seg)
Size tuplesort_estimate_shared(int nWorkers)
void tuplesort_end(Tuplesortstate *state)
void tuplesort_attach_shared(Sharedsort *shared, dsm_segment *seg)
struct SortCoordinateData * SortCoordinate
Tuplesortstate * tuplesort_begin_index_brin(int workMem, SortCoordinate coordinate, int sortopt)
BrinTuple * tuplesort_getbrintuple(Tuplesortstate *state, Size *len, bool forward)
void tuplesort_putbrintuple(Tuplesortstate *state, BrinTuple *tuple, Size size)
#define VACUUM_OPTION_PARALLEL_CLEANUP
void ExitParallelMode(void)
void EnterParallelMode(void)
bool RecoveryInProgress(void)
XLogRecPtr XLogInsert(RmgrId rmid, uint8 info)
void XLogRegisterData(const void *data, uint32 len)
XLogRecPtr log_newpage_buffer(Buffer buffer, bool page_std)
void XLogRegisterBuffer(uint8 block_id, Buffer buffer, uint8 flags)
void XLogBeginInsert(void)