-
Notifications
You must be signed in to change notification settings - Fork 159
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
allow concurrent Puts and proxied Gets #323
Conversation
19b97f6
to
1621686
Compare
Thanks @mostynb . Taking a look! |
The diff is a bit confusing, it might be easier to look at the entire file: |
cache/disk/disk.go
Outdated
if err != nil { | ||
log.Println("Failed to proxy Put:", err) | ||
} else { | ||
c.proxy.Put(kind, hash, size, rc) // Async, should be fast. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if err != nil {
log.Println("Failed to proxy Put:", err)
return err
}
// Doesn't block, should be fast
c.proxy.Put(kind, hash, size, rc)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It feels wrong to let this proxy.Put block cause the entire operation to fail (proxy.Put is allowed to fail silently). But if os.Open(tf.Name())
fails then the commit
is very likely to fail too, I guess.
if size > 0 { | ||
// If we know the size, attempt to reserve that much space. | ||
tryProxy = c.lru.Reserve(size) | ||
} else { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When/How does this happen that we don't know the size of a blob?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The HTTP API allows Content-Length to be unset/-1.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to support chunked encoding though? I believe both client and server always know the length of the content in advance?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry, I got mixed up. Content-Length is not relevant for http Get. I was trying to refer to the fact that incoming HTTP Get requests only specify the hash, without the size:
Line 196 in fe90bd1
rdr, sizeBytes, err := h.cache.Get(kind, hash, -1) |
Also, in the gRPC API clients do provide the expected size for CAS items, but cannot do so for the action cache. So these also result in a -1 here.
if !existingItem.(*lruItem).committed { | ||
inProgress = true | ||
val, available := c.lru.Get(key) | ||
if available { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If you move computing tryProxy
above the if available
block you can flatten it
if available {
item := val.(*lruItem)
if isSizeMismatch(size, item.size) {
return nil, -1, tryPoxy
}
fileInfo, err = os.Stat(blobPath)
if err != nil {
return nil, -1, tryProxy
}
foundSize := fileInfo.Size()
if isSizeMismatch(size, foundSize) {
return nil, -1, tryProxy
}
f, err := os.Open(blobPath)
if err != nil {
return nil, -1, tryProxy
}
return f, foundSize, false
}
I think this is a lot easier to read but also makes it clear that err
is not being handled. IIUC at least if os.Open()
errs we should surface this no?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One issue with that setup is that calculating tryProxy
will (usually) need to reserve space, but we do not want to reserve space if the blob available locally. I guess we could do that with a more complicated deferred function (maybe unreserve space, then unlock the mutex).
cache/disk/lru.go
Outdated
@@ -120,6 +124,44 @@ func (c *sizedLRU) MaxSize() int64 { | |||
return c.maxSize | |||
} | |||
|
|||
func (c *sizedLRU) Reserve(size int64) bool { | |||
if size < 0 || size > c.maxSize || c.reservedSize+size > c.maxSize { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can be simplified to
if size < 0 || c.reservedSize+size > c.maxSize
When would size be < 0?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a safety check for HTTP API request where Content-Length is unset/-1.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also shouldn't you be checking c.currentSize instead of c.reservedSize?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Checking with reservedSize
instead of currentSize
is definitely a bug.
But just checking c.currentSize+size > c.maxSize
for going over the upper bound is problematic if the result overflows int64. Maybe unlikely in practice, but size
comes directly from the client. If the client sends an unusually value, then size > c.maxSize
is likely to catch this, but I think I can come up with a more correct check.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, wait... We shouldn't check c.currentSize+size > c.maxSize
at the start, since we can evict items to make space.
This code was checking c.reservedSize+size > c.maxSize
because if that fails, we can't evict items to reserve enough space. I will change that part back for now, while still considering how to handle overflow.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Refactored this a bit to improve the overflow checking, and added a comment about why we check size+reservedSize.
cache/disk/lru.go
Outdated
@@ -27,16 +27,20 @@ type SizedLRU interface { | |||
Get(key Key) (value sizedItem, ok bool) | |||
Remove(key Key) | |||
Len() int | |||
CurrentSize() int64 | |||
CurrentSize() int64 // Returns the current used + reserved size. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Rename this method to CommittedAndReservedSize()? Might be interesting to export both as stats?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder what is most useful form? This is only used by Stats()
and tests. A pair of <current used + reserved size>, <reserved size>
seems reasonable and won't confuse existing users who are collecting and graphing "CurrSize".
Callers could use totalSize, reservedSize := c.CurrentSize()
which is pretty readable IMO.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Renamed to TotalSize()
and added ReservedSize()
. I left the label in the json output as CurrSize
.
cache/disk/lru.go
Outdated
if ele != nil { | ||
c.removeElement(ele) | ||
} else { | ||
return false // This should have been caught at the start. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's a bug if this happens?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shouldn't we fail somehow then and not just return false? This will put the cache in an undefined state?
cache/disk/lru.go
Outdated
newR := c.reservedSize - size | ||
|
||
if newC < 0 || newR < 0 { | ||
return false |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's a bug if this happens?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes.
Hello @mostynb, We have similar issue when using bazel 3.4.1 and bazel-remote:master instance with s3 storage as remote-cache. I've just deploy bazel-remote from your branch on our server but it's not resolving this issue.
or
Let me know, if I could help you with testing your fix. |
@dstranz: are there any errors in bazel-remote's logs when you encounter this error? And do the bazel client errors mention any specific blob hashes that we can look for in bazel-remote's logs? |
@mostynb I will try to compare build output with bazel-remote logs tomorrow and let you know. |
I had a long summer vacation and is now trying to catch up. I will try to find time to review and test next week. |
@mostynb Looking into bazel-remote logs was a very good point. I figure out that our issue was related to s3 configuration - after fixing it everything works perfectly. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm sorry for not answering for such a long time. I had a long vacation and I'm now trying to catch up.
Avoid having uncommitted entries in the LRU turned out really well and simplified a lot in your PR! Well done @mostynb!
Your new logic for when to call lru.Unreserve seems correct to me, but it is a bit hard to keep track of the special cases spread out. Have you considered simplifying that? (E.g. One pair of Reserve/defered-Unreserve in Put. And another pair in Get. But nothing in commit nor in availableOrTryProxy.)
My main question is about how performance is affected by holding lock while doing the system calls, see below.
cache/disk/disk.go
Outdated
fileInfo, err = os.Stat(blobPath) | ||
if err == nil { | ||
foundSize := fileInfo.Size() | ||
if !isSizeMismatch(size, foundSize) { | ||
var f *os.File | ||
f, err = os.Open(blobPath) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Have benchmarks been conducted, how holding the lock, during os.Stat and os.Open, affects performance when many concurrent Get and Put invocations?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't have access to a setup for high concurrency testing at the moment, So I have been mostly checking for correctness.
IIRC my reasoning here was that calling os.Stat()
and then os.Open()
without a lock is racy. But now I realise that we can open the file, release the lock, and then call File.Stat()
safely. So then we would be down to one atomic filesystem operation while checking the availability, and we can decide if it's OK for there to be a race between checking the index (with the lock) and checking the filesystem (without the lock).
If we release the lock before opening the file, there's a potential race condition with the item being purged (or overwritten, but I think that case is OK). The purged-before-open is most likely when the disk cache is small, eg as some users have when using a proxy backend. But in this scenario, we would just fall back to retrieving the file from the proxy backend. We currently have this race on master, and it's probably OK- the cache is too small for the workload and should be increased. I will push a change to do this.
cache/disk/disk.go
Outdated
} | ||
|
||
return err | ||
err = os.Rename(tempfile, finalPath) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Have benchmarks been conducted, how holding the lock during os.Rename, affects performance when many concurrent Get and Put invocations?
I have been thinking about an option E, in addition to the alternatives A-D that we discussed in #267
E: Avoid the need for holding lock during os.Rename, by embedding both the hash and the size in the path returned by cacheFilePath. So that concurrent Put requests with same hash but different size, could not cause size inconsistences depending on in which order their os.Rename are scheduled.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If the cache item filenames contain the size, then it would be difficult to service requests (like HTTP GET/HEAD) that do not specify a size.
I think we need the file renaming and insertion into the index to be atomic for a given key. Otherwise overlapping Put calls can leave the file on disk out of sync with the index.
Aside: I just noticed that os.Rename()
is not atomic, it calls os.Lstat()
and then syscall.Rename()
. We may as well just call syscall.Rename()
directly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, another idea... what if we write to a tempfile in the dir, then rename it to something which indicates that we have finished writing to it, then obtain the lock, and lru.Add an item to the LRU which contains the final filename and the same key as now. lru.Add could then return the previous filename if there was one. Then we release the lock and remove the old file that was returned (if there was one).
On startup, if there are multiple "finished" files for a given key we keep the one with the most recent atime and delete the others.
But I think this change might be best done in a followup PR, if calling syscall.Rename turns out to be too slow.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If the cache item filenames contain the size, then it would be difficult to service requests (like HTTP GET/HEAD) that do not specify a size.
For PUT we know the size after writeAndCloseFile finished for the temporary file. And for GET/HEAD with unspecified size, perhaps we could get the size from LRU, via the hash as key?
I think we need the file renaming and insertion into the index to be atomic for a given key. Otherwise overlapping Put calls can leave the file on disk out of sync with the index.
I’m thinking about if embedding both hash and size in the final file name, could avoid the risk for out of sync between file on disk and index.
Aside: I just noticed that os.Rename() is not atomic, it calls os.Lstat() and then syscall.Rename(). We may as well just call syscall.Rename() directly.
Good finding!
if err != nil { | ||
return err | ||
} | ||
|
||
if c.proxy != nil { | ||
rc, err := os.Open(tf.Name()) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Smart that you use the temporary file, instead of the final destination file, for the proxy, so that no lock is needed.
At Ericsson we deploy bazel-remote on 72 core machines. I plan to allocate some of them during the coming weekend, and evaluate if holding the mutex during syscall.Rename and os.Open, affects build performance or not, when there are many concurrent GET and PUT. The previous conversation is easily lost in the collapsed code review for outdated code above. @mostynb, do you have more reflections about embedding both hash and size in the file path, to potentially reduce the need for holding mutex during the syscalls? |
Thanks- that would be really useful.
Unfortunately I think it might be racy, eg:
A more complicated version might work, where we write to files of the form {hash}.tmp{unique} then move them to {hash}.done{unique} and then insert them into the index (including a field with the {unique} string). If we're replacing an item in the index then we remove the old {hash}.done{otherunique} file from disk after releasing the lock. That way there is much lower chance of collision between racing requests. But I think this change is large enough that we should land it in the more safe version (removing the files while holding the lock), and then follow up with an optimisation if it's too slow. Changing the on-disk format in this way probably requires a major version bump. |
Perhaps a {hash}-{size} named file would not have to be removed by a later PUT of same hash but different size. Instead a new entry could be added to the LRU’s double linked list, and the original entry could still be in the double linked list and be evicted the normal way. That might involve having hash+size as key in the LRU, just as the gRPC protocol seems to do conceptually. And for HTTP with unspecified size, complement with an additional map in LRU with only hash as key, towards the most recent regardless of size. I think that could avoid the problematic race and still not require holding mutex during syscall.Rename. However, I hope to get a chance to benchmark during the weekend. If no significant performance penalty, then we don’t need to bother about this and can keep it simple. :-) |
Instead of adding placeholder "uncommitted" items to the LRU, reserve space, then write the incoming blob to a randomly named tempfile (and verify it if possible). If everything went well, unreserve the space, add it to the LRU index and move the tempfile to the final location. This allows concurrent uploads (or proxy downloads) of the same blob, which might be slightly wasteful but meets all our consistency requirements. We can try to optimise this later, if needed. The "commit" now happens in the main part of the function and the deferred cleanup function is much simpler. We no longer have the "uncommitted item evicted" issue to consider, but we might run into cases where there are inserts that fail because there is too much space reserved. Fixes buchgr#267, buchgr#318.
2c826a6
to
983d785
Compare
I benchmarked three alternative implementations using mutexes in different ways:
Each implementation is benchmarked in two different scenarios. Each scenario is executed using up to 87 bazel clients started in parallell, for generating the load. Each client on separate machine. The bazel-remote server is deployed on a machine with 72 logical processors (2 x Xeon 6154) and 2 x 10 Gbit network interface (link aggregation). Scenario A: 0% Cache Hits
Scenario B: 100% Cache Hits
|
Thanks for running these benchmarks. Here are some initial questions/notes:
|
Thanks! I believe the logic is correct, and did not notice any errors (related to this commit) during the load test. If you decide to merge 983d785 now, despite the performance degradation, then it could perhaps be considered to embed both hash and file size in file path strings, the next time the on-disk format is changed for other reasons? (or address the degradation in other ways) For the hot-cache/100% cache hit scenario, with saturated network interface, I agree that something like bazelbuild/remote-apis#147 and/or bazelbuild/bazel#6862 would be needed. |
This PR is pretty large already, I think it's best to land it with the potential performance hit but continue working on improvements to see if we can win some of that back. This is surely faster than a failed build. We can change the on-disk format later if needed (and probably bump the version number to 2.0.0 in that case).
We can continue optimising after we get compression support (builds-without-the-bytes are already supported of course). |
Thanks @mostynb! Sounds good! |
At the moment, bazel-remote only allows a single concurrent Put for a given blob. If another Put starts, while one is ongoing, we discard the second copy and return quickly, but the blob is not yet available. This can cause trouble for the second client, which assumes that the blob is available as soon as its Put finishes. The same problem also exists for Get calls when using a proxy backend.
I guess this was not a problem when bazel-remote was first written and only supported the http cache protocol, because bazel serializes those requests more than it does with the newer gRPC protocol. With gRPC, a single bazel client is able to hit this problem.
This change refactors
disk.Cache
to allow concurrent Puts and proxied Gets, by reserving space in the LRU index and using pseudo-random tempfiles, While I was at it I extracted some common code into a couple of functions and reuse them in both Put and Get.Hopefully this fixes #267 and #318.