You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucy.apache.org by nw...@apache.org on 2017/04/16 10:28:52 UTC

[14/16] lucy git commit: Move LockFileLock to separate file

Move LockFileLock to separate file


Project: http://git-wip-us.apache.org/repos/asf/lucy/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucy/commit/6e8538a8
Tree: http://git-wip-us.apache.org/repos/asf/lucy/tree/6e8538a8
Diff: http://git-wip-us.apache.org/repos/asf/lucy/diff/6e8538a8

Branch: refs/heads/master
Commit: 6e8538a8834a49da7e8cce8b94decc130467b0f3
Parents: 2bd2bc6
Author: Nick Wellnhofer <we...@aevum.de>
Authored: Sun Feb 19 17:01:50 2017 +0100
Committer: Nick Wellnhofer <we...@aevum.de>
Committed: Mon Feb 20 16:51:32 2017 +0100

----------------------------------------------------------------------
 core/Lucy/Index/IndexManager.c   |   2 +-
 core/Lucy/Store/Lock.c           | 376 ++--------------------------------
 core/Lucy/Store/Lock.cfh         |  33 +--
 core/Lucy/Store/LockFileLock.c   | 363 ++++++++++++++++++++++++++++++++
 core/Lucy/Store/LockFileLock.cfh |  49 +++++
 test/Lucy/Test/Store/TestLock.c  |   2 +-
 6 files changed, 437 insertions(+), 388 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucy/blob/6e8538a8/core/Lucy/Index/IndexManager.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/IndexManager.c b/core/Lucy/Index/IndexManager.c
index bd3aa30..8e56509 100644
--- a/core/Lucy/Index/IndexManager.c
+++ b/core/Lucy/Index/IndexManager.c
@@ -25,7 +25,7 @@
 #include "Lucy/Index/Snapshot.h"
 #include "Lucy/Store/DirHandle.h"
 #include "Lucy/Store/Folder.h"
-#include "Lucy/Store/Lock.h"
+#include "Lucy/Store/LockFileLock.h"
 #include "Lucy/Util/IndexFileNames.h"
 #include "Lucy/Util/Json.h"
 #include "Lucy/Util/StringHelper.h"

http://git-wip-us.apache.org/repos/asf/lucy/blob/6e8538a8/core/Lucy/Store/Lock.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/Lock.c b/core/Lucy/Store/Lock.c
index e716099..7a4a060 100644
--- a/core/Lucy/Store/Lock.c
+++ b/core/Lucy/Store/Lock.c
@@ -15,18 +15,12 @@
  */
 
 #define C_LUCY_LOCK
-#define C_LUCY_LOCKFILELOCK
 #include "Lucy/Util/ToolSet.h"
 
-#include <stdio.h>
 #include <ctype.h>
 
 #include "Lucy/Store/Lock.h"
-#include "Lucy/Store/DirHandle.h"
 #include "Lucy/Store/Folder.h"
-#include "Lucy/Store/OutStream.h"
-#include "Lucy/Util/Json.h"
-#include "Lucy/Util/ProcessID.h"
 #include "Lucy/Util/Sleep.h"
 
 Lock*
@@ -76,6 +70,26 @@ Lock_Destroy_IMP(Lock *self) {
 }
 
 bool
+Lock_make_lock_dir(Folder *folder) {
+    String *lock_dir_name = SSTR_WRAP_C("locks");
+
+    if (!Folder_MkDir(folder, lock_dir_name)) {
+        Err *err = (Err*)INCREF(Err_get_error());
+        // Maybe our attempt failed because another process succeeded.
+        if (Folder_Find_Folder(folder, lock_dir_name)) {
+            DECREF(err);
+        }
+        else {
+            // Nope, everything failed, so bail out.
+            Err_set_error(err);
+            return false;
+        }
+    }
+
+    return true;
+}
+
+bool
 Lock_Obtain_Shared_IMP(Lock *self) {
     LockIVARS *const ivars = Lock_IVARS(self);
     int32_t time_left = ivars->interval == 0 ? 0 : ivars->timeout;
@@ -111,356 +125,6 @@ Lock_Obtain_Exclusive_IMP(Lock *self) {
 
 /***************************************************************************/
 
-#define LFLOCK_STATE_UNLOCKED          0
-#define LFLOCK_STATE_LOCKED_SHARED     1
-#define LFLOCK_STATE_LOCKED_EXCLUSIVE  2
-
-static bool
-S_request(LockFileLockIVARS *ivars, String *lock_path);
-
-static bool
-S_is_locked_exclusive(LockFileLockIVARS *ivars);
-
-static bool
-S_is_locked(LockFileLockIVARS *ivars);
-
-static bool
-S_is_shared_lock_file(LockFileLockIVARS *ivars, String *entry);
-
-static bool
-S_maybe_delete_file(LockFileLockIVARS *ivars, String *path,
-                    bool delete_mine, bool delete_other);
-
-LockFileLock*
-LFLock_new(Folder *folder, String *name, String *host, int32_t timeout,
-           int32_t interval, bool exclusive_only) {
-    LockFileLock *self = (LockFileLock*)Class_Make_Obj(LOCKFILELOCK);
-    return LFLock_init(self, folder, name, host, timeout, interval,
-                       exclusive_only);
-}
-
-LockFileLock*
-LFLock_init(LockFileLock *self, Folder *folder, String *name, String *host,
-            int32_t timeout, int32_t interval, bool exclusive_only) {
-    int pid = PID_getpid();
-    Lock_init((Lock*)self, folder, name, timeout, interval);
-    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-    ivars->host      = (String*)INCREF(host);
-    ivars->link_path = Str_newf("%o.%o.%i64", ivars->lock_path, host,
-                                (int64_t)pid);
-    ivars->exclusive_only = exclusive_only;
-    return self;
-}
-
-struct lockfile_context {
-    OutStream *outstream;
-    String *json;
-};
-
-static void
-S_write_lockfile_json(void *context) {
-    struct lockfile_context *stuff = (struct lockfile_context*)context;
-    size_t size = Str_Get_Size(stuff->json);
-    OutStream_Write_Bytes(stuff->outstream, Str_Get_Ptr8(stuff->json), size);
-    OutStream_Close(stuff->outstream);
-}
-
-bool
-LFLock_Request_Shared_IMP(LockFileLock *self) {
-    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-
-    if (ivars->exclusive_only) {
-        THROW(ERR, "Can't request shared lock if exclusive_only is set");
-    }
-
-    if (ivars->state != LFLOCK_STATE_UNLOCKED) {
-        THROW(ERR, "Lock already acquired");
-    }
-
-    // TODO: The is_locked test and subsequent file creation is prone to a
-    // race condition. We could protect the whole process with an internal
-    // exclusive lock.
-
-    if (S_is_locked_exclusive(ivars)) {
-        String *msg = Str_newf("'%o.lock' is locked", ivars->name);
-        Err_set_error((Err*)LockErr_new(msg));
-        return false;
-    }
-
-    String *path = NULL;
-
-    uint32_t i = 0;
-    do {
-        DECREF(path);
-        path = Str_newf("locks/%o-%u32.lock", ivars->name, ++i);
-    } while (Folder_Exists(ivars->folder, path));
-
-    if (S_request(ivars, path)) {
-        ivars->shared_lock_path = path;
-        ivars->state = LFLOCK_STATE_LOCKED_SHARED;
-        return true;
-    }
-    else {
-        DECREF(path);
-        return false;
-    }
-}
-
-bool
-LFLock_Request_Exclusive_IMP(LockFileLock *self) {
-    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-
-    if (ivars->state != LFLOCK_STATE_UNLOCKED) {
-        THROW(ERR, "Lock already acquired");
-    }
-
-    // TODO: The is_locked test and subsequent file creation is prone to a
-    // race condition. We could protect the whole process with an internal
-    // exclusive lock.
-
-    if (ivars->exclusive_only
-        ? S_is_locked_exclusive(ivars)
-        : S_is_locked(ivars)
-       ) {
-        String *msg = Str_newf("'%o.lock' is locked", ivars->name);
-        Err_set_error((Err*)LockErr_new(msg));
-        return false;
-    }
-
-    if (S_request(ivars, ivars->lock_path)) {
-        ivars->state = LFLOCK_STATE_LOCKED_EXCLUSIVE;
-        return true;
-    }
-    else {
-        return false;
-    }
-}
-
-static bool
-S_request(LockFileLockIVARS *ivars, String *lock_path) {
-    bool success = false;
-
-    // Create the "locks" subdirectory if necessary.
-    String *lock_dir_name = SSTR_WRAP_C("locks");
-    if (!Folder_Exists(ivars->folder, lock_dir_name)) {
-        if (!Folder_MkDir(ivars->folder, lock_dir_name)) {
-            Err *err = (Err*)INCREF(Err_get_error());
-            // Maybe our attempt failed because another process succeeded.
-            if (Folder_Find_Folder(ivars->folder, lock_dir_name)) {
-                DECREF(err);
-            }
-            else {
-                // Nope, everything failed, so bail out.
-                Err_set_error(err);
-                return false;
-            }
-        }
-    }
-
-    // Prepare to write pid, lock name, and host to the lock file as JSON.
-    Hash *file_data = Hash_new(3);
-    Hash_Store_Utf8(file_data, "pid", 3,
-                    (Obj*)Str_newf("%i32", (int32_t)PID_getpid()));
-    Hash_Store_Utf8(file_data, "host", 4, INCREF(ivars->host));
-    Hash_Store_Utf8(file_data, "name", 4, INCREF(ivars->name));
-    String *json = Json_to_json((Obj*)file_data);
-    DECREF(file_data);
-
-    // Write to a temporary file, then use the creation of a hard link to
-    // ensure atomic but non-destructive creation of the lockfile with its
-    // complete contents.
-
-    OutStream *outstream = Folder_Open_Out(ivars->folder, ivars->link_path);
-    if (!outstream) {
-        ERR_ADD_FRAME(Err_get_error());
-        DECREF(json);
-        return false;
-    }
-
-    struct lockfile_context context;
-    context.outstream = outstream;
-    context.json = json;
-    Err *json_error = Err_trap(S_write_lockfile_json, &context);
-    DECREF(outstream);
-    DECREF(json);
-    if (json_error) {
-        Err_set_error(json_error);
-    }
-    else {
-        success = Folder_Hard_Link(ivars->folder, ivars->link_path,
-                                   lock_path);
-        if (!success) {
-            // TODO: Only return a LockErr if errno == EEXIST, otherwise
-            // return a normal Err.
-            Err *hard_link_err = (Err*)CERTIFY(Err_get_error(), ERR);
-            String *msg = Str_newf("Failed to obtain lock at '%o': %o",
-                                   lock_path, Err_Get_Mess(hard_link_err));
-            Err_set_error((Err*)LockErr_new(msg));
-        }
-    }
-
-    // Verify that our temporary file got zapped.
-    bool deletion_failed = !Folder_Delete(ivars->folder, ivars->link_path);
-    if (deletion_failed) {
-        String *mess = MAKE_MESS("Failed to delete '%o'", ivars->link_path);
-        Err_throw_mess(ERR, mess);
-    }
-
-    return success;
-}
-
-void
-LFLock_Release_IMP(LockFileLock *self) {
-    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-
-    if (ivars->state == LFLOCK_STATE_UNLOCKED) {
-        THROW(ERR, "Lock not acquired");
-    }
-
-    if (ivars->state == LFLOCK_STATE_LOCKED_EXCLUSIVE) {
-        if (Folder_Exists(ivars->folder, ivars->lock_path)) {
-            S_maybe_delete_file(ivars, ivars->lock_path, true, false);
-        }
-    }
-    else { // Shared lock.
-        if (Folder_Exists(ivars->folder, ivars->shared_lock_path)) {
-            S_maybe_delete_file(ivars, ivars->shared_lock_path, true, false);
-        }
-
-        // Empty out lock_path.
-        DECREF(ivars->shared_lock_path);
-        ivars->shared_lock_path = NULL;
-    }
-
-    ivars->state = LFLOCK_STATE_UNLOCKED;
-}
-
-static bool
-S_is_locked_exclusive(LockFileLockIVARS *ivars) {
-    return Folder_Exists(ivars->folder, ivars->lock_path)
-           && !S_maybe_delete_file(ivars, ivars->lock_path, false, true);
-}
-
-static bool
-S_is_locked(LockFileLockIVARS *ivars) {
-    if (S_is_locked_exclusive(ivars)) { return true; }
-
-    // Check for shared lock.
-
-    String *lock_dir_name = SSTR_WRAP_C("locks");
-    if (!Folder_Find_Folder(ivars->folder, lock_dir_name)) {
-        return false;
-    }
-
-    bool locked = false;
-    DirHandle *dh = Folder_Open_Dir(ivars->folder, lock_dir_name);
-    if (!dh) { RETHROW(INCREF(Err_get_error())); }
-
-    while (DH_Next(dh)) {
-        String *entry = DH_Get_Entry(dh);
-        if (S_is_shared_lock_file(ivars, entry)) {
-            String *candidate = Str_newf("%o/%o", lock_dir_name, entry);
-            if (!S_maybe_delete_file(ivars, candidate, false, true)) {
-                locked = true;
-            }
-            DECREF(candidate);
-        }
-        DECREF(entry);
-    }
-
-    DECREF(dh);
-    return locked;
-}
-
-static bool
-S_is_shared_lock_file(LockFileLockIVARS *ivars, String *entry) {
-    // Translation:  $match = $entry =~ /^\Q$name-\d+\.lock\z/
-    bool match = false;
-
-    // $name
-    if (Str_Starts_With(entry, ivars->name)) {
-        StringIterator *iter = Str_Top(entry);
-        StrIter_Advance(iter, Str_Length(ivars->name));
-
-        // Hyphen-minus
-        if (StrIter_Next(iter) == '-') {
-            int32_t code_point = StrIter_Next(iter);
-
-            // Digit
-            if (code_point >= '0' && code_point <= '9') {
-                // Optional digits
-                do {
-                    code_point = StrIter_Next(iter);
-                } while (code_point >= '0' && code_point <= '9');
-
-                // ".lock"
-                match = code_point == '.'
-                        && StrIter_Starts_With_Utf8(iter, "lock", 4)
-                        && StrIter_Advance(iter, SIZE_MAX) == 4;
-            }
-        }
-
-        DECREF(iter);
-    }
-
-    return match;
-}
-
-static bool
-S_maybe_delete_file(LockFileLockIVARS *ivars, String *path,
-                    bool delete_mine, bool delete_other) {
-    Folder *folder  = ivars->folder;
-    bool    success = false;
-
-    Hash *hash = (Hash*)Json_slurp_json(folder, path);
-    if (hash != NULL && Obj_is_a((Obj*)hash, HASH)) {
-        String *pid_buf = (String*)Hash_Fetch_Utf8(hash, "pid", 3);
-        String *host    = (String*)Hash_Fetch_Utf8(hash, "host", 4);
-        String *name    = (String*)Hash_Fetch_Utf8(hash, "name", 4);
-
-        // Match hostname and lock name.
-        if (host != NULL
-            && Str_is_a(host, STRING)
-            && Str_Equals(host, (Obj*)ivars->host)
-            && name != NULL
-            && Str_is_a(name, STRING)
-            && Str_Equals(name, (Obj*)ivars->name)
-            && pid_buf != NULL
-            && Str_is_a(pid_buf, STRING)
-           ) {
-            // Verify that pid is either mine or dead.
-            int pid = (int)Str_To_I64(pid_buf);
-            if ((delete_mine && pid == PID_getpid())  // This process.
-                || (delete_other && !PID_active(pid)) // Dead pid.
-               ) {
-                if (Folder_Delete(folder, path)) {
-                    success = true;
-                }
-                else {
-                    String *mess
-                        = MAKE_MESS("Can't delete '%o'", path);
-                    DECREF(hash);
-                    Err_throw_mess(ERR, mess);
-                }
-            }
-        }
-    }
-    DECREF(hash);
-
-    return success;
-}
-
-void
-LFLock_Destroy_IMP(LockFileLock *self) {
-    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-    if (ivars->state != LFLOCK_STATE_UNLOCKED) { LFLock_Release(self); }
-    DECREF(ivars->host);
-    DECREF(ivars->link_path);
-    SUPER_DESTROY(self, LOCKFILELOCK);
-}
-
-/***************************************************************************/
-
 LockErr*
 LockErr_new(String *message) {
     LockErr *self = (LockErr*)Class_Make_Obj(LOCKERR);

http://git-wip-us.apache.org/repos/asf/lucy/blob/6e8538a8/core/Lucy/Store/Lock.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/Lock.cfh b/core/Lucy/Store/Lock.cfh
index 11f31c0..09c2c54 100644
--- a/core/Lucy/Store/Lock.cfh
+++ b/core/Lucy/Store/Lock.cfh
@@ -49,6 +49,9 @@ abstract class Lucy::Store::Lock inherits Clownfish::Obj {
     init(Lock *self, Folder *folder, String *name, int32_t timeout = 0,
          int32_t interval = 100);
 
+    inert bool
+    make_lock_dir(Folder *folder);
+
     /** Call [](.Request_Shared) once per `interval` until [](.Request_Shared)
      * returns success or the `timeout` has been reached.
      *
@@ -99,36 +102,6 @@ abstract class Lucy::Store::Lock inherits Clownfish::Obj {
     Destroy(Lock *self);
 }
 
-class Lucy::Store::LockFileLock nickname LFLock
-    inherits Lucy::Store::Lock {
-
-    String *host;
-    String *shared_lock_path;
-    String *link_path;
-    int     state;
-    bool    exclusive_only;
-
-    inert incremented LockFileLock*
-    new(Folder *folder, String *name, String *host, int32_t timeout = 0,
-        int32_t interval = 100, bool exclusive_only);
-
-    public inert LockFileLock*
-    init(LockFileLock *self, Folder *folder, String *name, String *host,
-         int32_t timeout = 0, int32_t interval = 100, bool exclusive_only);
-
-    public bool
-    Request_Shared(LockFileLock *self);
-
-    public bool
-    Request_Exclusive(LockFileLock *self);
-
-    public void
-    Release(LockFileLock *self);
-
-    public void
-    Destroy(LockFileLock *self);
-}
-
 /** Lock exception.
  *
  * LockErr is a subclass of [Err](cfish:cfish.Err) which indicates

http://git-wip-us.apache.org/repos/asf/lucy/blob/6e8538a8/core/Lucy/Store/LockFileLock.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/LockFileLock.c b/core/Lucy/Store/LockFileLock.c
new file mode 100644
index 0000000..19d624e
--- /dev/null
+++ b/core/Lucy/Store/LockFileLock.c
@@ -0,0 +1,363 @@
+/* Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#define C_LUCY_LOCKFILELOCK
+#include "Lucy/Util/ToolSet.h"
+
+#include "Lucy/Store/LockFileLock.h"
+#include "Lucy/Store/DirHandle.h"
+#include "Lucy/Store/Folder.h"
+#include "Lucy/Store/OutStream.h"
+#include "Lucy/Util/Json.h"
+#include "Lucy/Util/ProcessID.h"
+
+#define LFLOCK_STATE_UNLOCKED          0
+#define LFLOCK_STATE_LOCKED_SHARED     1
+#define LFLOCK_STATE_LOCKED_EXCLUSIVE  2
+
+static bool
+S_request(LockFileLockIVARS *ivars, String *lock_path);
+
+static bool
+S_is_locked_exclusive(LockFileLockIVARS *ivars);
+
+static bool
+S_is_locked(LockFileLockIVARS *ivars);
+
+static bool
+S_is_shared_lock_file(LockFileLockIVARS *ivars, String *entry);
+
+static bool
+S_maybe_delete_file(LockFileLockIVARS *ivars, String *path,
+                    bool delete_mine, bool delete_other);
+
+LockFileLock*
+LFLock_new(Folder *folder, String *name, String *host, int32_t timeout,
+           int32_t interval, bool exclusive_only) {
+    LockFileLock *self = (LockFileLock*)Class_Make_Obj(LOCKFILELOCK);
+    return LFLock_init(self, folder, name, host, timeout, interval,
+                       exclusive_only);
+}
+
+LockFileLock*
+LFLock_init(LockFileLock *self, Folder *folder, String *name, String *host,
+            int32_t timeout, int32_t interval, bool exclusive_only) {
+    int pid = PID_getpid();
+    Lock_init((Lock*)self, folder, name, timeout, interval);
+    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+    ivars->host      = (String*)INCREF(host);
+    ivars->link_path = Str_newf("%o.%o.%i64", ivars->lock_path, host,
+                                (int64_t)pid);
+    ivars->exclusive_only = exclusive_only;
+    return self;
+}
+
+struct lockfile_context {
+    OutStream *outstream;
+    String *json;
+};
+
+static void
+S_write_lockfile_json(void *context) {
+    struct lockfile_context *stuff = (struct lockfile_context*)context;
+    size_t size = Str_Get_Size(stuff->json);
+    OutStream_Write_Bytes(stuff->outstream, Str_Get_Ptr8(stuff->json), size);
+    OutStream_Close(stuff->outstream);
+}
+
+bool
+LFLock_Request_Shared_IMP(LockFileLock *self) {
+    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+
+    if (ivars->exclusive_only) {
+        THROW(ERR, "Can't request shared lock if exclusive_only is set");
+    }
+
+    if (ivars->state != LFLOCK_STATE_UNLOCKED) {
+        THROW(ERR, "Lock already acquired");
+    }
+
+    // TODO: The is_locked test and subsequent file creation is prone to a
+    // race condition. We could protect the whole process with an internal
+    // exclusive lock.
+
+    if (S_is_locked_exclusive(ivars)) {
+        String *msg = Str_newf("'%o.lock' is locked", ivars->name);
+        Err_set_error((Err*)LockErr_new(msg));
+        return false;
+    }
+
+    String *path = NULL;
+
+    uint32_t i = 0;
+    do {
+        DECREF(path);
+        path = Str_newf("locks/%o-%u32.lock", ivars->name, ++i);
+    } while (Folder_Exists(ivars->folder, path));
+
+    if (S_request(ivars, path)) {
+        ivars->shared_lock_path = path;
+        ivars->state = LFLOCK_STATE_LOCKED_SHARED;
+        return true;
+    }
+    else {
+        DECREF(path);
+        return false;
+    }
+}
+
+bool
+LFLock_Request_Exclusive_IMP(LockFileLock *self) {
+    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+
+    if (ivars->state != LFLOCK_STATE_UNLOCKED) {
+        THROW(ERR, "Lock already acquired");
+    }
+
+    // TODO: The is_locked test and subsequent file creation is prone to a
+    // race condition. We could protect the whole process with an internal
+    // exclusive lock.
+
+    if (ivars->exclusive_only
+        ? S_is_locked_exclusive(ivars)
+        : S_is_locked(ivars)
+       ) {
+        String *msg = Str_newf("'%o.lock' is locked", ivars->name);
+        Err_set_error((Err*)LockErr_new(msg));
+        return false;
+    }
+
+    if (S_request(ivars, ivars->lock_path)) {
+        ivars->state = LFLOCK_STATE_LOCKED_EXCLUSIVE;
+        return true;
+    }
+    else {
+        return false;
+    }
+}
+
+static bool
+S_request(LockFileLockIVARS *ivars, String *lock_path) {
+    bool success = false;
+
+    // Create the "locks" subdirectory if necessary.
+    String *lock_dir_name = SSTR_WRAP_C("locks");
+    if (!Folder_Exists(ivars->folder, lock_dir_name)) {
+        if (!Lock_make_lock_dir(ivars->folder)) { return false; }
+    }
+
+    // Prepare to write pid, lock name, and host to the lock file as JSON.
+    Hash *file_data = Hash_new(3);
+    Hash_Store_Utf8(file_data, "pid", 3,
+                    (Obj*)Str_newf("%i32", (int32_t)PID_getpid()));
+    Hash_Store_Utf8(file_data, "host", 4, INCREF(ivars->host));
+    Hash_Store_Utf8(file_data, "name", 4, INCREF(ivars->name));
+    String *json = Json_to_json((Obj*)file_data);
+    DECREF(file_data);
+
+    // Write to a temporary file, then use the creation of a hard link to
+    // ensure atomic but non-destructive creation of the lockfile with its
+    // complete contents.
+
+    OutStream *outstream = Folder_Open_Out(ivars->folder, ivars->link_path);
+    if (!outstream) {
+        ERR_ADD_FRAME(Err_get_error());
+        DECREF(json);
+        return false;
+    }
+
+    struct lockfile_context context;
+    context.outstream = outstream;
+    context.json = json;
+    Err *json_error = Err_trap(S_write_lockfile_json, &context);
+    DECREF(outstream);
+    DECREF(json);
+    if (json_error) {
+        Err_set_error(json_error);
+    }
+    else {
+        success = Folder_Hard_Link(ivars->folder, ivars->link_path,
+                                   lock_path);
+        if (!success) {
+            // TODO: Only return a LockErr if errno == EEXIST, otherwise
+            // return a normal Err.
+            Err *hard_link_err = (Err*)CERTIFY(Err_get_error(), ERR);
+            String *msg = Str_newf("Failed to obtain lock at '%o': %o",
+                                   lock_path, Err_Get_Mess(hard_link_err));
+            Err_set_error((Err*)LockErr_new(msg));
+        }
+    }
+
+    // Verify that our temporary file got zapped.
+    bool deletion_failed = !Folder_Delete(ivars->folder, ivars->link_path);
+    if (deletion_failed) {
+        String *mess = MAKE_MESS("Failed to delete '%o'", ivars->link_path);
+        Err_throw_mess(ERR, mess);
+    }
+
+    return success;
+}
+
+void
+LFLock_Release_IMP(LockFileLock *self) {
+    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+
+    if (ivars->state == LFLOCK_STATE_UNLOCKED) {
+        THROW(ERR, "Lock not acquired");
+    }
+
+    if (ivars->state == LFLOCK_STATE_LOCKED_EXCLUSIVE) {
+        if (Folder_Exists(ivars->folder, ivars->lock_path)) {
+            S_maybe_delete_file(ivars, ivars->lock_path, true, false);
+        }
+    }
+    else { // Shared lock.
+        if (Folder_Exists(ivars->folder, ivars->shared_lock_path)) {
+            S_maybe_delete_file(ivars, ivars->shared_lock_path, true, false);
+        }
+
+        // Empty out lock_path.
+        DECREF(ivars->shared_lock_path);
+        ivars->shared_lock_path = NULL;
+    }
+
+    ivars->state = LFLOCK_STATE_UNLOCKED;
+}
+
+static bool
+S_is_locked_exclusive(LockFileLockIVARS *ivars) {
+    return Folder_Exists(ivars->folder, ivars->lock_path)
+           && !S_maybe_delete_file(ivars, ivars->lock_path, false, true);
+}
+
+static bool
+S_is_locked(LockFileLockIVARS *ivars) {
+    if (S_is_locked_exclusive(ivars)) { return true; }
+
+    // Check for shared lock.
+
+    String *lock_dir_name = SSTR_WRAP_C("locks");
+    if (!Folder_Find_Folder(ivars->folder, lock_dir_name)) {
+        return false;
+    }
+
+    bool locked = false;
+    DirHandle *dh = Folder_Open_Dir(ivars->folder, lock_dir_name);
+    if (!dh) { RETHROW(INCREF(Err_get_error())); }
+
+    while (DH_Next(dh)) {
+        String *entry = DH_Get_Entry(dh);
+        if (S_is_shared_lock_file(ivars, entry)) {
+            String *candidate = Str_newf("%o/%o", lock_dir_name, entry);
+            if (!S_maybe_delete_file(ivars, candidate, false, true)) {
+                locked = true;
+            }
+            DECREF(candidate);
+        }
+        DECREF(entry);
+    }
+
+    DECREF(dh);
+    return locked;
+}
+
+static bool
+S_is_shared_lock_file(LockFileLockIVARS *ivars, String *entry) {
+    // Translation:  $match = $entry =~ /^\Q$name-\d+\.lock\z/
+    bool match = false;
+
+    // $name
+    if (Str_Starts_With(entry, ivars->name)) {
+        StringIterator *iter = Str_Top(entry);
+        StrIter_Advance(iter, Str_Length(ivars->name));
+
+        // Hyphen-minus
+        if (StrIter_Next(iter) == '-') {
+            int32_t code_point = StrIter_Next(iter);
+
+            // Digit
+            if (code_point >= '0' && code_point <= '9') {
+                // Optional digits
+                do {
+                    code_point = StrIter_Next(iter);
+                } while (code_point >= '0' && code_point <= '9');
+
+                // ".lock"
+                match = code_point == '.'
+                        && StrIter_Starts_With_Utf8(iter, "lock", 4)
+                        && StrIter_Advance(iter, SIZE_MAX) == 4;
+            }
+        }
+
+        DECREF(iter);
+    }
+
+    return match;
+}
+
+static bool
+S_maybe_delete_file(LockFileLockIVARS *ivars, String *path,
+                    bool delete_mine, bool delete_other) {
+    Folder *folder  = ivars->folder;
+    bool    success = false;
+
+    Hash *hash = (Hash*)Json_slurp_json(folder, path);
+    if (hash != NULL && Obj_is_a((Obj*)hash, HASH)) {
+        String *pid_buf = (String*)Hash_Fetch_Utf8(hash, "pid", 3);
+        String *host    = (String*)Hash_Fetch_Utf8(hash, "host", 4);
+        String *name    = (String*)Hash_Fetch_Utf8(hash, "name", 4);
+
+        // Match hostname and lock name.
+        if (host != NULL
+            && Str_is_a(host, STRING)
+            && Str_Equals(host, (Obj*)ivars->host)
+            && name != NULL
+            && Str_is_a(name, STRING)
+            && Str_Equals(name, (Obj*)ivars->name)
+            && pid_buf != NULL
+            && Str_is_a(pid_buf, STRING)
+           ) {
+            // Verify that pid is either mine or dead.
+            int pid = (int)Str_To_I64(pid_buf);
+            if ((delete_mine && pid == PID_getpid())  // This process.
+                || (delete_other && !PID_active(pid)) // Dead pid.
+               ) {
+                if (Folder_Delete(folder, path)) {
+                    success = true;
+                }
+                else {
+                    String *mess
+                        = MAKE_MESS("Can't delete '%o'", path);
+                    DECREF(hash);
+                    Err_throw_mess(ERR, mess);
+                }
+            }
+        }
+    }
+    DECREF(hash);
+
+    return success;
+}
+
+void
+LFLock_Destroy_IMP(LockFileLock *self) {
+    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+    if (ivars->state != LFLOCK_STATE_UNLOCKED) { LFLock_Release(self); }
+    DECREF(ivars->host);
+    DECREF(ivars->link_path);
+    SUPER_DESTROY(self, LOCKFILELOCK);
+}
+

http://git-wip-us.apache.org/repos/asf/lucy/blob/6e8538a8/core/Lucy/Store/LockFileLock.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/LockFileLock.cfh b/core/Lucy/Store/LockFileLock.cfh
new file mode 100644
index 0000000..30fc23a
--- /dev/null
+++ b/core/Lucy/Store/LockFileLock.cfh
@@ -0,0 +1,49 @@
+/* Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+parcel Lucy;
+
+class Lucy::Store::LockFileLock nickname LFLock
+    inherits Lucy::Store::Lock {
+
+    String *host;
+    String *shared_lock_path;
+    String *link_path;
+    int     state;
+    bool    exclusive_only;
+
+    inert incremented LockFileLock*
+    new(Folder *folder, String *name, String *host, int32_t timeout = 0,
+        int32_t interval = 100, bool exclusive_only);
+
+    public inert LockFileLock*
+    init(LockFileLock *self, Folder *folder, String *name, String *host,
+         int32_t timeout = 0, int32_t interval = 100, bool exclusive_only);
+
+    public bool
+    Request_Shared(LockFileLock *self);
+
+    public bool
+    Request_Exclusive(LockFileLock *self);
+
+    public void
+    Release(LockFileLock *self);
+
+    public void
+    Destroy(LockFileLock *self);
+}
+
+

http://git-wip-us.apache.org/repos/asf/lucy/blob/6e8538a8/test/Lucy/Test/Store/TestLock.c
----------------------------------------------------------------------
diff --git a/test/Lucy/Test/Store/TestLock.c b/test/Lucy/Test/Store/TestLock.c
index b52f366..df3eb20 100644
--- a/test/Lucy/Test/Store/TestLock.c
+++ b/test/Lucy/Test/Store/TestLock.c
@@ -30,7 +30,7 @@
 #include "Lucy/Test/Store/TestLock.h"
 #include "Clownfish/TestHarness/TestBatchRunner.h"
 #include "Lucy/Store/FSFolder.h"
-#include "Lucy/Store/Lock.h"
+#include "Lucy/Store/LockFileLock.h"
 #include "Lucy/Store/RAMFolder.h"
 #include "Lucy/Util/Json.h"