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:42 UTC

[04/16] lucy git commit: Merge SharedLock into LockFileLock

Merge SharedLock into LockFileLock

Supplying the lock type when requesting a lock better matches other
file lock APIs. Merging the implementations allows to convert
Maybe_Delete_File into a static function.

Make shared and exclusive locks check for locks of the other type
except for exclusive locks created with `exclusive_only`.

Clear_Stale is broken for shared locks but will be removed with the
following commit.


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

Branch: refs/heads/master
Commit: bb83d0d77bf99ed09b2bbad782c3c1d3b94f04a9
Parents: 2051451
Author: Nick Wellnhofer <we...@aevum.de>
Authored: Thu Feb 16 15:58:45 2017 +0100
Committer: Nick Wellnhofer <we...@aevum.de>
Committed: Mon Feb 20 16:26:21 2017 +0100

----------------------------------------------------------------------
 core/Lucy/Docs/FileLocking.md             |   5 +-
 core/Lucy/Index/BackgroundMerger.c        |   4 +-
 core/Lucy/Index/FilePurger.c              |   4 +-
 core/Lucy/Index/IndexManager.c            |  11 +-
 core/Lucy/Index/Indexer.c                 |   6 +-
 core/Lucy/Index/PolyReader.c              |   4 +-
 core/Lucy/Store/Lock.c                    | 326 +++++++++++++++++++------
 core/Lucy/Store/Lock.cfh                  |  86 ++++---
 core/Lucy/Store/LockFactory.c             |  15 +-
 core/Lucy/Store/LockFactory.cfh           |  21 +-
 core/Lucy/Store/SharedLock.c              | 173 -------------
 core/Lucy/Store/SharedLock.cfh            |  77 ------
 go/build.go                               |   6 +-
 go/lucy/index_test.go                     |   2 +-
 go/lucy/store.go                          |  28 ++-
 go/lucy/store_test.go                     |  62 +++--
 perl/buildlib/Lucy/Build/Binding/Store.pm |   2 +-
 perl/t/105-folder.t                       |  34 +--
 perl/t/106-locking.t                      |   9 +-
 perl/t/110-shared_lock.t                  |  57 +++--
 perl/t/111-index_manager.t                |  15 +-
 21 files changed, 449 insertions(+), 498 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Docs/FileLocking.md
----------------------------------------------------------------------
diff --git a/core/Lucy/Docs/FileLocking.md b/core/Lucy/Docs/FileLocking.md
index 911df21..2a6f5b8 100644
--- a/core/Lucy/Docs/FileLocking.md
+++ b/core/Lucy/Docs/FileLocking.md
@@ -43,9 +43,8 @@ process will crash with a "Stale NFS filehandle" exception.
 
 Under normal circumstances, it is neither necessary nor desirable for
 IndexReaders to secure read locks against an index, but for NFS we have to
-make an exception.  LockFactory's [](lucy.LockFactory.Make_Shared_Lock) method exists for this
-reason; supplying an IndexManager instance to IndexReader's constructor
-activates an internal locking mechanism using [](lucy.LockFactory.Make_Shared_Lock) which
+make an exception.  Supplying an IndexManager instance to IndexReader's
+constructor activates an internal locking mechanism using shared locks which
 prevents concurrent indexing processes from deleting files that are needed
 by active readers.
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Index/BackgroundMerger.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/BackgroundMerger.c b/core/Lucy/Index/BackgroundMerger.c
index 8f36050..661b9a6 100644
--- a/core/Lucy/Index/BackgroundMerger.c
+++ b/core/Lucy/Index/BackgroundMerger.c
@@ -530,7 +530,7 @@ S_obtain_write_lock(BackgroundMerger *self) {
     BackgroundMergerIVARS *const ivars = BGMerger_IVARS(self);
     Lock *write_lock = IxManager_Make_Write_Lock(ivars->manager);
     Lock_Clear_Stale(write_lock);
-    if (Lock_Obtain(write_lock)) {
+    if (Lock_Obtain_Exclusive(write_lock)) {
         // Only assign if successful, otherwise DESTROY unlocks -- bad!
         ivars->write_lock = write_lock;
     }
@@ -544,7 +544,7 @@ S_obtain_merge_lock(BackgroundMerger *self) {
     BackgroundMergerIVARS *const ivars = BGMerger_IVARS(self);
     Lock *merge_lock = IxManager_Make_Merge_Lock(ivars->manager);
     Lock_Clear_Stale(merge_lock);
-    if (Lock_Obtain(merge_lock)) {
+    if (Lock_Obtain_Exclusive(merge_lock)) {
         // Only assign if successful, same rationale as above.
         ivars->merge_lock = merge_lock;
     }

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Index/FilePurger.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/FilePurger.c b/core/Lucy/Index/FilePurger.c
index 55ade17..3980c8f 100644
--- a/core/Lucy/Index/FilePurger.c
+++ b/core/Lucy/Index/FilePurger.c
@@ -79,7 +79,7 @@ FilePurger_Purge_Snapshots_IMP(FilePurger *self, Snapshot *current) {
 
     // Obtain deletion lock, purge files, release deletion lock.
     Lock_Clear_Stale(deletion_lock);
-    if (Lock_Obtain(deletion_lock)) {
+    if (Lock_Obtain_Exclusive(deletion_lock)) {
         Folder *folder    = ivars->folder;
         Hash   *failures  = Hash_new(16);
         Hash   *spared    = Hash_new(32);
@@ -148,7 +148,7 @@ FilePurger_Purge_Aborted_Merge_IMP(FilePurger *self) {
     Lock         *merge_lock = IxManager_Make_Merge_Lock(manager);
 
     Lock_Clear_Stale(merge_lock);
-    if (!Lock_Is_Locked(merge_lock)) {
+    if (!Lock_Is_Locked_Exclusive(merge_lock)) {
         Hash *merge_data = IxManager_Read_Merge_Data(manager);
         Obj  *cutoff = merge_data
                        ? Hash_Fetch_Utf8(merge_data, "cutoff", 6)

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Index/IndexManager.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/IndexManager.c b/core/Lucy/Index/IndexManager.c
index 15367fa..15cbbd9 100644
--- a/core/Lucy/Index/IndexManager.c
+++ b/core/Lucy/Index/IndexManager.c
@@ -275,7 +275,8 @@ IxManager_Make_Write_Lock_IMP(IndexManager *self) {
     LockFactory *lock_factory = S_obtain_lock_factory(self);
     return LockFact_Make_Lock(lock_factory, write_lock_name,
                               (int32_t)ivars->write_lock_timeout,
-                              (int32_t)ivars->write_lock_interval);
+                              (int32_t)ivars->write_lock_interval,
+                              true);
 }
 
 Lock*
@@ -285,7 +286,8 @@ IxManager_Make_Deletion_Lock_IMP(IndexManager *self) {
     LockFactory *lock_factory = S_obtain_lock_factory(self);
     return LockFact_Make_Lock(lock_factory, lock_name,
                               (int32_t)ivars->deletion_lock_timeout,
-                              (int32_t)ivars->deletion_lock_interval);
+                              (int32_t)ivars->deletion_lock_interval,
+                              true);
 }
 
 Lock*
@@ -295,7 +297,8 @@ IxManager_Make_Merge_Lock_IMP(IndexManager *self) {
     LockFactory *lock_factory = S_obtain_lock_factory(self);
     return LockFact_Make_Lock(lock_factory, merge_lock_name,
                               (int32_t)ivars->merge_lock_timeout,
-                              (int32_t)ivars->merge_lock_interval);
+                              (int32_t)ivars->merge_lock_interval,
+                              true);
 }
 
 void
@@ -353,7 +356,7 @@ IxManager_Make_Snapshot_Read_Lock_IMP(IndexManager *self,
     size_t lock_name_len = Str_Length(filename) - (sizeof(".json") - 1);
     String *lock_name = Str_SubString(filename, 0, lock_name_len);
 
-    Lock *lock = LockFact_Make_Shared_Lock(lock_factory, lock_name, 1000, 100);
+    Lock *lock = LockFact_Make_Lock(lock_factory, lock_name, 1000, 100, false);
 
     DECREF(lock_name);
     return lock;

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Index/Indexer.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/Indexer.c b/core/Lucy/Index/Indexer.c
index 031237f..97763e3 100644
--- a/core/Lucy/Index/Indexer.c
+++ b/core/Lucy/Index/Indexer.c
@@ -98,7 +98,7 @@ Indexer_init(Indexer *self, Schema *schema, Obj *index,
     // Get a write lock for this folder.
     Lock *write_lock = IxManager_Make_Write_Lock(ivars->manager);
     Lock_Clear_Stale(write_lock);
-    if (Lock_Obtain(write_lock)) {
+    if (Lock_Obtain_Exclusive(write_lock)) {
         // Only assign if successful, otherwise DESTROY unlocks -- bad!
         ivars->write_lock = write_lock;
     }
@@ -170,7 +170,7 @@ Indexer_init(Indexer *self, Schema *schema, Obj *index,
     int64_t new_seg_num
         = IxManager_Highest_Seg_Num(ivars->manager, latest_snapshot) + 1;
     Lock *merge_lock = IxManager_Make_Merge_Lock(ivars->manager);
-    if (Lock_Is_Locked(merge_lock)) {
+    if (Lock_Is_Locked_Exclusive(merge_lock)) {
         // If there's a background merge process going on, stay out of its
         // way.
         Hash *merge_data = IxManager_Read_Merge_Data(ivars->manager);
@@ -398,7 +398,7 @@ S_maybe_merge(Indexer *self, Vector *seg_readers) {
     bool      merge_happened  = false;
     size_t    num_seg_readers = Vec_Get_Size(seg_readers);
     Lock     *merge_lock      = IxManager_Make_Merge_Lock(ivars->manager);
-    bool      got_merge_lock  = Lock_Obtain(merge_lock);
+    bool      got_merge_lock  = Lock_Obtain_Exclusive(merge_lock);
     int64_t   cutoff;
 
     if (got_merge_lock) {

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Index/PolyReader.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/PolyReader.c b/core/Lucy/Index/PolyReader.c
index e843bcd..d4166c6 100644
--- a/core/Lucy/Index/PolyReader.c
+++ b/core/Lucy/Index/PolyReader.c
@@ -471,7 +471,7 @@ S_obtain_deletion_lock(PolyReader *self) {
     PolyReaderIVARS *const ivars = PolyReader_IVARS(self);
     ivars->deletion_lock = IxManager_Make_Deletion_Lock(ivars->manager);
     Lock_Clear_Stale(ivars->deletion_lock);
-    if (!Lock_Obtain(ivars->deletion_lock)) {
+    if (!Lock_Obtain_Exclusive(ivars->deletion_lock)) {
         DECREF(ivars->deletion_lock);
         ivars->deletion_lock = NULL;
         return false;
@@ -486,7 +486,7 @@ S_obtain_read_lock(PolyReader *self, String *snapshot_file_name) {
                                                          snapshot_file_name);
 
     Lock_Clear_Stale(ivars->read_lock);
-    if (!Lock_Obtain(ivars->read_lock)) {
+    if (!Lock_Obtain_Shared(ivars->read_lock)) {
         DECREF(ivars->read_lock);
         ivars->read_lock = NULL;
         return false;

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Store/Lock.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/Lock.c b/core/Lucy/Store/Lock.c
index 97592b0..0eb42da 100644
--- a/core/Lucy/Store/Lock.c
+++ b/core/Lucy/Store/Lock.c
@@ -22,6 +22,7 @@
 #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"
@@ -82,26 +83,38 @@ Lock_Get_Name_IMP(Lock *self) {
 }
 
 String*
-Lock_Get_Lock_Path_IMP(Lock *self) {
-    return Lock_IVARS(self)->lock_path;
-}
-
-String*
 Lock_Get_Host_IMP(Lock *self) {
     return Lock_IVARS(self)->host;
 }
 
 bool
-Lock_Obtain_IMP(Lock *self) {
+Lock_Obtain_Shared_IMP(Lock *self) {
+    LockIVARS *const ivars = Lock_IVARS(self);
+    int32_t time_left = ivars->interval == 0 ? 0 : ivars->timeout;
+    bool locked = Lock_Request_Shared(self);
+
+    while (!locked) {
+        time_left -= ivars->interval;
+        if (time_left <= 0) { break; }
+        Sleep_millisleep((uint32_t)ivars->interval);
+        locked = Lock_Request_Shared(self);
+    }
+
+    if (!locked) { ERR_ADD_FRAME(Err_get_error()); }
+    return locked;
+}
+
+bool
+Lock_Obtain_Exclusive_IMP(Lock *self) {
     LockIVARS *const ivars = Lock_IVARS(self);
     int32_t time_left = ivars->interval == 0 ? 0 : ivars->timeout;
-    bool locked = Lock_Request(self);
+    bool locked = Lock_Request_Exclusive(self);
 
     while (!locked) {
         time_left -= ivars->interval;
         if (time_left <= 0) { break; }
         Sleep_millisleep((uint32_t)ivars->interval);
-        locked = Lock_Request(self);
+        locked = Lock_Request_Exclusive(self);
     }
 
     if (!locked) { ERR_ADD_FRAME(Err_get_error()); }
@@ -110,27 +123,42 @@ Lock_Obtain_IMP(Lock *self) {
 
 /***************************************************************************/
 
+static bool
+S_request(LockFileLockIVARS *ivars, String *lock_path);
+
+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) {
+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);
+    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) {
+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, host, timeout, interval);
     LockFileLockIVARS *const ivars = LFLock_IVARS(self);
     ivars->link_path = Str_newf("%o.%o.%i64", ivars->lock_path, host,
                                 (int64_t)pid);
+    ivars->exclusive_only = exclusive_only;
     return self;
 }
 
-bool
-LFLock_Shared_IMP(LockFileLock *self) {
-    UNUSED_VAR(self); return false;
+String*
+LFLock_Get_Lock_Path_IMP(LockFileLock *self) {
+    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+    return ivars->shared_lock_path
+           ? ivars->shared_lock_path
+           : ivars->lock_path;
 }
 
 struct lockfile_context {
@@ -147,16 +175,75 @@ S_write_lockfile_json(void *context) {
 }
 
 bool
-LFLock_Request_IMP(LockFileLock *self) {
+LFLock_Request_Shared_IMP(LockFileLock *self) {
     LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-    bool success = false;
 
-    if (Folder_Exists(ivars->folder, ivars->lock_path)) {
-        Err_set_error((Err*)LockErr_new(Str_newf("Can't obtain lock: '%o' exists",
-                                                 ivars->lock_path)));
+    if (ivars->exclusive_only) {
+        THROW(ERR, "Can't request shared lock if exclusive_only is set");
+    }
+
+    // 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 (LFLock_Is_Locked_Exclusive(self)) {
+        String *msg = Str_newf("'%o.lock' is locked", ivars->name);
+        Err_set_error((Err*)LockErr_new(msg));
+        return false;
+    }
+
+    String *path = ivars->shared_lock_path;
+
+    // Null shared_lock_path indicates whether this particular instance is
+    // locked.
+    if (path && Folder_Exists(ivars->folder, path)) {
+        // Don't allow double obtain.
+        String *msg = Str_newf("Lock already obtained via '%o'", path);
+        Err_set_error((Err*)LockErr_new(msg));
+        return false;
+    }
+
+    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;
+        return true;
+    }
+    else {
+        DECREF(path);
+        ivars->shared_lock_path = NULL;
+        return false;
+    }
+}
+
+bool
+LFLock_Request_Exclusive_IMP(LockFileLock *self) {
+    LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+
+    // 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
+        ? LFLock_Is_Locked_Exclusive(self)
+        : LFLock_Is_Locked(self)
+       ) {
+        String *msg = Str_newf("'%o.lock' is locked", ivars->name);
+        Err_set_error((Err*)LockErr_new(msg));
         return false;
     }
 
+    return S_request(ivars, ivars->lock_path);
+}
+
+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)) {
@@ -205,17 +292,17 @@ LFLock_Request_IMP(LockFileLock *self) {
     DECREF(json);
     if (wrote_json) {
         success = Folder_Hard_Link(ivars->folder, ivars->link_path,
-                                   ivars->lock_path);
+                                   lock_path);
         if (!success) {
             Err *hard_link_err = (Err*)CERTIFY(Err_get_error(), ERR);
             Err_set_error((Err*)LockErr_new(Str_newf("Failed to obtain lock at '%o': %o",
-                                                     ivars->lock_path,
+                                                     lock_path,
                                                      Err_Get_Mess(hard_link_err))));
         }
     }
     else {
         Err_set_error((Err*)LockErr_new(Str_newf("Failed to obtain lock at '%o': %o",
-                                                 ivars->lock_path,
+                                                 lock_path,
                                                  Err_Get_Mess(json_error))));
         DECREF(json_error);
     }
@@ -233,79 +320,167 @@ LFLock_Request_IMP(LockFileLock *self) {
 void
 LFLock_Release_IMP(LockFileLock *self) {
     LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-    if (Folder_Exists(ivars->folder, ivars->lock_path)) {
-        LFLock_Maybe_Delete_File(self, ivars->lock_path, true, false);
+
+    if (ivars->shared_lock_path) {
+        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;
+    }
+    else {
+        if (Folder_Exists(ivars->folder, ivars->lock_path)) {
+            S_maybe_delete_file(ivars, ivars->lock_path, true, false);
+        }
     }
 }
 
 bool
-LFLock_Is_Locked_IMP(LockFileLock *self) {
+LFLock_Is_Locked_Exclusive_IMP(LockFileLock *self) {
     LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+
     return Folder_Exists(ivars->folder, ivars->lock_path);
 }
 
-void
-LFLock_Clear_Stale_IMP(LockFileLock *self) {
+bool
+LFLock_Is_Locked_IMP(LockFileLock *self) {
     LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-    LFLock_Maybe_Delete_File(self, ivars->lock_path, false, true);
+
+    // Check for exclusive lock.
+    if (Folder_Exists(ivars->folder, ivars->lock_path)) {
+        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;
+    }
+
+    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)) {
+            DECREF(entry);
+            DECREF(dh);
+            return true;
+        }
+        DECREF(entry);
+    }
+
+    DECREF(dh);
+    return false;
 }
 
-bool
-LFLock_Maybe_Delete_File_IMP(LockFileLock *self, String *path,
-                             bool delete_mine, bool delete_other) {
+void
+LFLock_Clear_Stale_IMP(LockFileLock *self) {
     LockFileLockIVARS *const ivars = LFLock_IVARS(self);
-    Folder *folder  = ivars->folder;
-    bool    success = false;
 
-    // Only delete locks that start with our lock name.
-    if (!Str_Starts_With_Utf8(path, "locks", 5)) {
-        return false;
+    if (ivars->shared_lock_path) {
+        String *lock_dir_name = SSTR_WRAP_C("locks");
+        if (!Folder_Find_Folder(ivars->folder, lock_dir_name)) {
+            return;
+        }
+
+        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);
+                S_maybe_delete_file(ivars, candidate, false, true);
+                DECREF(candidate);
+            }
+            DECREF(entry);
+        }
+
+        DECREF(dh);
     }
-    StringIterator *iter = Str_Top(path);
-    StrIter_Advance(iter, 5 + 1);
-    if (!StrIter_Starts_With(iter, ivars->name)) {
+    else {
+        S_maybe_delete_file(ivars, ivars->lock_path, false, true);
+    }
+}
+
+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 false;
     }
-    DECREF(iter);
 
-    // Attempt to delete dead lock file.
-    if (Folder_Exists(folder, path)) {
-        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)
+    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.
                ) {
-                // 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);
-                    }
+                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);
     }
+    DECREF(hash);
 
     return success;
 }
@@ -313,6 +488,7 @@ LFLock_Maybe_Delete_File_IMP(LockFileLock *self, String *path,
 void
 LFLock_Destroy_IMP(LockFileLock *self) {
     LockFileLockIVARS *const ivars = LFLock_IVARS(self);
+    DECREF(ivars->shared_lock_path);
     DECREF(ivars->link_path);
     SUPER_DESTROY(self, LOCKFILELOCK);
 }

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Store/Lock.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/Lock.cfh b/core/Lucy/Store/Lock.cfh
index 111a045..3f93dc0 100644
--- a/core/Lucy/Store/Lock.cfh
+++ b/core/Lucy/Store/Lock.cfh
@@ -43,41 +43,53 @@ public abstract class Lucy::Store::Lock inherits Clownfish::Obj {
      * consist solely of characters matching [-_.A-Za-z0-9].
      * @param host A unique per-machine identifier.
      * @param timeout Time in milliseconds to keep retrying before abandoning
-     * the attempt to [](cfish:.Obtain) a lock.
+     * the attempt to obtain a lock.
      * @param interval Time in milliseconds between retries.
      */
     public inert Lock*
     init(Lock *self, Folder *folder, String *name,
          String *host, int32_t timeout = 0, int32_t interval = 100);
 
-    /** Returns true if the Lock is shared, false if the Lock is exclusive.
+    /** Call [](.Request_Shared) once per `interval` until [](.Request_Shared)
+     * returns success or the `timeout` has been reached.
+     *
+     * @return true on success, false on failure (sets the global error object
+     * returned by [](cfish:cfish.Err.get_error)).
      */
-    public abstract bool
-    Shared(Lock *self);
+    public bool
+    Obtain_Shared(Lock *self);
 
-    /** Call [](cfish:.Request) once per `interval` until [](cfish:.Request) returns
-     * success or the `timeout` has been reached.
+    /** Call [](.Request_Exclusive) once per `interval` until
+     * [](.Request_Exclusive) returns success or the `timeout` has been
+     * reached.
      *
      * @return true on success, false on failure (sets the global error object
      * returned by [](cfish:cfish.Err.get_error)).
      */
     public bool
-    Obtain(Lock *self);
+    Obtain_Exclusive(Lock *self);
+
+    /** Make one attempt to acquire a shared lock.
+     *
+     * [](.Request_Shared) should not fail if another shared lock is held
+     * against the resource identified by `name` (though it might fail for
+     * other reasons).
+     *
+     * @return true on success, false on failure (sets the global error object
+     * returned by [](cfish:cfish.Err.get_error)).
+     */
+    public abstract bool
+    Request_Shared(Lock *self);
 
-    /** Make one attempt to acquire the lock.
+    /** Make one attempt to acquire an exclusive lock.
      *
-     * The semantics of [](cfish:.Request) differ depending on whether [](cfish:.Shared) returns
-     * true.  If the Lock is [](cfish:.Shared), then [](cfish:.Request) should not fail if
-     * another lock is held against the resource identified by
-     * `name` (though it might fail for other reasons).  If it is
-     * not [](cfish:.Shared) -- i.e. it's an exclusive (write) lock -- then other locks
-     * should cause [](cfish:.Request) to fail.
+     * Other locks should cause [](.Request_Exclusive) to fail.
      *
      * @return true on success, false on failure (sets the global error object
      * returned by [](cfish:cfish.Err.get_error)).
      */
     public abstract bool
-    Request(Lock *self);
+    Request_Exclusive(Lock *self);
 
     /** Release the lock.
      */
@@ -85,13 +97,21 @@ public abstract class Lucy::Store::Lock inherits Clownfish::Obj {
     Release(Lock *self);
 
     /** Indicate whether the resource identified by this lock's name is
-     * currently locked.
+     * currently locked in shared or exclusive mode.
      *
      * @return true if the resource is locked, false otherwise.
      */
     public abstract bool
     Is_Locked(Lock *self);
 
+    /** Indicate whether the resource identified by this lock's name is
+     * currently locked in exclusive mode.
+     *
+     * @return true if the resource is locked, false otherwise.
+     */
+    public abstract bool
+    Is_Locked_Exclusive(Lock *self);
+
     /** Release all locks that meet the following three conditions: the lock
      * name matches, the host id matches, and the process id that the lock
      * was created under no longer identifies an active process.
@@ -105,9 +125,6 @@ public abstract class Lucy::Store::Lock inherits Clownfish::Obj {
     String*
     Get_Host(Lock *self);
 
-    String*
-    Get_Lock_Path(Lock *self);
-
     public void
     Destroy(Lock *self);
 }
@@ -115,21 +132,23 @@ public abstract class Lucy::Store::Lock inherits Clownfish::Obj {
 class Lucy::Store::LockFileLock nickname LFLock
     inherits Lucy::Store::Lock {
 
+    String *shared_lock_path;
     String *link_path;
+    bool    exclusive_only;
 
     inert incremented LockFileLock*
-    new(Folder *folder, String *name, String *host,
-        int32_t timeout = 0, int32_t interval = 100);
+    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);
+    init(LockFileLock *self, Folder *folder, String *name, String *host,
+         int32_t timeout = 0, int32_t interval = 100, bool exclusive_only);
 
     public bool
-    Shared(LockFileLock *self);
+    Request_Shared(LockFileLock *self);
 
     public bool
-    Request(LockFileLock *self);
+    Request_Exclusive(LockFileLock *self);
 
     public void
     Release(LockFileLock *self);
@@ -137,21 +156,14 @@ class Lucy::Store::LockFileLock nickname LFLock
     public bool
     Is_Locked(LockFileLock *self);
 
+    public bool
+    Is_Locked_Exclusive(LockFileLock *self);
+
     public void
     Clear_Stale(LockFileLock *self);
 
-    /** Delete a given lock file which meets these conditions...
-     *
-     *    - lock name matches.
-     *    - host id matches.
-     *
-     * If delete_mine is false, don't delete a lock file which matches this
-     * process's pid.  If delete_other is false, don't delete lock files which
-     * don't match this process's pid.
-     */
-    bool
-    Maybe_Delete_File(LockFileLock *self, String *filepath,
-                      bool delete_mine, bool delete_other);
+    String*
+    Get_Lock_Path(LockFileLock *self);
 
     public void
     Destroy(LockFileLock *self);

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Store/LockFactory.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/LockFactory.c b/core/Lucy/Store/LockFactory.c
index d2042cd..bd501f3 100644
--- a/core/Lucy/Store/LockFactory.c
+++ b/core/Lucy/Store/LockFactory.c
@@ -23,7 +23,6 @@
 #include "Lucy/Store/LockFactory.h"
 #include "Lucy/Store/Folder.h"
 #include "Lucy/Store/Lock.h"
-#include "Lucy/Store/SharedLock.h"
 
 LockFactory*
 LockFact_new(Folder *folder, String *host) {
@@ -48,19 +47,11 @@ LockFact_Destroy_IMP(LockFactory *self) {
 }
 
 Lock*
-LockFact_Make_Lock_IMP(LockFactory *self, String *name,
-                       int32_t timeout, int32_t interval) {
+LockFact_Make_Lock_IMP(LockFactory *self, String *name, int32_t timeout,
+                       int32_t interval, bool exclusive_only) {
     LockFactoryIVARS *const ivars = LockFact_IVARS(self);
     return (Lock*)LFLock_new(ivars->folder, name, ivars->host, timeout,
-                             interval);
-}
-
-Lock*
-LockFact_Make_Shared_Lock_IMP(LockFactory *self, String *name,
-                              int32_t timeout, int32_t interval) {
-    LockFactoryIVARS *const ivars = LockFact_IVARS(self);
-    return (Lock*)ShLock_new(ivars->folder, name, ivars->host, timeout,
-                             interval);
+                             interval, exclusive_only);
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Store/LockFactory.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/LockFactory.cfh b/core/Lucy/Store/LockFactory.cfh
index 383d271..47152b6 100644
--- a/core/Lucy/Store/LockFactory.cfh
+++ b/core/Lucy/Store/LockFactory.cfh
@@ -46,32 +46,17 @@ public class Lucy::Store::LockFactory nickname LockFact
     public inert LockFactory*
     init(LockFactory *self, Folder *folder, String *host);
 
-    /** Return a Lock object, which, once [](cfish:Lock.Obtain) returns successfully,
-     * maintains an exclusive lock on a resource.
+    /** Return a Lock object.
      *
      * @param name A file-system-friendly id which identifies the
      * resource to be locked.
      * @param timeout Time in milliseconds to keep retrying before abandoning
-     * the attempt to [](cfish:Lock.Obtain) a lock.
+     * the attempt to obtain a lock.
      * @param interval Time in milliseconds between retries.
      */
     public incremented Lock*
     Make_Lock(LockFactory *self, String *name, int32_t timeout = 0,
-              int32_t interval = 100);
-
-    /** Return a Lock object for which [](cfish:Lock.Shared) returns true, and which
-     * maintains a non-exclusive lock on a resource once [](cfish:Lock.Obtain) returns
-     * success.
-     *
-     * @param name A file-system-friendly id which identifies the
-     * resource to be locked.
-     * @param timeout Time in milliseconds to keep retrying before abandoning
-     * the attempt to [](cfish:Lock.Obtain) a lock.
-     * @param interval Time in milliseconds between retries.
-     */
-    public incremented Lock*
-    Make_Shared_Lock(LockFactory *self, String *name,
-                     int32_t timeout = 0, int32_t interval = 100);
+              int32_t interval = 100, bool exclusive_only);
 
     public void
     Destroy(LockFactory *self);

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Store/SharedLock.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/SharedLock.c b/core/Lucy/Store/SharedLock.c
deleted file mode 100644
index 2563e5f..0000000
--- a/core/Lucy/Store/SharedLock.c
+++ /dev/null
@@ -1,173 +0,0 @@
-/* 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_SHAREDLOCK
-#include "Lucy/Util/ToolSet.h"
-
-#include <errno.h>
-#include <stdio.h>
-#include <ctype.h>
-
-#include "Lucy/Store/SharedLock.h"
-#include "Lucy/Store/DirHandle.h"
-#include "Lucy/Store/Folder.h"
-#include "Lucy/Store/OutStream.h"
-
-SharedLock*
-ShLock_new(Folder *folder, String *name, String *host,
-           int32_t timeout, int32_t interval) {
-    SharedLock *self = (SharedLock*)Class_Make_Obj(SHAREDLOCK);
-    return ShLock_init(self, folder, name, host, timeout, interval);
-}
-
-SharedLock*
-ShLock_init(SharedLock *self, Folder *folder, String *name,
-            String *host, int32_t timeout, int32_t interval) {
-    LFLock_init((LockFileLock*)self, folder, name, host, timeout, interval);
-    SharedLockIVARS *const ivars = ShLock_IVARS(self);
-
-    // Override.
-    DECREF(ivars->lock_path);
-    ivars->lock_path = Str_newf("");
-
-    return self;
-}
-
-bool
-ShLock_Shared_IMP(SharedLock *self) {
-    UNUSED_VAR(self);
-    return true;
-}
-
-bool
-ShLock_Request_IMP(SharedLock *self) {
-    SharedLockIVARS *const ivars = ShLock_IVARS(self);
-    uint32_t i = 0;
-    ShLock_Request_t super_request
-        = SUPER_METHOD_PTR(SHAREDLOCK, LUCY_ShLock_Request);
-
-    // Empty lock_path indicates whether this particular instance is locked.
-    if (ivars->lock_path
-        && !Str_Equals_Utf8(ivars->lock_path, "", 0)
-        && Folder_Exists(ivars->folder, ivars->lock_path)
-       ) {
-        // Don't allow double obtain.
-        Err_set_error((Err*)LockErr_new(Str_newf("Lock already obtained via '%o'",
-                                                 ivars->lock_path)));
-        return false;
-    }
-
-    do {
-        DECREF(ivars->lock_path);
-        ivars->lock_path = Str_newf("locks/%o-%u32.lock", ivars->name, ++i);
-    } while (Folder_Exists(ivars->folder, ivars->lock_path));
-
-    bool success = super_request(self);
-    if (!success) { ERR_ADD_FRAME(Err_get_error()); }
-    return success;
-}
-
-void
-ShLock_Release_IMP(SharedLock *self) {
-    SharedLockIVARS *const ivars = ShLock_IVARS(self);
-    if (ivars->lock_path && !Str_Equals_Utf8(ivars->lock_path, "", 0)) {
-        ShLock_Release_t super_release
-            = SUPER_METHOD_PTR(SHAREDLOCK, LUCY_ShLock_Release);
-        super_release(self);
-
-        // Empty out lock_path.
-        DECREF(ivars->lock_path);
-        ivars->lock_path = Str_newf("");
-    }
-}
-
-
-void
-ShLock_Clear_Stale_IMP(SharedLock *self) {
-    SharedLockIVARS *const ivars = ShLock_IVARS(self);
-
-    String *lock_dir_name = SSTR_WRAP_C("locks");
-    if (!Folder_Find_Folder(ivars->folder, lock_dir_name)) {
-        return;
-    }
-
-    DirHandle *dh = Folder_Open_Dir(ivars->folder, lock_dir_name);
-    if (!dh) { RETHROW(INCREF(Err_get_error())); }
-
-    // Take a stab at any file that begins with our lock name.
-    while (DH_Next(dh)) {
-        String *entry = DH_Get_Entry(dh);
-        if (Str_Starts_With(entry, ivars->name)
-            && Str_Ends_With_Utf8(entry, ".lock", 5)
-           ) {
-            String *candidate = Str_newf("%o/%o", lock_dir_name, entry);
-            ShLock_Maybe_Delete_File(self, candidate, false, true);
-            DECREF(candidate);
-        }
-        DECREF(entry);
-    }
-
-    DECREF(dh);
-}
-
-bool
-ShLock_Is_Locked_IMP(SharedLock *self) {
-    SharedLockIVARS *const ivars = ShLock_IVARS(self);
-
-    String *lock_dir_name = SSTR_WRAP_C("locks");
-    if (!Folder_Find_Folder(ivars->folder, lock_dir_name)) {
-        return 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);
-        // Translation:  $locked = 1 if $entry =~ /^\Q$name-\d+\.lock$/
-        if (Str_Starts_With(entry, ivars->name)) {
-            StringIterator *iter = Str_Top(entry);
-            StrIter_Advance(iter, Str_Length(ivars->name));
-            int32_t code_point = StrIter_Next(iter);
-            if (code_point == '-') {
-                code_point = StrIter_Next(iter);
-                if (code_point != STR_OOB && isdigit(code_point)) {
-                    while (STR_OOB != (code_point = StrIter_Next(iter))) {
-                        if (!isdigit(code_point)) { break; }
-                    }
-                    if (code_point == '.'
-                        && StrIter_Starts_With_Utf8(iter, "lock", 4)
-                    ) {
-                        StrIter_Advance(iter, 4);
-                        if (!StrIter_Has_Next(iter)) {
-                            DECREF(iter);
-                            DECREF(entry);
-                            DECREF(dh);
-                            return true;
-                        }
-                    }
-                }
-            }
-            DECREF(iter);
-        }
-        DECREF(entry);
-    }
-
-    DECREF(dh);
-    return false;
-}
-
-

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/core/Lucy/Store/SharedLock.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Store/SharedLock.cfh b/core/Lucy/Store/SharedLock.cfh
deleted file mode 100644
index ba0fca2..0000000
--- a/core/Lucy/Store/SharedLock.cfh
+++ /dev/null
@@ -1,77 +0,0 @@
-/* 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;
-
-/** Shared (read) lock.
- *
- * SharedLock's interface is nearly identical to that of its parent class
- * [](cfish:Lock), taking the same constructor arguments and
- * implementing the same list of methods.  It differs from Lock only in the
- * semantics of two methods.
- *
- * First, [](cfish:.Obtain) will not fail if another lock is held against the resource
- * identified by `name` (though it might fail for other reasons).
- *
- * Second, [](cfish:.Is_Locked) returns true so long as some lock, somewhere is holding
- * a lock on `name`.  That lock could be this instance, or it could
- * be another -- so is entirely possible to call [](cfish:.Release) successfully on a
- * SharedLock object yet still have [](cfish:.Is_Locked) return true.
- *
- * As currently implemented, SharedLock differs from Lock in that each caller
- * gets its own lockfile.  Lockfiles still have filenames which begin with the
- * lock name and end with ".lock", but each is also assigned a unique number
- * which gets pasted between: "foo-44.lock" instead of "foo.lock".  A
- * SharedLock is considered fully released when no lock files with a given
- * lock name are left.
- */
-class Lucy::Store::SharedLock nickname ShLock
-    inherits Lucy::Store::LockFileLock {
-
-    inert incremented SharedLock*
-    new(Folder *folder, String *name, String *host,
-        int32_t timeout = 0, int32_t interval = 100);
-
-    /**
-     * @param folder The Lucy::Store::Folder where the lock file will
-     * reside.
-     * @param name String identifying the resource to be locked.
-     * @param host An identifier which should be unique per-machine.
-     * @param timeout Time in milliseconds to keep retrying before abandoning
-     * the attempt to [](cfish:.Obtain) a lock.
-     * @param interval Time in milliseconds between retries.
-     */
-    public inert SharedLock*
-    init(SharedLock *self, Folder *folder, String *name,
-         String *host, int32_t timeout = 0, int32_t interval = 100);
-
-    public bool
-    Shared(SharedLock *self);
-
-    public bool
-    Request(SharedLock *self);
-
-    public void
-    Release(SharedLock *self);
-
-    public bool
-    Is_Locked(SharedLock *self);
-
-    public void
-    Clear_Stale(SharedLock *self);
-}
-
-

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/go/build.go
----------------------------------------------------------------------
diff --git a/go/build.go b/go/build.go
index b751ba5..16c31f2 100644
--- a/go/build.go
+++ b/go/build.go
@@ -453,8 +453,10 @@ func specClasses(parcel *cfc.Parcel) {
 	dhBinding.Register()
 
 	lockBinding := cfc.NewGoClass(parcel, "Lucy::Store::Lock")
-	lockBinding.SpecMethod("Request", "Request() error")
-	lockBinding.SpecMethod("Obtain", "Obtain() error")
+	lockBinding.SpecMethod("Request_Shared", "RequestShared() error")
+	lockBinding.SpecMethod("Request_Exclusive", "RequestExclusive() error")
+	lockBinding.SpecMethod("Obtain_Shared", "ObtainShared() error")
+	lockBinding.SpecMethod("Obtain_Exclusive", "ObtainExclusive() error")
 	lockBinding.SpecMethod("Release", "Release() error")
 	lockBinding.SpecMethod("Clear_Stale", "ClearStale() error")
 	lockBinding.Register()

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/go/lucy/index_test.go
----------------------------------------------------------------------
diff --git a/go/lucy/index_test.go b/go/lucy/index_test.go
index 30f7c4a..d47785e 100644
--- a/go/lucy/index_test.go
+++ b/go/lucy/index_test.go
@@ -196,7 +196,7 @@ func TestIndexManagerLocks(t *testing.T) {
 		t.Errorf("makeDeletionLock")
 	}
 	snapFile := "snapshot_4a.json"
-	if _, ok := manager.makeSnapshotReadLock(snapFile).(SharedLock); !ok {
+	if _, ok := manager.makeSnapshotReadLock(snapFile).(Lock); !ok {
 		t.Errorf("makeDeletionLock")
 	}
 }

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/go/lucy/store.go
----------------------------------------------------------------------
diff --git a/go/lucy/store.go b/go/lucy/store.go
index a543ece..9b9d8c3 100644
--- a/go/lucy/store.go
+++ b/go/lucy/store.go
@@ -718,9 +718,9 @@ func OpenFSDirHandle(path string) (dh FSDirHandle, err error) {
 	return dh, err
 }
 
-func (lock *LockIMP) Request() error {
+func (lock *LockIMP) RequestShared() error {
 	self := (*C.lucy_Lock)(clownfish.Unwrap(lock, "lock"))
-	success := C.LUCY_Lock_Request(self)
+	success := C.LUCY_Lock_Request_Shared(self)
 	if !success {
 		cfErr := C.cfish_Err_get_error();
 		return clownfish.WRAPAny(unsafe.Pointer(C.cfish_incref(unsafe.Pointer(cfErr)))).(error)
@@ -728,9 +728,29 @@ func (lock *LockIMP) Request() error {
 	return nil
 }
 
-func (lock *LockIMP) Obtain() error {
+func (lock *LockIMP) RequestExclusive() error {
 	self := (*C.lucy_Lock)(clownfish.Unwrap(lock, "lock"))
-	success := C.LUCY_Lock_Obtain(self)
+	success := C.LUCY_Lock_Request_Exclusive(self)
+	if !success {
+		cfErr := C.cfish_Err_get_error();
+		return clownfish.WRAPAny(unsafe.Pointer(C.cfish_incref(unsafe.Pointer(cfErr)))).(error)
+	}
+	return nil
+}
+
+func (lock *LockIMP) ObtainShared() error {
+	self := (*C.lucy_Lock)(clownfish.Unwrap(lock, "lock"))
+	success := C.LUCY_Lock_Obtain_Shared(self)
+	if !success {
+		cfErr := C.cfish_Err_get_error();
+		return clownfish.WRAPAny(unsafe.Pointer(C.cfish_incref(unsafe.Pointer(cfErr)))).(error)
+	}
+	return nil
+}
+
+func (lock *LockIMP) ObtainExclusive() error {
+	self := (*C.lucy_Lock)(clownfish.Unwrap(lock, "lock"))
+	success := C.LUCY_Lock_Obtain_Exclusive(self)
 	if !success {
 		cfErr := C.cfish_Err_get_error();
 		return clownfish.WRAPAny(unsafe.Pointer(C.cfish_incref(unsafe.Pointer(cfErr)))).(error)

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/go/lucy/store_test.go
----------------------------------------------------------------------
diff --git a/go/lucy/store_test.go b/go/lucy/store_test.go
index d87d539..680b9b2 100644
--- a/go/lucy/store_test.go
+++ b/go/lucy/store_test.go
@@ -646,10 +646,11 @@ func TestFSDirHandleAll(t *testing.T) {
 	runDirHandleCommonTests(t, folder, makeDH)
 }
 
-func runLockCommonTests(t *testing.T, makeLock func(string, string) Lock) {
+func TestLockFileLockAll(t *testing.T) {
 	var err error
-	lock := makeLock("foo", "dev.example.com")
-	other := makeLock("foo", "dev.example.com")
+	folder := NewRAMFolder("myindex")
+	lock := NewLockFileLock(folder, "foo", "dev.example.com", 0, 1, false)
+	other := NewLockFileLock(folder, "foo", "dev.example.com", 0, 1, false)
 
 	if got := lock.getName(); got != "foo" {
 		t.Errorf("getName: %v", got)
@@ -658,7 +659,7 @@ func runLockCommonTests(t *testing.T, makeLock func(string, string) Lock) {
 		t.Errorf("getHost: %v", got)
 	}
 
-	err = lock.Request()
+	err = lock.RequestShared()
 	if err != nil {
 		t.Errorf("Request: %v", err)
 	}
@@ -669,42 +670,51 @@ func runLockCommonTests(t *testing.T, makeLock func(string, string) Lock) {
 		// Lock path only valid when locked for shared locks.
 		t.Errorf("getLockPath should work")
 	}
-	err = other.Request()
-	if other.Shared() && err != nil {
-		t.Errorf("SharedLock Request should succeed: %v", err)
-	} else if !other.Shared() && err == nil {
-		t.Errorf("Request should fail for locked resource")
+	err = other.RequestShared()
+	if err != nil {
+		t.Errorf("Shared lock Request should succeed: %v", err)
 	}
 	err = lock.Release()
 	if err != nil {
-		t.Errorf("Request: %v", err)
+		t.Errorf("Release: %v", err)
 	}
 	other.Release()
-	err = lock.Obtain()
+	err = lock.ObtainShared()
 	if err != nil {
 		t.Errorf("Obtain: %v", err)
 	}
+	lock.Release()
 
-	err = lock.ClearStale()
+	err = lock.RequestExclusive()
 	if err != nil {
-		t.Errorf("Nothing for ClearStale to do, but should still suceed: %v", err)
+		t.Errorf("Request: %v", err)
 	}
-}
-
-func TestLockFileLockAll(t *testing.T) {
-	folder := NewRAMFolder("myindex")
-	makeLock := func(name, host string) Lock {
-		return NewLockFileLock(folder, name, host, 0, 1)
+	if !lock.IsLocked() {
+		t.Errorf("Lock should be locked, but IsLocked returned false")
 	}
-	runLockCommonTests(t, makeLock)
-}
+	if got := lock.getLockPath(); len(got) == 0 {
+		// Lock path only valid when locked for shared locks.
+		t.Errorf("getLockPath should work")
+	}
+	err = other.RequestExclusive()
+	if err == nil {
+		t.Errorf("Request should fail for locked resource")
+	}
+	err = lock.Release()
+	if err != nil {
+		t.Errorf("Release: %v", err)
+	}
+	other.Release()
+	err = lock.ObtainExclusive()
+	if err != nil {
+		t.Errorf("Obtain: %v", err)
+	}
+	lock.Release()
 
-func TestSharedLockAll(t *testing.T) {
-	folder := NewRAMFolder("myindex")
-	makeLock := func(name, host string) Lock {
-		return NewSharedLock(folder, name, host, 0, 1)
+	err = lock.ClearStale()
+	if err != nil {
+		t.Errorf("Nothing for ClearStale to do, but should still suceed: %v", err)
 	}
-	runLockCommonTests(t, makeLock)
 }
 
 func TestLockFactoryAll(t *testing.T) {

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/perl/buildlib/Lucy/Build/Binding/Store.pm
----------------------------------------------------------------------
diff --git a/perl/buildlib/Lucy/Build/Binding/Store.pm b/perl/buildlib/Lucy/Build/Binding/Store.pm
index 4eedcba..4a0dc54 100644
--- a/perl/buildlib/Lucy/Build/Binding/Store.pm
+++ b/perl/buildlib/Lucy/Build/Binding/Store.pm
@@ -255,7 +255,7 @@ B<host> - A unique per-machine identifier.
 =item *
 
 B<timeout> - Time in milliseconds to keep retrying before abandoning
-the attempt to L<obtain()|/obtain> a lock.
+the attempt to obtain a lock.
 
 =item *
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/perl/t/105-folder.t
----------------------------------------------------------------------
diff --git a/perl/t/105-folder.t b/perl/t/105-folder.t
index 735c162..3d00195 100644
--- a/perl/t/105-folder.t
+++ b/perl/t/105-folder.t
@@ -47,27 +47,33 @@ for my $folder ( $fs_folder, $ram_folder ) {
     is( $slurped, $king, "slurp_file works" );
 
     my $lock = Lucy::Store::LockFileLock->new(
-        host    => '',
-        folder  => $folder,
-        name    => 'lock_robster',
-        timeout => 0,
+        host           => '',
+        folder         => $folder,
+        name           => 'lock_robster',
+        timeout        => 0,
+        exclusive_only => 1,
     );
     my $competing_lock = Lucy::Store::LockFileLock->new(
-        host    => '',
-        folder  => $folder,
-        name    => 'lock_robster',
-        timeout => 0,
+        host           => '',
+        folder         => $folder,
+        name           => 'lock_robster',
+        timeout        => 0,
+        exclusive_only => 1,
     );
 
-    $lock->obtain;
-    ok( $lock->is_locked,         "lock is locked" );
-    ok( !$competing_lock->obtain, "shouldn't get lock on existing resource" );
-    ok( $lock->is_locked, "lock still locked after competing attempt" );
+    $lock->obtain_exclusive();
+    ok( $lock->is_locked_exclusive(), "lock is locked" );
+    ok( !$competing_lock->obtain_exclusive(),
+        "shouldn't get lock on existing resource"
+    );
+    ok( $lock->is_locked_exclusive(),
+        "lock still locked after competing attempt"
+    );
 
     $lock->release;
-    ok( !$lock->is_locked, "release works" );
+    ok( !$lock->is_locked_exclusive(), "release works" );
 
-    $lock->obtain;
+    $lock->obtain_exclusive();
     $folder->rename( from => 'king_of_rock', to => 'king_of_lock' );
     $lock->release;
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/perl/t/106-locking.t
----------------------------------------------------------------------
diff --git a/perl/t/106-locking.t b/perl/t/106-locking.t
index b002b0a..3a1670c 100644
--- a/perl/t/106-locking.t
+++ b/perl/t/106-locking.t
@@ -44,13 +44,14 @@ Dead_locks_are_removed: {
 
     sub make_lock {
         my $lock = Lucy::Store::LockFileLock->new(
-            timeout => 0,
-            name    => 'foo',
-            host    => '',
+            timeout        => 0,
+            name           => 'foo',
+            host           => '',
+            exclusive_only => 1,
             @_
         );
         $lock->clear_stale;
-        $lock->obtain or die "no dice";
+        $lock->obtain_exclusive() or die "no dice";
         return $lock;
     }
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/perl/t/110-shared_lock.t
----------------------------------------------------------------------
diff --git a/perl/t/110-shared_lock.t b/perl/t/110-shared_lock.t
index 0164cd5..5281c9f 100644
--- a/perl/t/110-shared_lock.t
+++ b/perl/t/110-shared_lock.t
@@ -21,40 +21,43 @@ use Lucy::Test;
 
 my $folder = Lucy::Store::RAMFolder->new;
 
-my $lock = Lucy::Store::SharedLock->new(
-    folder  => $folder,
-    name    => 'ness',
-    timeout => 0,
-    host    => 'nessie',
+my $lock = Lucy::Store::LockFileLock->new(
+    folder         => $folder,
+    name           => 'ness',
+    timeout        => 0,
+    host           => 'nessie',
+    exclusive_only => 0,
 );
 
-ok( !$lock->is_locked, "not locked yet" );
+ok( !$lock->is_locked(), "not locked yet" );
 
-ok( $lock->obtain,                        "obtain" );
-ok( $lock->is_locked,                     "is_locked" );
+ok( $lock->obtain_shared(),               "obtain" );
+ok( $lock->is_locked(),                   "is_locked" );
 ok( $folder->exists('locks/ness-1.lock'), "lockfile exists" );
 
-my $another_lock = Lucy::Store::SharedLock->new(
-    folder  => $folder,
-    name    => 'ness',
-    timeout => 0,
-    host    => 'nessie',
+my $another_lock = Lucy::Store::LockFileLock->new(
+    folder         => $folder,
+    name           => 'ness',
+    timeout        => 0,
+    host           => 'nessie',
+    exclusive_only => 0,
 );
-ok( $another_lock->obtain, "got a second lock on the same resource" );
+ok( $another_lock->obtain_shared(), "got a second lock on the same resource" );
 
 $lock->release;
-ok( $lock->is_locked,
+ok( $lock->is_locked(),
     "first lock released but still is_locked because of other lock" );
 
-my $ya_lock = Lucy::Store::SharedLock->new(
-    folder  => $folder,
-    name    => 'ness',
-    timeout => 0,
-    host    => 'nessie',
+my $ya_lock = Lucy::Store::LockFileLock->new(
+    folder         => $folder,
+    name           => 'ness',
+    timeout        => 0,
+    host           => 'nessie',
+    exclusive_only => 0,
 );
-ok( $ya_lock->obtain, "got yet another lock" );
+ok( $ya_lock->obtain_shared(), "got yet another lock" );
 
-ok( $lock->obtain, "got first lock again" );
+ok( $lock->obtain_shared(), "got first lock again" );
 is( $lock->get_lock_path, "locks/ness-3.lock",
     "first lock uses a different lock_path now" );
 
@@ -71,12 +74,12 @@ $lock->release;
 $another_lock->release;
 $ya_lock->release;
 
-ok( $lock->is_locked, "failed to release a lock with a different pid" );
+ok( $lock->is_locked(), "failed to release a lock with a different pid" );
 $lock->clear_stale;
-ok( !$lock->is_locked, "clear_stale" );
+ok( !$lock->is_locked(), "clear_stale" );
 
-ok( $lock->obtain,    "got lock again" );
-ok( $lock->is_locked, "it's locked" );
+ok( $lock->obtain_shared(), "got lock again" );
+ok( $lock->is_locked(), "it's locked" );
 
 # Rewrite lock file to spec a different host.
 $content = $folder->slurp_file("locks/ness-1.lock");
@@ -87,4 +90,4 @@ $outstream->print($content);
 $outstream->close;
 
 $lock->release;
-ok( $lock->is_locked, "don't delete lock belonging to another host" );
+ok( $lock->is_locked(), "don't delete lock belonging to another host" );

http://git-wip-us.apache.org/repos/asf/lucy/blob/bb83d0d7/perl/t/111-index_manager.t
----------------------------------------------------------------------
diff --git a/perl/t/111-index_manager.t b/perl/t/111-index_manager.t
index 76b4a09..16c98ea 100644
--- a/perl/t/111-index_manager.t
+++ b/perl/t/111-index_manager.t
@@ -32,7 +32,7 @@ sub recycle {
 
 package main;
 
-use Test::More tests => 16;
+use Test::More tests => 13;
 use Lucy::Test;
 
 my $folder = Lucy::Store::RAMFolder->new;
@@ -43,21 +43,14 @@ my $lock_factory = Lucy::Store::LockFactory->new(
 );
 
 my $lock = $lock_factory->make_lock(
-    name    => 'angie',
-    timeout => 1000,
+    name           => 'angie',
+    timeout        => 1000,
+    exclusive_only => 0,
 );
 isa_ok( $lock, 'Lucy::Store::Lock', "make_lock" );
 is( $lock->get_name, "angie", "correct lock name" );
 is( $lock->get_host, "me",    "correct host" );
 
-$lock = $lock_factory->make_shared_lock(
-    name    => 'fred',
-    timeout => 0,
-);
-is( ref($lock),      'Lucy::Store::SharedLock', "make_shared_lock" );
-is( $lock->get_name, "fred",                    "correct lock name" );
-is( $lock->get_host, "me",                      "correct host" );
-
 my $schema = Lucy::Test::TestSchema->new;
 $folder = Lucy::Store::RAMFolder->new;