summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorHeikki Linnakangas <heikki.linnakangas@iki.fi>2011-06-08 13:47:21 +0300
committerHeikki Linnakangas <heikki.linnakangas@iki.fi>2011-06-08 14:02:43 +0300
commit8f9622bbb3c02b06176760c3ca2d33c5b5f629a7 (patch)
tree28aa06c755c12951e7c99f11815a9379ba3c76d3
parent16925c1e1fa236e4d7d6c8b571890e7c777f75d7 (diff)
downloadpostgresql-8f9622bbb3c02b06176760c3ca2d33c5b5f629a7.tar.gz
Make DDL operations play nicely with Serializable Snapshot Isolation.
Truncating or dropping a table is treated like deletion of all tuples, and check for conflicts accordingly. If a table is clustered or rewritten by ALTER TABLE, all predicate locks on the heap are promoted to relation-level locks, because the tuple or page ids of any existing tuples will change and won't be valid after rewriting the table. Arguably ALTER TABLE should be treated like a mass-UPDATE of every row, but if you e.g change the datatype of a column, you could also argue that it's just a change to the physical layout, not a logical change. Reindexing promotes all locks on the index to relation-level lock on the heap. Kevin Grittner, with a lot of cosmetic changes by me.
-rw-r--r--src/backend/catalog/heap.c9
-rw-r--r--src/backend/catalog/index.c13
-rw-r--r--src/backend/commands/cluster.c9
-rw-r--r--src/backend/commands/tablecmds.c19
-rw-r--r--src/backend/storage/lmgr/predicate.c521
-rw-r--r--src/include/storage/predicate.h2
-rw-r--r--src/include/storage/predicate_internals.h6
7 files changed, 499 insertions, 80 deletions
diff --git a/src/backend/catalog/heap.c b/src/backend/catalog/heap.c
index 8639957147..a6e541d858 100644
--- a/src/backend/catalog/heap.c
+++ b/src/backend/catalog/heap.c
@@ -63,6 +63,7 @@
#include "parser/parse_relation.h"
#include "storage/bufmgr.h"
#include "storage/freespace.h"
+#include "storage/predicate.h"
#include "storage/smgr.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -1658,6 +1659,14 @@ heap_drop_with_catalog(Oid relid)
CheckTableNotInUse(rel, "DROP TABLE");
/*
+ * This effectively deletes all rows in the table, and may be done in a
+ * serializable transaction. In that case we must record a rw-conflict in
+ * to this transaction from each transaction holding a predicate lock on
+ * the table.
+ */
+ CheckTableForSerializableConflictIn(rel);
+
+ /*
* Delete pg_foreign_table tuple first.
*/
if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE)
diff --git a/src/backend/catalog/index.c b/src/backend/catalog/index.c
index 1b39e1683c..0898cf363e 100644
--- a/src/backend/catalog/index.c
+++ b/src/backend/catalog/index.c
@@ -54,6 +54,7 @@
#include "parser/parser.h"
#include "storage/bufmgr.h"
#include "storage/lmgr.h"
+#include "storage/predicate.h"
#include "storage/procarray.h"
#include "storage/smgr.h"
#include "utils/builtins.h"
@@ -1312,6 +1313,12 @@ index_drop(Oid indexId)
CheckTableNotInUse(userIndexRelation, "DROP INDEX");
/*
+ * All predicate locks on the index are about to be made invalid. Promote
+ * them to relation locks on the heap.
+ */
+ TransferPredicateLocksToHeapRelation(userIndexRelation);
+
+ /*
* Schedule physical removal of the files
*/
RelationDropStorage(userIndexRelation);
@@ -2799,6 +2806,12 @@ reindex_index(Oid indexId, bool skip_constraint_checks)
*/
CheckTableNotInUse(iRel, "REINDEX INDEX");
+ /*
+ * All predicate locks on the index are about to be made invalid. Promote
+ * them to relation locks on the heap.
+ */
+ TransferPredicateLocksToHeapRelation(iRel);
+
PG_TRY();
{
/* Suppress use of the target index while rebuilding it */
diff --git a/src/backend/commands/cluster.c b/src/backend/commands/cluster.c
index dc0f6059b0..0ab3a8bcfa 100644
--- a/src/backend/commands/cluster.c
+++ b/src/backend/commands/cluster.c
@@ -39,6 +39,7 @@
#include "optimizer/planner.h"
#include "storage/bufmgr.h"
#include "storage/lmgr.h"
+#include "storage/predicate.h"
#include "storage/procarray.h"
#include "storage/smgr.h"
#include "utils/acl.h"
@@ -385,6 +386,14 @@ cluster_rel(Oid tableOid, Oid indexOid, bool recheck, bool verbose,
if (OidIsValid(indexOid))
check_index_is_clusterable(OldHeap, indexOid, recheck, AccessExclusiveLock);
+ /*
+ * All predicate locks on the tuples or pages are about to be made
+ * invalid, because we move tuples around. Promote them to relation
+ * locks. Predicate locks on indexes will be promoted when they are
+ * reindexed.
+ */
+ TransferPredicateLocksToHeapRelation(OldHeap);
+
/* rebuild_relation does all the dirty work */
rebuild_relation(OldHeap, indexOid, freeze_min_age, freeze_table_age,
verbose);
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index 60b66ec5c5..6279f2bf9a 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -70,6 +70,7 @@
#include "storage/bufmgr.h"
#include "storage/lmgr.h"
#include "storage/lock.h"
+#include "storage/predicate.h"
#include "storage/smgr.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -1040,6 +1041,14 @@ ExecuteTruncate(TruncateStmt *stmt)
Oid toast_relid;
/*
+ * This effectively deletes all rows in the table, and may be done
+ * in a serializable transaction. In that case we must record a
+ * rw-conflict in to this transaction from each transaction
+ * holding a predicate lock on the table.
+ */
+ CheckTableForSerializableConflictIn(rel);
+
+ /*
* Need the full transaction-safe pushups.
*
* Create a new empty storage file for the relation, and assign it
@@ -3529,6 +3538,16 @@ ATRewriteTable(AlteredTableInfo *tab, Oid OIDNewHeap, LOCKMODE lockmode)
(errmsg("verifying table \"%s\"",
RelationGetRelationName(oldrel))));
+ if (newrel)
+ {
+ /*
+ * All predicate locks on the tuples or pages are about to be made
+ * invalid, because we move tuples around. Promote them to
+ * relation locks.
+ */
+ TransferPredicateLocksToHeapRelation(oldrel);
+ }
+
econtext = GetPerTupleExprContext(estate);
/*
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index 32124b5e5d..28da729b6d 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -155,6 +155,7 @@
* BlockNumber newblkno);
* PredicateLockPageCombine(Relation relation, BlockNumber oldblkno,
* BlockNumber newblkno);
+ * TransferPredicateLocksToHeapRelation(Relation relation)
* ReleasePredicateLocks(bool isCommit)
*
* conflict detection (may also trigger rollback)
@@ -162,6 +163,7 @@
* HeapTupleData *tup, Buffer buffer)
* CheckForSerializableConflictIn(Relation relation, HeapTupleData *tup,
* Buffer buffer)
+ * CheckTableForSerializableConflictIn(Relation relation)
*
* final rollback checking
* PreCommit_CheckForSerializationFailure(void)
@@ -257,10 +259,10 @@
#define SxactIsMarkedForDeath(sxact) (((sxact)->flags & SXACT_FLAG_MARKED_FOR_DEATH) != 0)
/*
- * When a public interface method is called for a split on an index relation,
- * this is the test to see if we should do a quick return.
+ * Is this relation exempt from predicate locking? We don't do predicate
+ * locking on system or temporary relations.
*/
-#define SkipSplitTracking(relation) \
+#define SkipPredicateLocksForRelation(relation) \
(((relation)->rd_id < FirstBootstrapObjectId) \
|| RelationUsesLocalBuffers(relation))
@@ -273,7 +275,7 @@
((!IsolationIsSerializable()) \
|| ((MySerializableXact == InvalidSerializableXact)) \
|| ReleasePredicateLocksIfROSafe() \
- || SkipSplitTracking(relation))
+ || SkipPredicateLocksForRelation(relation))
/*
@@ -374,11 +376,13 @@ static HTAB *PredicateLockHash;
static SHM_QUEUE *FinishedSerializableTransactions;
/*
- * Tag for a reserved entry in PredicateLockTargetHash; used to ensure
- * there's an element available for scratch space if we need it,
- * e.g. in PredicateLockPageSplit. This is an otherwise-invalid tag.
+ * Tag for a dummy entry in PredicateLockTargetHash. By temporarily removing
+ * this entry, you can ensure that there's enough scratch space available for
+ * inserting one entry in the hash table. This is an otherwise-invalid tag.
*/
-static const PREDICATELOCKTARGETTAG ReservedTargetTag = {0, 0, 0, 0, 0};
+static const PREDICATELOCKTARGETTAG ScratchTargetTag = {0, 0, 0, 0, 0};
+static uint32 ScratchTargetTagHash;
+static int ScratchPartitionLock;
/*
* The local hash table used to determine when to combine multiple fine-
@@ -420,6 +424,8 @@ static bool PredicateLockExists(const PREDICATELOCKTARGETTAG *targettag);
static bool GetParentPredicateLockTag(const PREDICATELOCKTARGETTAG *tag,
PREDICATELOCKTARGETTAG *parent);
static bool CoarserLockCovers(const PREDICATELOCKTARGETTAG *newtargettag);
+static void RemoveScratchTarget(bool lockheld);
+static void RestoreScratchTarget(bool lockheld);
static void RemoveTargetIfNoLongerUsed(PREDICATELOCKTARGET *target,
uint32 targettaghash);
static void DeleteChildTargetLocks(const PREDICATELOCKTARGETTAG *newtargettag);
@@ -434,6 +440,8 @@ static bool TransferPredicateLocksToNewTarget(const PREDICATELOCKTARGETTAG oldta
const PREDICATELOCKTARGETTAG newtargettag,
bool removeOld);
static void PredicateLockAcquire(const PREDICATELOCKTARGETTAG *targettag);
+static void DropAllPredicateLocksFromTable(const Relation relation,
+ bool transfer);
static void SetNewSxactGlobalXmin(void);
static bool ReleasePredicateLocksIfROSafe(void);
static void ClearOldPredicateLocks(void);
@@ -977,8 +985,8 @@ InitPredicateLocks(void)
bool found;
/*
- * Compute size of predicate lock target hashtable.
- * Note these calculations must agree with PredicateLockShmemSize!
+ * Compute size of predicate lock target hashtable. Note these
+ * calculations must agree with PredicateLockShmemSize!
*/
max_table_size = NPREDICATELOCKTARGETENTS();
@@ -1003,14 +1011,12 @@ InitPredicateLocks(void)
max_table_size *= 2;
/*
- * Reserve an entry in the hash table; we use it to make sure there's
+ * Reserve a dummy entry in the hash table; we use it to make sure there's
* always one entry available when we need to split or combine a page,
* because running out of space there could mean aborting a
* non-serializable transaction.
*/
- hash_search(PredicateLockTargetHash, &ReservedTargetTag,
- HASH_ENTER, NULL);
-
+ hash_search(PredicateLockTargetHash, &ScratchTargetTag, HASH_ENTER, NULL);
/*
* Allocate hash table for PREDICATELOCK structs. This stores per
@@ -1030,8 +1036,8 @@ InitPredicateLocks(void)
hash_flags);
/*
- * Compute size for serializable transaction hashtable.
- * Note these calculations must agree with PredicateLockShmemSize!
+ * Compute size for serializable transaction hashtable. Note these
+ * calculations must agree with PredicateLockShmemSize!
*/
max_table_size = (MaxBackends + max_prepared_xacts);
@@ -1165,6 +1171,10 @@ InitPredicateLocks(void)
* transactions.
*/
OldSerXidInit();
+
+ /* Pre-calculate the hash and partition lock of the scratch entry */
+ ScratchTargetTagHash = PredicateLockTargetTagHashCode(&ScratchTargetTag);
+ ScratchPartitionLock = PredicateLockHashPartitionLock(ScratchTargetTagHash);
}
/*
@@ -1759,6 +1769,54 @@ CoarserLockCovers(const PREDICATELOCKTARGETTAG *newtargettag)
}
/*
+ * Remove the dummy entry from the predicate lock target hash, to free up some
+ * scratch space. The caller must be holding SerializablePredicateLockListLock,
+ * and must restore the entry with RestoreScratchTarget() before releasing the
+ * lock.
+ *
+ * If lockheld is true, the caller is already holding the partition lock
+ * of the partition containing the scratch entry.
+ */
+static void
+RemoveScratchTarget(bool lockheld)
+{
+ bool found;
+
+ Assert(LWLockHeldByMe(SerializablePredicateLockListLock));
+
+ if (!lockheld)
+ LWLockAcquire(ScratchPartitionLock, LW_EXCLUSIVE);
+ hash_search_with_hash_value(PredicateLockTargetHash,
+ &ScratchTargetTag,
+ ScratchTargetTagHash,
+ HASH_REMOVE, &found);
+ Assert(found);
+ if (!lockheld)
+ LWLockRelease(ScratchPartitionLock);
+}
+
+/*
+ * Re-insert the dummy entry in predicate lock target hash.
+ */
+static void
+RestoreScratchTarget(bool lockheld)
+{
+ bool found;
+
+ Assert(LWLockHeldByMe(SerializablePredicateLockListLock));
+
+ if (!lockheld)
+ LWLockAcquire(ScratchPartitionLock, LW_EXCLUSIVE);
+ hash_search_with_hash_value(PredicateLockTargetHash,
+ &ScratchTargetTag,
+ ScratchTargetTagHash,
+ HASH_ENTER, &found);
+ Assert(!found);
+ if (!lockheld)
+ LWLockRelease(ScratchPartitionLock);
+}
+
+/*
* Check whether the list of related predicate locks is empty for a
* predicate lock target, and remove the target if it is.
*/
@@ -2317,8 +2375,8 @@ DeleteLockTarget(PREDICATELOCKTARGET *target, uint32 targettaghash)
*
* Returns true on success, or false if we ran out of shared memory to
* allocate the new target or locks. Guaranteed to always succeed if
- * removeOld is set (by using the reserved entry in
- * PredicateLockTargetHash for scratch space).
+ * removeOld is set (by using the scratch entry in PredicateLockTargetHash
+ * for scratch space).
*
* Warning: the "removeOld" option should be used only with care,
* because this function does not (indeed, can not) update other
@@ -2345,9 +2403,6 @@ TransferPredicateLocksToNewTarget(const PREDICATELOCKTARGETTAG oldtargettag,
LWLockId newpartitionLock;
bool found;
bool outOfShmem = false;
- uint32 reservedtargettaghash;
- LWLockId reservedpartitionLock;
-
Assert(LWLockHeldByMe(SerializablePredicateLockListLock));
@@ -2356,24 +2411,13 @@ TransferPredicateLocksToNewTarget(const PREDICATELOCKTARGETTAG oldtargettag,
oldpartitionLock = PredicateLockHashPartitionLock(oldtargettaghash);
newpartitionLock = PredicateLockHashPartitionLock(newtargettaghash);
- reservedtargettaghash = 0; /* Quiet compiler warnings. */
- reservedpartitionLock = 0; /* Quiet compiler warnings. */
-
if (removeOld)
{
/*
- * Remove the reserved entry to give us scratch space, so we know
- * we'll be able to create the new lock target.
+ * Remove the dummy entry to give us scratch space, so we know we'll
+ * be able to create the new lock target.
*/
- reservedtargettaghash = PredicateLockTargetTagHashCode(&ReservedTargetTag);
- reservedpartitionLock = PredicateLockHashPartitionLock(reservedtargettaghash);
- LWLockAcquire(reservedpartitionLock, LW_EXCLUSIVE);
- hash_search_with_hash_value(PredicateLockTargetHash,
- &ReservedTargetTag,
- reservedtargettaghash,
- HASH_REMOVE, &found);
- Assert(found);
- LWLockRelease(reservedpartitionLock);
+ RemoveScratchTarget(false);
}
/*
@@ -2431,6 +2475,10 @@ TransferPredicateLocksToNewTarget(const PREDICATELOCKTARGETTAG oldtargettag,
newpredlocktag.myTarget = newtarget;
+ /*
+ * Loop through all the locks on the old target, replacing them with
+ * locks on the new target.
+ */
oldpredlock = (PREDICATELOCK *)
SHMQueueNext(&(oldtarget->predicateLocks),
&(oldtarget->predicateLocks),
@@ -2530,19 +2578,238 @@ exit:
/* We shouldn't run out of memory if we're moving locks */
Assert(!outOfShmem);
- /* Put the reserved entry back */
- LWLockAcquire(reservedpartitionLock, LW_EXCLUSIVE);
- hash_search_with_hash_value(PredicateLockTargetHash,
- &ReservedTargetTag,
- reservedtargettaghash,
- HASH_ENTER, &found);
- Assert(!found);
- LWLockRelease(reservedpartitionLock);
+ /* Put the scrach entry back */
+ RestoreScratchTarget(false);
}
return !outOfShmem;
}
+/*
+ * Drop all predicate locks of any granularity from the specified relation,
+ * which can be a heap relation or an index relation. If 'transfer' is true,
+ * acquire a relation lock on the heap for any transactions with any lock(s)
+ * on the specified relation.
+ *
+ * This requires grabbing a lot of LW locks and scanning the entire lock
+ * target table for matches. That makes this more expensive than most
+ * predicate lock management functions, but it will only be called for DDL
+ * type commands that are expensive anyway, and there are fast returns when
+ * no serializable transactions are active or the relation is temporary.
+ *
+ * We don't use the TransferPredicateLocksToNewTarget function because it
+ * acquires its own locks on the partitions of the two targets involved,
+ * and we'll already be holding all partition locks.
+ *
+ * We can't throw an error from here, because the call could be from a
+ * transaction which is not serializable.
+ *
+ * NOTE: This is currently only called with transfer set to true, but that may
+ * change. If we decide to clean up the locks from a table on commit of a
+ * transaction which executed DROP TABLE, the false condition will be useful.
+ */
+static void
+DropAllPredicateLocksFromTable(const Relation relation, bool transfer)
+{
+ HASH_SEQ_STATUS seqstat;
+ PREDICATELOCKTARGET *oldtarget;
+ PREDICATELOCKTARGET *heaptarget;
+ Oid dbId;
+ Oid relId;
+ Oid heapId;
+ int i;
+ bool isIndex;
+ bool found;
+ uint32 heaptargettaghash;
+
+ /*
+ * Bail out quickly if there are no serializable transactions running.
+ * It's safe to check this without taking locks because the caller is
+ * holding an ACCESS EXCLUSIVE lock on the relation. No new locks which
+ * would matter here can be acquired while that is held.
+ */
+ if (!TransactionIdIsValid(PredXact->SxactGlobalXmin))
+ return;
+
+ if (SkipPredicateLocksForRelation(relation))
+ return;
+
+ dbId = relation->rd_node.dbNode;
+ relId = relation->rd_id;
+ if (relation->rd_index == NULL)
+ {
+ isIndex = false;
+ heapId = relId;
+ }
+ else
+ {
+ isIndex = true;
+ heapId = relation->rd_index->indrelid;
+ }
+ Assert(heapId != InvalidOid);
+ Assert(transfer || !isIndex); /* index OID only makes sense with
+ * transfer */
+
+ /* Retrieve first time needed, then keep. */
+ heaptargettaghash = 0;
+ heaptarget = NULL;
+
+ /* Acquire locks on all lock partitions */
+ LWLockAcquire(SerializablePredicateLockListLock, LW_EXCLUSIVE);
+ for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
+ LWLockAcquire(FirstPredicateLockMgrLock + i, LW_EXCLUSIVE);
+ LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
+
+ /*
+ * Remove the dummy entry to give us scratch space, so we know we'll be
+ * able to create the new lock target.
+ */
+ if (transfer)
+ RemoveScratchTarget(true);
+
+ /* Scan through target map */
+ hash_seq_init(&seqstat, PredicateLockTargetHash);
+
+ while ((oldtarget = (PREDICATELOCKTARGET *) hash_seq_search(&seqstat)))
+ {
+ PREDICATELOCK *oldpredlock;
+
+ /*
+ * Check whether this is a target which needs attention.
+ */
+ if (GET_PREDICATELOCKTARGETTAG_RELATION(oldtarget->tag) != relId)
+ continue; /* wrong relation id */
+ if (GET_PREDICATELOCKTARGETTAG_DB(oldtarget->tag) != dbId)
+ continue; /* wrong database id */
+ if (transfer && !isIndex
+ && GET_PREDICATELOCKTARGETTAG_TYPE(oldtarget->tag) == PREDLOCKTAG_RELATION)
+ continue; /* already the right lock */
+
+ /*
+ * If we made it here, we have work to do. We make sure the heap
+ * relation lock exists, then we walk the list of predicate locks for
+ * the old target we found, moving all locks to the heap relation lock
+ * -- unless they already hold that.
+ */
+
+ /*
+ * First make sure we have the heap relation target. We only need to
+ * do this once.
+ */
+ if (transfer && heaptarget == NULL)
+ {
+ PREDICATELOCKTARGETTAG heaptargettag;
+
+ SET_PREDICATELOCKTARGETTAG_RELATION(heaptargettag, dbId, heapId);
+ heaptargettaghash = PredicateLockTargetTagHashCode(&heaptargettag);
+ heaptarget = hash_search_with_hash_value(PredicateLockTargetHash,
+ &heaptargettag,
+ heaptargettaghash,
+ HASH_ENTER, &found);
+ if (!found)
+ SHMQueueInit(&heaptarget->predicateLocks);
+ }
+
+ /*
+ * Loop through all the locks on the old target, replacing them with
+ * locks on the new target.
+ */
+ oldpredlock = (PREDICATELOCK *)
+ SHMQueueNext(&(oldtarget->predicateLocks),
+ &(oldtarget->predicateLocks),
+ offsetof(PREDICATELOCK, targetLink));
+ while (oldpredlock)
+ {
+ PREDICATELOCK *nextpredlock;
+ PREDICATELOCK *newpredlock;
+ SerCommitSeqNo oldCommitSeqNo;
+ SERIALIZABLEXACT *oldXact;
+
+ nextpredlock = (PREDICATELOCK *)
+ SHMQueueNext(&(oldtarget->predicateLocks),
+ &(oldpredlock->targetLink),
+ offsetof(PREDICATELOCK, targetLink));
+
+ /*
+ * Remove the old lock first. This avoids the chance of running
+ * out of lock structure entries for the hash table.
+ */
+ oldCommitSeqNo = oldpredlock->commitSeqNo;
+ oldXact = oldpredlock->tag.myXact;
+
+ SHMQueueDelete(&(oldpredlock->xactLink));
+
+ /*
+ * No need for retail delete from oldtarget list, we're removing
+ * the whole target anyway.
+ */
+ hash_search(PredicateLockHash,
+ &oldpredlock->tag,
+ HASH_REMOVE, &found);
+ Assert(found);
+
+ if (transfer)
+ {
+ PREDICATELOCKTAG newpredlocktag;
+
+ newpredlocktag.myTarget = heaptarget;
+ newpredlocktag.myXact = oldXact;
+ newpredlock = (PREDICATELOCK *)
+ hash_search_with_hash_value
+ (PredicateLockHash,
+ &newpredlocktag,
+ PredicateLockHashCodeFromTargetHashCode(&newpredlocktag,
+ heaptargettaghash),
+ HASH_ENTER, &found);
+ if (!found)
+ {
+ SHMQueueInsertBefore(&(heaptarget->predicateLocks),
+ &(newpredlock->targetLink));
+ SHMQueueInsertBefore(&(newpredlocktag.myXact->predicateLocks),
+ &(newpredlock->xactLink));
+ newpredlock->commitSeqNo = oldCommitSeqNo;
+ }
+ else
+ {
+ if (newpredlock->commitSeqNo < oldCommitSeqNo)
+ newpredlock->commitSeqNo = oldCommitSeqNo;
+ }
+
+ Assert(newpredlock->commitSeqNo != 0);
+ Assert((newpredlock->commitSeqNo == InvalidSerCommitSeqNo)
+ || (newpredlock->tag.myXact == OldCommittedSxact));
+ }
+
+ oldpredlock = nextpredlock;
+ }
+
+ hash_search(PredicateLockTargetHash, &oldtarget->tag, HASH_REMOVE,
+ &found);
+ Assert(found);
+ }
+
+ /* Put the scratch entry back */
+ if (transfer)
+ RestoreScratchTarget(true);
+
+ /* Release locks in reverse order */
+ LWLockRelease(SerializableXactHashLock);
+ for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
+ LWLockRelease(FirstPredicateLockMgrLock + i);
+ LWLockRelease(SerializablePredicateLockListLock);
+}
+
+/*
+ * TransferPredicateLocksToHeapRelation
+ * For all transactions, transfer all predicate locks for the given
+ * relation to a single relation lock on the heap.
+ */
+void
+TransferPredicateLocksToHeapRelation(const Relation relation)
+{
+ DropAllPredicateLocksFromTable(relation, true);
+}
+
/*
* PredicateLockPageSplit
@@ -2567,21 +2834,19 @@ PredicateLockPageSplit(const Relation relation, const BlockNumber oldblkno,
bool success;
/*
- * Bail out quickly if there are no serializable transactions
- * running.
+ * Bail out quickly if there are no serializable transactions running.
*
- * It's safe to do this check without taking any additional
- * locks. Even if a serializable transaction starts concurrently,
- * we know it can't take any SIREAD locks on the page being split
- * because the caller is holding the associated buffer page lock.
- * Memory reordering isn't an issue; the memory barrier in the
- * LWLock acquisition guarantees that this read occurs while the
- * buffer page lock is held.
+ * It's safe to do this check without taking any additional locks. Even if
+ * a serializable transaction starts concurrently, we know it can't take
+ * any SIREAD locks on the page being split because the caller is holding
+ * the associated buffer page lock. Memory reordering isn't an issue; the
+ * memory barrier in the LWLock acquisition guarantees that this read
+ * occurs while the buffer page lock is held.
*/
if (!TransactionIdIsValid(PredXact->SxactGlobalXmin))
return;
- if (SkipSplitTracking(relation))
+ if (SkipPredicateLocksForRelation(relation))
return;
Assert(oldblkno != newblkno);
@@ -2764,7 +3029,7 @@ ReleasePredicateLocks(const bool isCommit)
* If this value is changing, we don't care that much whether we get the
* old or new value -- it is just used to determine how far
* GlobalSerizableXmin must advance before this transaction can be fully
- * cleaned up. The worst that could happen is we wait for one more
+ * cleaned up. The worst that could happen is we wait for one more
* transaction to complete before freeing some RAM; correctness of visible
* behavior is not affected.
*/
@@ -3610,15 +3875,14 @@ CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag)
if (sxact == MySerializableXact)
{
/*
- * If we're getting a write lock on a tuple, we don't need
- * a predicate (SIREAD) lock on the same tuple. We can
- * safely remove our SIREAD lock, but we'll defer doing so
- * until after the loop because that requires upgrading to
- * an exclusive partition lock.
+ * If we're getting a write lock on a tuple, we don't need a
+ * predicate (SIREAD) lock on the same tuple. We can safely remove
+ * our SIREAD lock, but we'll defer doing so until after the loop
+ * because that requires upgrading to an exclusive partition lock.
*
- * We can't use this optimization within a subtransaction
- * because the subtransaction could roll back, and we
- * would be left without any lock at the top level.
+ * We can't use this optimization within a subtransaction because
+ * the subtransaction could roll back, and we would be left
+ * without any lock at the top level.
*/
if (!IsSubTransaction()
&& GET_PREDICATELOCKTARGETTAG_OFFSET(*targettag))
@@ -3660,14 +3924,12 @@ CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag)
LWLockRelease(partitionLock);
/*
- * If we found one of our own SIREAD locks to remove, remove it
- * now.
+ * If we found one of our own SIREAD locks to remove, remove it now.
*
- * At this point our transaction already has an ExclusiveRowLock
- * on the relation, so we are OK to drop the predicate lock on the
- * tuple, if found, without fearing that another write against the
- * tuple will occur before the MVCC information makes it to the
- * buffer.
+ * At this point our transaction already has an ExclusiveRowLock on the
+ * relation, so we are OK to drop the predicate lock on the tuple, if
+ * found, without fearing that another write against the tuple will occur
+ * before the MVCC information makes it to the buffer.
*/
if (mypredlock != NULL)
{
@@ -3679,9 +3941,9 @@ CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag)
LWLockAcquire(SerializableXactHashLock, LW_EXCLUSIVE);
/*
- * Remove the predicate lock from shared memory, if it wasn't
- * removed while the locks were released. One way that could
- * happen is from autovacuum cleaning up an index.
+ * Remove the predicate lock from shared memory, if it wasn't removed
+ * while the locks were released. One way that could happen is from
+ * autovacuum cleaning up an index.
*/
predlockhashcode = PredicateLockHashCodeFromTargetHashCode
(&mypredlocktag, targettaghash);
@@ -3710,13 +3972,13 @@ CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag)
LWLockRelease(SerializableXactHashLock);
LWLockRelease(partitionLock);
LWLockRelease(SerializablePredicateLockListLock);
-
+
if (rmpredlock != NULL)
{
/*
- * Remove entry in local lock table if it exists. It's OK
- * if it doesn't exist; that means the lock was
- * transferred to a new target by a different backend.
+ * Remove entry in local lock table if it exists. It's OK if it
+ * doesn't exist; that means the lock was transferred to a new
+ * target by a different backend.
*/
hash_search_with_hash_value(LocalPredicateLockHash,
targettag, targettaghash,
@@ -3792,6 +4054,113 @@ CheckForSerializableConflictIn(const Relation relation, const HeapTuple tuple,
}
/*
+ * CheckTableForSerializableConflictIn
+ * The entire table is going through a DDL-style logical mass delete
+ * like TRUNCATE or DROP TABLE. If that causes a rw-conflict in from
+ * another serializable transaction, take appropriate action.
+ *
+ * While these operations do not operate entirely within the bounds of
+ * snapshot isolation, they can occur inside a serializable transaction, and
+ * will logically occur after any reads which saw rows which were destroyed
+ * by these operations, so we do what we can to serialize properly under
+ * SSI.
+ *
+ * The relation passed in must be a heap relation. Any predicate lock of any
+ * granularity on the heap will cause a rw-conflict in to this transaction.
+ * Predicate locks on indexes do not matter because they only exist to guard
+ * against conflicting inserts into the index, and this is a mass *delete*.
+ * When a table is truncated or dropped, the index will also be truncated
+ * or dropped, and we'll deal with locks on the index when that happens.
+ *
+ * Dropping or truncating a table also needs to drop any existing predicate
+ * locks on heap tuples or pages, because they're about to go away. This
+ * should be done before altering the predicate locks because the transaction
+ * could be rolled back because of a conflict, in which case the lock changes
+ * are not needed. (At the moment, we don't actually bother to drop the
+ * existing locks on a dropped or truncated table at the moment. That might
+ * lead to some false positives, but it doesn't seem worth the trouble.)
+ */
+void
+CheckTableForSerializableConflictIn(const Relation relation)
+{
+ HASH_SEQ_STATUS seqstat;
+ PREDICATELOCKTARGET *target;
+ Oid dbId;
+ Oid heapId;
+ int i;
+
+ /*
+ * Bail out quickly if there are no serializable transactions running.
+ * It's safe to check this without taking locks because the caller is
+ * holding an ACCESS EXCLUSIVE lock on the relation. No new locks which
+ * would matter here can be acquired while that is held.
+ */
+ if (!TransactionIdIsValid(PredXact->SxactGlobalXmin))
+ return;
+
+ if (SkipSerialization(relation))
+ return;
+
+ Assert(relation->rd_index == NULL); /* not an index relation */
+
+ dbId = relation->rd_node.dbNode;
+ heapId = relation->rd_id;
+
+ LWLockAcquire(SerializablePredicateLockListLock, LW_EXCLUSIVE);
+ for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
+ LWLockAcquire(FirstPredicateLockMgrLock + i, LW_SHARED);
+ LWLockAcquire(SerializableXactHashLock, LW_SHARED);
+
+ /* Scan through target list */
+ hash_seq_init(&seqstat, PredicateLockTargetHash);
+
+ while ((target = (PREDICATELOCKTARGET *) hash_seq_search(&seqstat)))
+ {
+ PREDICATELOCK *predlock;
+
+ /*
+ * Check whether this is a target which needs attention.
+ */
+ if (GET_PREDICATELOCKTARGETTAG_RELATION(target->tag) != heapId)
+ continue; /* wrong relation id */
+ if (GET_PREDICATELOCKTARGETTAG_DB(target->tag) != dbId)
+ continue; /* wrong database id */
+
+ /*
+ * Loop through locks for this target and flag conflicts.
+ */
+ predlock = (PREDICATELOCK *)
+ SHMQueueNext(&(target->predicateLocks),
+ &(target->predicateLocks),
+ offsetof(PREDICATELOCK, targetLink));
+ while (predlock)
+ {
+ PREDICATELOCK *nextpredlock;
+
+ nextpredlock = (PREDICATELOCK *)
+ SHMQueueNext(&(target->predicateLocks),
+ &(predlock->targetLink),
+ offsetof(PREDICATELOCK, targetLink));
+
+ if (predlock->tag.myXact != MySerializableXact
+ && !RWConflictExists(predlock->tag.myXact,
+ (SERIALIZABLEXACT *) MySerializableXact))
+ FlagRWConflict(predlock->tag.myXact,
+ (SERIALIZABLEXACT *) MySerializableXact);
+
+ predlock = nextpredlock;
+ }
+ }
+
+ /* Release locks in reverse order */
+ LWLockRelease(SerializableXactHashLock);
+ for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
+ LWLockRelease(FirstPredicateLockMgrLock + i);
+ LWLockRelease(SerializablePredicateLockListLock);
+}
+
+
+/*
* Flag a rw-dependency between two serializable transactions.
*
* The caller is responsible for ensuring that we have a LW lock on
diff --git a/src/include/storage/predicate.h b/src/include/storage/predicate.h
index 77ae8f904d..760c76cff0 100644
--- a/src/include/storage/predicate.h
+++ b/src/include/storage/predicate.h
@@ -49,11 +49,13 @@ extern void PredicateLockPage(const Relation relation, const BlockNumber blkno);
extern void PredicateLockTuple(const Relation relation, const HeapTuple tuple);
extern void PredicateLockPageSplit(const Relation relation, const BlockNumber oldblkno, const BlockNumber newblkno);
extern void PredicateLockPageCombine(const Relation relation, const BlockNumber oldblkno, const BlockNumber newblkno);
+extern void TransferPredicateLocksToHeapRelation(const Relation relation);
extern void ReleasePredicateLocks(const bool isCommit);
/* conflict detection (may also trigger rollback) */
extern void CheckForSerializableConflictOut(const bool valid, const Relation relation, const HeapTuple tuple, const Buffer buffer);
extern void CheckForSerializableConflictIn(const Relation relation, const HeapTuple tuple, const Buffer buffer);
+extern void CheckTableForSerializableConflictIn(const Relation relation);
/* final rollback checking */
extern void PreCommit_CheckForSerializationFailure(void);
diff --git a/src/include/storage/predicate_internals.h b/src/include/storage/predicate_internals.h
index b144ab319a..56a01f0b91 100644
--- a/src/include/storage/predicate_internals.h
+++ b/src/include/storage/predicate_internals.h
@@ -273,9 +273,7 @@ typedef struct PREDICATELOCKTARGETTAG
* up the targets as the related tuples are pruned or vacuumed, we check the
* xmin on access. This should be far less costly.
*/
-typedef struct PREDICATELOCKTARGET PREDICATELOCKTARGET;
-
-struct PREDICATELOCKTARGET
+typedef struct PREDICATELOCKTARGET
{
/* hash key */
PREDICATELOCKTARGETTAG tag; /* unique identifier of lockable object */
@@ -283,7 +281,7 @@ struct PREDICATELOCKTARGET
/* data */
SHM_QUEUE predicateLocks; /* list of PREDICATELOCK objects assoc. with
* predicate lock target */
-};
+} PREDICATELOCKTARGET;
/*