Split ExecStoreTuple into ExecStoreHeapTuple and ExecStoreBufferHeapTuple.

Upcoming changes introduce further types of tuple table slots, in
preparation of making table storage pluggable. New storage methods
will have different representation of tuples, therefore the slot
accessor should refer explicitly to heap tuples.

Instead of just renaming the functions, split it into one function
that accepts heap tuples not residing in buffers, and one accepting
ones in buffers.  Previously one function was used for both, but that
was a bit awkward already, and splitting will allow us to represent
slot types for tuples in buffers and normal memory separately.

This is split out from the patch introducing abstract slots, as this
largely consists out of mechanical changes.

Author: Ashutosh Bapat
Reviewed-By: Andres Freund
Discussion: https://postgr.es/m/20180220224318.gw4oe5jadhpmcdnm@alap3.anarazel.de
This commit is contained in:
Andres Freund 2018-09-25 16:27:48 -07:00
parent bbdfbb9154
commit 29c94e03c7
32 changed files with 173 additions and 146 deletions

View File

@ -1443,10 +1443,9 @@ postgresIterateForeignScan(ForeignScanState *node)
/* /*
* Return the next tuple. * Return the next tuple.
*/ */
ExecStoreTuple(fsstate->tuples[fsstate->next_tuple++], ExecStoreHeapTuple(fsstate->tuples[fsstate->next_tuple++],
slot, slot,
InvalidBuffer, false);
false);
return slot; return slot;
} }
@ -3517,7 +3516,7 @@ store_returning_result(PgFdwModifyState *fmstate,
NULL, NULL,
fmstate->temp_cxt); fmstate->temp_cxt);
/* tuple will be deleted when it is cleared from the slot */ /* tuple will be deleted when it is cleared from the slot */
ExecStoreTuple(newtup, slot, InvalidBuffer, true); ExecStoreHeapTuple(newtup, slot, true);
} }
PG_CATCH(); PG_CATCH();
{ {
@ -3790,7 +3789,7 @@ get_returning_data(ForeignScanState *node)
dmstate->retrieved_attrs, dmstate->retrieved_attrs,
node, node,
dmstate->temp_cxt); dmstate->temp_cxt);
ExecStoreTuple(newtup, slot, InvalidBuffer, false); ExecStoreHeapTuple(newtup, slot, false);
} }
PG_CATCH(); PG_CATCH();
{ {

View File

@ -4502,14 +4502,14 @@ ProjIndexIsUnchanged(Relation relation, HeapTuple oldtup, HeapTuple newtup)
int i; int i;
ResetExprContext(econtext); ResetExprContext(econtext);
ExecStoreTuple(oldtup, slot, InvalidBuffer, false); ExecStoreHeapTuple(oldtup, slot, false);
FormIndexDatum(indexInfo, FormIndexDatum(indexInfo,
slot, slot,
estate, estate,
old_values, old_values,
old_isnull); old_isnull);
ExecStoreTuple(newtup, slot, InvalidBuffer, false); ExecStoreHeapTuple(newtup, slot, false);
FormIndexDatum(indexInfo, FormIndexDatum(indexInfo,
slot, slot,
estate, estate,

View File

@ -2866,7 +2866,7 @@ IndexBuildHeapRangeScan(Relation heapRelation,
MemoryContextReset(econtext->ecxt_per_tuple_memory); MemoryContextReset(econtext->ecxt_per_tuple_memory);
/* Set up for predicate or expression evaluation */ /* Set up for predicate or expression evaluation */
ExecStoreTuple(heapTuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(heapTuple, slot, false);
/* /*
* In a partial index, discard tuples that don't satisfy the * In a partial index, discard tuples that don't satisfy the
@ -3015,7 +3015,7 @@ IndexCheckExclusion(Relation heapRelation,
MemoryContextReset(econtext->ecxt_per_tuple_memory); MemoryContextReset(econtext->ecxt_per_tuple_memory);
/* Set up for predicate or expression evaluation */ /* Set up for predicate or expression evaluation */
ExecStoreTuple(heapTuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(heapTuple, slot, false);
/* /*
* In a partial index, ignore tuples that don't satisfy the predicate. * In a partial index, ignore tuples that don't satisfy the predicate.
@ -3436,7 +3436,7 @@ validate_index_heapscan(Relation heapRelation,
MemoryContextReset(econtext->ecxt_per_tuple_memory); MemoryContextReset(econtext->ecxt_per_tuple_memory);
/* Set up for predicate or expression evaluation */ /* Set up for predicate or expression evaluation */
ExecStoreTuple(heapTuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(heapTuple, slot, false);
/* /*
* In a partial index, discard tuples that don't satisfy the * In a partial index, discard tuples that don't satisfy the

View File

@ -96,7 +96,7 @@ CatalogIndexInsert(CatalogIndexState indstate, HeapTuple heapTuple)
/* Need a slot to hold the tuple being examined */ /* Need a slot to hold the tuple being examined */
slot = MakeSingleTupleTableSlot(RelationGetDescr(heapRelation)); slot = MakeSingleTupleTableSlot(RelationGetDescr(heapRelation));
ExecStoreTuple(heapTuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(heapTuple, slot, false);
/* /*
* for each index, form and insert the index tuple * for each index, form and insert the index tuple

View File

@ -799,7 +799,7 @@ compute_index_stats(Relation onerel, double totalrows,
ResetExprContext(econtext); ResetExprContext(econtext);
/* Set up for predicate or expression evaluation */ /* Set up for predicate or expression evaluation */
ExecStoreTuple(heapTuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(heapTuple, slot, false);
/* If index is partial, check predicate */ /* If index is partial, check predicate */
if (predicate != NULL) if (predicate != NULL)

View File

@ -124,7 +124,7 @@ unique_key_recheck(PG_FUNCTION_ARGS)
*/ */
slot = MakeSingleTupleTableSlot(RelationGetDescr(trigdata->tg_relation)); slot = MakeSingleTupleTableSlot(RelationGetDescr(trigdata->tg_relation));
ExecStoreTuple(new_row, slot, InvalidBuffer, false); ExecStoreHeapTuple(new_row, slot, false);
/* /*
* Typically the index won't have expressions, but if it does we need an * Typically the index won't have expressions, but if it does we need an

View File

@ -2684,7 +2684,7 @@ CopyFrom(CopyState cstate)
/* Place tuple in tuple slot --- but slot shouldn't free it */ /* Place tuple in tuple slot --- but slot shouldn't free it */
slot = myslot; slot = myslot;
ExecStoreTuple(tuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, slot, false);
/* Determine the partition to heap_insert the tuple into */ /* Determine the partition to heap_insert the tuple into */
if (proute) if (proute)
@ -3119,7 +3119,7 @@ CopyFromInsertBatch(CopyState cstate, EState *estate, CommandId mycid,
List *recheckIndexes; List *recheckIndexes;
cstate->cur_lineno = firstBufferedLineNo + i; cstate->cur_lineno = firstBufferedLineNo + i;
ExecStoreTuple(bufferedTuples[i], myslot, InvalidBuffer, false); ExecStoreHeapTuple(bufferedTuples[i], myslot, false);
recheckIndexes = recheckIndexes =
ExecInsertIndexTuples(myslot, &(bufferedTuples[i]->t_self), ExecInsertIndexTuples(myslot, &(bufferedTuples[i]->t_self),
estate, false, NULL, NIL); estate, false, NULL, NIL);

View File

@ -2337,7 +2337,7 @@ ExecuteCallStmt(CallStmt *stmt, ParamListInfo params, bool atomic, DestReceiver
rettupdata.t_tableOid = InvalidOid; rettupdata.t_tableOid = InvalidOid;
rettupdata.t_data = td; rettupdata.t_data = td;
slot = ExecStoreTuple(&rettupdata, tstate->slot, InvalidBuffer, false); slot = ExecStoreHeapTuple(&rettupdata, tstate->slot, false);
tstate->dest->receiveSlot(slot, tstate->dest); tstate->dest->receiveSlot(slot, tstate->dest);
end_tup_output(tstate); end_tup_output(tstate);

View File

@ -4776,7 +4776,7 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
* Process supplied expressions to replace selected columns. * Process supplied expressions to replace selected columns.
* Expression inputs come from the old tuple. * Expression inputs come from the old tuple.
*/ */
ExecStoreTuple(tuple, oldslot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, oldslot, false);
econtext->ecxt_scantuple = oldslot; econtext->ecxt_scantuple = oldslot;
foreach(l, tab->newvals) foreach(l, tab->newvals)
@ -4806,7 +4806,7 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
} }
/* Now check any constraints on the possibly-changed tuple */ /* Now check any constraints on the possibly-changed tuple */
ExecStoreTuple(tuple, newslot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, newslot, false);
econtext->ecxt_scantuple = newslot; econtext->ecxt_scantuple = newslot;
foreach(l, notnull_attrs) foreach(l, notnull_attrs)
@ -8526,7 +8526,7 @@ validateCheckConstraint(Relation rel, HeapTuple constrtup)
while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL) while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
{ {
ExecStoreTuple(tuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, slot, false);
if (!ExecCheck(exprstate, econtext)) if (!ExecCheck(exprstate, econtext))
ereport(ERROR, ereport(ERROR,

View File

@ -2571,7 +2571,7 @@ ExecBRInsertTriggers(EState *estate, ResultRelInfo *relinfo,
if (newslot->tts_tupleDescriptor != tupdesc) if (newslot->tts_tupleDescriptor != tupdesc)
ExecSetSlotDescriptor(newslot, tupdesc); ExecSetSlotDescriptor(newslot, tupdesc);
ExecStoreTuple(newtuple, newslot, InvalidBuffer, false); ExecStoreHeapTuple(newtuple, newslot, false);
slot = newslot; slot = newslot;
} }
return slot; return slot;
@ -2652,7 +2652,7 @@ ExecIRInsertTriggers(EState *estate, ResultRelInfo *relinfo,
if (newslot->tts_tupleDescriptor != tupdesc) if (newslot->tts_tupleDescriptor != tupdesc)
ExecSetSlotDescriptor(newslot, tupdesc); ExecSetSlotDescriptor(newslot, tupdesc);
ExecStoreTuple(newtuple, newslot, InvalidBuffer, false); ExecStoreHeapTuple(newtuple, newslot, false);
slot = newslot; slot = newslot;
} }
return slot; return slot;
@ -3078,7 +3078,7 @@ ExecBRUpdateTriggers(EState *estate, EPQState *epqstate,
if (newslot->tts_tupleDescriptor != tupdesc) if (newslot->tts_tupleDescriptor != tupdesc)
ExecSetSlotDescriptor(newslot, tupdesc); ExecSetSlotDescriptor(newslot, tupdesc);
ExecStoreTuple(newtuple, newslot, InvalidBuffer, false); ExecStoreHeapTuple(newtuple, newslot, false);
slot = newslot; slot = newslot;
} }
return slot; return slot;
@ -3186,7 +3186,7 @@ ExecIRUpdateTriggers(EState *estate, ResultRelInfo *relinfo,
if (newslot->tts_tupleDescriptor != tupdesc) if (newslot->tts_tupleDescriptor != tupdesc)
ExecSetSlotDescriptor(newslot, tupdesc); ExecSetSlotDescriptor(newslot, tupdesc);
ExecStoreTuple(newtuple, newslot, InvalidBuffer, false); ExecStoreHeapTuple(newtuple, newslot, false);
slot = newslot; slot = newslot;
} }
return slot; return slot;
@ -3514,7 +3514,7 @@ TriggerEnabled(EState *estate, ResultRelInfo *relinfo,
oldslot = estate->es_trig_oldtup_slot; oldslot = estate->es_trig_oldtup_slot;
if (oldslot->tts_tupleDescriptor != tupdesc) if (oldslot->tts_tupleDescriptor != tupdesc)
ExecSetSlotDescriptor(oldslot, tupdesc); ExecSetSlotDescriptor(oldslot, tupdesc);
ExecStoreTuple(oldtup, oldslot, InvalidBuffer, false); ExecStoreHeapTuple(oldtup, oldslot, false);
} }
if (HeapTupleIsValid(newtup)) if (HeapTupleIsValid(newtup))
{ {
@ -3528,7 +3528,7 @@ TriggerEnabled(EState *estate, ResultRelInfo *relinfo,
newslot = estate->es_trig_newtup_slot; newslot = estate->es_trig_newtup_slot;
if (newslot->tts_tupleDescriptor != tupdesc) if (newslot->tts_tupleDescriptor != tupdesc)
ExecSetSlotDescriptor(newslot, tupdesc); ExecSetSlotDescriptor(newslot, tupdesc);
ExecStoreTuple(newtup, newslot, InvalidBuffer, false); ExecStoreHeapTuple(newtup, newslot, false);
} }
/* /*

View File

@ -750,7 +750,7 @@ retry:
* Extract the index column values and isnull flags from the existing * Extract the index column values and isnull flags from the existing
* tuple. * tuple.
*/ */
ExecStoreTuple(tup, existing_slot, InvalidBuffer, false); ExecStoreHeapTuple(tup, existing_slot, false);
FormIndexDatum(indexInfo, existing_slot, estate, FormIndexDatum(indexInfo, existing_slot, estate,
existing_values, existing_isnull); existing_values, existing_isnull);

View File

@ -1957,7 +1957,7 @@ ExecPartitionCheckEmitError(ResultRelInfo *resultRelInfo,
{ {
tuple = do_convert_tuple(tuple, map); tuple = do_convert_tuple(tuple, map);
ExecSetSlotDescriptor(slot, tupdesc); ExecSetSlotDescriptor(slot, tupdesc);
ExecStoreTuple(tuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, slot, false);
} }
} }
@ -2036,7 +2036,7 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
{ {
tuple = do_convert_tuple(tuple, map); tuple = do_convert_tuple(tuple, map);
ExecSetSlotDescriptor(slot, tupdesc); ExecSetSlotDescriptor(slot, tupdesc);
ExecStoreTuple(tuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, slot, false);
} }
} }
@ -2084,7 +2084,7 @@ ExecConstraints(ResultRelInfo *resultRelInfo,
{ {
tuple = do_convert_tuple(tuple, map); tuple = do_convert_tuple(tuple, map);
ExecSetSlotDescriptor(slot, tupdesc); ExecSetSlotDescriptor(slot, tupdesc);
ExecStoreTuple(tuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, slot, false);
} }
} }
@ -2190,7 +2190,7 @@ ExecWithCheckOptions(WCOKind kind, ResultRelInfo *resultRelInfo,
{ {
tuple = do_convert_tuple(tuple, map); tuple = do_convert_tuple(tuple, map);
ExecSetSlotDescriptor(slot, tupdesc); ExecSetSlotDescriptor(slot, tupdesc);
ExecStoreTuple(tuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, slot, false);
} }
} }

View File

@ -258,7 +258,7 @@ ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd,
if (myslot != NULL && map != NULL) if (myslot != NULL && map != NULL)
{ {
tuple = do_convert_tuple(tuple, map); tuple = do_convert_tuple(tuple, map);
ExecStoreTuple(tuple, myslot, InvalidBuffer, true); ExecStoreHeapTuple(tuple, myslot, true);
slot = myslot; slot = myslot;
} }
@ -842,7 +842,7 @@ ConvertPartitionTupleSlot(TupleConversionMap *map,
*p_my_slot = new_slot; *p_my_slot = new_slot;
Assert(new_slot != NULL); Assert(new_slot != NULL);
ExecSetSlotDescriptor(new_slot, map->outdesc); ExecSetSlotDescriptor(new_slot, map->outdesc);
ExecStoreTuple(tuple, new_slot, InvalidBuffer, shouldFree); ExecStoreHeapTuple(tuple, new_slot, shouldFree);
return tuple; return tuple;
} }

View File

@ -146,7 +146,7 @@ retry:
if ((scantuple = index_getnext(scan, ForwardScanDirection)) != NULL) if ((scantuple = index_getnext(scan, ForwardScanDirection)) != NULL)
{ {
found = true; found = true;
ExecStoreTuple(scantuple, outslot, InvalidBuffer, false); ExecStoreHeapTuple(scantuple, outslot, false);
ExecMaterializeSlot(outslot); ExecMaterializeSlot(outslot);
xwait = TransactionIdIsValid(snap.xmin) ? xwait = TransactionIdIsValid(snap.xmin) ?
@ -310,7 +310,7 @@ retry:
continue; continue;
found = true; found = true;
ExecStoreTuple(scantuple, outslot, InvalidBuffer, false); ExecStoreHeapTuple(scantuple, outslot, false);
ExecMaterializeSlot(outslot); ExecMaterializeSlot(outslot);
xwait = TransactionIdIsValid(snap.xmin) ? xwait = TransactionIdIsValid(snap.xmin) ?

View File

@ -78,8 +78,8 @@ ExecScanFetch(ScanState *node,
return ExecClearTuple(slot); return ExecClearTuple(slot);
/* Store test tuple in the plan node's scan slot */ /* Store test tuple in the plan node's scan slot */
ExecStoreTuple(estate->es_epqTuple[scanrelid - 1], ExecStoreHeapTuple(estate->es_epqTuple[scanrelid - 1],
slot, InvalidBuffer, false); slot, false);
/* Check if it meets the access-method conditions */ /* Check if it meets the access-method conditions */
if (!(*recheckMtd) (node, slot)) if (!(*recheckMtd) (node, slot))

View File

@ -26,10 +26,10 @@
* *
* During ExecutorRun() * During ExecutorRun()
* ---------------- * ----------------
* - SeqNext() calls ExecStoreTuple() to place the tuple returned * - SeqNext() calls ExecStoreBufferHeapTuple() to place the tuple
* by the access methods into the scan tuple slot. * returned by the access methods into the scan tuple slot.
* *
* - ExecSeqScan() calls ExecStoreTuple() to take the result * - ExecSeqScan() calls ExecStoreHeapTuple() to take the result
* tuple from ExecProject() and place it into the result tuple slot. * tuple from ExecProject() and place it into the result tuple slot.
* *
* - ExecutePlan() calls the output function. * - ExecutePlan() calls the output function.
@ -287,48 +287,32 @@ ExecSetSlotDescriptor(TupleTableSlot *slot, /* slot to change */
} }
/* -------------------------------- /* --------------------------------
* ExecStoreTuple * ExecStoreHeapTuple
* *
* This function is used to store a physical tuple into a specified * This function is used to store an on-the-fly physical tuple into a specified
* slot in the tuple table. * slot in the tuple table.
* *
* tuple: tuple to store * tuple: tuple to store
* slot: slot to store it in * slot: slot to store it in
* buffer: disk buffer if tuple is in a disk page, else InvalidBuffer
* shouldFree: true if ExecClearTuple should pfree() the tuple * shouldFree: true if ExecClearTuple should pfree() the tuple
* when done with it * when done with it
* *
* If 'buffer' is not InvalidBuffer, the tuple table code acquires a pin * shouldFree is normally set 'true' for tuples constructed on-the-fly. But it
* on the buffer which is held until the slot is cleared, so that the tuple * can be 'false' when the referenced tuple is held in a tuple table slot
* won't go away on us. * belonging to a lower-level executor Proc node. In this case the lower-level
* * slot retains ownership and responsibility for eventually releasing the
* shouldFree is normally set 'true' for tuples constructed on-the-fly. * tuple. When this method is used, we must be certain that the upper-level
* It must always be 'false' for tuples that are stored in disk pages, * Proc node will lose interest in the tuple sooner than the lower-level one
* since we don't want to try to pfree those. * does! If you're not certain, copy the lower-level tuple with heap_copytuple
* * and let the upper-level table slot assume ownership of the copy!
* Another case where it is 'false' is when the referenced tuple is held
* in a tuple table slot belonging to a lower-level executor Proc node.
* In this case the lower-level slot retains ownership and responsibility
* for eventually releasing the tuple. When this method is used, we must
* be certain that the upper-level Proc node will lose interest in the tuple
* sooner than the lower-level one does! If you're not certain, copy the
* lower-level tuple with heap_copytuple and let the upper-level table
* slot assume ownership of the copy!
* *
* Return value is just the passed-in slot pointer. * Return value is just the passed-in slot pointer.
*
* NOTE: before PostgreSQL 8.1, this function would accept a NULL tuple
* pointer and effectively behave like ExecClearTuple (though you could
* still specify a buffer to pin, which would be an odd combination).
* This saved a couple lines of code in a few places, but seemed more likely
* to mask logic errors than to be really useful, so it's now disallowed.
* -------------------------------- * --------------------------------
*/ */
TupleTableSlot * TupleTableSlot *
ExecStoreTuple(HeapTuple tuple, ExecStoreHeapTuple(HeapTuple tuple,
TupleTableSlot *slot, TupleTableSlot *slot,
Buffer buffer, bool shouldFree)
bool shouldFree)
{ {
/* /*
* sanity checks * sanity checks
@ -336,8 +320,6 @@ ExecStoreTuple(HeapTuple tuple,
Assert(tuple != NULL); Assert(tuple != NULL);
Assert(slot != NULL); Assert(slot != NULL);
Assert(slot->tts_tupleDescriptor != NULL); Assert(slot->tts_tupleDescriptor != NULL);
/* passing shouldFree=true for a tuple on a disk page is not sane */
Assert(BufferIsValid(buffer) ? (!shouldFree) : true);
/* /*
* Free any old physical tuple belonging to the slot. * Free any old physical tuple belonging to the slot.
@ -359,22 +341,78 @@ ExecStoreTuple(HeapTuple tuple,
/* Mark extracted state invalid */ /* Mark extracted state invalid */
slot->tts_nvalid = 0; slot->tts_nvalid = 0;
/* Unpin any buffer pinned by the slot. */
if (BufferIsValid(slot->tts_buffer))
ReleaseBuffer(slot->tts_buffer);
slot->tts_buffer = InvalidBuffer;
return slot;
}
/* --------------------------------
* ExecStoreBufferHeapTuple
*
* This function is used to store an on-disk physical tuple from a buffer
* into a specified slot in the tuple table.
*
* tuple: tuple to store
* slot: slot to store it in
* buffer: disk buffer if tuple is in a disk page, else InvalidBuffer
*
* The tuple table code acquires a pin on the buffer which is held until the
* slot is cleared, so that the tuple won't go away on us.
*
* Return value is just the passed-in slot pointer.
* --------------------------------
*/
TupleTableSlot *
ExecStoreBufferHeapTuple(HeapTuple tuple,
TupleTableSlot *slot,
Buffer buffer)
{
/* /*
* If tuple is on a disk page, keep the page pinned as long as we hold a * sanity checks
* pointer into it. We assume the caller already has such a pin. */
Assert(tuple != NULL);
Assert(slot != NULL);
Assert(slot->tts_tupleDescriptor != NULL);
Assert(BufferIsValid(buffer));
/*
* Free any old physical tuple belonging to the slot.
*/
if (slot->tts_shouldFree)
heap_freetuple(slot->tts_tuple);
if (slot->tts_shouldFreeMin)
heap_free_minimal_tuple(slot->tts_mintuple);
/*
* Store the new tuple into the specified slot.
*/
slot->tts_isempty = false;
slot->tts_shouldFree = false;
slot->tts_shouldFreeMin = false;
slot->tts_tuple = tuple;
slot->tts_mintuple = NULL;
/* Mark extracted state invalid */
slot->tts_nvalid = 0;
/*
* Keep the disk page containing the given tuple pinned as long as we hold
* a pointer into it. We assume the caller already has such a pin.
* *
* This is coded to optimize the case where the slot previously held a * This is coded to optimize the case where the slot previously held a
* tuple on the same disk page: in that case releasing and re-acquiring * tuple on the same disk page: in that case releasing and re-acquiring the
* the pin is a waste of cycles. This is a common situation during * pin is a waste of cycles. This is a common situation during seqscans,
* seqscans, so it's worth troubling over. * so it's worth troubling over.
*/ */
if (slot->tts_buffer != buffer) if (slot->tts_buffer != buffer)
{ {
if (BufferIsValid(slot->tts_buffer)) if (BufferIsValid(slot->tts_buffer))
ReleaseBuffer(slot->tts_buffer); ReleaseBuffer(slot->tts_buffer);
slot->tts_buffer = buffer; slot->tts_buffer = buffer;
if (BufferIsValid(buffer)) IncrBufferRefCount(buffer);
IncrBufferRefCount(buffer);
} }
return slot; return slot;
@ -383,7 +421,7 @@ ExecStoreTuple(HeapTuple tuple,
/* -------------------------------- /* --------------------------------
* ExecStoreMinimalTuple * ExecStoreMinimalTuple
* *
* Like ExecStoreTuple, but insert a "minimal" tuple into the slot. * Like ExecStoreHeapTuple, but insert a "minimal" tuple into the slot.
* *
* No 'buffer' parameter since minimal tuples are never stored in relations. * No 'buffer' parameter since minimal tuples are never stored in relations.
* -------------------------------- * --------------------------------
@ -652,7 +690,7 @@ ExecFetchSlotTuple(TupleTableSlot *slot)
tuple = heap_expand_tuple(slot->tts_tuple, tuple = heap_expand_tuple(slot->tts_tuple,
slot->tts_tupleDescriptor); slot->tts_tupleDescriptor);
MemoryContextSwitchTo(oldContext); MemoryContextSwitchTo(oldContext);
slot = ExecStoreTuple(tuple, slot, InvalidBuffer, true); slot = ExecStoreHeapTuple(tuple, slot, true);
} }
return slot->tts_tuple; return slot->tts_tuple;
} }
@ -834,7 +872,7 @@ ExecCopySlot(TupleTableSlot *dstslot, TupleTableSlot *srcslot)
newTuple = ExecCopySlotTuple(srcslot); newTuple = ExecCopySlotTuple(srcslot);
MemoryContextSwitchTo(oldContext); MemoryContextSwitchTo(oldContext);
return ExecStoreTuple(newTuple, dstslot, InvalidBuffer, true); return ExecStoreHeapTuple(newTuple, dstslot, true);
} }

View File

@ -1799,10 +1799,9 @@ agg_retrieve_direct(AggState *aggstate)
* reserved for it. The tuple will be deleted when it is * reserved for it. The tuple will be deleted when it is
* cleared from the slot. * cleared from the slot.
*/ */
ExecStoreTuple(aggstate->grp_firstTuple, ExecStoreHeapTuple(aggstate->grp_firstTuple,
firstSlot, firstSlot,
InvalidBuffer, true);
true);
aggstate->grp_firstTuple = NULL; /* don't keep two pointers */ aggstate->grp_firstTuple = NULL; /* don't keep two pointers */
/* set up for first advance_aggregates call */ /* set up for first advance_aggregates call */

View File

@ -340,10 +340,9 @@ BitmapHeapNext(BitmapHeapScanState *node)
* Set up the result slot to point to this tuple. Note that the * Set up the result slot to point to this tuple. Note that the
* slot acquires a pin on the buffer. * slot acquires a pin on the buffer.
*/ */
ExecStoreTuple(&scan->rs_ctup, ExecStoreBufferHeapTuple(&scan->rs_ctup,
slot, slot,
scan->rs_cbuf, scan->rs_cbuf);
false);
/* /*
* If we are using lossy info, we have to recheck the qual * If we are using lossy info, we have to recheck the qual

View File

@ -257,11 +257,9 @@ gather_getnext(GatherState *gatherstate)
if (HeapTupleIsValid(tup)) if (HeapTupleIsValid(tup))
{ {
ExecStoreTuple(tup, /* tuple to store */ ExecStoreHeapTuple(tup, /* tuple to store */
fslot, /* slot in which to store the tuple */ fslot, /* slot to store the tuple */
InvalidBuffer, /* buffer associated with this true); /* pfree tuple when done with it */
* tuple */
true); /* pfree tuple when done with it */
return fslot; return fslot;
} }
} }

View File

@ -679,11 +679,10 @@ gather_merge_readnext(GatherMergeState *gm_state, int reader, bool nowait)
Assert(HeapTupleIsValid(tup)); Assert(HeapTupleIsValid(tup));
/* Build the TupleTableSlot for the given tuple */ /* Build the TupleTableSlot for the given tuple */
ExecStoreTuple(tup, /* tuple to store */ ExecStoreHeapTuple(tup, /* tuple to store */
gm_state->gm_slots[reader], /* slot in which to store the gm_state->gm_slots[reader], /* slot in which to store
* tuple */ * the tuple */
InvalidBuffer, /* no buffer associated with tuple */ true); /* pfree tuple when done with it */
true); /* pfree tuple when done with it */
return true; return true;
} }

View File

@ -199,7 +199,7 @@ IndexOnlyNext(IndexOnlyScanState *node)
*/ */
Assert(slot->tts_tupleDescriptor->natts == Assert(slot->tts_tupleDescriptor->natts ==
scandesc->xs_hitupdesc->natts); scandesc->xs_hitupdesc->natts);
ExecStoreTuple(scandesc->xs_hitup, slot, InvalidBuffer, false); ExecStoreHeapTuple(scandesc->xs_hitup, slot, false);
} }
else if (scandesc->xs_itup) else if (scandesc->xs_itup)
StoreIndexTuple(slot, scandesc->xs_itup, scandesc->xs_itupdesc); StoreIndexTuple(slot, scandesc->xs_itup, scandesc->xs_itupdesc);

View File

@ -140,10 +140,10 @@ IndexNext(IndexScanState *node)
* Note: we pass 'false' because tuples returned by amgetnext are * Note: we pass 'false' because tuples returned by amgetnext are
* pointers onto disk pages and must not be pfree()'d. * pointers onto disk pages and must not be pfree()'d.
*/ */
ExecStoreTuple(tuple, /* tuple to store */ ExecStoreBufferHeapTuple(tuple, /* tuple to store */
slot, /* slot to store in */ slot, /* slot to store in */
scandesc->xs_cbuf, /* buffer containing tuple */ scandesc->xs_cbuf); /* buffer containing
false); /* don't pfree */ * tuple */
/* /*
* If the index was lossy, we have to recheck the index quals using * If the index was lossy, we have to recheck the index quals using
@ -257,7 +257,7 @@ IndexNextWithReorder(IndexScanState *node)
tuple = reorderqueue_pop(node); tuple = reorderqueue_pop(node);
/* Pass 'true', as the tuple in the queue is a palloc'd copy */ /* Pass 'true', as the tuple in the queue is a palloc'd copy */
ExecStoreTuple(tuple, slot, InvalidBuffer, true); ExecStoreHeapTuple(tuple, slot, true);
return slot; return slot;
} }
} }
@ -284,13 +284,11 @@ next_indextuple:
/* /*
* Store the scanned tuple in the scan tuple slot of the scan state. * Store the scanned tuple in the scan tuple slot of the scan state.
* Note: we pass 'false' because tuples returned by amgetnext are
* pointers onto disk pages and must not be pfree()'d.
*/ */
ExecStoreTuple(tuple, /* tuple to store */ ExecStoreBufferHeapTuple(tuple, /* tuple to store */
slot, /* slot to store in */ slot, /* slot to store in */
scandesc->xs_cbuf, /* buffer containing tuple */ scandesc->xs_cbuf); /* buffer containing
false); /* don't pfree */ * tuple */
/* /*
* If the index was lossy, we have to recheck the index quals and * If the index was lossy, we have to recheck the index quals and

View File

@ -888,7 +888,7 @@ ldelete:;
if (slot->tts_tupleDescriptor != RelationGetDescr(resultRelationDesc)) if (slot->tts_tupleDescriptor != RelationGetDescr(resultRelationDesc))
ExecSetSlotDescriptor(slot, RelationGetDescr(resultRelationDesc)); ExecSetSlotDescriptor(slot, RelationGetDescr(resultRelationDesc));
ExecStoreTuple(&deltuple, slot, InvalidBuffer, false); ExecStoreHeapTuple(&deltuple, slot, false);
} }
rslot = ExecProcessReturning(resultRelInfo, slot, planSlot); rslot = ExecProcessReturning(resultRelInfo, slot, planSlot);
@ -1479,7 +1479,7 @@ ExecOnConflictUpdate(ModifyTableState *mtstate,
ExecCheckHeapTupleVisible(estate, &tuple, buffer); ExecCheckHeapTupleVisible(estate, &tuple, buffer);
/* Store target's existing tuple in the state's dedicated slot */ /* Store target's existing tuple in the state's dedicated slot */
ExecStoreTuple(&tuple, mtstate->mt_existing, buffer, false); ExecStoreBufferHeapTuple(&tuple, mtstate->mt_existing, buffer);
/* /*
* Make tuple and any needed join variables available to ExecQual and * Make tuple and any needed join variables available to ExecQual and

View File

@ -63,10 +63,9 @@ SampleNext(SampleScanState *node)
slot = node->ss.ss_ScanTupleSlot; slot = node->ss.ss_ScanTupleSlot;
if (tuple) if (tuple)
ExecStoreTuple(tuple, /* tuple to store */ ExecStoreBufferHeapTuple(tuple, /* tuple to store */
slot, /* slot to store in */ slot, /* slot to store in */
node->ss.ss_currentScanDesc->rs_cbuf, /* tuple's buffer */ node->ss.ss_currentScanDesc->rs_cbuf); /* tuple's buffer */
false); /* don't pfree this pointer */
else else
ExecClearTuple(slot); ExecClearTuple(slot);

View File

@ -84,15 +84,14 @@ SeqNext(SeqScanState *node)
* our scan tuple slot and return the slot. Note: we pass 'false' because * our scan tuple slot and return the slot. Note: we pass 'false' because
* tuples returned by heap_getnext() are pointers onto disk pages and were * tuples returned by heap_getnext() are pointers onto disk pages and were
* not created with palloc() and so should not be pfree()'d. Note also * not created with palloc() and so should not be pfree()'d. Note also
* that ExecStoreTuple will increment the refcount of the buffer; the * that ExecStoreHeapTuple will increment the refcount of the buffer; the
* refcount will not be dropped until the tuple table slot is cleared. * refcount will not be dropped until the tuple table slot is cleared.
*/ */
if (tuple) if (tuple)
ExecStoreTuple(tuple, /* tuple to store */ ExecStoreBufferHeapTuple(tuple, /* tuple to store */
slot, /* slot to store in */ slot, /* slot to store in */
scandesc->rs_cbuf, /* buffer associated with this scandesc->rs_cbuf); /* buffer associated
* tuple */ * with this tuple */
false); /* don't pfree this pointer */
else else
ExecClearTuple(slot); ExecClearTuple(slot);

View File

@ -267,10 +267,9 @@ setop_retrieve_direct(SetOpState *setopstate)
* for it. The tuple will be deleted when it is cleared from the * for it. The tuple will be deleted when it is cleared from the
* slot. * slot.
*/ */
ExecStoreTuple(setopstate->grp_firstTuple, ExecStoreHeapTuple(setopstate->grp_firstTuple,
resultTupleSlot, resultTupleSlot,
InvalidBuffer, true);
true);
setopstate->grp_firstTuple = NULL; /* don't keep two pointers */ setopstate->grp_firstTuple = NULL; /* don't keep two pointers */
/* Initialize working state for a new input tuple group */ /* Initialize working state for a new input tuple group */

View File

@ -377,20 +377,18 @@ TidNext(TidScanState *node)
if (heap_fetch(heapRelation, snapshot, tuple, &buffer, false, NULL)) if (heap_fetch(heapRelation, snapshot, tuple, &buffer, false, NULL))
{ {
/* /*
* store the scanned tuple in the scan tuple slot of the scan * Store the scanned tuple in the scan tuple slot of the scan
* state. Eventually we will only do this and not return a tuple. * state. Eventually we will only do this and not return a tuple.
* Note: we pass 'false' because tuples returned by amgetnext are
* pointers onto disk pages and were not created with palloc() and
* so should not be pfree()'d.
*/ */
ExecStoreTuple(tuple, /* tuple to store */ ExecStoreBufferHeapTuple(tuple, /* tuple to store */
slot, /* slot to store in */ slot, /* slot to store in */
buffer, /* buffer associated with tuple */ buffer); /* buffer associated with
false); /* don't pfree */ * tuple */
/* /*
* At this point we have an extra pin on the buffer, because * At this point we have an extra pin on the buffer, because
* ExecStoreTuple incremented the pin count. Drop our local pin. * ExecStoreHeapTuple incremented the pin count. Drop our local
* pin.
*/ */
ReleaseBuffer(buffer); ReleaseBuffer(buffer);

View File

@ -715,7 +715,7 @@ check_default_partition_contents(Relation parent, Relation default_rel,
while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL) while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL)
{ {
ExecStoreTuple(tuple, tupslot, InvalidBuffer, false); ExecStoreHeapTuple(tuple, tupslot, false);
econtext->ecxt_scantuple = tupslot; econtext->ecxt_scantuple = tupslot;
if (!ExecCheck(partqualstate, econtext)) if (!ExecCheck(partqualstate, econtext))

View File

@ -755,7 +755,7 @@ apply_handle_update(StringInfo s)
{ {
/* Process and store remote tuple in the slot */ /* Process and store remote tuple in the slot */
oldctx = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate)); oldctx = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
ExecStoreTuple(localslot->tts_tuple, remoteslot, InvalidBuffer, false); ExecStoreHeapTuple(localslot->tts_tuple, remoteslot, false);
slot_modify_cstrings(remoteslot, rel, newtup.values, newtup.changed); slot_modify_cstrings(remoteslot, rel, newtup.values, newtup.changed);
MemoryContextSwitchTo(oldctx); MemoryContextSwitchTo(oldctx);

View File

@ -5607,7 +5607,7 @@ get_actual_variable_range(PlannerInfo *root, VariableStatData *vardata,
indexscandir)) != NULL) indexscandir)) != NULL)
{ {
/* Extract the index column values from the heap tuple */ /* Extract the index column values from the heap tuple */
ExecStoreTuple(tup, slot, InvalidBuffer, false); ExecStoreHeapTuple(tup, slot, false);
FormIndexDatum(indexInfo, slot, estate, FormIndexDatum(indexInfo, slot, estate,
values, isnull); values, isnull);
@ -5640,7 +5640,7 @@ get_actual_variable_range(PlannerInfo *root, VariableStatData *vardata,
-indexscandir)) != NULL) -indexscandir)) != NULL)
{ {
/* Extract the index column values from the heap tuple */ /* Extract the index column values from the heap tuple */
ExecStoreTuple(tup, slot, InvalidBuffer, false); ExecStoreHeapTuple(tup, slot, false);
FormIndexDatum(indexInfo, slot, estate, FormIndexDatum(indexInfo, slot, estate,
values, isnull); values, isnull);

View File

@ -3792,11 +3792,11 @@ comparetup_cluster(const SortTuple *a, const SortTuple *b,
ecxt_scantuple = GetPerTupleExprContext(state->estate)->ecxt_scantuple; ecxt_scantuple = GetPerTupleExprContext(state->estate)->ecxt_scantuple;
ExecStoreTuple(ltup, ecxt_scantuple, InvalidBuffer, false); ExecStoreHeapTuple(ltup, ecxt_scantuple, false);
FormIndexDatum(state->indexInfo, ecxt_scantuple, state->estate, FormIndexDatum(state->indexInfo, ecxt_scantuple, state->estate,
l_index_values, l_index_isnull); l_index_values, l_index_isnull);
ExecStoreTuple(rtup, ecxt_scantuple, InvalidBuffer, false); ExecStoreHeapTuple(rtup, ecxt_scantuple, false);
FormIndexDatum(state->indexInfo, ecxt_scantuple, state->estate, FormIndexDatum(state->indexInfo, ecxt_scantuple, state->estate,
r_index_values, r_index_isnull); r_index_values, r_index_isnull);

View File

@ -153,10 +153,12 @@ extern void ExecResetTupleTable(List *tupleTable, bool shouldFree);
extern TupleTableSlot *MakeSingleTupleTableSlot(TupleDesc tupdesc); extern TupleTableSlot *MakeSingleTupleTableSlot(TupleDesc tupdesc);
extern void ExecDropSingleTupleTableSlot(TupleTableSlot *slot); extern void ExecDropSingleTupleTableSlot(TupleTableSlot *slot);
extern void ExecSetSlotDescriptor(TupleTableSlot *slot, TupleDesc tupdesc); extern void ExecSetSlotDescriptor(TupleTableSlot *slot, TupleDesc tupdesc);
extern TupleTableSlot *ExecStoreTuple(HeapTuple tuple, extern TupleTableSlot *ExecStoreHeapTuple(HeapTuple tuple,
TupleTableSlot *slot, TupleTableSlot *slot,
Buffer buffer, bool shouldFree);
bool shouldFree); extern TupleTableSlot *ExecStoreBufferHeapTuple(HeapTuple tuple,
TupleTableSlot *slot,
Buffer buffer);
extern TupleTableSlot *ExecStoreMinimalTuple(MinimalTuple mtup, extern TupleTableSlot *ExecStoreMinimalTuple(MinimalTuple mtup,
TupleTableSlot *slot, TupleTableSlot *slot,
bool shouldFree); bool shouldFree);