*** a/src/backend/storage/lmgr/predicate.c
--- b/src/backend/storage/lmgr/predicate.c
***************
*** 124,133 ****
   *	SerializableXactHashLock
   *		- Protects both PredXact and SerializableXidHash.
   *
-  *	PredicateLockNextRowLinkLock
-  *		- Protects the priorVersionOfRow and nextVersionOfRow fields of
-  *			PREDICATELOCKTARGET when linkage is being created or destroyed.
-  *
   *
   * Portions Copyright (c) 1996-2011, PostgreSQL Global Development Group
   * Portions Copyright (c) 1994, Regents of the University of California
--- 124,129 ----
***************
*** 444,451 **** static void ReleaseOneSerializableXact(SERIALIZABLEXACT *sxact, bool partial,
  						   bool summarize);
  static bool XidIsConcurrent(TransactionId xid);
  static void CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag);
- static bool CheckSingleTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag,
- 						  PREDICATELOCKTARGETTAG *nexttargettag);
  static void FlagRWConflict(SERIALIZABLEXACT *reader, SERIALIZABLEXACT *writer);
  static void OnConflict_CheckForSerializationFailure(const SERIALIZABLEXACT *reader,
  										SERIALIZABLEXACT *writer);
--- 440,445 ----
***************
*** 1044,1050 **** InitPredicateLocks(void)
  		PredXact->LastSxactCommitSeqNo = FirstNormalSerCommitSeqNo - 1;
  		PredXact->CanPartialClearThrough = 0;
  		PredXact->HavePartialClearedThrough = 0;
- 		PredXact->NeedTargetLinkCleanup = false;
  		requestSize = mul_size((Size) max_table_size,
  							   PredXactListElementDataSize);
  		PredXact->element = ShmemAlloc(requestSize);
--- 1038,1043 ----
***************
*** 1747,1753 **** static void
  RemoveTargetIfNoLongerUsed(PREDICATELOCKTARGET *target, uint32 targettaghash)
  {
  	PREDICATELOCKTARGET *rmtarget;
- 	PREDICATELOCKTARGET *next;
  
  	Assert(LWLockHeldByMe(SerializablePredicateLockListLock));
  
--- 1740,1745 ----
***************
*** 1755,1787 **** RemoveTargetIfNoLongerUsed(PREDICATELOCKTARGET *target, uint32 targettaghash)
  	if (!SHMQueueEmpty(&target->predicateLocks))
  		return;
  
- 	/* Can't remove it if there are locks for a prior row version. */
- 	LWLockAcquire(PredicateLockNextRowLinkLock, LW_EXCLUSIVE);
- 	if (target->priorVersionOfRow != NULL)
- 	{
- 		LWLockRelease(PredicateLockNextRowLinkLock);
- 		return;
- 	}
- 
- 	/*
- 	 * We are going to release this target,  This requires that we let the
- 	 * next version of the row (if any) know that it's previous version is
- 	 * done.
- 	 *
- 	 * It might be that the link was all that was keeping the other target
- 	 * from cleanup, but we can't clean that up here -- LW locking is all
- 	 * wrong for that.	We'll pass the HTAB in the general cleanup function to
- 	 * get rid of such "dead" targets.
- 	 */
- 	next = target->nextVersionOfRow;
- 	if (next != NULL)
- 	{
- 		next->priorVersionOfRow = NULL;
- 		if (SHMQueueEmpty(&next->predicateLocks))
- 			PredXact->NeedTargetLinkCleanup = true;
- 	}
- 	LWLockRelease(PredicateLockNextRowLinkLock);
- 
  	/* Actually remove the target. */
  	rmtarget = hash_search_with_hash_value(PredicateLockTargetHash,
  										   &target->tag,
--- 1747,1752 ----
***************
*** 2065,2075 **** CreatePredicateLock(const PREDICATELOCKTARGETTAG *targettag,
  				 errmsg("out of shared memory"),
  				 errhint("You might need to increase max_pred_locks_per_transaction.")));
  	if (!found)
- 	{
  		SHMQueueInit(&(target->predicateLocks));
- 		target->priorVersionOfRow = NULL;
- 		target->nextVersionOfRow = NULL;
- 	}
  
  	/* We've got the sxact and target, make sure they're joined. */
  	locktag.myTarget = target;
--- 2030,2036 ----
***************
*** 2215,2220 **** PredicateLockTuple(const Relation relation, const HeapTuple tuple)
--- 2176,2182 ----
  {
  	PREDICATELOCKTARGETTAG tag;
  	ItemPointer tid;
+ 	TransactionId targetxmin;
  
  	if (SkipSerialization(relation))
  		return;
***************
*** 2224,2238 **** PredicateLockTuple(const Relation relation, const HeapTuple tuple)
  	 */
  	if (relation->rd_index == NULL)
  	{
! 		TransactionId myxid = GetTopTransactionIdIfAny();
  
  		if (TransactionIdIsValid(myxid))
  		{
! 			TransactionId xid = HeapTupleHeaderGetXmin(tuple->t_data);
! 
! 			if (TransactionIdFollowsOrEquals(xid, TransactionXmin))
  			{
! 				xid = SubTransGetTopmostTransaction(xid);
  				if (TransactionIdEquals(xid, myxid))
  				{
  					/* We wrote it; we already have a write lock. */
--- 2186,2201 ----
  	 */
  	if (relation->rd_index == NULL)
  	{
! 		TransactionId	myxid;
  
+ 		targetxmin = HeapTupleHeaderGetXmin(tuple->t_data);
+ 
+ 		myxid = GetTopTransactionIdIfAny();
  		if (TransactionIdIsValid(myxid))
  		{
! 			if (TransactionIdFollowsOrEquals(targetxmin, TransactionXmin))
  			{
! 				TransactionId xid = SubTransGetTopmostTransaction(targetxmin);
  				if (TransactionIdEquals(xid, myxid))
  				{
  					/* We wrote it; we already have a write lock. */
***************
*** 2241,2246 **** PredicateLockTuple(const Relation relation, const HeapTuple tuple)
--- 2204,2211 ----
  			}
  		}
  	}
+ 	else
+ 		targetxmin = InvalidTransactionId;
  
  	/*
  	 * Do quick-but-not-definitive test for a relation lock first.	This will
***************
*** 2259,2374 **** PredicateLockTuple(const Relation relation, const HeapTuple tuple)
  									 relation->rd_node.dbNode,
  									 relation->rd_id,
  									 ItemPointerGetBlockNumber(tid),
! 									 ItemPointerGetOffsetNumber(tid));
  	PredicateLockAcquire(&tag);
  }
  
  /*
!  * If the old tuple has any predicate locks, create a lock target for the
!  * new tuple and point them at each other.	Conflict detection needs to
!  * look for locks against prior versions of the row.
   */
  void
  PredicateLockTupleRowVersionLink(const Relation relation,
  								 const HeapTuple oldTuple,
  								 const HeapTuple newTuple)
  {
! 	PREDICATELOCKTARGETTAG oldtargettag;
  	PREDICATELOCKTARGETTAG newtargettag;
- 	PREDICATELOCKTARGET *oldtarget;
- 	PREDICATELOCKTARGET *newtarget;
- 	PREDICATELOCKTARGET *next;
- 	uint32		oldtargettaghash;
- 	LWLockId	oldpartitionLock;
- 	uint32		newtargettaghash;
- 	LWLockId	newpartitionLock;
- 	bool		found;
  
! 	SET_PREDICATELOCKTARGETTAG_TUPLE(oldtargettag,
  									 relation->rd_node.dbNode,
  									 relation->rd_id,
  							  ItemPointerGetBlockNumber(&(oldTuple->t_self)),
! 							ItemPointerGetOffsetNumber(&(oldTuple->t_self)));
! 	oldtargettaghash = PredicateLockTargetTagHashCode(&oldtargettag);
! 	oldpartitionLock = PredicateLockHashPartitionLock(oldtargettaghash);
  
  	SET_PREDICATELOCKTARGETTAG_TUPLE(newtargettag,
  									 relation->rd_node.dbNode,
  									 relation->rd_id,
  							  ItemPointerGetBlockNumber(&(newTuple->t_self)),
! 							ItemPointerGetOffsetNumber(&(newTuple->t_self)));
! 	newtargettaghash = PredicateLockTargetTagHashCode(&newtargettag);
! 	newpartitionLock = PredicateLockHashPartitionLock(newtargettaghash);
  
! 	/* Lock lower numbered partition first. */
! 	if (oldpartitionLock < newpartitionLock)
! 	{
! 		LWLockAcquire(oldpartitionLock, LW_SHARED);
! 		LWLockAcquire(newpartitionLock, LW_EXCLUSIVE);
! 	}
! 	else if (newpartitionLock < oldpartitionLock)
! 	{
! 		LWLockAcquire(newpartitionLock, LW_EXCLUSIVE);
! 		LWLockAcquire(oldpartitionLock, LW_SHARED);
! 	}
! 	else
! 		LWLockAcquire(newpartitionLock, LW_EXCLUSIVE);
  
! 	oldtarget = (PREDICATELOCKTARGET *)
! 		hash_search_with_hash_value(PredicateLockTargetHash,
! 									&oldtargettag, oldtargettaghash,
! 									HASH_FIND, NULL);
  
! 	/* Only need to link if there is an old target already. */
! 	if (oldtarget)
! 	{
! 		LWLockAcquire(PredicateLockNextRowLinkLock, LW_EXCLUSIVE);
! 
! 		/* Guard against stale pointers from rollback. */
! 		next = oldtarget->nextVersionOfRow;
! 		if (next != NULL)
! 		{
! 			next->priorVersionOfRow = NULL;
! 			oldtarget->nextVersionOfRow = NULL;
! 		}
! 
! 		/* Find or create the new target, and link old and new. */
! 		newtarget = (PREDICATELOCKTARGET *)
! 			hash_search_with_hash_value(PredicateLockTargetHash,
! 										&newtargettag, newtargettaghash,
! 										HASH_ENTER, &found);
! 		if (!newtarget)
! 			ereport(ERROR,
! 					(errcode(ERRCODE_OUT_OF_MEMORY),
! 					 errmsg("out of shared memory"),
! 					 errhint("You might need to increase max_pred_locks_per_transaction.")));
! 		if (!found)
! 		{
! 			SHMQueueInit(&(newtarget->predicateLocks));
! 			newtarget->nextVersionOfRow = NULL;
! 		}
! 		else
! 			Assert(newtarget->priorVersionOfRow == NULL);
! 
! 		newtarget->priorVersionOfRow = oldtarget;
! 		oldtarget->nextVersionOfRow = newtarget;
! 
! 		LWLockRelease(PredicateLockNextRowLinkLock);
! 	}
! 
! 	/* Release lower number partition last. */
! 	if (oldpartitionLock < newpartitionLock)
! 	{
! 		LWLockRelease(newpartitionLock);
! 		LWLockRelease(oldpartitionLock);
! 	}
! 	else if (newpartitionLock < oldpartitionLock)
! 	{
! 		LWLockRelease(oldpartitionLock);
! 		LWLockRelease(newpartitionLock);
! 	}
! 	else
! 		LWLockRelease(newpartitionLock);
  }
  
  
--- 2224,2271 ----
  									 relation->rd_node.dbNode,
  									 relation->rd_id,
  									 ItemPointerGetBlockNumber(tid),
! 									 ItemPointerGetOffsetNumber(tid),
! 									 targetxmin);
  	PredicateLockAcquire(&tag);
  }
  
  /*
!  * If the old tuple has any predicate locks, copy them to the new target.
   */
  void
  PredicateLockTupleRowVersionLink(const Relation relation,
  								 const HeapTuple oldTuple,
  								 const HeapTuple newTuple)
  {
! 	PREDICATELOCKTARGETTAG oldtupletargettag;
! 	PREDICATELOCKTARGETTAG oldpagetargettag;
  	PREDICATELOCKTARGETTAG newtargettag;
  
! 	SET_PREDICATELOCKTARGETTAG_TUPLE(oldtupletargettag,
  									 relation->rd_node.dbNode,
  									 relation->rd_id,
  							  ItemPointerGetBlockNumber(&(oldTuple->t_self)),
! 							ItemPointerGetOffsetNumber(&(oldTuple->t_self)),
! 									 HeapTupleHeaderGetXmin(oldTuple->t_data));
! 
! 	SET_PREDICATELOCKTARGETTAG_PAGE(oldpagetargettag,
! 									relation->rd_node.dbNode,
! 									relation->rd_id,
! 							  ItemPointerGetBlockNumber(&(oldTuple->t_self)));
  
  	SET_PREDICATELOCKTARGETTAG_TUPLE(newtargettag,
  									 relation->rd_node.dbNode,
  									 relation->rd_id,
  							  ItemPointerGetBlockNumber(&(newTuple->t_self)),
! 							ItemPointerGetOffsetNumber(&(newTuple->t_self)),
! 									 HeapTupleHeaderGetXmin(newTuple->t_data));
  
! 	LWLockAcquire(SerializablePredicateLockListLock, LW_EXCLUSIVE);
  
! 	TransferPredicateLocksToNewTarget(oldtupletargettag, newtargettag, false);
! 	TransferPredicateLocksToNewTarget(oldpagetargettag, newtargettag, false);
  
! 	LWLockRelease(SerializablePredicateLockListLock);
  }
  
  
***************
*** 2533,2543 **** TransferPredicateLocksToNewTarget(const PREDICATELOCKTARGETTAG oldtargettag,
  
  		/* If we created a new entry, initialize it */
  		if (!found)
- 		{
  			SHMQueueInit(&(newtarget->predicateLocks));
- 			newtarget->priorVersionOfRow = NULL;
- 			newtarget->nextVersionOfRow = NULL;
- 		}
  
  		newpredlocktag.myTarget = newtarget;
  
--- 2430,2436 ----
***************
*** 3132,3140 **** ClearOldPredicateLocks(void)
  {
  	SERIALIZABLEXACT *finishedSxact;
  	PREDICATELOCK *predlock;
- 	int			i;
- 	HASH_SEQ_STATUS seqstat;
- 	PREDICATELOCKTARGET *locktarget;
  
  	LWLockAcquire(SerializableFinishedListLock, LW_EXCLUSIVE);
  	finishedSxact = (SERIALIZABLEXACT *)
--- 3025,3030 ----
***************
*** 3232,3266 **** ClearOldPredicateLocks(void)
  
  	LWLockRelease(SerializablePredicateLockListLock);
  	LWLockRelease(SerializableFinishedListLock);
- 
- 	if (!PredXact->NeedTargetLinkCleanup)
- 		return;
- 
- 	/*
- 	 * Clean up any targets which were disconnected from a prior version with
- 	 * no predicate locks attached.
- 	 */
- 	for (i = 0; i < NUM_PREDICATELOCK_PARTITIONS; i++)
- 		LWLockAcquire(FirstPredicateLockMgrLock + i, LW_EXCLUSIVE);
- 	LWLockAcquire(PredicateLockNextRowLinkLock, LW_SHARED);
- 
- 	hash_seq_init(&seqstat, PredicateLockTargetHash);
- 	while ((locktarget = (PREDICATELOCKTARGET *) hash_seq_search(&seqstat)))
- 	{
- 		if (SHMQueueEmpty(&locktarget->predicateLocks)
- 			&& locktarget->priorVersionOfRow == NULL
- 			&& locktarget->nextVersionOfRow == NULL)
- 		{
- 			hash_search(PredicateLockTargetHash, &locktarget->tag,
- 						HASH_REMOVE, NULL);
- 		}
- 	}
- 
- 	PredXact->NeedTargetLinkCleanup = false;
- 
- 	LWLockRelease(PredicateLockNextRowLinkLock);
- 	for (i = NUM_PREDICATELOCK_PARTITIONS - 1; i >= 0; i--)
- 		LWLockRelease(FirstPredicateLockMgrLock + i);
  }
  
  /*
--- 3122,3127 ----
***************
*** 3682,3713 **** CheckForSerializableConflictOut(const bool visible, const Relation relation,
  static void
  CheckTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag)
  {
- 	PREDICATELOCKTARGETTAG nexttargettag = { 0 };
- 	PREDICATELOCKTARGETTAG thistargettag;
- 
- 	for (;;)
- 	{
- 		if (!CheckSingleTargetForConflictsIn(targettag, &nexttargettag))
- 			break;
- 		thistargettag = nexttargettag;
- 		targettag = &thistargettag;
- 	}
- }
- 
- /*
-  * Check a particular target for rw-dependency conflict in. If the tuple
-  * has prior versions, returns true and *nexttargettag is set to the tag
-  * of the prior tuple version.
-  */
- static bool
- CheckSingleTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag,
- 								PREDICATELOCKTARGETTAG *nexttargettag)
- {
  	uint32		targettaghash;
  	LWLockId	partitionLock;
  	PREDICATELOCKTARGET *target;
  	PREDICATELOCK *predlock;
- 	bool		hasnexttarget = false;
  
  	Assert(MySerializableXact != InvalidSerializableXact);
  
--- 3543,3552 ----
***************
*** 3717,3723 **** CheckSingleTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag,
  	targettaghash = PredicateLockTargetTagHashCode(targettag);
  	partitionLock = PredicateLockHashPartitionLock(targettaghash);
  	LWLockAcquire(partitionLock, LW_SHARED);
- 	LWLockAcquire(PredicateLockNextRowLinkLock, LW_SHARED);
  	target = (PREDICATELOCKTARGET *)
  		hash_search_with_hash_value(PredicateLockTargetHash,
  									targettag, targettaghash,
--- 3556,3561 ----
***************
*** 3725,3747 **** CheckSingleTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag,
  	if (!target)
  	{
  		/* Nothing has this target locked; we're done here. */
- 		LWLockRelease(PredicateLockNextRowLinkLock);
  		LWLockRelease(partitionLock);
! 		return false;
  	}
  
  	/*
- 	 * If the target is linked to a prior version of the row, save the tag so
- 	 * that it can be used for iterative calls to this function.
- 	 */
- 	if (target->priorVersionOfRow != NULL)
- 	{
- 		*nexttargettag = target->priorVersionOfRow->tag;
- 		hasnexttarget = true;
- 	}
- 	LWLockRelease(PredicateLockNextRowLinkLock);
- 
- 	/*
  	 * Each lock for an overlapping transaction represents a conflict: a
  	 * rw-dependency in to this transaction.
  	 */
--- 3563,3573 ----
  	if (!target)
  	{
  		/* Nothing has this target locked; we're done here. */
  		LWLockRelease(partitionLock);
! 		return;
  	}
  
  	/*
  	 * Each lock for an overlapping transaction represents a conflict: a
  	 * rw-dependency in to this transaction.
  	 */
***************
*** 3848,3854 **** CheckSingleTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag,
  					 * the target, bail out before re-acquiring the locks.
  					 */
  					if (rmtarget)
! 						return hasnexttarget;
  
  					/*
  					 * The list has been altered.  Start over at the front.
--- 3674,3680 ----
  					 * the target, bail out before re-acquiring the locks.
  					 */
  					if (rmtarget)
! 						return;
  
  					/*
  					 * The list has been altered.  Start over at the front.
***************
*** 3895,3902 **** CheckSingleTargetForConflictsIn(PREDICATELOCKTARGETTAG *targettag,
  	}
  	LWLockRelease(SerializableXactHashLock);
  	LWLockRelease(partitionLock);
- 
- 	return hasnexttarget;
  }
  
  /*
--- 3721,3726 ----
***************
*** 3943,3949 **** CheckForSerializableConflictIn(const Relation relation, const HeapTuple tuple,
  										 relation->rd_node.dbNode,
  										 relation->rd_id,
  						 ItemPointerGetBlockNumber(&(tuple->t_data->t_ctid)),
! 					   ItemPointerGetOffsetNumber(&(tuple->t_data->t_ctid)));
  		CheckTargetForConflictsIn(&targettag);
  	}
  
--- 3767,3774 ----
  										 relation->rd_node.dbNode,
  										 relation->rd_id,
  						 ItemPointerGetBlockNumber(&(tuple->t_data->t_ctid)),
! 					   ItemPointerGetOffsetNumber(&(tuple->t_data->t_ctid)),
! 					   HeapTupleHeaderGetXmin(tuple->t_data));
  		CheckTargetForConflictsIn(&targettag);
  	}
  
*** a/src/include/storage/lwlock.h
--- b/src/include/storage/lwlock.h
***************
*** 78,84 **** typedef enum LWLockId
  	SerializableFinishedListLock,
  	SerializablePredicateLockListLock,
  	OldSerXidLock,
- 	PredicateLockNextRowLinkLock,
  	/* Individual lock IDs end here */
  	FirstBufMappingLock,
  	FirstLockMgrLock = FirstBufMappingLock + NUM_BUFFER_PARTITIONS,
--- 78,83 ----
*** a/src/include/storage/predicate_internals.h
--- b/src/include/storage/predicate_internals.h
***************
*** 150,157 **** typedef struct PredXactListData
  	SerCommitSeqNo HavePartialClearedThrough;	/* have cleared through this
  												 * seq no */
  	SERIALIZABLEXACT *OldCommittedSxact;		/* shared copy of dummy sxact */
- 	bool		NeedTargetLinkCleanup;	/* to save cleanup effort for rare
- 										 * case */
  
  	PredXactListElement element;
  } PredXactListData;
--- 150,155 ----
***************
*** 231,239 **** typedef struct SERIALIZABLEXID
  
  /*
   * The PREDICATELOCKTARGETTAG struct identifies a database object which can
!  * be the target of predicate locks.  It is designed to fit into 16 bytes
!  * with no padding.  Note that this would need adjustment if we widen Oid or
!  * BlockNumber to more than 32 bits.
   *
   * TODO SSI: If we always use the same fields for the same type of value, we
   * should rename these.  Holding off until it's clear there are no exceptions.
--- 229,241 ----
  
  /*
   * The PREDICATELOCKTARGETTAG struct identifies a database object which can
!  * be the target of predicate locks.
!  *
!  * locktag_field6 was added to allow slack space to be filled so that stack-
!  * allocated tags wouldn't have random bytes in the middle.  Moving the only
!  * uint16 to the end didn't work, because the hash function being used
!  * doesn't properly respect tag length -- it will go to a four byte boundary
!  * past the end of the tag.
   *
   * TODO SSI: If we always use the same fields for the same type of value, we
   * should rename these.  Holding off until it's clear there are no exceptions.
***************
*** 247,254 **** typedef struct PREDICATELOCKTARGETTAG
  	uint32		locktag_field1; /* a 32-bit ID field */
  	uint32		locktag_field2; /* a 32-bit ID field */
  	uint32		locktag_field3; /* a 32-bit ID field */
  	uint16		locktag_field4; /* a 16-bit ID field */
! 	uint16		locktag_field5; /* a 16-bit ID field */
  } PREDICATELOCKTARGETTAG;
  
  /*
--- 249,257 ----
  	uint32		locktag_field1; /* a 32-bit ID field */
  	uint32		locktag_field2; /* a 32-bit ID field */
  	uint32		locktag_field3; /* a 32-bit ID field */
+ 	uint32		locktag_field5; /* a 32-bit ID field */
  	uint16		locktag_field4; /* a 16-bit ID field */
! 	uint16		locktag_field6; /* filler */
  } PREDICATELOCKTARGETTAG;
  
  /*
***************
*** 260,271 **** typedef struct PREDICATELOCKTARGETTAG
   * already have one.  An entry is removed when the last lock is removed from
   * its list.
   *
!  * Because a check for predicate locks on a tuple target should also find
!  * locks on previous versions of the same row, if there are any created by
!  * overlapping transactions, we keep a pointer to the target for the prior
!  * version of the row.	We also keep a pointer to the next version of the
!  * row, so that when we no longer have any predicate locks and the back
!  * pointer is clear, we can clean up the prior pointer for the next version.
   */
  typedef struct PREDICATELOCKTARGET PREDICATELOCKTARGET;
  
--- 263,273 ----
   * already have one.  An entry is removed when the last lock is removed from
   * its list.
   *
!  * Because a particular target might become obsolete, due to update to a new
!  * version, before the reading transaction is obsolete, we need some way to
!  * prevent errors from reuse of a tuple ID.  Rather than attempting to clean
!  * 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;
  
***************
*** 277,291 **** struct PREDICATELOCKTARGET
  	/* data */
  	SHM_QUEUE	predicateLocks; /* list of PREDICATELOCK objects assoc. with
  								 * predicate lock target */
- 
- 	/*
- 	 * The following two pointers are only used for tuple locks, and are only
- 	 * consulted for conflict detection and cleanup; not for granularity
- 	 * promotion.
- 	 */
- 	PREDICATELOCKTARGET *priorVersionOfRow;		/* what other locks to check */
- 	PREDICATELOCKTARGET *nextVersionOfRow;		/* who has pointer here for
- 												 * more targets */
  };
  
  
--- 279,284 ----
***************
*** 387,407 **** typedef struct PredicateLockData
  	 (locktag).locktag_field2 = (reloid), \
  	 (locktag).locktag_field3 = InvalidBlockNumber, \
  	 (locktag).locktag_field4 = InvalidOffsetNumber, \
! 	 (locktag).locktag_field5 = 0)
  
  #define SET_PREDICATELOCKTARGETTAG_PAGE(locktag,dboid,reloid,blocknum) \
  	((locktag).locktag_field1 = (dboid), \
  	 (locktag).locktag_field2 = (reloid), \
  	 (locktag).locktag_field3 = (blocknum), \
  	 (locktag).locktag_field4 = InvalidOffsetNumber, \
! 	 (locktag).locktag_field5 = 0)
  
! #define SET_PREDICATELOCKTARGETTAG_TUPLE(locktag,dboid,reloid,blocknum,offnum) \
  	((locktag).locktag_field1 = (dboid), \
  	 (locktag).locktag_field2 = (reloid), \
  	 (locktag).locktag_field3 = (blocknum), \
  	 (locktag).locktag_field4 = (offnum), \
! 	 (locktag).locktag_field5 = 0)
  
  #define GET_PREDICATELOCKTARGETTAG_DB(locktag) \
  	((locktag).locktag_field1)
--- 380,403 ----
  	 (locktag).locktag_field2 = (reloid), \
  	 (locktag).locktag_field3 = InvalidBlockNumber, \
  	 (locktag).locktag_field4 = InvalidOffsetNumber, \
! 	 (locktag).locktag_field5 = InvalidTransactionId, \
! 	 (locktag).locktag_field6 = 0)
  
  #define SET_PREDICATELOCKTARGETTAG_PAGE(locktag,dboid,reloid,blocknum) \
  	((locktag).locktag_field1 = (dboid), \
  	 (locktag).locktag_field2 = (reloid), \
  	 (locktag).locktag_field3 = (blocknum), \
  	 (locktag).locktag_field4 = InvalidOffsetNumber, \
! 	 (locktag).locktag_field5 = InvalidTransactionId, \
! 	 (locktag).locktag_field6 = 0)
  
! #define SET_PREDICATELOCKTARGETTAG_TUPLE(locktag,dboid,reloid,blocknum,offnum,xmin) \
  	((locktag).locktag_field1 = (dboid), \
  	 (locktag).locktag_field2 = (reloid), \
  	 (locktag).locktag_field3 = (blocknum), \
  	 (locktag).locktag_field4 = (offnum), \
! 	 (locktag).locktag_field5 = (xmin), \
! 	 (locktag).locktag_field6 = 0)
  
  #define GET_PREDICATELOCKTARGETTAG_DB(locktag) \
  	((locktag).locktag_field1)
***************
*** 411,416 **** typedef struct PredicateLockData
--- 407,414 ----
  	((locktag).locktag_field3)
  #define GET_PREDICATELOCKTARGETTAG_OFFSET(locktag) \
  	((locktag).locktag_field4)
+ #define GET_PREDICATELOCKTARGETTAG_XMIN(locktag) \
+ 	((locktag).locktag_field5)
  #define GET_PREDICATELOCKTARGETTAG_TYPE(locktag)							 \
  	(((locktag).locktag_field4 != InvalidOffsetNumber) ? PREDLOCKTAG_TUPLE : \
  	 (((locktag).locktag_field3 != InvalidBlockNumber) ? PREDLOCKTAG_PAGE :   \
