Re: Transactions and temp tables

Поиск
Список
Период
Сортировка
От Heikki Linnakangas
Тема Re: Transactions and temp tables
Дата
Msg-id 49232993.1050801@enterprisedb.com
обсуждение исходный текст
Ответ на Re: Transactions and temp tables  (Heikki Linnakangas <heikki.linnakangas@enterprisedb.com>)
Ответы Re: Transactions and temp tables  (Emmanuel Cecchet <manu@frogthinker.org>)
Список pgsql-hackers
Heikki Linnakangas wrote:
> Somehow this feels pretty baroque, though. Perhaps a better approach
> would be to add a new AtPrepare_OnCommitActions function to tablecmds.c,
> that gets called before AtPrepare_Locks. It would scan through the
> on_commits list, and release all locks for the "PREPARE-safe" temp
> tables, and throw the error if necessary. I'll try that next.

Here's what I ended up with. I morphed the on commit action registration
into tracking of all temporary relations.

This only allows access to ON COMMIT DELETE ROWS temp tables. Accessing
other temporary tables, and creating or dropping tables in the
transaction is still forbidden.

It took me a couple of iterations to handle toast tables and indexes
correctly. More testing would be appreciated with more complex cases
like VACUUM FULL, subtransactions etc.

--
   Heikki Linnakangas
   EnterpriseDB   http://www.enterprisedb.com
*** src/backend/access/heap/heapam.c
--- src/backend/access/heap/heapam.c
***************
*** 51,56 ****
--- 51,57 ----
  #include "access/xlogutils.h"
  #include "catalog/catalog.h"
  #include "catalog/namespace.h"
+ #include "commands/tablecmds.h"
  #include "miscadmin.h"
  #include "pgstat.h"
  #include "storage/bufmgr.h"
***************
*** 878,884 **** relation_open(Oid relationId, LOCKMODE lockmode)

      /* Make note that we've accessed a temporary relation */
      if (r->rd_istemp)
!         MyXactAccessedTempRel = true;

      pgstat_initstats(r);

--- 879,885 ----

      /* Make note that we've accessed a temporary relation */
      if (r->rd_istemp)
!         register_temp_rel_access(relationId);

      pgstat_initstats(r);

***************
*** 926,932 **** try_relation_open(Oid relationId, LOCKMODE lockmode)

      /* Make note that we've accessed a temporary relation */
      if (r->rd_istemp)
!         MyXactAccessedTempRel = true;

      pgstat_initstats(r);

--- 927,933 ----

      /* Make note that we've accessed a temporary relation */
      if (r->rd_istemp)
!         register_temp_rel_access(relationId);

      pgstat_initstats(r);

***************
*** 976,982 **** relation_open_nowait(Oid relationId, LOCKMODE lockmode)

      /* Make note that we've accessed a temporary relation */
      if (r->rd_istemp)
!         MyXactAccessedTempRel = true;

      pgstat_initstats(r);

--- 977,983 ----

      /* Make note that we've accessed a temporary relation */
      if (r->rd_istemp)
!         register_temp_rel_access(relationId);

      pgstat_initstats(r);

*** src/backend/access/transam/xact.c
--- src/backend/access/transam/xact.c
***************
*** 67,80 **** int            CommitDelay = 0;    /* precommit delay in microseconds */
  int            CommitSiblings = 5; /* # concurrent xacts needed to sleep */

  /*
-  * MyXactAccessedTempRel is set when a temporary relation is accessed.
-  * We don't allow PREPARE TRANSACTION in that case.  (This is global
-  * so that it can be set from heapam.c.)
-  */
- bool        MyXactAccessedTempRel = false;
-
-
- /*
   *    transaction states - transaction state from server perspective
   */
  typedef enum TransState
--- 67,72 ----
***************
*** 1467,1473 **** StartTransaction(void)
      XactIsoLevel = DefaultXactIsoLevel;
      XactReadOnly = DefaultXactReadOnly;
      forceSyncCommit = false;
-     MyXactAccessedTempRel = false;

      /*
       * reinitialize within-transaction counters
--- 1459,1464 ----
***************
*** 1798,1823 **** PrepareTransaction(void)

      /* NOTIFY and flatfiles will be handled below */

-     /*
-      * Don't allow PREPARE TRANSACTION if we've accessed a temporary table
-      * in this transaction.  Having the prepared xact hold locks on another
-      * backend's temp table seems a bad idea --- for instance it would prevent
-      * the backend from exiting.  There are other problems too, such as how
-      * to clean up the source backend's local buffers and ON COMMIT state
-      * if the prepared xact includes a DROP of a temp table.
-      *
-      * We must check this after executing any ON COMMIT actions, because
-      * they might still access a temp relation.
-      *
-      * XXX In principle this could be relaxed to allow some useful special
-      * cases, such as a temp table created and dropped all within the
-      * transaction.  That seems to require much more bookkeeping though.
-      */
-     if (MyXactAccessedTempRel)
-         ereport(ERROR,
-                 (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-                  errmsg("cannot PREPARE a transaction that has operated on temporary tables")));
-
      /* Prevent cancel/die interrupt while cleaning up */
      HOLD_INTERRUPTS();

--- 1789,1794 ----
***************
*** 1861,1866 **** PrepareTransaction(void)
--- 1832,1838 ----
      AtPrepare_Notify();
      AtPrepare_UpdateFlatFiles();
      AtPrepare_Inval();
+     AtPrepare_on_commit_actions();
      AtPrepare_Locks();
      AtPrepare_PgStat();

*** src/backend/catalog/heap.c
--- src/backend/catalog/heap.c
***************
*** 39,44 ****
--- 39,45 ----
  #include "catalog/heap.h"
  #include "catalog/index.h"
  #include "catalog/indexing.h"
+ #include "catalog/namespace.h"
  #include "catalog/pg_attrdef.h"
  #include "catalog/pg_constraint.h"
  #include "catalog/pg_inherits.h"
***************
*** 1062,1071 **** heap_create_with_catalog(const char *relname,
      StoreConstraints(new_rel_desc, cooked_constraints);

      /*
!      * If there's a special on-commit action, remember it
       */
!     if (oncommit != ONCOMMIT_NOOP)
!         register_on_commit_action(relid, oncommit);

      /*
       * ok, the relation has been cataloged, so close our relations and return
--- 1063,1072 ----
      StoreConstraints(new_rel_desc, cooked_constraints);

      /*
!      * Register to the on commit action handler, if it's a temporary table.
       */
!     if (isTempOrToastNamespace(relnamespace))
!         register_temp_rel(relid, relkind, oncommit);

      /*
       * ok, the relation has been cataloged, so close our relations and return
***************
*** 1443,1451 **** heap_drop_with_catalog(Oid relid)
      relation_close(rel, NoLock);

      /*
!      * Forget any ON COMMIT action for the rel
       */
!     remove_on_commit_action(relid);

      /*
       * Flush the relation from the relcache.  We want to do this before
--- 1444,1452 ----
      relation_close(rel, NoLock);

      /*
!      * Unregister any ON COMMIT action for the relation.
       */
!     unregister_temp_rel(relid);

      /*
       * Flush the relation from the relcache.  We want to do this before
*** src/backend/catalog/index.c
--- src/backend/catalog/index.c
***************
*** 873,878 **** index_create(Oid heapRelationId,
--- 873,886 ----
      }

      /*
+      * Register to the on commit action handler, if it's a temporary table.
+      * Indexes can't have any ON COMMIT actions, but the registration is
+      * still needed to handle PREPARE TRANSACTION correctly.
+      */
+     if (isTempOrToastNamespace(namespaceId))
+         register_temp_rel(indexRelationId, RELKIND_INDEX, ONCOMMIT_NOOP);
+
+     /*
       * Close the heap and index; but we keep the locks that we acquired above
       * until end of transaction.
       */
***************
*** 953,958 **** index_drop(Oid indexId)
--- 961,971 ----
      heap_close(indexRelation, RowExclusiveLock);

      /*
+      * Unregister from the on commit action handler.
+      */
+     unregister_temp_rel(indexId);
+
+     /*
       * if it has any expression columns, we might have stored statistics about
       * them.
       */
*** src/backend/commands/tablecmds.c
--- src/backend/commands/tablecmds.c
***************
*** 83,88 ****
--- 83,89 ----
  typedef struct OnCommitItem
  {
      Oid            relid;            /* relid of relation */
+     char        relkind;        /* relkind of relation */
      OnCommitAction oncommit;    /* what to do at end of xact */

      /*
***************
*** 94,102 **** typedef struct OnCommitItem
       */
      SubTransactionId creating_subid;
      SubTransactionId deleting_subid;
  } OnCommitItem;

! static List *on_commits = NIL;


  /*
--- 95,104 ----
       */
      SubTransactionId creating_subid;
      SubTransactionId deleting_subid;
+     SubTransactionId accessing_subid;
  } OnCommitItem;

! static HTAB *on_commits = NULL;


  /*
***************
*** 7561,7600 **** AlterSeqNamespaces(Relation classRel, Relation rel,


  /*
!  * This code supports
   *    CREATE TEMP TABLE ... ON COMMIT { DROP | PRESERVE ROWS | DELETE ROWS }
   *
   * Because we only support this for TEMP tables, it's sufficient to remember
   * the state in a backend-local data structure.
   */

  /*
!  * Register a newly-created relation's ON COMMIT action.
   */
  void
! register_on_commit_action(Oid relid, OnCommitAction action)
  {
      OnCommitItem *oc;
!     MemoryContext oldcxt;

!     /*
!      * We needn't bother registering the relation unless there is an ON COMMIT
!      * action we need to take.
!      */
!     if (action == ONCOMMIT_NOOP || action == ONCOMMIT_PRESERVE_ROWS)
!         return;

!     oldcxt = MemoryContextSwitchTo(CacheMemoryContext);

-     oc = (OnCommitItem *) palloc(sizeof(OnCommitItem));
      oc->relid = relid;
      oc->oncommit = action;
      oc->creating_subid = GetCurrentSubTransactionId();
      oc->deleting_subid = InvalidSubTransactionId;
!
!     on_commits = lcons(oc, on_commits);
!
!     MemoryContextSwitchTo(oldcxt);
  }

  /*
--- 7563,7615 ----


  /*
!  * This code keeps track of all temporary relations in the current session,
!  * to support
   *    CREATE TEMP TABLE ... ON COMMIT { DROP | PRESERVE ROWS | DELETE ROWS }
   *
+  * and to implement the special handling of temporary relations at
+  * PREPARE TRANSACTION.
+  *
   * Because we only support this for TEMP tables, it's sufficient to remember
   * the state in a backend-local data structure.
   */

  /*
!  * Register a newly-created temporary relation.
   */
  void
! register_temp_rel(Oid relid, char relkind, OnCommitAction action)
  {
      OnCommitItem *oc;
!     bool          found;

!     if (on_commits == NULL)
!     {
!         HASHCTL        hash_ctl;
!
!         /* Initialize hash table */
!         memset(&hash_ctl, 0, sizeof(hash_ctl));
!         hash_ctl.keysize = sizeof(Oid);
!         hash_ctl.entrysize = sizeof(OnCommitItem);
!         hash_ctl.hcxt = CacheMemoryContext;
!         hash_ctl.hash = oid_hash;
!
!         on_commits = hash_create("On commit actions",
!                                  4, /* small initial size to make scans fast */
!                                  &hash_ctl,
!                                  HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
!     }

!     oc = (OnCommitItem *) hash_search(on_commits, &relid, HASH_ENTER, &found);
!     if (found)
!         elog(ERROR, "relid already in on commit action table");

      oc->relid = relid;
+     oc->relkind = relkind;
      oc->oncommit = action;
      oc->creating_subid = GetCurrentSubTransactionId();
      oc->deleting_subid = InvalidSubTransactionId;
!     oc->accessing_subid = InvalidSubTransactionId;
  }

  /*
***************
*** 7603,7622 **** register_on_commit_action(Oid relid, OnCommitAction action)
   * Actually, we only mark the OnCommitItem entry as to be deleted after commit.
   */
  void
! remove_on_commit_action(Oid relid)
  {
!     ListCell   *l;

!     foreach(l, on_commits)
!     {
!         OnCommitItem *oc = (OnCommitItem *) lfirst(l);

!         if (oc->relid == relid)
!         {
!             oc->deleting_subid = GetCurrentSubTransactionId();
!             break;
!         }
!     }
  }

  /*
--- 7618,7650 ----
   * Actually, we only mark the OnCommitItem entry as to be deleted after commit.
   */
  void
! unregister_temp_rel(Oid relid)
  {
!     OnCommitItem *oc;

!     if (on_commits == NULL)
!         return;

!     oc = (OnCommitItem *) hash_search(on_commits, &relid, HASH_FIND, NULL);
!
!     if (oc != NULL)
!         oc->deleting_subid = GetCurrentSubTransactionId();
! }
!
!
! /*
!  * Make note that a temporary relation has been accessed in this transaction.
!  * This should be called whenever a lock is acquired on the relation, except
!  * for transient locks that are always released before commit.
!  * relation_open() will usually do that for you.
!  */
! void
! register_temp_rel_access(Oid relid)
! {
!     OnCommitItem *oc;
!
!     oc = (OnCommitItem *) hash_search(on_commits, &relid, HASH_FIND, NULL);
!     oc->accessing_subid = GetCurrentSubTransactionId();
  }

  /*
***************
*** 7628,7672 **** remove_on_commit_action(Oid relid)
  void
  PreCommit_on_commit_actions(void)
  {
!     ListCell   *l;
      List       *oids_to_truncate = NIL;

!     foreach(l, on_commits)
!     {
!         OnCommitItem *oc = (OnCommitItem *) lfirst(l);
!
!         /* Ignore entry if already dropped in this xact */
!         if (oc->deleting_subid != InvalidSubTransactionId)
!             continue;

!         switch (oc->oncommit)
          {
!             case ONCOMMIT_NOOP:
!             case ONCOMMIT_PRESERVE_ROWS:
!                 /* Do nothing (there shouldn't be such entries, actually) */
!                 break;
!             case ONCOMMIT_DELETE_ROWS:
!                 oids_to_truncate = lappend_oid(oids_to_truncate, oc->relid);
!                 break;
!             case ONCOMMIT_DROP:
!                 {
!                     ObjectAddress object;
!
!                     object.classId = RelationRelationId;
!                     object.objectId = oc->relid;
!                     object.objectSubId = 0;
!                     performDeletion(&object, DROP_CASCADE);

!                     /*
!                      * Note that table deletion will call
!                      * remove_on_commit_action, so the entry should get marked
!                      * as deleted.
!                      */
!                     Assert(oc->deleting_subid != InvalidSubTransactionId);
                      break;
!                 }
          }
      }
      if (oids_to_truncate != NIL)
      {
          heap_truncate(oids_to_truncate);
--- 7656,7712 ----
  void
  PreCommit_on_commit_actions(void)
  {
!     HASH_SEQ_STATUS seq_status;
      List       *oids_to_truncate = NIL;
+     OnCommitItem *oc;

!     if (on_commits == NULL)
!         return;

!     hash_seq_init(&seq_status, on_commits);
!     PG_TRY();
!     {
!         while ((oc = hash_seq_search(&seq_status)) != NULL)
          {
!             /* Ignore entry if already dropped in this xact */
!             if (oc->deleting_subid != InvalidSubTransactionId)
!                 continue;

!             switch (oc->oncommit)
!             {
!                 case ONCOMMIT_NOOP:
!                 case ONCOMMIT_PRESERVE_ROWS:
!                     /* Do nothing */
                      break;
!                 case ONCOMMIT_DELETE_ROWS:
!                     oids_to_truncate = lappend_oid(oids_to_truncate, oc->relid);
!                     break;
!                 case ONCOMMIT_DROP:
!                     {
!                         ObjectAddress object;
!
!                         object.classId = RelationRelationId;
!                         object.objectId = oc->relid;
!                         object.objectSubId = 0;
!                         performDeletion(&object, DROP_CASCADE);
!
!                         /*
!                          * Note that table deletion will call
!                          * remove_on_commit_action, so the entry should get
!                          * marked as deleted.
!                          */
!                         Assert(oc->deleting_subid != InvalidSubTransactionId);
!                         break;
!                     }
!             }
          }
      }
+     PG_CATCH();
+     {
+         hash_seq_term(&seq_status);
+     }
+     PG_END_TRY();
+
      if (oids_to_truncate != NIL)
      {
          heap_truncate(oids_to_truncate);
***************
*** 7675,7680 **** PreCommit_on_commit_actions(void)
--- 7715,7821 ----
  }

  /*
+  * Don't allow PREPARE TRANSACTION if we've accessed a temporary table in
+  * this transaction.  Having the prepared xact hold locks on another
+  * backend's temp table seems a bad idea --- for instance it would prevent
+  * the backend from exiting.  There are other problems too, such as how
+  * to clean up the source backend's local buffers and ON COMMIT state
+  * if the prepared xact includes a DROP of a temp table.
+  *
+  * We must check this after executing any ON COMMIT actions, because
+  * they might still access a temp relation.
+  *
+  * However, since PostgreSQL 8.4, we do allow PREPARE TRANSATION if the
+  * transaction has accessed a temporary table with ON COMMIT DELETE ROWS
+  * action, as long as the transaction hasn't created or dropped one.  We
+  * work around the lock problem by releasing the locks early, in the PREPARE
+  * phase. That doesn't violate the two-phase locking protocol (not to be
+  * confused with two-phase commit), as the lock would be released at the
+  * 2nd phase commit or rollback anyway, and the transaction won't acquire
+  * any more locks after PREPARE.
+  *
+  * The reason why we only allow that for ON COMMIT DELETE ROWS tables, is that
+  * some commands like VACUUM FULL and CREATE INDEX get confused if there's
+  * xids in the table that belong to an in-progress transaction. They don't
+  * normally encounter such tuples because they grab an AccessExclusiveLock.
+  *
+  * XXX In principle this could be relaxed to allow some other useful special
+  * cases, such as a temp table created and dropped all within the
+  * transaction.  We would need to at least drop the local buffers, however.
+  * It might also be possible to allow read-only access to temp tables, they
+  * shouldn't pose a problem to commands like VACUUM FULL and CREATE INDEX.
+  */
+ void
+ AtPrepare_on_commit_actions(void)
+ {
+     HASH_SEQ_STATUS seq_status;
+     OnCommitItem *oc;
+
+     if (on_commits == NULL)
+         return;
+
+     hash_seq_init(&seq_status, on_commits);
+     while ((oc = hash_seq_search(&seq_status)) != NULL)
+     {
+         /*
+          * If the transaction hasn't touched the relation at all, we're fine.
+          */
+         if (oc->accessing_subid == InvalidSubTransactionId &&
+             oc->creating_subid == InvalidSubTransactionId &&
+             oc->deleting_subid == InvalidSubTransactionId)
+             continue;
+
+         /*
+          * Allow access to ON COMMIT DELETE ROWS temporary tables,
+          * that have not been created or dropped in this transaction.
+          */
+         if ((oc->oncommit == ONCOMMIT_DELETE_ROWS || oc->relkind != RELKIND_RELATION) &&
+             oc->creating_subid == InvalidSubTransactionId &&
+             oc->deleting_subid == InvalidSubTransactionId)
+         {
+             /*
+              * XXX If we could release the locks on the relation here,
+              * AtPrepare_Locks() wouldn't need to treat temp locks specially,
+              * and we wouldn't need the is_temp_rel() function below. But we
+              * don't know what locks we're holding, and there's no function
+              * to release all our locks on a given object.
+              */
+             continue;
+         }
+         else
+         {
+             hash_seq_term(&seq_status);
+             /* Determine appropriate error message */
+             if (oc->creating_subid != InvalidSubTransactionId ||
+                 oc->deleting_subid != InvalidSubTransactionId)
+                 ereport(ERROR,
+                         (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+                          errmsg("cannot PREPARE a transaction that has created or dropped a temporary relation")));
+             else
+                 ereport(ERROR,
+                         (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+                          errmsg("cannot PREPARE a transaction that has accessed a temporary table not defined with ON
COMMITDELETE ROWS"))); 
+         }
+     }
+ }
+
+ /*
+  * Is the given relation a temporary table in this backend?
+  */
+ bool
+ is_temp_rel(Oid relid)
+ {
+     OnCommitItem *oc;
+
+     /* you can't access a temp table if there isn't any */
+     Assert(on_commits != NULL);
+
+     oc = (OnCommitItem *) hash_search(on_commits, &relid, HASH_FIND, NULL);
+
+     return (oc != NULL);
+ }
+
+ /*
   * Post-commit or post-abort cleanup for ON COMMIT management.
   *
   * All we do here is remove no-longer-needed OnCommitItem entries.
***************
*** 7685,7718 **** PreCommit_on_commit_actions(void)
  void
  AtEOXact_on_commit_actions(bool isCommit)
  {
!     ListCell   *cur_item;
!     ListCell   *prev_item;

!     prev_item = NULL;
!     cur_item = list_head(on_commits);

!     while (cur_item != NULL)
      {
-         OnCommitItem *oc = (OnCommitItem *) lfirst(cur_item);
-
          if (isCommit ? oc->deleting_subid != InvalidSubTransactionId :
              oc->creating_subid != InvalidSubTransactionId)
          {
!             /* cur_item must be removed */
!             on_commits = list_delete_cell(on_commits, cur_item, prev_item);
!             pfree(oc);
!             if (prev_item)
!                 cur_item = lnext(prev_item);
!             else
!                 cur_item = list_head(on_commits);
          }
          else
          {
!             /* cur_item must be preserved */
              oc->creating_subid = InvalidSubTransactionId;
              oc->deleting_subid = InvalidSubTransactionId;
!             prev_item = cur_item;
!             cur_item = lnext(prev_item);
          }
      }
  }
--- 7826,7852 ----
  void
  AtEOXact_on_commit_actions(bool isCommit)
  {
!     HASH_SEQ_STATUS seq_status;
!     OnCommitItem *oc;

!     if (on_commits == NULL)
!         return;

!     hash_seq_init(&seq_status, on_commits);
!     while ((oc = hash_seq_search(&seq_status)) != NULL)
      {
          if (isCommit ? oc->deleting_subid != InvalidSubTransactionId :
              oc->creating_subid != InvalidSubTransactionId)
          {
!             /* current item must be removed */
!             hash_search(on_commits, &oc->relid, HASH_REMOVE, NULL);
          }
          else
          {
!             /* current item must be preserved */
              oc->creating_subid = InvalidSubTransactionId;
              oc->deleting_subid = InvalidSubTransactionId;
!             oc->accessing_subid = InvalidSubTransactionId;
          }
      }
  }
***************
*** 7728,7762 **** void
  AtEOSubXact_on_commit_actions(bool isCommit, SubTransactionId mySubid,
                                SubTransactionId parentSubid)
  {
!     ListCell   *cur_item;
!     ListCell   *prev_item;

!     prev_item = NULL;
!     cur_item = list_head(on_commits);

!     while (cur_item != NULL)
      {
-         OnCommitItem *oc = (OnCommitItem *) lfirst(cur_item);
-
          if (!isCommit && oc->creating_subid == mySubid)
          {
!             /* cur_item must be removed */
!             on_commits = list_delete_cell(on_commits, cur_item, prev_item);
!             pfree(oc);
!             if (prev_item)
!                 cur_item = lnext(prev_item);
!             else
!                 cur_item = list_head(on_commits);
          }
          else
          {
!             /* cur_item must be preserved */
              if (oc->creating_subid == mySubid)
                  oc->creating_subid = parentSubid;
              if (oc->deleting_subid == mySubid)
                  oc->deleting_subid = isCommit ? parentSubid : InvalidSubTransactionId;
!             prev_item = cur_item;
!             cur_item = lnext(prev_item);
          }
      }
  }
--- 7862,7890 ----
  AtEOSubXact_on_commit_actions(bool isCommit, SubTransactionId mySubid,
                                SubTransactionId parentSubid)
  {
!     HASH_SEQ_STATUS seq_status;
!     OnCommitItem *oc;

!     if (on_commits == NULL)
!         return;

!     hash_seq_init(&seq_status, on_commits);
!     while ((oc = hash_seq_search(&seq_status)) != NULL)
      {
          if (!isCommit && oc->creating_subid == mySubid)
          {
!             /* current item must be removed */
!             hash_search(on_commits, &oc->relid, HASH_REMOVE, NULL);
          }
          else
          {
!             /* current item must be preserved */
              if (oc->creating_subid == mySubid)
                  oc->creating_subid = parentSubid;
              if (oc->deleting_subid == mySubid)
                  oc->deleting_subid = isCommit ? parentSubid : InvalidSubTransactionId;
!             if (oc->accessing_subid == mySubid)
!                 oc->accessing_subid = isCommit ? parentSubid : InvalidSubTransactionId;
          }
      }
  }
*** src/backend/storage/lmgr/lock.c
--- src/backend/storage/lmgr/lock.c
***************
*** 35,40 ****
--- 35,41 ----
  #include "access/transam.h"
  #include "access/twophase.h"
  #include "access/twophase_rmgr.h"
+ #include "commands/tablecmds.h"
  #include "miscadmin.h"
  #include "pg_trace.h"
  #include "pgstat.h"
***************
*** 1816,1834 **** GetLockConflicts(const LOCKTAG *locktag, LOCKMODE lockmode)
      return vxids;
  }


  /*
   * AtPrepare_Locks
   *        Do the preparatory work for a PREPARE: make 2PC state file records
   *        for all locks currently held.
   *
!  * Non-transactional locks are ignored, as are VXID locks.
   *
   * There are some special cases that we error out on: we can't be holding
!  * any session locks (should be OK since only VACUUM uses those) and we
!  * can't be holding any locks on temporary objects (since that would mess
!  * up the current backend if it tries to exit before the prepared xact is
!  * committed).
   */
  void
  AtPrepare_Locks(void)
--- 1817,1882 ----
      return vxids;
  }

+ /*
+  * A helper function to determine whether the given lock should be
+  * prepared
+  */
+ static bool
+ is_preparable_locktag(LOCKTAG *lock)
+ {
+     /* Ignore nontransactional locks */
+     if (!LockMethods[lock->locktag_lockmethodid]->transactional)
+         return false;
+
+     /* Locks on temporary tables are dropped in PREPARE phase */
+     switch (lock->locktag_type)
+     {
+         case LOCKTAG_RELATION_EXTEND:
+         case LOCKTAG_PAGE:
+         case LOCKTAG_TUPLE:
+             /*
+              * These lock types are only held transiently, we shouldn't
+              * be holding any at the end of transaction.
+              */
+             elog(ERROR, "unexpected lock type at PREPARE TRANSACTION");
+
+         case LOCKTAG_VIRTUALTRANSACTION:
+             /*
+              * Ignore VXID locks.  We don't want those to be held by
+              * prepared transactions, since they aren't meaningful after
+              * a restart.
+              */
+             return false;
+
+         case LOCKTAG_RELATION:
+             /*
+              * Locks on temporary relations are not persisted. That means
+              * that they're dropped early, in the PREPARE phase. We only
+              * allow that in some narrow cases, which
+              * AtPrepare_on_commit_actions has already checked for.
+              */
+             /* field1 is dboid, field2 is reloid for all of these */
+             if ((Oid) lock->locktag_field1 == InvalidOid)
+                 break;   /* shared, so not temp */
+             if (is_temp_rel((Oid) lock->locktag_field2))
+                 return false;
+         default:
+             break;
+     }
+     return true;
+ }

  /*
   * AtPrepare_Locks
   *        Do the preparatory work for a PREPARE: make 2PC state file records
   *        for all locks currently held.
   *
!  * Non-transactional and VXID locks are ignored. Locks on temporary objects
!  * are also ignored, because that would mess up the current backend if it
!  * tries to exit before the prepared xact is committed.
   *
   * There are some special cases that we error out on: we can't be holding
!  * any session locks (should be OK since only VACUUM uses those).
   */
  void
  AtPrepare_Locks(void)
***************
*** 1848,1868 **** AtPrepare_Locks(void)
          LOCALLOCKOWNER *lockOwners = locallock->lockOwners;
          int            i;

-         /* Ignore nontransactional locks */
-         if (!LockMethods[LOCALLOCK_LOCKMETHOD(*locallock)]->transactional)
-             continue;
-
-         /*
-          * Ignore VXID locks.  We don't want those to be held by prepared
-          * transactions, since they aren't meaningful after a restart.
-          */
-         if (locallock->tag.lock.locktag_type == LOCKTAG_VIRTUALTRANSACTION)
-             continue;
-
          /* Ignore it if we don't actually hold the lock */
          if (locallock->nLocks <= 0)
              continue;

          /* Scan to verify there are no session locks */
          for (i = locallock->numLockOwners - 1; i >= 0; i--)
          {
--- 1896,1909 ----
          LOCALLOCKOWNER *lockOwners = locallock->lockOwners;
          int            i;

          /* Ignore it if we don't actually hold the lock */
          if (locallock->nLocks <= 0)
              continue;

+         /* Does the lock need to be persisted? */
+         if (!is_preparable_locktag(&locallock->tag.lock))
+             continue;
+
          /* Scan to verify there are no session locks */
          for (i = locallock->numLockOwners - 1; i >= 0; i--)
          {
***************
*** 1936,1947 **** PostPrepare_Locks(TransactionId xid)
              continue;
          }

!         /* Ignore nontransactional locks */
!         if (!LockMethods[LOCALLOCK_LOCKMETHOD(*locallock)]->transactional)
!             continue;
!
!         /* Ignore VXID locks */
!         if (locallock->tag.lock.locktag_type == LOCKTAG_VIRTUALTRANSACTION)
              continue;

          /* We already checked there are no session locks */
--- 1977,1984 ----
              continue;
          }

!         /* Does the lock need to be persisted? */
!         if (!is_preparable_locktag(&locallock->tag.lock))
              continue;

          /* We already checked there are no session locks */
***************
*** 1985,1996 **** PostPrepare_Locks(TransactionId xid)

              lock = proclock->tag.myLock;

!             /* Ignore nontransactional locks */
!             if (!LockMethods[LOCK_LOCKMETHOD(*lock)]->transactional)
!                 goto next_item;
!
!             /* Ignore VXID locks */
!             if (lock->tag.locktag_type == LOCKTAG_VIRTUALTRANSACTION)
                  goto next_item;

              PROCLOCK_PRINT("PostPrepare_Locks", proclock);
--- 2022,2029 ----

              lock = proclock->tag.myLock;

!             /* Does the lock need to be persisted? */
!             if (!is_preparable_locktag(&lock->tag))
                  goto next_item;

              PROCLOCK_PRINT("PostPrepare_Locks", proclock);
*** src/include/access/xact.h
--- src/include/access/xact.h
***************
*** 44,52 **** extern bool XactReadOnly;
  /* Asynchronous commits */
  extern bool XactSyncCommit;

- /* Kluge for 2PC support */
- extern bool MyXactAccessedTempRel;
-
  /*
   *    start- and end-of-transaction callbacks for dynamically loaded modules
   */
--- 44,49 ----
*** src/include/commands/tablecmds.h
--- src/include/commands/tablecmds.h
***************
*** 61,69 **** extern AttrNumber *varattnos_map(TupleDesc old, TupleDesc new);
  extern AttrNumber *varattnos_map_schema(TupleDesc old, List *schema);
  extern void change_varattnos_of_a_node(Node *node, const AttrNumber *newattno);

! extern void register_on_commit_action(Oid relid, OnCommitAction action);
! extern void remove_on_commit_action(Oid relid);

  extern void PreCommit_on_commit_actions(void);
  extern void AtEOXact_on_commit_actions(bool isCommit);
  extern void AtEOSubXact_on_commit_actions(bool isCommit,
--- 61,72 ----
  extern AttrNumber *varattnos_map_schema(TupleDesc old, List *schema);
  extern void change_varattnos_of_a_node(Node *node, const AttrNumber *newattno);

! extern void register_temp_rel(Oid relid, char relkind, OnCommitAction action);
! extern void unregister_temp_rel(Oid relid);
! extern void register_temp_rel_access(Oid relid);
! extern bool is_temp_rel(Oid relid);

+ extern void AtPrepare_on_commit_actions(void);
  extern void PreCommit_on_commit_actions(void);
  extern void AtEOXact_on_commit_actions(bool isCommit);
  extern void AtEOSubXact_on_commit_actions(bool isCommit,

В списке pgsql-hackers по дате отправления:

Предыдущее
От: Tom Lane
Дата:
Сообщение: Re: Reducing some DDL Locks to ShareLock
Следующее
От: "Dann Corbit"
Дата:
Сообщение: Installation oddity -- installer insists that PostgreSQL has failed to start, even though it is started and functioning correctly