Skip to content
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

feat: prefetch accounts and access keys #7590

Merged
merged 30 commits into from
Sep 14, 2022

Conversation

jakmeier
Copy link
Contributor

@jakmeier jakmeier commented Sep 8, 2022

Introduces the concept of prefetching data from the DB while applying chunks.

This is non-speculative prefetching only for now. In other words, the future is not speculatively predicted, only data is fetched that is guaranteed to be useful. More details on that inside runtime/runtime/src/prefetch.rs.

No protocol change is needed for this. In general, prefetching how it has been implemented is (supposed to be) invisible in all possible ways, other than trie storage latency.

Performance wise, this is going to make the worst-case assumption for all action receipts better. The worst case is that two accounts and one access keys have to be fetched from disk for every receipt. This IO cost dominates the gas cost for action receipt creation.

Prefetching this data opens the door to potentially reducing this cost. This could affect all actions but is particularly relevant for redistributing gas costs around function call actions, see also #6992.


Test plan

Tests check that the prefetcher loads the trie nodes that are expected into the staging area and that they are removed from it afterwards.

@jakmeier jakmeier requested a review from a team as a code owner September 8, 2022 22:43
@jakmeier jakmeier requested review from matklad and Longarithm and removed request for matklad September 8, 2022 22:43
/// Work items are defined as `TrieKey` because currently the only
/// work is to prefetch a trie key. If other IO work is added, consider
/// changing the queue to an enum.
work_queue: Arc<Mutex<BoundedQueue<TrieKey>>>,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If I got it right, BoundedQueue is not useful here - if it is full, it pops and returns element from tail, instead of returning new element back. We may need different queue implementation here, sorry if naming was misleading

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I see, my mistake. I switched to the bounded queue last minute, doing manual checks over a VecDequeue should be simple enough anyway, I don't think I need a new queue type.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

probably I should add tests for the memory bounds too, this one should be caught in a test IMO

Copy link
Contributor

@Ekleog Ekleog left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are quite a few things I did not understand, so I commented on them. Also maybe a few ideas for consideration in the inline comments? Though given the things I don't understand yet I'm not sure all of them would be good ones.

Also, this review is focused only on the concurrency parts of it, as the storage parts of things is beyond what I'm used to.

core/store/src/trie/prefetching_trie_storage.rs Outdated Show resolved Hide resolved
/// pre-fetcher uses this in read-only mode to avoid premature evictions.
shard_cache: TrieCache,
/// Shared with parent `TrieCachingStorage`.
prefetching: Arc<Mutex<PrefetchStagingArea>>,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If I understand correctly, the reads and writes that can happen here are:

  1. Adding a slot when submitting a job. This mutates the hashmap itself
  2. Filling a slot when a job completes. This mutates only one value of the hashmap
  3. Reading a slot when a job is needed. This reads only one value of the hashmap
  4. Releasing a slot after the job is needed. This writes the hashmap

With this in mind, to ideally encode this in the rust type system, I would:

  1. Hoist the Mutex inside PrefetchStagingArea so that it's Sync and users of the type don't need to care about synchronization (AFAICT we never use nor will use PrefetchStagingArea outside of a Mutex)
  2. Hoist the slots for step 2 and 3 outside of the HashMap: now, the HashMap is Hash → Id with an invariant that Id is unique in the HashMap
  3. Store the Slots outside of the HashMap, in a Vec<Mutex> indexed by the Id above. So this Vec would be stored outside of the HashMap mutex described in step 1. Hopefully we don't need to change the Vec size while running, if we did need to we could use a RwLock around the Vec because it should basically never happen

Now, steps 2 and 3 would probably be increasing code complexity for little benefit so long as there's little contention, and my guess would be there would be little contention. So maybe just consider step 1 for now, as it cleans up the API, and keep steps 2 and 3 for if benchmarks say it'd be a good idea?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I really like your suggestions here! I will probably only incorporate 1. for now, to make the code as simple as possible.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Applies step one in my latest commit

// The shard cache mutex plus the prefetch staging area mutex are used for
// that in combination. Let's call the first lock S and the second P.
// The rules for S and P are:
// 1. To avoid deadlocks, S must always be requested before P, if they are
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Step 1 would also solve this in a typesystem-based way, so this comment would become unnecessary

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure how step 1 helps here. I think I applied it roughly like you suggested. But the shard cache still lives next to the staging area in TriePrefetchingStorage and in TrieCachingStorage. Both could still lock in the wrong order from a type-system point of view.
Did I misunderstand step 1, or am I missing something else? It would be lovely if this could be incorporated in the type system!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My thinking was that if the mutex is inside PrefetchStagingArea, then it is the only thing that will ever lock it, and by scoping it cannot hold the lock for long enough to cause a deadlock. That said I haven't re-reviewed the PR yet so I may come back in a bit with additional details

core/store/src/trie/prefetching_trie_storage.rs Outdated Show resolved Hide resolved
core/store/src/trie/prefetching_trie_storage.rs Outdated Show resolved Hide resolved
@@ -1319,6 +1335,10 @@ impl Runtime {
}

// And then we process the new incoming receipts. These are receipts from other shards.
if let Some(prefetcher) = &mut prefetcher {
prefetcher.clear();
let _queue_full = prefetcher.input_receipts(&incoming_receipts);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And same here, this could be moved alongside the two other input_receipts I think

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Indeed it could and I had it this way in my first implementation.
I think it makes more sense here because we want to fetch this data after all delayed receipts have finished, if they even finish all in this block. Also, the bounded queue of requests will have more space again even if the local receipts already filled it up.
That was my thinking. But I am open to arguments to move it up if you still think it makes more sense there.

@@ -1165,6 +1167,11 @@ impl Runtime {

let trie = Rc::new(trie);
let mut state_update = TrieUpdate::new(trie.clone());
let mut prefetcher = TriePrefetcher::new(trie.clone());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't understand, why do you recreate a prefetcher from scratch, including spinning up new threads, for each new block? Would it not be possible to just reuse the same thread pool and metadata for all the blocks, just clearing out the slots to make space for the new prefetch requests?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, it would be nicer to keep the prefetcher and threads alive, yeah. The trie root of each IO thread's PrefetchingTrieStorage would have to be updated, as each chunk operates on a different trie. It can also be tricky because chunks are already executed by a rayon iterator, so we don't even know exactly how many chunks are being processed at a time.
And we would need to keep state between chunks somewhere and pass it down. In other words, the change would spill much further up in the transaction runtime, while I was trying to keep it local.

How bad do you think it is to create a new set of threads each time? If it is only a problem due to performance concerns, it might be okay to current design, my tests showed that creating the threads only costs single digit micro seconds.

@@ -1331,6 +1351,11 @@ impl Runtime {
}
}

// No more receipts are executed on this trie, stop any pending prefetches on it.
if let Some(prefetcher) = &prefetcher {
prefetcher.stop_prefetching();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should maybe be part of the impl Drop for TriePrefetcher

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's already what the current impl Drop for TriePrefetcher does effectively. It is required anyway for all failure cases that may happen above.

Would you like to see drop(prefetcher) here instead of calling stop_preftching?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm I think it would be dropped by end-of-scope anyway? (unless the code changes, I'l re-mention it in the new if it's important anyway)

runtime/runtime/src/prefetch.rs Outdated Show resolved Hide resolved

/// Start prefetching data for processing the receipts.
///
/// Returns an error if the prefetching queue is full.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thinking about it now, AFAICT the queue is already semantically bounded by the number of incoming transactions and receipts, which are all already held in memory, and gets cleared at the end of each block.

So I'm starting to wonder whether the queue should not just be unbounded, as sure it'll increase the memory use constant a bit, but this way it avoids the situation where data that could have been prefetched was not due to the queue being bounded?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With the follow-up PR that is in the works, the amount of requests per receipts will no longer be bounded by a reasonable constant. So, while in the context of only prefetching accounts and access keys your argument makes a lot of sense, the more general prefetching I want to introduce will potentially read many items per receipts.
To give a concrete example, imagine 10 receipts are ready. Each of them is a IO heavy function call that we can somehow prefetch. Each will end up using 299 Tgas, so in reality we will only process 4 of them. Fetching data for all 10 receipts would be too much in this case.
Using a bounded queue is a way to prevent that we will prefetch more data than we can feasibly fetch within one block, without the need to know or predict how many of the receipts will be executed this block.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would it actually be bad to prefetch too much? Actually, thinking about it: when switching to a new trie root, should we not dump all the prefetched and unused-yet data into shard cache, as it'll most likely be used for the next block anyway? (Though we shouldn't do that if it'd affect gas pricing, but I don't know about that)

jakmeier and others added 5 commits September 10, 2022 23:37
Co-authored-by: Léo Gaspard <github@leo.gaspard.ninja>
- use crossbeam `ArrayQueue`
    - this also fixes wrong usage of `BoundedQueue`
- put arc and mutex inside `PrefetchStagingArea`
- fixed false comments
const MAX_PREFETCH_STAGING_MEMORY: usize = 200 * 1024 * 1024;
/// How much memory capacity is reserved for each prefetch request.
/// Set to 4MiB, the same as `max_length_storage_value`.
const PREFETCH_RESERVED_BYTES_PER_SLOT: usize = 4 * 1024 * 1024;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It means that if 50 slots are occupied, we can't insert new prefetch requests anymore. One receipt can include a batch of 100 storage reads, which itself may trigger, say 20 nodes read, so such scenario would require 2K slots.
Is prefetcher fast enough to keep the queue not full? If not, can we call Trie::get_ref instead of Trie::get - so we could reduce value size limit to 1K - node size limit?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I should clarify this in the comment: This is the reserved bytes before we start fetching, at which point we assume the worst case. But once we have the value, we will use the actual size. So with 8 IO threads, we only charge this pessimistic value for 8 slots at a time. With that in mind, the prefetcher is fast enough.
If we wanted to go for many more threads, I think your idea would be perfect to address this issue. But as things stand, I tend towards prefetching everything.

- by default have prefetching disabled, allow enabling it with config
- keep prefetcher and IO threads alive between chunks
- use crossbeam bounded channel
- remove stop_io atomic boolean
- change some comments
@jakmeier
Copy link
Contributor Author

@Longarithm @mm-near @Ekleog
I pushed a few more changes.

  1. Now the feature has to be enabled in the config, otherwise it's dead code.
  2. Using crossbeam channels removed some ugly sleeps, as recv blocks until a message arrives or the channel is disconnected.
  3. I moved the ownership of PrefetchApi to be alongside the shard caches in ShardTries. This means they are kept alive between chunks of the same shard, and shared in case of multiple chunks of the same shard being applied simultaneously. A clone of the api object is then placed inside TrieCachingStorage, which the node runtime can pick up to send requests.
  4. To make simultaneous chunk processing work, I include the trie root in every request. Not pretty but by far the simplest solution that I could come up with.
  5. As a nice side-effect, the atomic boolean to manually kill threads is no longer needed.

I think I broke some tests with the config, need to check that out tomorrow... My own tests are running fine though, so prefetching still works as intended, if enabled in config.

core/store/src/trie/prefetching_trie_storage.rs Outdated Show resolved Hide resolved
#[derive(Clone, Debug)]
enum PrefetchSlot {
PendingPrefetch,
PendingFetch,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What happens if Prefetching failed ? What do we put in the slot ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Prefetching is always done by hash, and hashes must be present in node storage. So if there is a failure, it is an actual IO error or a missing node, neither of which we can handle anyway.
I've now changed it that we will at least remove the reserved slot in case of a failure, but I don't see any value in putting the error value in there. Does that make sense?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah - removing the reserved slot makes sense. thanks. -- this will be especially important for sweatcoin and other future optimizers -- that might in theory be requesting things that don't exist in storage.

core/store/src/trie/prefetching_trie_storage.rs Outdated Show resolved Hide resolved
core/store/src/trie/prefetching_trie_storage.rs Outdated Show resolved Hide resolved
.map_err(|_| StorageError::StorageInternalError)?
.ok_or_else(|| {
StorageError::StorageInconsistentState("Trie node missing".to_string())
})?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

question about error handling:

so if we have an error here -- the self.prefetching.slots will not be updated - so it will keep having PendingPrefetch there -- and blocking_get will block forever ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are right, this isn't ideal. Error are unrecoverable at this point but we don't want to end up in a infinite loop anyway.
The fix I just pushed handles None and Err separately and has some comments. And we release the slot such that any thread waiting on the value can progress. Additionally, the main thread will try again fetching the data on its own, just in case something is fishy with the store in the prefetcher.

core/store/src/trie/prefetching_trie_storage.rs Outdated Show resolved Hide resolved
core/store/src/trie/trie_storage.rs Show resolved Hide resolved

// Insert value to shard cache, if its size is small enough.
// It is fine to have a size limit for shard cache and **not** have a limit for chunk cache, because key
// is always a value hash, so for each key there could be only one value, and it is impossible to have
// **different** values for the given key in shard and chunk caches.
if val.len() < TrieConfig::max_cached_value_size() {
let mut guard = self.shard_cache.0.lock().expect(POISONED_LOCK_ERR);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so in case of the "old" behaviour -- we acquire the lock again (while holding it already) - does it lead to deadlocks ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right, good catch. I think we should just drop the guard in any case above. The only parallel access you could have to the shard cache in the absence of prefetchers would be when multiple chunks for the same shard are applied at the same time. And I see no reason that we need to keep the lock in between, in fact releasing it makes that scenario much more efficient.

- comments
- avoid double lock of same mutex
- handle prefetcher errors more gracefully
@Longarithm
Copy link
Member

I don't have prior experience with prefetching logic. I spent several hours reading, the implementation looks clean and reasonable. From what I see, now we have a memory shared with caching storage and prefetchers, from which we take fetched values - which make strong sense.

I leave an approval, with one more comment I came up recently.

core/store/src/trie/trie_storage.rs Outdated Show resolved Hide resolved
core/store/src/trie/trie_storage.rs Outdated Show resolved Hide resolved
std::mem::drop(guard);

val = match prefetch_state {
// Slot reserved for us, or no space left. Main thread should fetch data from DB.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we are the "main" thread, right ?

especially the "Slot reserved" case deserve a better comment -- AFAIK this means, that we asked the prefetcher, but it didn't pick it up yet -- so we're doing it outselves..

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added some more comments, let me know @mm-near if you think it is still unclear

#[derive(Clone, Debug)]
enum PrefetchSlot {
PendingPrefetch,
PendingFetch,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah - removing the reserved slot makes sense. thanks. -- this will be especially important for sweatcoin and other future optimizers -- that might in theory be requesting things that don't exist in storage.

@jakmeier
Copy link
Contributor Author

I don't have prior experience with prefetching logic. I spent several hours reading, the implementation looks clean and reasonable. From what I see, now we have a memory shared with caching storage and prefetchers, from which we take fetched values - which make strong sense.

I leave an approval, with one more comment I came up recently.

That's great, thanks for the review! It is a complex PR on several dimensions. I really appreciate it that you put in the time as someone who understand the trie storage details better than I do. That's where we really needed your expertise a reviewer the most.

For prefetching details, I was able to talk it through with several people in person and I think the design makes sense in principle.

Copy link
Contributor

@Ekleog Ekleog left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Requesting changes for the "leaking keys in prefetch staging area" issue, that'd mean prefetching would stop working after enough blocks happen until the process is restarted.

Overall LGTM otherwise, though I added comments and am wondering about gas costs' relationship with shard cache

// `blocking_get` will return None if the prefetch slot has been removed
// by the main thread and the value inserted into the shard cache.
let mut guard = self.shard_cache.0.lock().expect(POISONED_LOCK_ERR);
guard.get(hash)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this useful? I'm under the impression that the return value of TriePrefetchingStorage-backed tries is just ignored. So if the value was inserted in the shard cache anyway, I think we can just return Arc::new([])?

(And maybe do that literally in every return of TriePrefetchingStorage, as it could lead to surprising behavior otherwise)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

only the final return value is ignored, all the node that are fetched through the same interface are required to traverse the trie

so if I understand the question right, then yes, it is very useful :)

})
.ok_or_else(|| {
// This could only happen if this thread started prefetching a value
// while also another thread was already prefetching it. Then the other
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here, I think the locking on the prefetching slots hashmap is enough to prevent this situation from ever happening, and it could semantically be a panic? Not that it's not good to handle it anyway, but I'm thinking the comment and error message could be more explicit

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't understand, as far as I see we are not holding the lock for the prefetching area continuously.

core/store/src/trie/prefetching_trie_storage.rs Outdated Show resolved Hide resolved
/// Queued up work will not be finished. But trie keys that are already
/// being fetched will finish.
pub fn clear(&self) {
while let Ok(_dropped) = self.work_queue_rx.try_recv() {}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this could cause spurious prefetch failures if two chunks are being processed in parallel nad one finishes while the other is still prefetching. That said it's probably something we can postpone the fix for later, so long as an issue is kept to track this lack

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

there are such issues, I tried to cover them in new comments and also explain how we deal with them currently

core/store/src/trie/shard_tries.rs Outdated Show resolved Hide resolved
guard.put(*hash, val.clone());
} else {
self.metrics.shard_cache_too_large.inc();
near_o11y::io_trace!(count: "shard_cache_too_large");
}

if let Some(prefetcher) = &self.prefetch_api {
// Only release after insertion in shard cache. See comment on fn release.
prefetcher.prefetching.release(hash);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If one of the read_from_db calls above early-return, or if future code changes lead to other early-return points being added in this function, we'll leak one slot. Not sure how bad it is though, as the prefetch area should be emptied after each chunk, so maybe we can just live with this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess yeah, but if we have a storage error on the main thread I am pretty sure we panic on the caller site anyway. Storage errors of that kind are unrecoverable AFAIK.
Anyway, with the clear between chunks, I believe this should also be handled indirectly now.

@@ -1331,6 +1351,11 @@ impl Runtime {
}
}

// No more receipts are executed on this trie, stop any pending prefetches on it.
if let Some(prefetcher) = &prefetcher {
prefetcher.clear();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We definitely need to remove from the prefetch staging area all the keys that were related to this trie root here, as otherwise all the keys that were prefetched and not consumed because processing had to stop would leak and take space forever, making prefetching less and less efficient

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

runtime/runtime/src/prefetch.rs Outdated Show resolved Hide resolved
runtime/runtime/src/prefetch.rs Show resolved Hide resolved
runtime/runtime/src/prefetch.rs Outdated Show resolved Hide resolved
@jakmeier jakmeier requested review from Ekleog and mm-near and removed request for Ekleog September 14, 2022 06:35
Copy link
Contributor

@Ekleog Ekleog left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall LGTM! I just think we should open an issue to track improving the "drop the whole hashmap upon chunk end" behavior, as it's something that'll go worse the more our system is under load, so we should probably fix it before it becomes a problem we see in the real world as then it'd be too late.

Also for all my comments about size_guard handling, I'd feel even better if it were possible, instead, to do the refactor "size+hashmap -> SizeBoundHashMap" struct as we were discussing yesterday. It'd reduce the risk some of the size computations done here were wrong (or will become wrong with future changes), as I'm not sure I didn't miss any place size is being updated. (While I do think that the current code is correct, an underflow may have pretty bad consequences, so...)

/// Get prefetched value if available and otherwise atomically set
/// prefetcher state to being fetched by main thread.
pub(crate) fn get_or_set_fetching(&self, key: CryptoHash) -> PrefetcherResult {
self.get_and_set_if_empty(key, PrefetchSlot::PendingFetch)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: this function could probably just be removed and inlined to its (I think) only caller, as I don't think it clarifies code much

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it's done to keep PrefetchSlot private to this module, which I feel does clarify code a lot in some sense

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh yes I didn't think of visibility concerns :)


fn insert_fetched(&self, key: CryptoHash, value: Arc<[u8]>) {
let mut guard = self.0.lock().expect(POISONED_LOCK_ERR);
guard.size_bytes += value.len();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: use Self::reserved_memory here

/// Reserved memory capacity for a value from the prefetching area.
fn reserved_memory(dropped: PrefetchSlot) -> usize {
match dropped {
PrefetchSlot::Done(value) => value.len(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe this should be value.len() + 16 (fat arc), or even also counting the key size, so that we avoid issues around spamming the prefetch cache with empty or near-empty values? (Though with the clean up and the fact we don't prefetch whatever the user wants yet it's probably not a big deal anyway, but just for future-proofing)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

my personal feeling is that a boundary on value sizes is easier to reason about than actual memory usage, so I'd like to keep it as it is for now
I would change my opinion if the total size was really large - but this should stay relatively small anyway.

return PrefetcherResult::MemoryLimitReached;
}
entry.insert(set_if_empty);
guard.size_bytes += PREFETCH_RESERVED_BYTES_PER_SLOT;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here, this should be Self::reserved_memory in case we ever add a caller that inputs a Done directly to the cache

for (_key, dropped) in guard.slots.drain() {
reclaimed += PrefetchStagingArea::reserved_memory(dropped);
}
guard.size_bytes -= reclaimed;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not just update the size in the loop? It's not even an atomic so performance-wise it should be the exact same.

Andeven more: why not just reset the size to 0 flat? It should end up at 0 anyway, so I can see the "reclaimed" variable being useful exclusively within a debug_assert.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in the loop is not possible due to double borrow, but I can see the argument to set it to 0 flat

// It only means we were not able to mark it as already being fetched, which in turn could lead to
// a prefetcher trying to fetch the same value before we can put it in the shard cache.
PrefetcherResult::SlotReserved | PrefetcherResult::MemoryLimitReached => {
self.read_from_db(hash)?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm no prefetch_miss metric here, just for completeness?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

actual metrics are added in the other PR, and the io trace metrics you see here are for post-processing where we can just subtract one number from another to get misses anyway

Copy link
Contributor

@Ekleog Ekleog left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks great to me! :D

@near-bulldozer near-bulldozer bot merged commit 113f66c into near:master Sep 14, 2022
jakmeier added a commit to jakmeier/nearcore that referenced this pull request Sep 15, 2022
Introduces the concept of prefetching data from the DB while applying chunks.

This is non-speculative prefetching only for now. In other words, the future is not speculatively predicted, only data is fetched that is guaranteed to be useful. More details on that inside `runtime/runtime/src/prefetch.rs`.

No protocol change is needed for this. In general, prefetching how it has been implemented is (supposed to be) invisible in all possible ways, other than trie storage latency.

Performance wise, this is going to make the worst-case assumption for all action receipts better. The worst case is that two accounts and one access keys have to be fetched from disk for every receipt. This IO cost dominates the gas cost for action receipt creation.

Prefetching this data opens the door to potentially reducing this cost. This could affect all actions but is particularly relevant for redistributing gas costs around function call actions, see also near#6992.

----

Tests check that the prefetcher loads the trie nodes that are expected into the staging area and that they are removed from it afterwards.
jakmeier added a commit to jakmeier/nearcore that referenced this pull request Sep 15, 2022
Introduces the concept of prefetching data from the DB while applying chunks.

This is non-speculative prefetching only for now. In other words, the future is not speculatively predicted, only data is fetched that is guaranteed to be useful. More details on that inside `runtime/runtime/src/prefetch.rs`.

No protocol change is needed for this. In general, prefetching how it has been implemented is (supposed to be) invisible in all possible ways, other than trie storage latency.

Performance wise, this is going to make the worst-case assumption for all action receipts better. The worst case is that two accounts and one access keys have to be fetched from disk for every receipt. This IO cost dominates the gas cost for action receipt creation.

Prefetching this data opens the door to potentially reducing this cost. This could affect all actions but is particularly relevant for redistributing gas costs around function call actions, see also near#6992.

----

Tests check that the prefetcher loads the trie nodes that are expected into the staging area and that they are removed from it afterwards.
nikurt pushed a commit that referenced this pull request Nov 9, 2022
Introduces the concept of prefetching data from the DB while applying chunks.

This is non-speculative prefetching only for now. In other words, the future is not speculatively predicted, only data is fetched that is guaranteed to be useful. More details on that inside `runtime/runtime/src/prefetch.rs`.

No protocol change is needed for this. In general, prefetching how it has been implemented is (supposed to be) invisible in all possible ways, other than trie storage latency.

Performance wise, this is going to make the worst-case assumption for all action receipts better. The worst case is that two accounts and one access keys have to be fetched from disk for every receipt. This IO cost dominates the gas cost for action receipt creation.

Prefetching this data opens the door to potentially reducing this cost. This could affect all actions but is particularly relevant for redistributing gas costs around function call actions, see also #6992.

----
### Test plan
Tests check that the prefetcher loads the trie nodes that are expected into the staging area and that they are removed from it afterwards.
@jakmeier jakmeier deleted the prefetch-accounts branch November 22, 2022 14:30
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants