bool amcaninclude;
/* does AM use maintenance_work_mem? */
bool amusemaintenanceworkmem;
+ /* does AM summarize tuples, with at least all tuples in the block
+ * summarized in one summary */
+ bool amsummarizing;
/* OR of parallel vacuum flags */
uint8 amparallelvacuumoptions;
/* type of data stored in index, or InvalidOid if variable */
null, independently of <structfield>amoptionalkey</structfield>.
</para>
+ <para>
+ The <structfield>amsummarizing</structfield> flag indicates whether the
+ access method summarizes the indexed tuples, with summarizing granularity
+ of at least per block.
+ Access methods that do not point to individual tuples, but to block ranges
+ (like <acronym>BRIN</acronym>), may allow the <acronym>HOT</acronym> optimization
+ to continue. This does not apply to attributes referenced in index
+ predicates, an update of such attribute always disables <acronym>HOT</acronym>.
+ </para>
+
</sect1>
<sect1 id="index-functions">
amroutine->amcanparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
+ amroutine->amsummarizing = true;
amroutine->amparallelvacuumoptions =
VACUUM_OPTION_PARALLEL_CLEANUP;
amroutine->amkeytype = InvalidOid;
amroutine->amcanparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = true;
+ amroutine->amsummarizing = false;
amroutine->amparallelvacuumoptions =
VACUUM_OPTION_PARALLEL_BULKDEL | VACUUM_OPTION_PARALLEL_CLEANUP;
amroutine->amkeytype = InvalidOid;
amroutine->amcanparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
+ amroutine->amsummarizing = false;
amroutine->amparallelvacuumoptions =
VACUUM_OPTION_PARALLEL_BULKDEL | VACUUM_OPTION_PARALLEL_COND_CLEANUP;
amroutine->amkeytype = InvalidOid;
amroutine->amcanparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
+ amroutine->amsummarizing = false;
amroutine->amparallelvacuumoptions =
VACUUM_OPTION_PARALLEL_BULKDEL;
amroutine->amkeytype = INT4OID;
TM_Result
heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
CommandId cid, Snapshot crosscheck, bool wait,
- TM_FailureData *tmfd, LockTupleMode *lockmode)
+ TM_FailureData *tmfd, LockTupleMode *lockmode,
+ TU_UpdateIndexes *update_indexes)
{
TM_Result result;
TransactionId xid = GetCurrentTransactionId();
Bitmapset *hot_attrs;
+ Bitmapset *sum_attrs;
Bitmapset *key_attrs;
Bitmapset *id_attrs;
Bitmapset *interesting_attrs;
bool have_tuple_lock = false;
bool iscombo;
bool use_hot_update = false;
+ bool summarized_update = false;
bool key_intact;
bool all_visible_cleared = false;
bool all_visible_cleared_new = false;
* Note that we get copies of each bitmap, so we need not worry about
* relcache flush happening midway through.
*/
- hot_attrs = RelationGetIndexAttrBitmap(relation, INDEX_ATTR_BITMAP_ALL);
+ hot_attrs = RelationGetIndexAttrBitmap(relation,
+ INDEX_ATTR_BITMAP_HOT_BLOCKING);
+ sum_attrs = RelationGetIndexAttrBitmap(relation,
+ INDEX_ATTR_BITMAP_SUMMARIZED);
key_attrs = RelationGetIndexAttrBitmap(relation, INDEX_ATTR_BITMAP_KEY);
id_attrs = RelationGetIndexAttrBitmap(relation,
INDEX_ATTR_BITMAP_IDENTITY_KEY);
interesting_attrs = NULL;
interesting_attrs = bms_add_members(interesting_attrs, hot_attrs);
+ interesting_attrs = bms_add_members(interesting_attrs, sum_attrs);
interesting_attrs = bms_add_members(interesting_attrs, key_attrs);
interesting_attrs = bms_add_members(interesting_attrs, id_attrs);
UnlockTupleTuplock(relation, &(oldtup.t_self), *lockmode);
if (vmbuffer != InvalidBuffer)
ReleaseBuffer(vmbuffer);
+ *update_indexes = TU_None;
+
bms_free(hot_attrs);
+ bms_free(sum_attrs);
bms_free(key_attrs);
bms_free(id_attrs);
bms_free(modified_attrs);
* changed.
*/
if (!bms_overlap(modified_attrs, hot_attrs))
+ {
use_hot_update = true;
+
+ /*
+ * If none of the columns that are used in hot-blocking indexes
+ * were updated, we can apply HOT, but we do still need to check
+ * if we need to update the summarizing indexes, and update those
+ * indexes if the columns were updated, or we may fail to detect
+ * e.g. value bound changes in BRIN minmax indexes.
+ */
+ if (bms_overlap(modified_attrs, sum_attrs))
+ summarized_update = true;
+ }
}
else
{
heap_freetuple(heaptup);
}
+ /*
+ * If it is a HOT update, the update may still need to update summarized
+ * indexes, lest we fail to update those summaries and get incorrect
+ * results (for example, minmax bounds of the block may change with this
+ * update).
+ */
+ if (use_hot_update)
+ {
+ if (summarized_update)
+ *update_indexes = TU_Summarizing;
+ else
+ *update_indexes = TU_None;
+ }
+ else
+ *update_indexes = TU_All;
+
if (old_key_tuple != NULL && old_key_copied)
heap_freetuple(old_key_tuple);
bms_free(hot_attrs);
+ bms_free(sum_attrs);
bms_free(key_attrs);
bms_free(id_attrs);
bms_free(modified_attrs);
* via ereport().
*/
void
-simple_heap_update(Relation relation, ItemPointer otid, HeapTuple tup)
+simple_heap_update(Relation relation, ItemPointer otid, HeapTuple tup,
+ TU_UpdateIndexes *update_indexes)
{
TM_Result result;
TM_FailureData tmfd;
result = heap_update(relation, otid, tup,
GetCurrentCommandId(true), InvalidSnapshot,
true /* wait for commit */ ,
- &tmfd, &lockmode);
+ &tmfd, &lockmode, update_indexes);
switch (result)
{
case TM_SelfModified:
heapam_tuple_update(Relation relation, ItemPointer otid, TupleTableSlot *slot,
CommandId cid, Snapshot snapshot, Snapshot crosscheck,
bool wait, TM_FailureData *tmfd,
- LockTupleMode *lockmode, bool *update_indexes)
+ LockTupleMode *lockmode, TU_UpdateIndexes *update_indexes)
{
bool shouldFree = true;
HeapTuple tuple = ExecFetchSlotHeapTuple(slot, true, &shouldFree);
tuple->t_tableOid = slot->tts_tableOid;
result = heap_update(relation, otid, tuple, cid, crosscheck, wait,
- tmfd, lockmode);
+ tmfd, lockmode, update_indexes);
ItemPointerCopy(&tuple->t_self, &slot->tts_tid);
/*
* Note: heap_update returns the tid (location) of the new tuple in the
* t_self field.
*
- * If it's a HOT update, we mustn't insert new index entries.
+ * If the update is not HOT, we must update all indexes. If the update
+ * is HOT, it could be that we updated summarized columns, so we either
+ * update only summarized indexes, or none at all.
*/
- *update_indexes = result == TM_Ok && !HeapTupleIsHeapOnly(tuple);
+ if (result != TM_Ok)
+ {
+ Assert(*update_indexes == TU_None);
+ *update_indexes = TU_None;
+ }
+ else if (!HeapTupleIsHeapOnly(tuple))
+ Assert(*update_indexes == TU_All);
+ else
+ Assert((*update_indexes == TU_Summarizing) ||
+ (*update_indexes == TU_None));
if (shouldFree)
pfree(tuple);
amroutine->amcanparallel = true;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
+ amroutine->amsummarizing = false;
amroutine->amparallelvacuumoptions =
VACUUM_OPTION_PARALLEL_BULKDEL | VACUUM_OPTION_PARALLEL_COND_CLEANUP;
amroutine->amkeytype = InvalidOid;
amroutine->amcanparallel = false;
amroutine->amcaninclude = true;
amroutine->amusemaintenanceworkmem = false;
+ amroutine->amsummarizing = false;
amroutine->amparallelvacuumoptions =
VACUUM_OPTION_PARALLEL_BULKDEL | VACUUM_OPTION_PARALLEL_COND_CLEANUP;
amroutine->amkeytype = InvalidOid;
simple_table_tuple_update(Relation rel, ItemPointer otid,
TupleTableSlot *slot,
Snapshot snapshot,
- bool *update_indexes)
+ TU_UpdateIndexes *update_indexes)
{
TM_Result result;
TM_FailureData tmfd;
oldInfo->ii_Unique,
oldInfo->ii_NullsNotDistinct,
false, /* not ready for inserts */
- true);
+ true,
+ indexRelation->rd_indam->amsummarizing);
/*
* Extract the list of column names and the column numbers for the new
indexStruct->indisunique,
indexStruct->indnullsnotdistinct,
indexStruct->indisready,
- false);
+ false,
+ index->rd_indam->amsummarizing);
/* fill in attribute numbers */
for (i = 0; i < numAtts; i++)
indexStruct->indisunique,
indexStruct->indnullsnotdistinct,
indexStruct->indisready,
- false);
+ false,
+ index->rd_indam->amsummarizing);
/* fill in attribute numbers */
for (i = 0; i < numAtts; i++)
* This is effectively a cut-down version of ExecInsertIndexTuples.
*/
static void
-CatalogIndexInsert(CatalogIndexState indstate, HeapTuple heapTuple)
+CatalogIndexInsert(CatalogIndexState indstate, HeapTuple heapTuple,
+ TU_UpdateIndexes updateIndexes)
{
int i;
int numIndexes;
IndexInfo **indexInfoArray;
Datum values[INDEX_MAX_KEYS];
bool isnull[INDEX_MAX_KEYS];
+ bool onlySummarized = (updateIndexes == TU_Summarizing);
/*
* HOT update does not require index inserts. But with asserts enabled we
* table/index.
*/
#ifndef USE_ASSERT_CHECKING
- if (HeapTupleIsHeapOnly(heapTuple))
+ if (HeapTupleIsHeapOnly(heapTuple) && !onlySummarized)
return;
#endif
+ /* When only updating summarized indexes, the tuple has to be HOT. */
+ Assert((!onlySummarized) || HeapTupleIsHeapOnly(heapTuple));
+
/*
* Get information from the state structure. Fall out if nothing to do.
*/
/* see earlier check above */
#ifdef USE_ASSERT_CHECKING
- if (HeapTupleIsHeapOnly(heapTuple))
+ if (HeapTupleIsHeapOnly(heapTuple) && !onlySummarized)
{
Assert(!ReindexIsProcessingIndex(RelationGetRelid(index)));
continue;
}
#endif /* USE_ASSERT_CHECKING */
+ /*
+ * Skip insertions into non-summarizing indexes if we only need
+ * to update summarizing indexes.
+ */
+ if (onlySummarized && !indexInfo->ii_Summarizing)
+ continue;
+
/*
* FormIndexDatum fills in its values and isnull parameters with the
* appropriate values for the column(s) of the index.
simple_heap_insert(heapRel, tup);
- CatalogIndexInsert(indstate, tup);
+ CatalogIndexInsert(indstate, tup, TU_All);
CatalogCloseIndexes(indstate);
}
simple_heap_insert(heapRel, tup);
- CatalogIndexInsert(indstate, tup);
+ CatalogIndexInsert(indstate, tup, TU_All);
}
/*
tuple = ExecFetchSlotHeapTuple(slot[i], true, &should_free);
tuple->t_tableOid = slot[i]->tts_tableOid;
- CatalogIndexInsert(indstate, tuple);
+ CatalogIndexInsert(indstate, tuple, TU_All);
if (should_free)
heap_freetuple(tuple);
CatalogTupleUpdate(Relation heapRel, ItemPointer otid, HeapTuple tup)
{
CatalogIndexState indstate;
+ TU_UpdateIndexes updateIndexes = TU_All;
CatalogTupleCheckConstraints(heapRel, tup);
indstate = CatalogOpenIndexes(heapRel);
- simple_heap_update(heapRel, otid, tup);
+ simple_heap_update(heapRel, otid, tup, &updateIndexes);
- CatalogIndexInsert(indstate, tup);
+ CatalogIndexInsert(indstate, tup, updateIndexes);
CatalogCloseIndexes(indstate);
}
CatalogTupleUpdateWithInfo(Relation heapRel, ItemPointer otid, HeapTuple tup,
CatalogIndexState indstate)
{
+ TU_UpdateIndexes updateIndexes = TU_All;
+
CatalogTupleCheckConstraints(heapRel, tup);
- simple_heap_update(heapRel, otid, tup);
+ simple_heap_update(heapRel, otid, tup, &updateIndexes);
- CatalogIndexInsert(indstate, tup);
+ CatalogIndexInsert(indstate, tup, updateIndexes);
}
/*
recheckIndexes =
ExecInsertIndexTuples(resultRelInfo,
buffer->slots[i], estate, false,
- false, NULL, NIL);
+ false, NULL, NIL, false);
ExecARInsertTriggers(estate, resultRelInfo,
slots[i], recheckIndexes,
cstate->transition_capture);
false,
false,
NULL,
- NIL);
+ NIL,
+ false);
}
/* AFTER ROW INSERT Triggers */
Form_pg_am accessMethodForm;
IndexAmRoutine *amRoutine;
bool amcanorder;
+ bool amsummarizing;
int16 *coloptions;
IndexInfo *indexInfo;
int numberOfAttributes;
ReleaseSysCache(tuple);
amcanorder = amRoutine->amcanorder;
+ amsummarizing = amRoutine->amsummarizing;
/*
* Compute the operator classes, collations, and exclusion operators for
* ii_NumIndexKeyAttrs with same value.
*/
indexInfo = makeIndexInfo(numberOfAttributes, numberOfAttributes,
- accessMethodId, NIL, NIL, false, false, false, false);
+ accessMethodId, NIL, NIL, false, false,
+ false, false, amsummarizing);
typeObjectId = palloc_array(Oid, numberOfAttributes);
collationObjectId = palloc_array(Oid, numberOfAttributes);
classObjectId = palloc_array(Oid, numberOfAttributes);
Form_pg_am accessMethodForm;
IndexAmRoutine *amRoutine;
bool amcanorder;
+ bool amissummarizing;
amoptions_function amoptions;
bool partitioned;
bool safe_index;
amcanorder = amRoutine->amcanorder;
amoptions = amRoutine->amoptions;
+ amissummarizing = amRoutine->amsummarizing;
pfree(amRoutine);
ReleaseSysCache(tuple);
stmt->unique,
stmt->nulls_not_distinct,
!concurrent,
- concurrent);
+ concurrent,
+ amissummarizing);
typeObjectId = palloc_array(Oid, numberOfAttributes);
collationObjectId = palloc_array(Oid, numberOfAttributes);
* into all the relations indexing the result relation
* when a heap tuple is inserted into the result relation.
*
- * When 'update' is true, executor is performing an UPDATE
- * that could not use an optimization like heapam's HOT (in
- * more general terms a call to table_tuple_update() took
- * place and set 'update_indexes' to true). Receiving this
- * hint makes us consider if we should pass down the
- * 'indexUnchanged' hint in turn. That's something that we
- * figure out for each index_insert() call iff 'update' is
- * true. (When 'update' is false we already know not to pass
- * the hint to any index.)
+ * When 'update' is true and 'onlySummarizing' is false,
+ * executor is performing an UPDATE that could not use an
+ * optimization like heapam's HOT (in more general terms a
+ * call to table_tuple_update() took place and set
+ * 'update_indexes' to TUUI_All). Receiving this hint makes
+ * us consider if we should pass down the 'indexUnchanged'
+ * hint in turn. That's something that we figure out for
+ * each index_insert() call iff 'update' is true.
+ * (When 'update' is false we already know not to pass the
+ * hint to any index.)
+ *
+ * If onlySummarizing is set, an equivalent optimization to
+ * HOT has been applied and any updated columns are indexed
+ * only by summarizing indexes (or in more general terms a
+ * call to table_tuple_update() took place and set
+ * 'update_indexes' to TUUI_Summarizing). We can (and must)
+ * therefore only update the indexes that have
+ * 'amsummarizing' = true.
*
* Unique and exclusion constraints are enforced at the same
* time. This returns a list of index OIDs for any unique or
bool update,
bool noDupErr,
bool *specConflict,
- List *arbiterIndexes)
+ List *arbiterIndexes,
+ bool onlySummarizing)
{
ItemPointer tupleid = &slot->tts_tid;
List *result = NIL;
if (!indexInfo->ii_ReadyForInserts)
continue;
+ /*
+ * Skip processing of non-summarizing indexes if we only
+ * update summarizing indexes
+ */
+ if (onlySummarizing && !indexInfo->ii_Summarizing)
+ continue;
+
/* Check for partial index */
if (indexInfo->ii_Predicate != NIL)
{
if (resultRelInfo->ri_NumIndices > 0)
recheckIndexes = ExecInsertIndexTuples(resultRelInfo,
slot, estate, false, false,
- NULL, NIL);
+ NULL, NIL, false);
/* AFTER ROW INSERT Triggers */
ExecARInsertTriggers(estate, resultRelInfo, slot,
if (!skip_tuple)
{
List *recheckIndexes = NIL;
- bool update_indexes;
+ TU_UpdateIndexes update_indexes;
/* Compute stored generated columns */
if (rel->rd_att->constr &&
simple_table_tuple_update(rel, tid, slot, estate->es_snapshot,
&update_indexes);
- if (resultRelInfo->ri_NumIndices > 0 && update_indexes)
+ if (resultRelInfo->ri_NumIndices > 0 && (update_indexes != TU_None))
recheckIndexes = ExecInsertIndexTuples(resultRelInfo,
slot, estate, true, false,
- NULL, NIL);
+ NULL, NIL,
+ (update_indexes == TU_Summarizing));
/* AFTER ROW UPDATE Triggers */
ExecARUpdateTriggers(estate, resultRelInfo,
typedef struct UpdateContext
{
bool updated; /* did UPDATE actually occur? */
- bool updateIndexes; /* index update required? */
bool crossPartUpdate; /* was it a cross-partition update? */
+ TU_UpdateIndexes updateIndexes; /* Which index updates are required? */
/*
* Lock mode to acquire on the latest tuple version before performing
recheckIndexes = ExecInsertIndexTuples(resultRelInfo,
slot, estate, false, true,
&specConflict,
- arbiterIndexes);
+ arbiterIndexes,
+ false);
/* adjust the tuple's state accordingly */
table_tuple_complete_speculative(resultRelationDesc, slot,
if (resultRelInfo->ri_NumIndices > 0)
recheckIndexes = ExecInsertIndexTuples(resultRelInfo,
slot, estate, false,
- false, NULL, NIL);
+ false, NULL, NIL,
+ false);
}
}
List *recheckIndexes = NIL;
/* insert index entries for tuple if necessary */
- if (resultRelInfo->ri_NumIndices > 0 && updateCxt->updateIndexes)
+ if (resultRelInfo->ri_NumIndices > 0 && (updateCxt->updateIndexes != TU_None))
recheckIndexes = ExecInsertIndexTuples(resultRelInfo,
slot, context->estate,
true, false,
- NULL, NIL);
+ NULL, NIL,
+ (updateCxt->updateIndexes == TU_Summarizing));
/* AFTER ROW UPDATE Triggers */
ExecARUpdateTriggers(context->estate, resultRelInfo,
*/
IndexInfo *
makeIndexInfo(int numattrs, int numkeyattrs, Oid amoid, List *expressions,
- List *predicates, bool unique, bool nulls_not_distinct, bool isready, bool concurrent)
+ List *predicates, bool unique, bool nulls_not_distinct,
+ bool isready, bool concurrent, bool summarizing)
{
IndexInfo *n = makeNode(IndexInfo);
n->ii_CheckedUnchanged = false;
n->ii_IndexUnchanged = false;
n->ii_Concurrent = concurrent;
+ n->ii_Summarizing = summarizing;
+
+ /* summarizing indexes cannot contain non-key attributes */
+ Assert(!summarizing || (numkeyattrs == numattrs));
/* expressions */
n->ii_Expressions = expressions;
list_free_deep(relation->rd_fkeylist);
list_free(relation->rd_indexlist);
list_free(relation->rd_statlist);
- bms_free(relation->rd_indexattr);
bms_free(relation->rd_keyattr);
bms_free(relation->rd_pkattr);
bms_free(relation->rd_idattr);
+ bms_free(relation->rd_hotblockingattr);
+ bms_free(relation->rd_summarizedattr);
if (relation->rd_pubdesc)
pfree(relation->rd_pubdesc);
if (relation->rd_options)
Bitmapset *
RelationGetIndexAttrBitmap(Relation relation, IndexAttrBitmapKind attrKind)
{
- Bitmapset *indexattrs; /* indexed columns */
Bitmapset *uindexattrs; /* columns in unique indexes */
Bitmapset *pkindexattrs; /* columns in the primary index */
Bitmapset *idindexattrs; /* columns in the replica identity */
+ Bitmapset *hotblockingattrs; /* columns with HOT blocking indexes */
+ Bitmapset *summarizedattrs; /* columns with summarizing indexes */
List *indexoidlist;
List *newindexoidlist;
Oid relpkindex;
MemoryContext oldcxt;
/* Quick exit if we already computed the result. */
- if (relation->rd_indexattr != NULL)
+ if (relation->rd_attrsvalid)
{
switch (attrKind)
{
- case INDEX_ATTR_BITMAP_ALL:
- return bms_copy(relation->rd_indexattr);
case INDEX_ATTR_BITMAP_KEY:
return bms_copy(relation->rd_keyattr);
case INDEX_ATTR_BITMAP_PRIMARY_KEY:
return bms_copy(relation->rd_pkattr);
case INDEX_ATTR_BITMAP_IDENTITY_KEY:
return bms_copy(relation->rd_idattr);
+ case INDEX_ATTR_BITMAP_HOT_BLOCKING:
+ return bms_copy(relation->rd_hotblockingattr);
+ case INDEX_ATTR_BITMAP_SUMMARIZED:
+ return bms_copy(relation->rd_summarizedattr);
default:
elog(ERROR, "unknown attrKind %u", attrKind);
}
* CONCURRENTLY is far enough along that we should ignore the index, it
* won't be returned at all by RelationGetIndexList.
*/
- indexattrs = NULL;
uindexattrs = NULL;
pkindexattrs = NULL;
idindexattrs = NULL;
+ hotblockingattrs = NULL;
+ summarizedattrs = NULL;
foreach(l, indexoidlist)
{
Oid indexOid = lfirst_oid(l);
bool isKey; /* candidate key */
bool isPK; /* primary key */
bool isIDKey; /* replica identity index */
+ Bitmapset **attrs;
indexDesc = index_open(indexOid, AccessShareLock);
/* Is this index the configured (or default) replica identity? */
isIDKey = (indexOid == relreplindex);
+ /*
+ * If the index is summarizing, it doesn't block HOT updates, but we
+ * may still need to update it (if the attributes were modified). So
+ * decide which bitmap we'll update in the following loop.
+ */
+ if (indexDesc->rd_indam->amsummarizing)
+ attrs = &summarizedattrs;
+ else
+ attrs = &hotblockingattrs;
+
/* Collect simple attribute references */
for (i = 0; i < indexDesc->rd_index->indnatts; i++)
{
/*
* Since we have covering indexes with non-key columns, we must
* handle them accurately here. non-key columns must be added into
- * indexattrs, since they are in index, and HOT-update shouldn't
- * miss them. Obviously, non-key columns couldn't be referenced by
+ * hotblockingattrs or summarizedattrs, since they are in index,
+ * and update shouldn't miss them.
+ *
+ * Summarizing indexes do not block HOT, but do need to be updated
+ * when the column value changes, thus require a separate
+ * attribute bitmapset.
+ *
+ * Obviously, non-key columns couldn't be referenced by
* foreign key or identity key. Hence we do not include them into
* uindexattrs, pkindexattrs and idindexattrs bitmaps.
*/
if (attrnum != 0)
{
- indexattrs = bms_add_member(indexattrs,
- attrnum - FirstLowInvalidHeapAttributeNumber);
+ *attrs = bms_add_member(*attrs,
+ attrnum - FirstLowInvalidHeapAttributeNumber);
if (isKey && i < indexDesc->rd_index->indnkeyatts)
uindexattrs = bms_add_member(uindexattrs,
}
/* Collect all attributes used in expressions, too */
- pull_varattnos(indexExpressions, 1, &indexattrs);
+ pull_varattnos(indexExpressions, 1, attrs);
/* Collect all attributes in the index predicate, too */
- pull_varattnos(indexPredicate, 1, &indexattrs);
+ pull_varattnos(indexPredicate, 1, attrs);
index_close(indexDesc, AccessShareLock);
}
bms_free(uindexattrs);
bms_free(pkindexattrs);
bms_free(idindexattrs);
- bms_free(indexattrs);
+ bms_free(hotblockingattrs);
+ bms_free(summarizedattrs);
goto restart;
}
/* Don't leak the old values of these bitmaps, if any */
- bms_free(relation->rd_indexattr);
- relation->rd_indexattr = NULL;
+ relation->rd_attrsvalid = false;
bms_free(relation->rd_keyattr);
relation->rd_keyattr = NULL;
bms_free(relation->rd_pkattr);
relation->rd_pkattr = NULL;
bms_free(relation->rd_idattr);
relation->rd_idattr = NULL;
+ bms_free(relation->rd_hotblockingattr);
+ relation->rd_hotblockingattr = NULL;
+ bms_free(relation->rd_summarizedattr);
+ relation->rd_summarizedattr = NULL;
/*
* Now save copies of the bitmaps in the relcache entry. We intentionally
- * set rd_indexattr last, because that's the one that signals validity of
+ * set rd_attrsvalid last, because that's the one that signals validity of
* the values; if we run out of memory before making that copy, we won't
* leave the relcache entry looking like the other ones are valid but
* empty.
relation->rd_keyattr = bms_copy(uindexattrs);
relation->rd_pkattr = bms_copy(pkindexattrs);
relation->rd_idattr = bms_copy(idindexattrs);
- relation->rd_indexattr = bms_copy(indexattrs);
+ relation->rd_hotblockingattr = bms_copy(hotblockingattrs);
+ relation->rd_summarizedattr = bms_copy(summarizedattrs);
+ relation->rd_attrsvalid = true;
MemoryContextSwitchTo(oldcxt);
/* We return our original working copy for caller to play with */
switch (attrKind)
{
- case INDEX_ATTR_BITMAP_ALL:
- return indexattrs;
case INDEX_ATTR_BITMAP_KEY:
return uindexattrs;
case INDEX_ATTR_BITMAP_PRIMARY_KEY:
return pkindexattrs;
case INDEX_ATTR_BITMAP_IDENTITY_KEY:
return idindexattrs;
+ case INDEX_ATTR_BITMAP_HOT_BLOCKING:
+ return hotblockingattrs;
+ case INDEX_ATTR_BITMAP_SUMMARIZED:
+ return summarizedattrs;
default:
elog(ERROR, "unknown attrKind %u", attrKind);
return NULL;
rel->rd_indexlist = NIL;
rel->rd_pkindex = InvalidOid;
rel->rd_replidindex = InvalidOid;
- rel->rd_indexattr = NULL;
+ rel->rd_attrsvalid = false;
rel->rd_keyattr = NULL;
rel->rd_pkattr = NULL;
rel->rd_idattr = NULL;
bool amcaninclude;
/* does AM use maintenance_work_mem? */
bool amusemaintenanceworkmem;
+ /* does AM store tuple information only at block granularity? */
+ bool amsummarizing;
/* OR of parallel vacuum flags. See vacuum.h for flags. */
uint8 amparallelvacuumoptions;
/* type of data stored in index, or InvalidOid if variable */
extern TM_Result heap_update(Relation relation, ItemPointer otid,
HeapTuple newtup,
CommandId cid, Snapshot crosscheck, bool wait,
- struct TM_FailureData *tmfd, LockTupleMode *lockmode);
+ struct TM_FailureData *tmfd, LockTupleMode *lockmode,
+ TU_UpdateIndexes *update_indexes);
extern TM_Result heap_lock_tuple(Relation relation, HeapTuple tuple,
CommandId cid, LockTupleMode mode, LockWaitPolicy wait_policy,
bool follow_updates,
extern void simple_heap_insert(Relation relation, HeapTuple tup);
extern void simple_heap_delete(Relation relation, ItemPointer tid);
extern void simple_heap_update(Relation relation, ItemPointer otid,
- HeapTuple tup);
+ HeapTuple tup, TU_UpdateIndexes *update_indexes);
extern TransactionId heap_index_delete_tuples(Relation rel,
TM_IndexDeleteOp *delstate);
TM_WouldBlock
} TM_Result;
+/*
+ * Result codes for table_update(..., update_indexes*..).
+ * Used to determine which indexes to update.
+ */
+typedef enum TU_UpdateIndexes
+{
+ /* No indexed columns were updated (incl. TID addressing of tuple) */
+ TU_None,
+
+ /* A non-summarizing indexed column was updated, or the TID has changed */
+ TU_All,
+
+ /* Only summarized columns were updated, TID is unchanged */
+ TU_Summarizing
+} TU_UpdateIndexes;
+
/*
* When table_tuple_update, table_tuple_delete, or table_tuple_lock fail
* because the target tuple is already outdated, they fill in this struct to
bool wait,
TM_FailureData *tmfd,
LockTupleMode *lockmode,
- bool *update_indexes);
+ TU_UpdateIndexes *update_indexes);
/* see table_tuple_lock() for reference about parameters */
TM_Result (*tuple_lock) (Relation rel,
table_tuple_update(Relation rel, ItemPointer otid, TupleTableSlot *slot,
CommandId cid, Snapshot snapshot, Snapshot crosscheck,
bool wait, TM_FailureData *tmfd, LockTupleMode *lockmode,
- bool *update_indexes)
+ TU_UpdateIndexes *update_indexes)
{
return rel->rd_tableam->tuple_update(rel, otid, slot,
cid, snapshot, crosscheck,
Snapshot snapshot);
extern void simple_table_tuple_update(Relation rel, ItemPointer otid,
TupleTableSlot *slot, Snapshot snapshot,
- bool *update_indexes);
+ TU_UpdateIndexes *update_indexes);
/* ----------------------------------------------------------------------------
TupleTableSlot *slot, EState *estate,
bool update,
bool noDupErr,
- bool *specConflict, List *arbiterIndexes);
+ bool *specConflict, List *arbiterIndexes,
+ bool onlySummarizing);
extern bool ExecCheckIndexConstraints(ResultRelInfo *resultRelInfo,
TupleTableSlot *slot,
EState *estate, ItemPointer conflictTid,
* IndexUnchanged aminsert hint, cached for retail inserts
* Concurrent are we doing a concurrent index build?
* BrokenHotChain did we detect any broken HOT chains?
+ * Summarizing is it a summarizing index?
* ParallelWorkers # of workers requested (excludes leader)
* Am Oid of index AM
* AmCache private cache area for index AM
bool ii_IndexUnchanged;
bool ii_Concurrent;
bool ii_BrokenHotChain;
+ bool ii_Summarizing;
int ii_ParallelWorkers;
Oid ii_Am;
void *ii_AmCache;
extern IndexInfo *makeIndexInfo(int numattrs, int numkeyattrs, Oid amoid,
List *expressions, List *predicates,
- bool unique, bool nulls_not_distinct, bool isready, bool concurrent);
+ bool unique, bool nulls_not_distinct,
+ bool isready, bool concurrent,
+ bool summarizing);
extern DefElem *makeDefElem(char *name, Node *arg, int location);
extern DefElem *makeDefElemExtended(char *nameSpace, char *name, Node *arg,
List *rd_statlist; /* list of OIDs of extended stats */
/* data managed by RelationGetIndexAttrBitmap: */
- Bitmapset *rd_indexattr; /* identifies columns used in indexes */
+ bool rd_attrsvalid; /* are bitmaps of attrs valid? */
Bitmapset *rd_keyattr; /* cols that can be ref'd by foreign keys */
Bitmapset *rd_pkattr; /* cols included in primary key */
Bitmapset *rd_idattr; /* included in replica identity index */
+ Bitmapset *rd_hotblockingattr; /* cols blocking HOT update */
+ Bitmapset *rd_summarizedattr; /* cols indexed by summarizing indexes */
PublicationDesc *rd_pubdesc; /* publication descriptor, or NULL */
typedef enum IndexAttrBitmapKind
{
- INDEX_ATTR_BITMAP_ALL,
INDEX_ATTR_BITMAP_KEY,
INDEX_ATTR_BITMAP_PRIMARY_KEY,
- INDEX_ATTR_BITMAP_IDENTITY_KEY
+ INDEX_ATTR_BITMAP_IDENTITY_KEY,
+ INDEX_ATTR_BITMAP_HOT_BLOCKING,
+ INDEX_ATTR_BITMAP_SUMMARIZED
} IndexAttrBitmapKind;
extern Bitmapset *RelationGetIndexAttrBitmap(Relation relation,
amroutine->amcanparallel = false;
amroutine->amcaninclude = false;
amroutine->amusemaintenanceworkmem = false;
+ amroutine->amsummarizing = false;
amroutine->amparallelvacuumoptions = VACUUM_OPTION_NO_PARALLEL;
amroutine->amkeytype = InvalidOid;
t
(1 row)
+-- test BRIN index doesn't block HOT update
+CREATE TABLE brin_hot (
+ id integer PRIMARY KEY,
+ val integer NOT NULL
+) WITH (autovacuum_enabled = off, fillfactor = 70);
+INSERT INTO brin_hot SELECT *, 0 FROM generate_series(1, 235);
+CREATE INDEX val_brin ON brin_hot using brin(val);
+CREATE FUNCTION wait_for_hot_stats() RETURNS void AS $$
+DECLARE
+ start_time timestamptz := clock_timestamp();
+ updated bool;
+BEGIN
+ -- we don't want to wait forever; loop will exit after 30 seconds
+ FOR i IN 1 .. 300 LOOP
+ SELECT (pg_stat_get_tuples_hot_updated('brin_hot'::regclass::oid) > 0) INTO updated;
+ EXIT WHEN updated;
+
+ -- wait a little
+ PERFORM pg_sleep_for('100 milliseconds');
+ -- reset stats snapshot so we can test again
+ PERFORM pg_stat_clear_snapshot();
+ END LOOP;
+ -- report time waited in postmaster log (where it won't change test output)
+ RAISE log 'wait_for_hot_stats delayed % seconds',
+ EXTRACT(epoch FROM clock_timestamp() - start_time);
+END
+$$ LANGUAGE plpgsql;
+UPDATE brin_hot SET val = -3 WHERE id = 42;
+-- We can't just call wait_for_hot_stats() at this point, because we only
+-- transmit stats when the session goes idle, and we probably didn't
+-- transmit the last couple of counts yet thanks to the rate-limiting logic
+-- in pgstat_report_stat(). But instead of waiting for the rate limiter's
+-- timeout to elapse, let's just start a new session. The old one will
+-- then send its stats before dying.
+\c -
+SELECT wait_for_hot_stats();
+ wait_for_hot_stats
+--------------------
+
+(1 row)
+
+SELECT pg_stat_get_tuples_hot_updated('brin_hot'::regclass::oid);
+ pg_stat_get_tuples_hot_updated
+--------------------------------
+ 1
+(1 row)
+
+DROP TABLE brin_hot;
+DROP FUNCTION wait_for_hot_stats();
+-- Test handling of index predicates - updating attributes in precicates
+-- should not block HOT when summarizing indexes are involved. We update
+-- a row that was not indexed due to the index predicate, and becomes
+-- indexable - the HOT-updated tuple is forwarded to the BRIN index.
+CREATE TABLE brin_hot_2 (a int, b int);
+INSERT INTO brin_hot_2 VALUES (1, 100);
+CREATE INDEX ON brin_hot_2 USING brin (b) WHERE a = 2;
+UPDATE brin_hot_2 SET a = 2;
+EXPLAIN (COSTS OFF) SELECT * FROM brin_hot_2 WHERE a = 2 AND b = 100;
+ QUERY PLAN
+-----------------------------------
+ Seq Scan on brin_hot_2
+ Filter: ((a = 2) AND (b = 100))
+(2 rows)
+
+SELECT COUNT(*) FROM brin_hot_2 WHERE a = 2 AND b = 100;
+ count
+-------
+ 1
+(1 row)
+
+SET enable_seqscan = off;
+EXPLAIN (COSTS OFF) SELECT * FROM brin_hot_2 WHERE a = 2 AND b = 100;
+ QUERY PLAN
+---------------------------------------------
+ Bitmap Heap Scan on brin_hot_2
+ Recheck Cond: ((b = 100) AND (a = 2))
+ -> Bitmap Index Scan on brin_hot_2_b_idx
+ Index Cond: (b = 100)
+(4 rows)
+
+SELECT COUNT(*) FROM brin_hot_2 WHERE a = 2 AND b = 100;
+ count
+-------
+ 1
+(1 row)
+
+DROP TABLE brin_hot_2;
+-- Test that updates to indexed columns are still propagated to the
+-- BRIN column.
+CREATE TABLE brin_hot_3 (a int, filler text) WITH (fillfactor = 10);
+INSERT INTO brin_hot_3 SELECT 1, repeat(' ', 500) FROM generate_series(1, 20);
+CREATE INDEX ON brin_hot_3 USING brin (a) WITH (pages_per_range = 1);
+UPDATE brin_hot_3 SET a = 2;
+EXPLAIN (COSTS OFF) SELECT * FROM brin_hot_3 WHERE a = 2;
+ QUERY PLAN
+---------------------------------------------
+ Bitmap Heap Scan on brin_hot_3
+ Recheck Cond: (a = 2)
+ -> Bitmap Index Scan on brin_hot_3_a_idx
+ Index Cond: (a = 2)
+(4 rows)
+
+SELECT COUNT(*) FROM brin_hot_3 WHERE a = 2;
+ count
+-------
+ 20
+(1 row)
+
+DROP TABLE brin_hot_3;
+SET enable_seqscan = on;
-- End of Stats Test
FROM pg_stat_io \gset
SELECT :io_stats_post_reset < :io_stats_pre_reset;
+
+-- test BRIN index doesn't block HOT update
+CREATE TABLE brin_hot (
+ id integer PRIMARY KEY,
+ val integer NOT NULL
+) WITH (autovacuum_enabled = off, fillfactor = 70);
+
+INSERT INTO brin_hot SELECT *, 0 FROM generate_series(1, 235);
+CREATE INDEX val_brin ON brin_hot using brin(val);
+
+CREATE FUNCTION wait_for_hot_stats() RETURNS void AS $$
+DECLARE
+ start_time timestamptz := clock_timestamp();
+ updated bool;
+BEGIN
+ -- we don't want to wait forever; loop will exit after 30 seconds
+ FOR i IN 1 .. 300 LOOP
+ SELECT (pg_stat_get_tuples_hot_updated('brin_hot'::regclass::oid) > 0) INTO updated;
+ EXIT WHEN updated;
+
+ -- wait a little
+ PERFORM pg_sleep_for('100 milliseconds');
+ -- reset stats snapshot so we can test again
+ PERFORM pg_stat_clear_snapshot();
+ END LOOP;
+ -- report time waited in postmaster log (where it won't change test output)
+ RAISE log 'wait_for_hot_stats delayed % seconds',
+ EXTRACT(epoch FROM clock_timestamp() - start_time);
+END
+$$ LANGUAGE plpgsql;
+
+UPDATE brin_hot SET val = -3 WHERE id = 42;
+
+-- We can't just call wait_for_hot_stats() at this point, because we only
+-- transmit stats when the session goes idle, and we probably didn't
+-- transmit the last couple of counts yet thanks to the rate-limiting logic
+-- in pgstat_report_stat(). But instead of waiting for the rate limiter's
+-- timeout to elapse, let's just start a new session. The old one will
+-- then send its stats before dying.
+\c -
+
+SELECT wait_for_hot_stats();
+SELECT pg_stat_get_tuples_hot_updated('brin_hot'::regclass::oid);
+
+DROP TABLE brin_hot;
+DROP FUNCTION wait_for_hot_stats();
+
+-- Test handling of index predicates - updating attributes in precicates
+-- should not block HOT when summarizing indexes are involved. We update
+-- a row that was not indexed due to the index predicate, and becomes
+-- indexable - the HOT-updated tuple is forwarded to the BRIN index.
+CREATE TABLE brin_hot_2 (a int, b int);
+INSERT INTO brin_hot_2 VALUES (1, 100);
+CREATE INDEX ON brin_hot_2 USING brin (b) WHERE a = 2;
+
+UPDATE brin_hot_2 SET a = 2;
+
+EXPLAIN (COSTS OFF) SELECT * FROM brin_hot_2 WHERE a = 2 AND b = 100;
+SELECT COUNT(*) FROM brin_hot_2 WHERE a = 2 AND b = 100;
+
+SET enable_seqscan = off;
+
+EXPLAIN (COSTS OFF) SELECT * FROM brin_hot_2 WHERE a = 2 AND b = 100;
+SELECT COUNT(*) FROM brin_hot_2 WHERE a = 2 AND b = 100;
+
+DROP TABLE brin_hot_2;
+
+-- Test that updates to indexed columns are still propagated to the
+-- BRIN column.
+CREATE TABLE brin_hot_3 (a int, filler text) WITH (fillfactor = 10);
+INSERT INTO brin_hot_3 SELECT 1, repeat(' ', 500) FROM generate_series(1, 20);
+CREATE INDEX ON brin_hot_3 USING brin (a) WITH (pages_per_range = 1);
+UPDATE brin_hot_3 SET a = 2;
+
+EXPLAIN (COSTS OFF) SELECT * FROM brin_hot_3 WHERE a = 2;
+SELECT COUNT(*) FROM brin_hot_3 WHERE a = 2;
+
+DROP TABLE brin_hot_3;
+
+SET enable_seqscan = on;
+
-- End of Stats Test