diff options
author | Googler <noreply@google.com> | 2018-04-27 10:37:02 -0700 |
---|---|---|
committer | Adin Scannell <ascannell@google.com> | 2018-04-28 01:44:26 -0400 |
commit | d02b74a5dcfed4bfc8f2f8e545bca4d2afabb296 (patch) | |
tree | 54f95eef73aee6bacbfc736fffc631be2605ed53 /pkg/sentry/fs/lock | |
parent | f70210e742919f40aa2f0934a22f1c9ba6dada62 (diff) |
Check in gVisor.
PiperOrigin-RevId: 194583126
Change-Id: Ica1d8821a90f74e7e745962d71801c598c652463
Diffstat (limited to 'pkg/sentry/fs/lock')
-rw-r--r-- | pkg/sentry/fs/lock/BUILD | 72 | ||||
-rw-r--r-- | pkg/sentry/fs/lock/lock.go | 457 | ||||
-rw-r--r-- | pkg/sentry/fs/lock/lock_range_test.go | 136 | ||||
-rw-r--r-- | pkg/sentry/fs/lock/lock_set_functions.go | 69 | ||||
-rw-r--r-- | pkg/sentry/fs/lock/lock_test.go | 1059 |
5 files changed, 1793 insertions, 0 deletions
diff --git a/pkg/sentry/fs/lock/BUILD b/pkg/sentry/fs/lock/BUILD new file mode 100644 index 000000000..c15dde800 --- /dev/null +++ b/pkg/sentry/fs/lock/BUILD @@ -0,0 +1,72 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_generics:defs.bzl", "go_template_instance") +load("//tools/go_stateify:defs.bzl", "go_stateify") + +go_stateify( + name = "lock_state", + srcs = [ + "lock.go", + "lock_range.go", + "lock_set.go", + ], + out = "lock_state.go", + package = "lock", +) + +go_template_instance( + name = "lock_range", + out = "lock_range.go", + package = "lock", + prefix = "Lock", + template = "//pkg/segment:generic_range", + types = { + "T": "uint64", + }, +) + +go_template_instance( + name = "lock_set", + out = "lock_set.go", + consts = { + "minDegree": "3", + }, + package = "lock", + prefix = "Lock", + template = "//pkg/segment:generic_set", + types = { + "Key": "uint64", + "Range": "LockRange", + "Value": "Lock", + "Functions": "lockSetFunctions", + }, +) + +go_library( + name = "lock", + srcs = [ + "lock.go", + "lock_range.go", + "lock_set.go", + "lock_set_functions.go", + "lock_state.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs/lock", + visibility = ["//pkg/sentry:internal"], + deps = [ + "//pkg/log", + "//pkg/state", + "//pkg/waiter", + ], +) + +go_test( + name = "lock_test", + size = "small", + srcs = [ + "lock_range_test.go", + "lock_test.go", + ], + embed = [":lock"], +) diff --git a/pkg/sentry/fs/lock/lock.go b/pkg/sentry/fs/lock/lock.go new file mode 100644 index 000000000..24d54c989 --- /dev/null +++ b/pkg/sentry/fs/lock/lock.go @@ -0,0 +1,457 @@ +// Copyright 2018 Google Inc. +// +// Licensed 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. + +// Package lock is the API for POSIX-style advisory regional file locks and +// BSD-style full file locks. +// +// Callers needing to enforce these types of locks, like sys_fcntl, can call +// LockRegion and UnlockRegion on a thread-safe set of Locks. Locks are +// specific to a unique file (unique device/inode pair) and for this reason +// should not be shared between files. +// +// A Lock has a set of holders identified by UniqueID. Normally this is the +// pid of the thread attempting to acquire the lock. +// +// Since these are advisory locks, they do not need to be integrated into +// Reads/Writes and for this reason there is no way to *check* if a lock is +// held. One can only attempt to take a lock or unlock an existing lock. +// +// A Lock in a set of Locks is typed: it is either a read lock with any number +// of readers and no writer, or a write lock with no readers. +// +// As expected from POSIX, any attempt to acquire a write lock on a file region +// when there already exits a write lock held by a different uid will fail. Any +// attempt to acquire a write lock on a file region when there is more than one +// reader will fail. Any attempt to acquire a read lock on a file region when +// there is already a writer will fail. +// +// In special cases, a read lock may be upgraded to a write lock and a write lock +// can be downgraded to a read lock. This can only happen if: +// +// * read lock upgrade to write lock: There can be only one reader and the reader +// must be the same as the requested write lock holder. +// +// * write lock downgrade to read lock: The writer must be the same as the requested +// read lock holder. +// +// UnlockRegion always succeeds. If LockRegion fails the caller should normally +// interpret this as "try again later". +package lock + +import ( + "fmt" + "math" + "sync" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// LockType is a type of regional file lock. +type LockType int + +// UniqueID is a unique identifier of the holder of a regional file lock. +type UniqueID uint64 + +const ( + // ReadLock describes a POSIX regional file lock to be taken + // read only. There may be multiple of these locks on a single + // file region as long as there is no writer lock on the same + // region. + ReadLock LockType = iota + + // WriteLock describes a POSIX regional file lock to be taken + // write only. There may be only a single holder of this lock + // and no read locks. + WriteLock +) + +// LockEOF is the maximal possible end of a regional file lock. +const LockEOF = math.MaxUint64 + +// Lock is a regional file lock. It consists of either a single writer +// or a set of readers. +// +// A Lock may be upgraded from a read lock to a write lock only if there +// is a single reader and that reader has the same uid as the write lock. +// +// A Lock may be downgraded from a write lock to a read lock only if +// the write lock's uid is the same as the read lock. +type Lock struct { + // Readers are the set of read lock holders identified by UniqueID. + // If len(Readers) > 0 then HasWriter must be false. + Readers map[UniqueID]bool + + // HasWriter indicates that this is a write lock held by a single + // UniqueID. + HasWriter bool + + // Writer is only valid if HasWriter is true. It identifies a + // single write lock holder. + Writer UniqueID +} + +// Locks is a thread-safe wrapper around a LockSet. +type Locks struct { + // mu protects locks below. + mu sync.Mutex `state:"nosave"` + + // locks is the set of region locks currently held on an Inode. + locks LockSet + + // blockedQueue is the queue of waiters that are waiting on a lock. + blockedQueue waiter.Queue +} + +// Blocker is the interface used for blocking locks. Passing a nil Blocker +// will be treated as non-blocking. +type Blocker interface { + Block(C chan struct{}) error +} + +const ( + // EventMaskAll is the mask we will always use for locks, by using the + // same mask all the time we can wake up everyone anytime the lock + // changes state. + EventMaskAll waiter.EventMask = 0xFFFF +) + +// LockRegion attempts to acquire a typed lock for the uid on a region +// of a file. Returns true if successful in locking the region. If false +// is returned, the caller should normally interpret this as "try again later" if +// accquiring the lock in a non-blocking mode or "interrupted" if in a blocking mode. +// Blocker is the interface used to provide blocking behavior, passing a nil Blocker +// will result in non-blocking behavior. +func (l *Locks) LockRegion(uid UniqueID, t LockType, r LockRange, block Blocker) bool { + for { + l.mu.Lock() + + // Blocking locks must run in a loop because we'll be woken up whenever an unlock event + // happens for this lock. We will then attempt to take the lock again and if it fails + // continue blocking. + res := l.locks.lock(uid, t, r) + if !res && block != nil { + e, ch := waiter.NewChannelEntry(nil) + l.blockedQueue.EventRegister(&e, EventMaskAll) + l.mu.Unlock() + if err := block.Block(ch); err != nil { + // We were interrupted, the caller can translate this to EINTR if applicable. + l.blockedQueue.EventUnregister(&e) + return false + } + l.blockedQueue.EventUnregister(&e) + continue // Try again now that someone has unlocked. + } + + l.mu.Unlock() + return res + } +} + +// UnlockRegion attempts to release a lock for the uid on a region of a file. +// This operation is always successful, even if there did not exist a lock on +// the requested region held by uid in the first place. +func (l *Locks) UnlockRegion(uid UniqueID, r LockRange) { + l.mu.Lock() + defer l.mu.Unlock() + l.locks.unlock(uid, r) + + // Now that we've released the lock, we need to wake up any waiters. + l.blockedQueue.Notify(EventMaskAll) +} + +// makeLock returns a new typed Lock that has either uid as its only reader +// or uid as its only writer. +func makeLock(uid UniqueID, t LockType) Lock { + value := Lock{Readers: make(map[UniqueID]bool)} + switch t { + case ReadLock: + value.Readers[uid] = true + case WriteLock: + value.HasWriter = true + value.Writer = uid + default: + panic(fmt.Sprintf("makeLock: invalid lock type %d", t)) + } + return value +} + +// isHeld returns true if uid is a holder of Lock. +func (l Lock) isHeld(uid UniqueID) bool { + if l.HasWriter && l.Writer == uid { + return true + } + return l.Readers[uid] +} + +// lock sets uid as a holder of a typed lock on Lock. +// +// Preconditions: canLock is true for the range containing this Lock. +func (l *Lock) lock(uid UniqueID, t LockType) { + switch t { + case ReadLock: + // If we are already a reader, then this is a no-op. + if l.Readers[uid] { + return + } + // We cannot downgrade a write lock to a read lock unless the + // uid is the same. + if l.HasWriter { + if l.Writer != uid { + panic(fmt.Sprintf("lock: cannot downgrade write lock to read lock for uid %d, writer is %d", uid, l.Writer)) + } + // Ensure that there is only one reader if upgrading. + l.Readers = make(map[UniqueID]bool) + // Ensure that there is no longer a writer. + l.HasWriter = false + } + l.Readers[uid] = true + return + case WriteLock: + // If we are already the writer, then this is a no-op. + if l.HasWriter && l.Writer == uid { + return + } + // We can only upgrade a read lock to a write lock if there + // is only one reader and that reader has the same uid as + // the write lock. + if readers := len(l.Readers); readers > 0 { + if readers != 1 { + panic(fmt.Sprintf("lock: cannot upgrade read lock to write lock for uid %d, too many readers %v", uid, l.Readers)) + } + if !l.Readers[uid] { + panic(fmt.Sprintf("lock: cannot upgrade read lock to write lock for uid %d, conflicting reader %v", uid, l.Readers)) + } + } + // Ensure that there is only a writer. + l.Readers = make(map[UniqueID]bool) + l.HasWriter = true + l.Writer = uid + default: + panic(fmt.Sprintf("lock: invalid lock type %d", t)) + } +} + +// lockable returns true if check returns true for every Lock in LockRange. +// Further, check should return true if Lock meets the callers requirements +// for locking Lock. +func (l LockSet) lockable(r LockRange, check func(value Lock) bool) bool { + // Get our starting point. + seg := l.LowerBoundSegment(r.Start) + for seg.Ok() && seg.Start() < r.End { + // Note that we don't care about overruning the end of the + // last segment because if everything checks out we'll just + // split the last segment. + if !check(seg.Value()) { + return false + } + // Jump to the next segment, ignoring gaps, for the same + // reason we ignored the first gap. + seg = seg.NextSegment() + } + // No conflict, we can get a lock for uid over the entire range. + return true +} + +// canLock returns true if uid will be able to take a Lock of type t on the +// entire range specified by LockRange. +func (l LockSet) canLock(uid UniqueID, t LockType, r LockRange) bool { + switch t { + case ReadLock: + return l.lockable(r, func(value Lock) bool { + // If there is no writer, there's no problem adding + // another reader. + if !value.HasWriter { + return true + } + // If there is a writer, then it must be the same uid + // in order to downgrade the lock to a read lock. + return value.Writer == uid + }) + case WriteLock: + return l.lockable(r, func(value Lock) bool { + // If there are only readers. + if !value.HasWriter { + // Then this uid can only take a write lock if + // this is a private upgrade, meaning that the + // only reader is uid. + return len(value.Readers) == 1 && value.Readers[uid] + } + // If the uid is already a writer on this region, then + // adding a write lock would be a no-op. + return value.Writer == uid + }) + default: + panic(fmt.Sprintf("canLock: invalid lock type %d", t)) + } +} + +// lock returns true if uid took a lock of type t on the entire range of LockRange. +// +// Preconditions: r.Start <= r.End (will panic otherwise). +func (l *LockSet) lock(uid UniqueID, t LockType, r LockRange) bool { + if r.Start > r.End { + panic(fmt.Sprintf("lock: r.Start %d > r.End %d", r.Start, r.End)) + } + + // Don't attempt to insert anything with a range of 0 and treat this + // as a successful no-op. + if r.Length() == 0 { + return true + } + + // Do a first-pass check. We *could* hold onto the segments we + // checked if canLock would return true, but traversing the segment + // set should be fast and this keeps things simple. + if !l.canLock(uid, t, r) { + return false + } + // Get our starting point. + seg, gap := l.Find(r.Start) + if gap.Ok() { + // Fill in the gap and get the next segment to modify. + seg = l.Insert(gap, gap.Range().Intersect(r), makeLock(uid, t)).NextSegment() + } else if seg.Start() < r.Start { + // Get our first segment to modify. + _, seg = l.Split(seg, r.Start) + } + for seg.Ok() && seg.Start() < r.End { + // Split the last one if necessary. + if seg.End() > r.End { + seg, _ = l.SplitUnchecked(seg, r.End) + } + + // Set the lock on the segment. This is guaranteed to + // always be safe, given canLock above. + value := seg.ValuePtr() + value.lock(uid, t) + + // Fill subsequent gaps. + gap = seg.NextGap() + if gr := gap.Range().Intersect(r); gr.Length() > 0 { + seg = l.Insert(gap, gr, makeLock(uid, t)).NextSegment() + } else { + seg = gap.NextSegment() + } + } + return true +} + +// unlock is always successful. If uid has no locks held for the range LockRange, +// unlock is a no-op. +// +// Preconditions: same as lock. +func (l *LockSet) unlock(uid UniqueID, r LockRange) { + if r.Start > r.End { + panic(fmt.Sprintf("unlock: r.Start %d > r.End %d", r.Start, r.End)) + } + + // Same as setlock. + if r.Length() == 0 { + return + } + + // Get our starting point. + seg := l.LowerBoundSegment(r.Start) + for seg.Ok() && seg.Start() < r.End { + // If this segment doesn't have a lock from uid then + // there is no need to fragment the set with Isolate (below). + // In this case just move on to the next segment. + if !seg.Value().isHeld(uid) { + seg = seg.NextSegment() + continue + } + + // Ensure that if we need to unlock a sub-segment that + // we don't unlock/remove that entire segment. + seg = l.Isolate(seg, r) + + value := seg.Value() + var remove bool + if value.HasWriter && value.Writer == uid { + // If we are unlocking a writer, then since there can + // only ever be one writer and no readers, then this + // lock should always be removed from the set. + remove = true + } else if value.Readers[uid] { + // If uid is the last reader, then just remove the entire + // segment. + if len(value.Readers) == 1 { + remove = true + } else { + // Otherwise we need to remove this reader without + // affecting any other segment's readers. To do + // this, we need to make a copy of the Readers map + // and not add this uid. + newValue := Lock{Readers: make(map[UniqueID]bool)} + for k, v := range value.Readers { + if k != uid { + newValue.Readers[k] = v + } + } + seg.SetValue(newValue) + } + } + if remove { + seg = l.Remove(seg).NextSegment() + } else { + seg = seg.NextSegment() + } + } +} + +// ComputeRange takes a positive file offset and computes the start of a LockRange +// using start (relative to offset) and the end of the LockRange using length. The +// values of start and length may be negative but the resulting LockRange must +// preserve that LockRange.Start < LockRange.End and LockRange.Start > 0. +func ComputeRange(start, length, offset int64) (LockRange, error) { + offset += start + // fcntl(2): "l_start can be a negative number provided the offset + // does not lie before the start of the file" + if offset < 0 { + return LockRange{}, syscall.EINVAL + } + + // fcntl(2): Specifying 0 for l_len has the special meaning: lock all + // bytes starting at the location specified by l_whence and l_start + // through to the end of file, no matter how large the file grows. + end := uint64(LockEOF) + if length > 0 { + // fcntl(2): If l_len is positive, then the range to be locked + // covers bytes l_start up to and including l_start+l_len-1. + // + // Since LockRange.End is exclusive we need not -1 from length.. + end = uint64(offset + length) + } else if length < 0 { + // fcntl(2): If l_len is negative, the interval described by + // lock covers bytes l_start+l_len up to and including l_start-1. + // + // Since LockRange.End is exclusive we need not -1 from offset. + signedEnd := offset + // Add to offset using a negative length (subtract). + offset += length + if offset < 0 { + return LockRange{}, syscall.EINVAL + } + if signedEnd < offset { + return LockRange{}, syscall.EOVERFLOW + } + // At this point signedEnd cannot be negative, + // since we asserted that offset is not negative + // and it is not less than offset. + end = uint64(signedEnd) + } + // Offset is guaranteed to be positive at this point. + return LockRange{Start: uint64(offset), End: end}, nil +} diff --git a/pkg/sentry/fs/lock/lock_range_test.go b/pkg/sentry/fs/lock/lock_range_test.go new file mode 100644 index 000000000..06a37c701 --- /dev/null +++ b/pkg/sentry/fs/lock/lock_range_test.go @@ -0,0 +1,136 @@ +// Copyright 2018 Google Inc. +// +// Licensed 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. + +package lock + +import ( + "syscall" + "testing" +) + +func TestComputeRange(t *testing.T) { + tests := []struct { + // Description of test. + name string + + // Requested start of the lock range. + start int64 + + // Requested length of the lock range, + // can be negative :( + length int64 + + // Pre-computed file offset based on whence. + // Will be added to start. + offset int64 + + // Expected error. + err error + + // If error is nil, the expected LockRange. + LockRange + }{ + { + name: "offset, start, and length all zero", + LockRange: LockRange{Start: 0, End: LockEOF}, + }, + { + name: "zero offset, zero start, positive length", + start: 0, + length: 4096, + offset: 0, + LockRange: LockRange{Start: 0, End: 4096}, + }, + { + name: "zero offset, negative start", + start: -4096, + offset: 0, + err: syscall.EINVAL, + }, + { + name: "large offset, negative start, positive length", + start: -2048, + length: 2048, + offset: 4096, + LockRange: LockRange{Start: 2048, End: 4096}, + }, + { + name: "large offset, negative start, zero length", + start: -2048, + length: 0, + offset: 4096, + LockRange: LockRange{Start: 2048, End: LockEOF}, + }, + { + name: "zero offset, zero start, negative length", + start: 0, + length: -4096, + offset: 0, + err: syscall.EINVAL, + }, + { + name: "large offset, zero start, negative length", + start: 0, + length: -4096, + offset: 4096, + LockRange: LockRange{Start: 0, End: 4096}, + }, + { + name: "offset, start, and length equal, length is negative", + start: 1024, + length: -1024, + offset: 1024, + LockRange: LockRange{Start: 1024, End: 2048}, + }, + { + name: "offset, start, and length equal, start is negative", + start: -1024, + length: 1024, + offset: 1024, + LockRange: LockRange{Start: 0, End: 1024}, + }, + { + name: "offset, start, and length equal, offset is negative", + start: 1024, + length: 1024, + offset: -1024, + LockRange: LockRange{Start: 0, End: 1024}, + }, + { + name: "offset, start, and length equal, all negative", + start: -1024, + length: -1024, + offset: -1024, + err: syscall.EINVAL, + }, + { + name: "offset, start, and length equal, all positive", + start: 1024, + length: 1024, + offset: 1024, + LockRange: LockRange{Start: 2048, End: 3072}, + }, + } + + for _, test := range tests { + rng, err := ComputeRange(test.start, test.length, test.offset) + if err != test.err { + t.Errorf("%s: lockRange(%d, %d, %d) got error %v, want %v", test.name, test.start, test.length, test.offset, err, test.err) + continue + } + if err == nil && rng != test.LockRange { + t.Errorf("%s: lockRange(%d, %d, %d) got LockRange %v, want %v", test.name, test.start, test.length, test.offset, rng, test.LockRange) + } + } +} diff --git a/pkg/sentry/fs/lock/lock_set_functions.go b/pkg/sentry/fs/lock/lock_set_functions.go new file mode 100644 index 000000000..e16f485be --- /dev/null +++ b/pkg/sentry/fs/lock/lock_set_functions.go @@ -0,0 +1,69 @@ +// Copyright 2018 Google Inc. +// +// Licensed 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. + +package lock + +import ( + "math" +) + +// LockSet maps a set of Locks into a file. The key is the file offset. + +type lockSetFunctions struct{} + +func (lockSetFunctions) MinKey() uint64 { + return 0 +} + +func (lockSetFunctions) MaxKey() uint64 { + return math.MaxUint64 +} + +func (lockSetFunctions) ClearValue(l *Lock) { + *l = Lock{} +} + +func (lockSetFunctions) Merge(r1 LockRange, val1 Lock, r2 LockRange, val2 Lock) (Lock, bool) { + // Merge only if the Readers/Writers are identical. + if len(val1.Readers) != len(val2.Readers) { + return Lock{}, false + } + for k := range val1.Readers { + if !val2.Readers[k] { + return Lock{}, false + } + } + if val1.HasWriter != val2.HasWriter { + return Lock{}, false + } + if val1.HasWriter { + if val1.Writer != val2.Writer { + return Lock{}, false + } + } + return val1, true +} + +func (lockSetFunctions) Split(r LockRange, val Lock, split uint64) (Lock, Lock) { + // Copy the segment so that split segments don't contain map references + // to other segments. + val0 := Lock{Readers: make(map[UniqueID]bool)} + for k, v := range val.Readers { + val0.Readers[k] = v + } + val0.HasWriter = val.HasWriter + val0.Writer = val.Writer + + return val, val0 +} diff --git a/pkg/sentry/fs/lock/lock_test.go b/pkg/sentry/fs/lock/lock_test.go new file mode 100644 index 000000000..c60f5f7a2 --- /dev/null +++ b/pkg/sentry/fs/lock/lock_test.go @@ -0,0 +1,1059 @@ +// Copyright 2018 Google Inc. +// +// Licensed 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. + +package lock + +import ( + "reflect" + "testing" +) + +type entry struct { + Lock + LockRange +} + +func equals(e0, e1 []entry) bool { + if len(e0) != len(e1) { + return false + } + for i := range e0 { + for k := range e0[i].Lock.Readers { + if !e1[i].Lock.Readers[k] { + return false + } + } + for k := range e1[i].Lock.Readers { + if !e0[i].Lock.Readers[k] { + return false + } + } + if !reflect.DeepEqual(e0[i].LockRange, e1[i].LockRange) { + return false + } + if e0[i].Lock.HasWriter != e1[i].Lock.HasWriter { + return false + } + if e0[i].Lock.Writer != e1[i].Lock.Writer { + return false + } + } + return true +} + +// fill a LockSet with consecutive region locks. Will panic if +// LockRanges are not consecutive. +func fill(entries []entry) LockSet { + l := LockSet{} + for _, e := range entries { + gap := l.FindGap(e.LockRange.Start) + if !gap.Ok() { + panic("cannot insert into existing segment") + } + l.Insert(gap, e.LockRange, e.Lock) + } + return l +} + +func TestCanLockEmpty(t *testing.T) { + l := LockSet{} + + // Expect to be able to take any locks given that the set is empty. + eof := l.FirstGap().End() + r := LockRange{0, eof} + if !l.canLock(1, ReadLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", ReadLock, r, 1) + } + if !l.canLock(2, ReadLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", ReadLock, r, 2) + } + if !l.canLock(1, WriteLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", WriteLock, r, 1) + } + if !l.canLock(2, WriteLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", WriteLock, r, 2) + } +} + +func TestCanLock(t *testing.T) { + // + -------------- + ---------- + -------------- + --------- + + // | Readers 1 & 2 | Readers 1 | Readers 1 & 3 | Writer 1 | + // + ------------- + ---------- + -------------- + --------- + + // 0 1024 2048 3072 4096 + l := fill([]entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{1: true, 2: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{1: true}}, + LockRange: LockRange{1024, 2048}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{1: true, 3: true}}, + LockRange: LockRange{2048, 3072}, + }, + { + Lock: Lock{HasWriter: true, Writer: 1}, + LockRange: LockRange{3072, 4096}, + }, + }) + + // Now that we have a mildly interesting layout, try some checks on different + // ranges, uids, and lock types. + // + // Expect to be able to extend the read lock, despite the writer lock, because + // the writer has the same uid as the requested read lock. + r := LockRange{0, 8192} + if !l.canLock(1, ReadLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", ReadLock, r, 1) + } + // Expect to *not* be able to extend the read lock since there is an overlapping + // writer region locked by someone other than the uid. + if l.canLock(2, ReadLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got true, want false", ReadLock, r, 2) + } + // Expect to be able to extend the read lock if there are only other readers in + // the way. + r = LockRange{64, 3072} + if !l.canLock(2, ReadLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", ReadLock, r, 2) + } + // Expect to be able to set a read lock beyond the range of any existing locks. + r = LockRange{4096, 10240} + if !l.canLock(2, ReadLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", ReadLock, r, 2) + } + + // Expect to not be able to take a write lock with other readers in the way. + r = LockRange{0, 8192} + if l.canLock(1, WriteLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got true, want false", WriteLock, r, 1) + } + // Expect to be able to extend the write lock for the same uid. + r = LockRange{3072, 8192} + if !l.canLock(1, WriteLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", WriteLock, r, 1) + } + // Expect to not be able to overlap a write lock for two different uids. + if l.canLock(2, WriteLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got true, want false", WriteLock, r, 2) + } + // Expect to be able to set a write lock that is beyond the range of any + // existing locks. + r = LockRange{8192, 10240} + if !l.canLock(2, WriteLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", WriteLock, r, 2) + } + // Expect to be able to upgrade a read lock (any portion of it). + r = LockRange{1024, 2048} + if !l.canLock(1, WriteLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", WriteLock, r, 1) + } + r = LockRange{1080, 2000} + if !l.canLock(1, WriteLock, r) { + t.Fatalf("canLock type %d for range %v and uid %d got false, want true", WriteLock, r, 1) + } +} + +func TestSetLock(t *testing.T) { + tests := []struct { + // description of test. + name string + + // LockSet entries to pre-fill. + before []entry + + // Description of region to lock: + // + // start is the file offset of the lock. + start uint64 + // end is the end file offset of the lock. + end uint64 + // uid of lock attempter. + uid UniqueID + // lock type requested. + lockType LockType + + // success is true if taking the above + // lock should succeed. + success bool + + // Expected layout of the set after locking + // if success is true. + after []entry + }{ + { + name: "set zero length ReadLock on empty set", + start: 0, + end: 0, + uid: 0, + lockType: ReadLock, + success: true, + }, + { + name: "set zero length WriteLock on empty set", + start: 0, + end: 0, + uid: 0, + lockType: WriteLock, + success: true, + }, + { + name: "set ReadLock on empty set", + start: 0, + end: LockEOF, + uid: 0, + lockType: ReadLock, + success: true, + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + }, + { + name: "set WriteLock on empty set", + start: 0, + end: LockEOF, + uid: 0, + lockType: WriteLock, + success: true, + // + ----------------------------------------- + + // | Writer 0 | + // + ----------------------------------------- + + // 0 max uint64 + after: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, LockEOF}, + }, + }, + }, + { + name: "set ReadLock on WriteLock same uid", + // + ----------------------------------------- + + // | Writer 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: 4096, + uid: 0, + lockType: ReadLock, + success: true, + // + ----------- + --------------------------- + + // | Readers 0 | Writer 0 | + // + ----------- + --------------------------- + + // 0 4096 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, 4096}, + }, + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "set WriteLock on ReadLock same uid", + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: 4096, + uid: 0, + lockType: WriteLock, + success: true, + // + ----------- + --------------------------- + + // | Writer 0 | Readers 0 | + // + ----------- + --------------------------- + + // 0 4096 max uint64 + after: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, 4096}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "set ReadLock on WriteLock different uid", + // + ----------------------------------------- + + // | Writer 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: 4096, + uid: 1, + lockType: ReadLock, + success: false, + }, + { + name: "set WriteLock on ReadLock different uid", + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: 4096, + uid: 1, + lockType: WriteLock, + success: false, + }, + { + name: "split ReadLock for overlapping lock at start 0", + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: 4096, + uid: 1, + lockType: ReadLock, + success: true, + // + -------------- + --------------------------- + + // | Readers 0 & 1 | Readers 0 | + // + -------------- + --------------------------- + + // 0 4096 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{0, 4096}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "split ReadLock for overlapping lock at non-zero start", + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 4096, + end: 8192, + uid: 1, + lockType: ReadLock, + success: true, + // + ---------- + -------------- + ----------- + + // | Readers 0 | Readers 0 & 1 | Readers 0 | + // + ---------- + -------------- + ----------- + + // 0 4096 8192 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, 4096}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{4096, 8192}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{8192, LockEOF}, + }, + }, + }, + { + name: "fill front gap with ReadLock", + // + --------- + ---------------------------- + + // | gap | Readers 0 | + // + --------- + ---------------------------- + + // 0 1024 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{1024, LockEOF}, + }, + }, + start: 0, + end: 8192, + uid: 0, + lockType: ReadLock, + success: true, + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + }, + { + name: "fill end gap with ReadLock", + // + ---------------------------- + + // | Readers 0 | + // + ---------------------------- + + // 0 4096 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, 4096}, + }, + }, + start: 1024, + end: LockEOF, + uid: 0, + lockType: ReadLock, + success: true, + // Note that this is not merged after lock does a Split. This is + // fine because the two locks will still *behave* as one. In other + // words we can fragment any lock all we want and semantically it + // makes no difference. + // + // + ----------- + --------------------------- + + // | Readers 0 | Readers 0 | + // + ----------- + --------------------------- + + // 0 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{1024, LockEOF}, + }, + }, + }, + { + name: "fill gap with ReadLock and split", + // + --------- + ---------------------------- + + // | gap | Readers 0 | + // + --------- + ---------------------------- + + // 0 1024 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{1024, LockEOF}, + }, + }, + start: 0, + end: 4096, + uid: 1, + lockType: ReadLock, + success: true, + // + --------- + ------------- + ------------- + + // | Reader 1 | Readers 0 & 1 | Reader 0 | + // + ----------+ ------------- + ------------- + + // 0 1024 4096 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{1: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{1024, 4096}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "upgrade ReadLock to WriteLock for single uid fill gap", + // + ------------- + --------- + --- + ------------- + + // | Readers 0 & 1 | Readers 0 | gap | Readers 0 & 2 | + // + ------------- + --------- + --- + ------------- + + // 0 1024 2048 4096 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{1024, 2048}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 2: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + start: 1024, + end: 4096, + uid: 0, + lockType: WriteLock, + success: true, + // + ------------- + -------- + ------------- + + // | Readers 0 & 1 | Writer 0 | Readers 0 & 2 | + // + ------------- + -------- + ------------- + + // 0 1024 4096 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{1024, 4096}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 2: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "upgrade ReadLock to WriteLock for single uid keep gap", + // + ------------- + --------- + --- + ------------- + + // | Readers 0 & 1 | Readers 0 | gap | Readers 0 & 2 | + // + ------------- + --------- + --- + ------------- + + // 0 1024 2048 4096 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{1024, 2048}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 2: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + start: 1024, + end: 3072, + uid: 0, + lockType: WriteLock, + success: true, + // + ------------- + -------- + --- + ------------- + + // | Readers 0 & 1 | Writer 0 | gap | Readers 0 & 2 | + // + ------------- + -------- + --- + ------------- + + // 0 1024 3072 4096 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{1024, 3072}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 2: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "fail to upgrade ReadLock to WriteLock with conflicting Reader", + // + ------------- + --------- + + // | Readers 0 & 1 | Readers 0 | + // + ------------- + --------- + + // 0 1024 2048 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{1024, 2048}, + }, + }, + start: 0, + end: 2048, + uid: 0, + lockType: WriteLock, + success: false, + }, + { + name: "take WriteLock on whole file if all uids are the same", + // + ------------- + --------- + --------- + ---------- + + // | Writer 0 | Readers 0 | Readers 0 | Readers 0 | + // + ------------- + --------- + --------- + ---------- + + // 0 1024 2048 4096 max uint64 + before: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{1024, 2048}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{2048, 4096}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + start: 0, + end: LockEOF, + uid: 0, + lockType: WriteLock, + success: true, + // We do not manually merge locks. Semantically a fragmented lock + // held by the same uid will behave as one lock so it makes no difference. + // + // + ------------- + ---------------------------- + + // | Writer 0 | Writer 0 | + // + ------------- + ---------------------------- + + // 0 1024 max uint64 + after: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{1024, LockEOF}, + }, + }, + }, + } + + for _, test := range tests { + l := fill(test.before) + + r := LockRange{Start: test.start, End: test.end} + success := l.lock(test.uid, test.lockType, r) + var got []entry + for seg := l.FirstSegment(); seg.Ok(); seg = seg.NextSegment() { + got = append(got, entry{ + Lock: seg.Value(), + LockRange: seg.Range(), + }) + } + + if success != test.success { + t.Errorf("%s: setlock(%v, %+v, %d, %d) got success %v, want %v", test.name, test.before, r, test.uid, test.lockType, success, test.success) + continue + } + + if success { + if !equals(got, test.after) { + t.Errorf("%s: got set %+v, want %+v", test.name, got, test.after) + } + } + } +} + +func TestUnlock(t *testing.T) { + tests := []struct { + // description of test. + name string + + // LockSet entries to pre-fill. + before []entry + + // Description of region to unlock: + // + // start is the file start of the lock. + start uint64 + // end is the end file start of the lock. + end uint64 + // uid of lock holder. + uid UniqueID + + // Expected layout of the set after unlocking. + after []entry + }{ + { + name: "unlock zero length on empty set", + start: 0, + end: 0, + uid: 0, + }, + { + name: "unlock on empty set (no-op)", + start: 0, + end: LockEOF, + uid: 0, + }, + { + name: "unlock uid not locked (no-op)", + // + --------------------------- + + // | Readers 1 & 2 | + // + --------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{1: true, 2: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 1024, + end: 4096, + uid: 0, + // + --------------------------- + + // | Readers 1 & 2 | + // + --------------------------- + + // 0 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{1: true, 2: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + }, + { + name: "unlock ReadLock over entire file", + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: LockEOF, + uid: 0, + }, + { + name: "unlock WriteLock over entire file", + // + ----------------------------------------- + + // | Writer 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: LockEOF, + uid: 0, + }, + { + name: "unlock partial ReadLock (start)", + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: 4096, + uid: 0, + // + ------ + --------------------------- + + // | gap | Readers 0 | + // +------- + --------------------------- + + // 0 4096 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "unlock partial WriteLock (start)", + // + ----------------------------------------- + + // | Writer 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 0, + end: 4096, + uid: 0, + // + ------ + --------------------------- + + // | gap | Writer 0 | + // +------- + --------------------------- + + // 0 4096 max uint64 + after: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "unlock partial ReadLock (end)", + // + ----------------------------------------- + + // | Readers 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 4096, + end: LockEOF, + uid: 0, + // + --------------------------- + + // | Readers 0 | + // +---------------------------- + + // 0 4096 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true}}, + LockRange: LockRange{0, 4096}, + }, + }, + }, + { + name: "unlock partial WriteLock (end)", + // + ----------------------------------------- + + // | Writer 0 | + // + ----------------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 4096, + end: LockEOF, + uid: 0, + // + --------------------------- + + // | Writer 0 | + // +---------------------------- + + // 0 4096 + after: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, 4096}, + }, + }, + }, + { + name: "unlock for single uid", + // + ------------- + --------- + ------------------- + + // | Readers 0 & 1 | Writer 0 | Readers 0 & 1 & 2 | + // + ------------- + --------- + ------------------- + + // 0 1024 4096 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{1024, 4096}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true, 2: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + start: 0, + end: LockEOF, + uid: 0, + // + --------- + --- + --------------- + + // | Readers 1 | gap | Readers 1 & 2 | + // + --------- + --- + --------------- + + // 0 1024 4096 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{1: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{1: true, 2: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "unlock subsection locked", + // + ------------------------------- + + // | Readers 0 & 1 & 2 | + // + ------------------------------- + + // 0 max uint64 + before: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true, 2: true}}, + LockRange: LockRange{0, LockEOF}, + }, + }, + start: 1024, + end: 4096, + uid: 0, + // + ----------------- + ------------- + ----------------- + + // | Readers 0 & 1 & 2 | Readers 1 & 2 | Readers 0 & 1 & 2 | + // + ----------------- + ------------- + ----------------- + + // 0 1024 4096 max uint64 + after: []entry{ + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true, 2: true}}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{1: true, 2: true}}, + LockRange: LockRange{1024, 4096}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true, 2: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "unlock mid-gap to increase gap", + // + --------- + ----- + ------------------- + + // | Writer 0 | gap | Readers 0 & 1 | + // + --------- + ----- + ------------------- + + // 0 1024 4096 max uint64 + before: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + start: 8, + end: 2048, + uid: 0, + // + --------- + ----- + ------------------- + + // | Writer 0 | gap | Readers 0 & 1 | + // + --------- + ----- + ------------------- + + // 0 8 4096 max uint64 + after: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, 8}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + }, + { + name: "unlock split region on uid mid-gap", + // + --------- + ----- + ------------------- + + // | Writer 0 | gap | Readers 0 & 1 | + // + --------- + ----- + ------------------- + + // 0 1024 4096 max uint64 + before: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{4096, LockEOF}, + }, + }, + start: 2048, + end: 8192, + uid: 0, + // + --------- + ----- + --------- + ------------- + + // | Writer 0 | gap | Readers 1 | Readers 0 & 1 | + // + --------- + ----- + --------- + ------------- + + // 0 1024 4096 8192 max uint64 + after: []entry{ + { + Lock: Lock{HasWriter: true, Writer: 0}, + LockRange: LockRange{0, 1024}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{1: true}}, + LockRange: LockRange{4096, 8192}, + }, + { + Lock: Lock{Readers: map[UniqueID]bool{0: true, 1: true}}, + LockRange: LockRange{8192, LockEOF}, + }, + }, + }, + } + + for _, test := range tests { + l := fill(test.before) + + r := LockRange{Start: test.start, End: test.end} + l.unlock(test.uid, r) + var got []entry + for seg := l.FirstSegment(); seg.Ok(); seg = seg.NextSegment() { + got = append(got, entry{ + Lock: seg.Value(), + LockRange: seg.Range(), + }) + } + if !equals(got, test.after) { + t.Errorf("%s: got set %+v, want %+v", test.name, got, test.after) + } + } +} |