-
Notifications
You must be signed in to change notification settings - Fork 53
Use EnumerateChildrenAsync in for enumerating HAMT links #19
Conversation
For comparison, much time does it take without this patch? |
@Stebalien I am going to start that shortly, but don't expect an update at least on the internet time till at least tomorrow :) |
Np. I'll be traveling 3pm onward anyways. |
Follow up on timing: For internet fetch, I ran For local machine, I get the following less than impressive results: Old Code:
This PR w/ existing Fetch Concurrency:
FetchConcurrency at 256:
My intuition here is the speed limit is either the size of the WantList -- since there's only a single peer (I ran this disconnected from the internet) -- or just the CPU (which gets pegged hard in all these cases) ^^ @Stebalien |
hamt/hamt.go
Outdated
@@ -408,6 +415,35 @@ func (ds *Shard) ForEachLink(ctx context.Context, f func(*ipld.Link) error) erro | |||
}) | |||
} | |||
|
|||
func (ds *Shard) makeAsyncTrieGetLinks(cb func(*ipld.Link) error) dag.GetLinks { |
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.
Could we add some documentation for this method? Especially explaining what is the callback argument cb
(also please consider expanding the name to something more descriptive) supposed/expected to do.
Big perf win over internet! |
hamt/hamt.go
Outdated
return nil | ||
}) | ||
|
||
err := dag.EnumerateChildrenAsync(ctx, getLinks, ds.nd.Cid(), func(c cid.Cid) bool { return true }) |
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 standard thing to pass as the visit function is a cid.Set's 'Visit' function.
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.
@whyrusleeping Does that make sense in this case? Given that the act of walking the child is what produces the shard output, I feel like we want a universal true here -- i.e. even if for some reason there were identical CIDs for two seperate shards, we will want to walk both shards so their directory output gets added to the overall list?
At the same time I'm having trouble imagining how this would be the case for a directory. Still I can definitely add it.
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 went ahead and added this: 3738990
Didn't break tests, so I have no problem with having it here
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.
even if for some reason there were identical CIDs for two seperate shards,
If there are two separate shards in the first place it would mean that they hold entries with different names (otherwise all those entries would be in the same shard) and that would mean in turn that the CIDs are different (since the names of the links they contain are different).
That being said, that reasoning is not at all obvious to a new reader who wouldn't know how the HAMT works, the set implies that we may want to avoid some repetitions (that in fact won't happen) whereas we actually want to walk all the shards, so if we're keeping the set could we add a comment line explaining this?
hamt/hamt_test.go
Outdated
@@ -288,6 +291,9 @@ func TestSetAfterMarshal(t *testing.T) { | |||
} | |||
} | |||
|
|||
// fails | |||
nds.nd.Cid() |
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.
We noticed something and want to get some feedback. This line nds.nd.Cid()
results in a nil exception
because while trying to calculate the cid
some of the links are nil
. We ran into this, because we need to pass the cid
to @whyrusleeping's EnumerateChildrenAsync
method. We have a question:
This behavior is not new. If you go to master and put the same line in the same place it will also result in the same error. Which we've determined is because calling nds.Set
does NOT populate the nds.nd.links
, but it does add nds.children
. This was not an issue before because EnumLinks
walked the children and not the links, but this new solution deals only with ProtoNode
and not Shard
. Is this ok? Does this test reflect something that another developer in the future might do: i.e. create a shard, add a bunch of nodes, and then run EnumLinks
on it?
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.
Honestly, I have no idea. The only people that know what's going on here are @whyrusleeping and @Stebalien.
Let me give you my take from an outside developer who's trying to figure out the logic here with the few comments I see on this package.
Yes, inserting a new link in the directory only saves it in the children
slice of shards but not in the node links where it's just placing a nil
placeholder,
Lines 341 to 342 in 06009a2
ds.children = append(ds.children[:i], append([]child{sv}, ds.children[i:]...)...) | |
ds.nd.SetLinks(append(ds.nd.Links()[:i], append([]*ipld.Link{nil}, ds.nd.Links()[i:]...)...)) |
From the getChild
comments and logic I'm inferring that the children
slice is just a cache of the links of the node in nd
which have the "actual truth" of the state of the HAMT,
Lines 256 to 259 in 06009a2
// getChild returns the i'th child of this shard. If it is cached in the | |
// children array, it will return it from there. Otherwise, it loads the child | |
// node from disk. | |
func (ds *Shard) getChild(ctx context.Context, i int) (child, error) { |
since when the links of the internal node are being accessed they are never checked for nil
values (it's always assumed that you can read the link Name
).
Lines 276 to 280 in 06009a2
// loadChild reads the i'th child node of this shard from disk and returns it | |
// as a 'child' interface | |
func (ds *Shard) loadChild(ctx context.Context, i int) (child, error) { | |
lnk := ds.nd.Links()[i] | |
if len(lnk.Name) < ds.maxpadlen { |
The Node
method does seem to guarantee a node that will have link integrity since it contemplates accessing both slices depending on the case (something about a "child unloaded" case I'm not really sure what it's referring to),
Lines 152 to 173 in 06009a2
ch := ds.children[cindex] | |
if ch != nil { | |
clnk, err := ch.Link() | |
if err != nil { | |
return nil, err | |
} | |
err = out.AddRawLink(ds.linkNamePrefix(i)+ch.Label(), clnk) | |
if err != nil { | |
return nil, err | |
} | |
} else { | |
// child unloaded, just copy in link with updated name | |
lnk := ds.nd.Links()[cindex] | |
label := lnk.Name[ds.maxpadlen:] | |
err := out.AddRawLink(ds.linkNamePrefix(i)+label, lnk) | |
if err != nil { | |
return nil, err | |
} | |
} | |
cindex++ |
that doesn't panic in my tests but I guess it's very expensive to be computing it in every EnumLinks
call.
Also, I'm not sure if the internal ProtoNode
of the shard (nd
) should be trusted as a faithful representations of the shard at every moment of its operation. From my (very limited) understanding ProtoNode
s are not mutable objects, or at least they should be handled with care if they want to be used to that end, I would normally advise to make a copy of the node and work on that. The Cid
method itself is very dangerous since it relies on cached versions of the computed hash, a cache that is not always invalidated when the node is modified, e.g., the SetLinks
method used here doesn't do it (the same with the weird link copy operations in rmChild
) compared with methods like AddRawLink
which does invalidate it. As a simple example, if you add a Cid
call to the beginning of any of these tests (just after the root shard creation), following calls won't panic (because they actually won't access the node links, just its cached hash).
Anyway, my answer is still "I'm not really sure", just wanted to give you some background here.
(Btw, we really need to shape up the code in this package, big time. Any external audit of this package -and by external I just mean anyone besides the three developers that have done any commits to the hamt.go
file- is really a very expensive operation. My first impression when I landed in the TestSetAfterMarshal
function and saw the three variables named nd
, ds
, and nds
was of anxiety, and it goes downhill from there.. This shouldn't be so hard as anyone can grasp the general idea of redistributing link entries in many nodes instead of overloading just one, but I feel that the implementation details are somewhat obscure, maxpadlen
? makeDirWidth
talks about size and width.. I'll open a separate issue about this, sorry for the rant :)
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.
@schomatis thank you for a thorough and thoughtful response, I'm processing it all now
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.
Unfortunately, I'm also not sure what's going on here. I'm guessing this has something to do with lazily calculating everything when we call Shard.Node()
but I don't see us actually doing that.
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 guessing this has something to do with lazily calculating everything when we call
Shard.Node()
Yes, I think the safest bet at this point would be to iterate the links in the way Node()
does,
Lines 149 to 178 in 3738990
cindex := 0 | |
// TODO: optimized 'for each set bit' | |
for i := 0; i < ds.tableSize; i++ { | |
if !ds.bitfield.Bit(i) { | |
continue | |
} | |
ch := ds.children[cindex] | |
if ch != nil { | |
clnk, err := ch.Link() | |
if err != nil { | |
return nil, err | |
} | |
err = out.AddRawLink(ds.linkNamePrefix(i)+ch.Label(), clnk) | |
if err != nil { | |
return nil, err | |
} | |
} else { | |
// child unloaded, just copy in link with updated name | |
lnk := ds.nd.Links()[cindex] | |
label := lnk.Name[ds.maxpadlen:] | |
err := out.AddRawLink(ds.linkNamePrefix(i)+label, lnk) | |
if err != nil { | |
return nil, err | |
} | |
} | |
cindex++ | |
} |
(without the expensive DAGService.Add
call or marshaling the data) which could be encapsulated in a function to avoid repeating code.
@schomatis what was your "over the internet" perf win? |
I was replicating @hannahhoward's test from #19 (comment)
and on my local machine (with a below average connection) the |
3738990
to
c5f9067
Compare
@schomatis make some readability changes -- see 45f703f per your request. Let me know what you think. |
c5f9067
to
45f703f
Compare
I don't want to block this, but I would like to get some compassion to how well this change compares to my more comprehensive solution at ipfs/kubo#4979. Even if we don't go with that solution it will give us an idea of what we should aim to achieve. |
hamt/hamt.go
Outdated
childShards = append(childShards, lnk) | ||
} else { | ||
sv := directoryShard.makeShardValue(lnk) | ||
onShardValue(sv) |
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.
unhandled error
@kevina I know a little bit about the code so I'll take a shot. I think that after some discussion we relaxed the requirement to return the listing in a stable order. Our reasoning was that readdir(), as an example, doesn't guarantee a repeatable order, and that the retrieval order of entries in the sharded structure is effectively random anyway (although it could be stable). If a sorted output is needed then the results can be sorted after fetching. Once that restriction was lifted it meant that we could reuse existing code to do the async walk of the DAG, and that code had some advantages like; it already existed and was more general purpose (dealing with Nodes and []*Link), and it didn't bother to wait for all children within a node before recursing which we think (probably) gives us a higher degree of parallelism in some tree shapes and (quite probably) gives us a quicker time to begin returning any actual directory entries. Which is good, but will require additional changes to the API to stream results. I think I have that right. People can chime in if I've misunderstood something. |
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.
other than my nits, seems fine to me.
Great! The |
hamt/hamt.go
Outdated
|
||
childShards := make([]*ipld.Link, 0, len(directoryShard.children)) | ||
for idx := range directoryShard.children { | ||
lnk := directoryShard.nd.Links()[idx] |
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.
Just so it doesn't get lost in the noise, we need to figure how to walk the trie, relying on the links at the DAG layer isn't enough since they don't always represent the current state of the shard. An approach similar to Node()
should be used.
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.
@schomatis can you explain further? We just marshalled this thing via NewHamtFromDag so presumably our links and children would have to correspond. What are the other ways that we could have a mismatch here? Or are you talking about something 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.
Sorry, you're right, I got confused with the range
over the uninitialized children
slice, but it is allocated with a space equal to the node's links, so the range
over the indexes should work.
- makeAsyncTrieGetLinks -- returns a function that can be used as a GetLinks function with EnumerateChildrenAsyc - Mutex on appending links in EnumLinks
Due to new architecture of EnumLinks, remove order concerns and prevent crashes on calls to fetch CID
5ad84cb
to
94b0134
Compare
…ding-fetch-4908 Use EnumerateChildrenAsync in for enumerating HAMT links This commit was moved from ipfs/go-unixfs@2ae0707
Goals
Speed up LS operation on sharded directories #4908
Test the use of EnumerateChildrenAsync as a fast way of fetching an arbitrary MerkleDAG w/o concern for order (ipfs/kubo#5487)
Implementation
In the
EnumLinks
method for HAMT, instead of calling ForEachLink (synchronous), useEnumerateChildrenAsync
EnumerateChildrenAsync
requires a root CID, aGetLinks
function which returns a list of child IPLD Links to traverse for a given CID, and avisit
function which returns a boolean of whether to traverse a given CID. In our case,visit
is a function that just returns true (always visit), and filtering nodes to visit is instead handled by filtering the child links returned by ourgetLinks
function. ThegetLinks
function is generated bymakeAsyncTrieGetLinks
and does the following:makeAsyncTrieGetLinks
so thatEnumLinks
can add the value to the links it will returnEnumerateChildrenAsync
will visit themEnumLinks
uses a mutex when appending to the list of links so that appends are thread safeFor Discussion
In tandem with increasing the FetchConcurrency for
EnumerateChildrenAsync
to 128 and adding sessions to the LS operation, this produces the following results for fetching the Wikipedia directory (runningipfs repo gc && time ipfs ls --resolve-type=false /ipfs/QmT5NvUtoM5nWFfrQdVrFtvGfKFmG7AHE8P34isapyhCxX/wiki/
):From the internet:
ipfs ls --resolve-type=false 6.08s user 0.86s system 0% cpu 38:30.81 total
From a machine on local network:
ipfs ls --resolve-type=false 6.68s user 1.13s system 5% cpu 2:17.42 total
The improvements are simply not as dramatic with the current
FetchConcurrency
of 8. I am also going to experiment with addingGetMany
calls inside make trie as a way of prefetching children with greater overall concurrency without changingFetchConcurrency
.I also wonder if it is neccesary to actually call NewHamtFromDag and whether there are potentially improvements in performance/memory/GC usage by skipping this extra processing step.
I am going to submit a seperate PR for adding sessions to LS, which I think can be merged independently.
Also still need to add tests -- this is not a final implementation
child of ipfs/kubo#5487