You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@subversion.apache.org by gl...@apache.org on 2013/06/29 21:53:43 UTC

svn commit: r867809 - in /subversion/trunk/subversion/libsvn_fs_fs: fs.c fs.h fs_fs.c structure

Author: glasser
Date: Fri Nov  9 01:50:29 2007
New Revision: 867809

URL: http://svn.apache.org/r867809
Log:
Lock the FSFS 'transaction-current' file under its own
'txn-current-lock' file instead of 'write-lock', so that new
transactions and commits don't block each other.

In order to do so, abstract out the locking code a little more.  While
we're at it, make the locking code do "lock,(error:create,lock)"
instead of "stat,(missing:create),lock", to save a stat in the common
case.

* subversion/libsvn_fs_fs/fs.h
  (PATH_TXN_CURRENT_LOCK): New filename.  (And reflow the block.)
  (fs_fs_shared_data_t): If thready, add a txn_current_lock mutex
   field to protect the transaction-current file from concurrent
   access by multiple threads of one process.

* subversion/libsvn_fs_fs/fs.c
  (fs_serialized_init): Create the txn_current_lock mutex, if thready.

* subversion/libsvn_fs_fs/fs_fs.c
  (path_txn_current): New  helper function for getting the path of the
   transaction-current file.
  (path_txn_current_lock): New  helper function for getting the path of the
   txn-current-lock file.
  (get_lock_on_filesystem): Rename from get_write_lock, and generalize
   to lock a given file instead of 'write-lock'.  (Also move earlier
   in the file.)  Remove the stat (see above).
  (with_some_lock): Extract most of svn_fs_fs__with_write_lock into
   this function, which takes as argument a lock filename and (if
   threading) a mutex, instead of a svn_fs_t.
  (svn_fs_fs__with_write_lock): Now just a wrapper around
   with_some_lock (and moved earlier in the file).
  (with_txn_current_lock): New function which runs its body with the
   txn-current-lock file (and mutex) locked.
  (get_and_increment_txn_key_body): Use new path_txn_current helper
   function.
  (create_txn_dir): Call get_and_increment_txn_key_body using
   with_txn_current_lock instead of svn_fs_fs__with_write_lock (this
   change is the whole point of the revision).
  (svn_fs_fs__create): Create the txn-current-lock file when creating
   the repo., and use the path_txn_current helper.

* subversion/libsvn_fs_fs/structure
  (Layout of the FS directory): Document that transaction-current is
   modified under the lock of txn-current-lock.

Review by: malcolm
(earlier version, helpful suggestions)

Modified:
    subversion/trunk/subversion/libsvn_fs_fs/fs.c
    subversion/trunk/subversion/libsvn_fs_fs/fs.h
    subversion/trunk/subversion/libsvn_fs_fs/fs_fs.c
    subversion/trunk/subversion/libsvn_fs_fs/structure

Modified: subversion/trunk/subversion/libsvn_fs_fs/fs.c
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_fs_fs/fs.c?rev=867809&r1=867808&r2=867809&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_fs_fs/fs.c (original)
+++ subversion/trunk/subversion/libsvn_fs_fs/fs.c Fri Nov  9 01:50:29 2007
@@ -110,6 +110,13 @@ fs_serialized_init(svn_fs_t *fs, apr_poo
       if (status)
         return svn_error_wrap_apr(status,
                                   _("Can't create FSFS txn list mutex"));
+
+      /* ... not to mention locking the transaction-current file. */
+      status = apr_thread_mutex_create(&ffsd->txn_current_lock,
+                                       APR_THREAD_MUTEX_DEFAULT, common_pool);
+      if (status)
+        return svn_error_wrap_apr(status,
+                                  _("Can't create FSFS txn-current mutex"));
 #endif
 
       key = apr_pstrdup(common_pool, key);

Modified: subversion/trunk/subversion/libsvn_fs_fs/fs.h
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_fs_fs/fs.h?rev=867809&r1=867808&r2=867809&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_fs_fs/fs.h (original)
+++ subversion/trunk/subversion/libsvn_fs_fs/fs.h Fri Nov  9 01:50:29 2007
@@ -38,15 +38,16 @@ extern "C" {
    native filesystem directories and revision files. */
 
 /* Names of special files in the fs_fs filesystem. */
-#define PATH_FORMAT        "format"        /* Contains format number */
-#define PATH_UUID          "uuid"          /* Contains UUID */
-#define PATH_CURRENT       "current"       /* Youngest revision */
-#define PATH_LOCK_FILE     "write-lock"    /* Revision lock file */
-#define PATH_REVS_DIR      "revs"          /* Directory of revisions */
-#define PATH_REVPROPS_DIR  "revprops"      /* Directory of revprops */
-#define PATH_TXNS_DIR      "transactions"  /* Directory of transactions */
-#define PATH_TXN_CURRENT   "transaction-current" /* File with next txn key */
-#define PATH_LOCKS_DIR     "locks"         /* Directory of locks */
+#define PATH_FORMAT           "format"           /* Contains format number */
+#define PATH_UUID             "uuid"             /* Contains UUID */
+#define PATH_CURRENT          "current"          /* Youngest revision */
+#define PATH_LOCK_FILE        "write-lock"       /* Revision lock file */
+#define PATH_REVS_DIR         "revs"             /* Directory of revisions */
+#define PATH_REVPROPS_DIR     "revprops"         /* Directory of revprops */
+#define PATH_TXNS_DIR         "transactions"     /* Directory of transactions */
+#define PATH_TXN_CURRENT      "transaction-current" /* File with next txn key */
+#define PATH_TXN_CURRENT_LOCK "txn-current-lock" /* Lock for txn-current */
+#define PATH_LOCKS_DIR         "locks"           /* Directory of locks */
 
 /* Names of special files and file extensions for transactions */
 #define PATH_CHANGES       "changes"       /* Records changes made so far */
@@ -138,6 +139,10 @@ typedef struct
   /* A lock for intra-process synchronization when grabbing the
      repository write lock. */
   apr_thread_mutex_t *fs_write_lock;
+
+  /* A lock for intra-process synchronization when locking the
+     transaction-current file. */
+  apr_thread_mutex_t *txn_current_lock;
 #endif
 
   /* The common pool, under which this object is allocated, subpools

Modified: subversion/trunk/subversion/libsvn_fs_fs/fs_fs.c
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_fs_fs/fs_fs.c?rev=867809&r1=867808&r2=867809&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_fs_fs/fs_fs.c (original)
+++ subversion/trunk/subversion/libsvn_fs_fs/fs_fs.c Fri Nov  9 01:50:29 2007
@@ -145,6 +145,18 @@ svn_fs_fs__path_current(svn_fs_t *fs, ap
 }
 
 static const char *
+path_txn_current(svn_fs_t *fs, apr_pool_t *pool)
+{
+  return svn_path_join(fs->path, PATH_TXN_CURRENT, pool);
+}
+
+static const char *
+path_txn_current_lock(svn_fs_t *fs, apr_pool_t *pool)
+{
+  return svn_path_join(fs->path, PATH_TXN_CURRENT_LOCK, pool);
+}
+
+static const char *
 path_lock(svn_fs_t *fs, apr_pool_t *pool)
 {
   return svn_path_join(fs->path, PATH_LOCK_FILE, pool);
@@ -394,6 +406,119 @@ with_txnlist_lock(svn_fs_t *fs,
   return err;
 }
 
+
+/* Get a lock on empty file LOCK_FILENAME, creating it in POOL. */
+static svn_error_t *
+get_lock_on_filesystem(const char *lock_filename,
+               apr_pool_t *pool)
+{
+  svn_error_t *err = svn_io_file_lock2(lock_filename, TRUE, FALSE, pool);
+
+  if (err && APR_STATUS_IS_ENOENT(err->apr_err))
+    {
+      /* No lock file?  No big deal; these are just empty files
+         anyway.  Create it and try again. */
+      svn_error_clear(err);
+      err = NULL;
+
+      SVN_ERR(svn_io_file_create(lock_filename, "", pool));
+      SVN_ERR(svn_io_file_lock2(lock_filename, TRUE, FALSE, pool));
+    }
+
+  return err;
+}
+
+/* Obtain a write lock on the file LOCK_FILENAME (protecting with
+   LOCK_MUTEX if APR is threaded) in a subpool of POOL, call BODY with
+   BATON and that subpool, destroy the subpool (releasing the write
+   lock) and return what BODY returned. */
+static svn_error_t *
+with_some_lock(svn_error_t *(*body)(void *baton,
+                                    apr_pool_t *pool),
+               void *baton,
+               const char *lock_filename,
+#if APR_HAS_THREADS
+               apr_thread_mutex_t *lock_mutex,
+#endif
+               apr_pool_t *pool)
+{
+  apr_pool_t *subpool = svn_pool_create(pool);
+  svn_error_t *err;
+
+#if APR_HAS_THREADS
+  apr_status_t status;
+
+  /* POSIX fcntl locks are per-process, so we need to serialize locks
+     within the process. */
+  status = apr_thread_mutex_lock(lock_mutex);
+  if (status)
+    return svn_error_wrap_apr(status, 
+                              _("Can't grab FSFS mutex for '%s'"),
+                              lock_filename);
+#endif
+
+  err = get_lock_on_filesystem(lock_filename, subpool);
+
+  if (!err)
+    err = body(baton, subpool);
+
+  svn_pool_destroy(subpool);
+
+#if APR_HAS_THREADS
+  status = apr_thread_mutex_unlock(lock_mutex);
+  if (status && !err)
+    return svn_error_wrap_apr(status,
+                              _("Can't ungrab FSFS mutex for '%s'"),
+                              lock_filename);
+#endif
+
+  return err;
+}
+
+svn_error_t *
+svn_fs_fs__with_write_lock(svn_fs_t *fs,
+                           svn_error_t *(*body)(void *baton,
+                                                apr_pool_t *pool),
+                           void *baton,
+                           apr_pool_t *pool)
+{
+#if APR_HAS_THREADS
+  fs_fs_data_t *ffd = fs->fsap_data;
+  fs_fs_shared_data_t *ffsd = ffd->shared;
+  apr_thread_mutex_t *mutex = ffsd->fs_write_lock;
+#endif
+
+  return with_some_lock(body, baton,
+                        path_lock(fs, pool),
+#if APR_HAS_THREADS
+                        mutex,
+#endif
+                        pool);
+}
+
+/* Run BODY (with BATON and POOL) while the transaction-current file
+   of FS is locked. */
+svn_error_t *
+with_txn_current_lock(svn_fs_t *fs,
+                      svn_error_t *(*body)(void *baton,
+                                           apr_pool_t *pool),
+                      void *baton,
+                      apr_pool_t *pool)
+{
+#if APR_HAS_THREADS
+  fs_fs_data_t *ffd = fs->fsap_data;
+  fs_fs_shared_data_t *ffsd = ffd->shared;
+  apr_thread_mutex_t *mutex = ffsd->txn_current_lock;
+#endif
+
+  return with_some_lock(body, baton,
+                        path_txn_current_lock(fs, pool),
+#if APR_HAS_THREADS
+                        mutex,
+#endif
+                        pool);
+}
+
 /* A structure used by unlock_proto_rev() and unlock_proto_rev_body(),
    which see. */
 struct unlock_proto_rev_baton
@@ -3183,9 +3308,7 @@ static svn_error_t *
 get_and_increment_txn_key_body(void *baton, apr_pool_t *pool)
 {
   struct get_and_increment_txn_key_baton *cb = baton;
-  const char *txn_current_filename = svn_path_join(cb->fs->path,
-                                                   PATH_TXN_CURRENT,
-                                                   pool);
+  const char *txn_current_filename = path_txn_current(cb->fs, pool);
   apr_file_t *txn_current_file;
   const char *tmp_filename;
   char next_txn_id[MAX_KEY_SIZE+3];
@@ -3262,10 +3385,10 @@ create_txn_dir(const char **id_p, svn_fs
      number the transaction is based off into the transaction id. */
   cb.pool = pool;
   cb.fs = fs;
-  SVN_ERR(svn_fs_fs__with_write_lock(fs,
-                                     get_and_increment_txn_key_body,
-                                     &cb,
-                                     pool));
+  SVN_ERR(with_txn_current_lock(fs,
+                                get_and_increment_txn_key_body,
+                                &cb,
+                                pool));
   *id_p = apr_psprintf(pool, "%ld-%s", rev, cb.txn_id);
 
   txn_dir = svn_path_join_many(pool,
@@ -4679,67 +4802,6 @@ write_final_current(svn_fs_t *fs,
   return write_current(fs, rev, new_node_id, new_copy_id, pool);
 }
 
-/* Get a write lock in FS, creating it in POOL. */
-static svn_error_t *
-get_write_lock(svn_fs_t *fs,
-               apr_pool_t *pool)
-{
-  const char *lock_filename;
-  svn_node_kind_t kind;
-
-  lock_filename = path_lock(fs, pool);
-
-  /* svn 1.1.1 and earlier deferred lock file creation to the first
-     commit.  So in case the repository was created by an earlier
-     version of svn, check the lock file here. */
-  SVN_ERR(svn_io_check_path(lock_filename, &kind, pool));
-  if ((kind == svn_node_unknown) || (kind == svn_node_none))
-    SVN_ERR(svn_io_file_create(lock_filename, "", pool));
-
-  SVN_ERR(svn_io_file_lock2(lock_filename, TRUE, FALSE, pool));
-
-  return SVN_NO_ERROR;
-}
-
-svn_error_t *
-svn_fs_fs__with_write_lock(svn_fs_t *fs,
-                           svn_error_t *(*body)(void *baton,
-                                                apr_pool_t *pool),
-                           void *baton,
-                           apr_pool_t *pool)
-{
-  apr_pool_t *subpool = svn_pool_create(pool);
-  svn_error_t *err;
-
-#if APR_HAS_THREADS
-  fs_fs_data_t *ffd = fs->fsap_data;
-  fs_fs_shared_data_t *ffsd = ffd->shared;
-  apr_status_t status;
-
-  /* POSIX fcntl locks are per-process, so we need to serialize locks
-     within the process. */
-  status = apr_thread_mutex_lock(ffsd->fs_write_lock);
-  if (status)
-    return svn_error_wrap_apr(status, _("Can't grab FSFS repository mutex"));
-#endif
-
-  err = get_write_lock(fs, subpool);
-
-  if (!err)
-    err = body(baton, subpool);
-
-  svn_pool_destroy(subpool);
-
-#if APR_HAS_THREADS
-  status = apr_thread_mutex_unlock(ffsd->fs_write_lock);
-  if (status && !err)
-    return svn_error_wrap_apr(status,
-                              _("Can't ungrab FSFS repository mutex"));
-#endif
-
-  return err;
-}
-
 /* Verify that the user registed with FS has all the locks necessary to
    permit all the changes associate with TXN_NAME.
    The FS write lock is assumed to be held by the caller. */
@@ -5114,8 +5176,12 @@ svn_fs_fs__create(svn_fs_t *fs,
   /* Create the transaction-current file if the repository supports
      the transaction sequence file. */
   if (format >= SVN_FS_FS__MIN_TXN_CURRENT_FORMAT)
-    SVN_ERR(svn_io_file_create(svn_path_join(path, PATH_TXN_CURRENT, pool),
-                               "0\n", pool));
+    {
+      SVN_ERR(svn_io_file_create(path_txn_current(fs, pool),
+                                 "0\n", pool));
+      SVN_ERR(svn_io_file_create(path_txn_current_lock(fs, pool),
+                                 "", pool));
+    }
 
   /* This filesystem is ready.  Stamp it with a format number. */
   SVN_ERR(write_format(path_format(fs, pool),

Modified: subversion/trunk/subversion/libsvn_fs_fs/structure
URL: http://svn.apache.org/viewvc/subversion/trunk/subversion/libsvn_fs_fs/structure?rev=867809&r1=867808&r2=867809&view=diff
==============================================================================
--- subversion/trunk/subversion/libsvn_fs_fs/structure (original)
+++ subversion/trunk/subversion/libsvn_fs_fs/structure Fri Nov  9 01:50:29 2007
@@ -47,6 +47,7 @@ repository) is:
   current             File specifying current revision and next node/copy id
   fs-type             File identifying this filesystem as an FSFS filesystem
   write-lock          Empty file, locked to serialise writers
+  txn-current-lock    Empty file, locked to serialise 'transaction-current'
   uuid                File containing the UUID of the repository
   format              File containing the format number of this filesystem
 
@@ -71,9 +72,9 @@ that contains only a base-36 number.  Th
 in the next transaction name, along with the revision number the
 transaction is based on.  This sequence number ensures that
 transaction names are not reused, even if the transaction is aborted
-and a new transaction based on the same revision is begun.  The
-"transaction-current" file is read and written under the fs-wide
-write-lock.
+and a new transaction based on the same revision is begun.  The only
+operation that FSFS performs on this file is "get and increment";
+the "txn-current-lock" file is locked during this operation.
 
 Filesystem formats
 ------------------